/** * Copyright 2015 StreamSets Inc. * * Licensed under the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package com.streamsets.datacollector.runner.preview; import com.codahale.metrics.ExponentiallyDecayingReservoir; import com.codahale.metrics.Histogram; import com.codahale.metrics.MetricRegistry; import com.codahale.metrics.Timer; import com.google.common.base.Throwables; import com.streamsets.datacollector.config.PipelineConfiguration; import com.streamsets.datacollector.config.StageType; import com.streamsets.datacollector.el.PipelineEL; import com.streamsets.datacollector.main.RuntimeInfo; import com.streamsets.datacollector.metrics.MetricsConfigurator; import com.streamsets.datacollector.restapi.bean.MetricRegistryJson; import com.streamsets.datacollector.runner.BatchContextImpl; import com.streamsets.datacollector.runner.BatchListener; import com.streamsets.datacollector.runner.FullPipeBatch; import com.streamsets.datacollector.runner.MultiplexerPipe; import com.streamsets.datacollector.runner.Observer; import com.streamsets.datacollector.runner.ObserverPipe; import com.streamsets.datacollector.runner.PipeContext; import com.streamsets.datacollector.runner.PipeRunner; import com.streamsets.datacollector.runner.PipelineRunner; import com.streamsets.datacollector.runner.PipelineRuntimeException; import com.streamsets.datacollector.runner.PushSourceContextDelegate; import com.streamsets.datacollector.runner.RunnerPool; import com.streamsets.datacollector.runner.RuntimeStats; import com.streamsets.datacollector.runner.SourceOffsetTracker; import com.streamsets.datacollector.runner.SourcePipe; import com.streamsets.datacollector.runner.StageContext; import com.streamsets.datacollector.runner.StageOutput; import com.streamsets.datacollector.runner.StagePipe; import com.streamsets.datacollector.runner.production.BadRecordsHandler; import com.streamsets.datacollector.runner.production.ReportErrorDelegate; import com.streamsets.datacollector.runner.production.StatsAggregationHandler; import com.streamsets.pipeline.api.BatchContext; import com.streamsets.pipeline.api.PushSource; import com.streamsets.pipeline.api.Source; import com.streamsets.pipeline.api.StageException; import com.streamsets.pipeline.api.impl.ErrorMessage; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; public class PreviewPipelineRunner implements PipelineRunner, PushSourceContextDelegate, ReportErrorDelegate { private static final Logger LOG = LoggerFactory.getLogger(PreviewPipelineRunner.class); private final RuntimeInfo runtimeInfo; private final SourceOffsetTracker offsetTracker; private final int batchSize; private final int batches; private final boolean skipTargets; private final MetricRegistry metrics; private final List<List<StageOutput>> batchesOutput; private final String name; private final String rev; private final Timer processingTimer; // Exception thrown while executing the pipeline private volatile Throwable exceptionFromExecution = null; private SourcePipe originPipe; private List<PipeRunner> pipes; private RunnerPool<PipeRunner> runnerPool; private BadRecordsHandler badRecordsHandler; private StatsAggregationHandler statsAggregationHandler; private Map<String, StageOutput> stagesToSkip; private AtomicInteger batchesProcessed; private PipelineConfiguration pipelineConfiguration; public PreviewPipelineRunner(String name, String rev, RuntimeInfo runtimeInfo, SourceOffsetTracker offsetTracker, int batchSize, int batches, boolean skipTargets) { this.name = name; this.rev = rev; this.runtimeInfo = runtimeInfo; this.offsetTracker = offsetTracker; this.batchSize = batchSize; this.batches = batches; this.skipTargets = skipTargets; this.metrics = new MetricRegistry(); processingTimer = MetricsConfigurator.createTimer(metrics, "pipeline.batchProcessing", name, rev); batchesOutput = Collections.synchronizedList(new ArrayList<List<StageOutput>>()); } @Override public MetricRegistryJson getMetricRegistryJson() { return null; } @Override public void errorNotification(SourcePipe originPipe, List<PipeRunner> pipes, Throwable throwable) { } @Override public RuntimeInfo getRuntimeInfo() { return runtimeInfo; } @Override public boolean isPreview() { return true; } @Override public MetricRegistry getMetrics() { return metrics; } @Override @SuppressWarnings("unchecked") public void run( SourcePipe originPipe, List<PipeRunner> pipes, BadRecordsHandler badRecordsHandler, StatsAggregationHandler statsAggregationHandler ) throws StageException, PipelineRuntimeException { run(originPipe, pipes, badRecordsHandler, Collections.EMPTY_LIST, statsAggregationHandler); } @Override public void run( SourcePipe originPipe, List<PipeRunner> pipes, BadRecordsHandler badRecordsHandler, List<StageOutput> stageOutputsToOverride, StatsAggregationHandler statsAggregationHandler ) throws StageException, PipelineRuntimeException { this.originPipe = originPipe; this.pipes = pipes; this.badRecordsHandler = badRecordsHandler; this.statsAggregationHandler = statsAggregationHandler; this.runnerPool = new RunnerPool<>(pipes, new RuntimeStats(), new Histogram(new ExponentiallyDecayingReservoir())); stagesToSkip = new HashMap<>(); for (StageOutput stageOutput : stageOutputsToOverride) { stagesToSkip.put(stageOutput.getInstanceName(), stageOutput); } if (originPipe.getStage().getStage() instanceof PushSource) { runPushSource(); } else { runPollSource(); } } private void runPushSource() throws StageException, PipelineRuntimeException { // This object will receive delegated calls from the push origin callbacks originPipe.getStage().setPushSourceContextDelegate(this); // Counter of batches that were already processed batchesProcessed = new AtomicInteger(0); if(stagesToSkip.containsKey(originPipe.getStage().getInfo().getInstanceName())) { // We're skipping the origin's execution, so let's run the pipeline in "usual" manner runPollSource(); } else { // Push origin will block on the call until the either all data have been consumed or the pipeline stopped originPipe.process(offsetTracker.getOffsets(), batchSize, this); } // If execution failed on exception, we should propagate it up if(exceptionFromExecution != null) { Throwables.propagateIfInstanceOf(exceptionFromExecution, StageException.class); Throwables.propagateIfInstanceOf(exceptionFromExecution, PipelineRuntimeException.class); Throwables.propagate(exceptionFromExecution); } } @Override public BatchContext startBatch() { FullPipeBatch pipeBatch = new FullPipeBatch(null, null, batchSize, true); BatchContextImpl batchContext = new BatchContextImpl(pipeBatch); originPipe.prepareBatchContext(batchContext); // Since the origin owns the threads in PushSource, need to re-populate the PipelineEL on every batch PipelineEL.setConstantsInContext(pipelineConfiguration, originPipe.getStage().getContext().getUserContext()); return batchContext; } @Override public boolean processBatch(BatchContext batchCtx, String entityName, String entityOffset) { try { BatchContextImpl batchContext = (BatchContextImpl) batchCtx; // Finish origin processing originPipe.finishBatchContext(batchContext); // Run rest of the pipeline runSourceLessBatch( batchContext.getStartTime(), batchContext.getPipeBatch(), entityName, entityOffset ); // Increment amount of intercepted batches by one and end the processing if we have desirable amount int count = batchesProcessed.incrementAndGet(); if(count >= batches) { ((StageContext)originPipe.getStage().getContext()).setStop(true); } // Not doing any commits in the preview return true; } catch (Throwable e) { LOG.error("Error while executing preview", e); // Remember the exception so that we can re-throw it later synchronized (this) { if(exceptionFromExecution == null) { exceptionFromExecution = e; } } // We got exception while executing pipeline which is a signal that we should stop processing ((StageContext)originPipe.getStage().getContext()).setStop(true); return false; } finally { PipelineEL.unsetConstantsInContext(); } } @Override public void commitOffset(String entityName, String entityOffset) { // Not doing anything in preview } private void runPollSource() throws StageException, PipelineRuntimeException { for (int i = 0; i < batches; i++) { FullPipeBatch pipeBatch = new FullPipeBatch( Source.POLL_SOURCE_OFFSET_KEY, offsetTracker.getOffsets().get(Source.POLL_SOURCE_OFFSET_KEY), batchSize, true ); long start = System.currentTimeMillis(); // Process origin data StageOutput originOutput = stagesToSkip.get(originPipe.getStage().getInfo().getInstanceName()); if(originOutput == null) { originPipe.process(pipeBatch); } else { pipeBatch.overrideStageOutput(originPipe, originOutput); } runSourceLessBatch( start, pipeBatch, Source.POLL_SOURCE_OFFSET_KEY, pipeBatch.getNewOffset() ); } } private void runSourceLessBatch( long start, FullPipeBatch pipeBatch, String offsetEntity, String newOffset ) throws StageException, PipelineRuntimeException { PipeRunner pipeRunner = null; try { pipeRunner = runnerPool.getRunner(); pipeRunner.forEach(pipe -> { StageOutput stageOutput = stagesToSkip.get(pipe.getStage().getInfo().getInstanceName()); if (stageOutput == null || (pipe instanceof ObserverPipe) || (pipe instanceof MultiplexerPipe)) { if (!skipTargets || !pipe.getStage().getDefinition().getType().isOneOf(StageType.TARGET, StageType.EXECUTOR)) { pipe.process(pipeBatch); } else { pipeBatch.skipStage(pipe); } } else { if (pipe instanceof StagePipe) { pipeBatch.overrideStageOutput((StagePipe) pipe, stageOutput); } } }); } finally { if(pipeRunner != null) { runnerPool.returnRunner(pipeRunner); } } offsetTracker.commitOffset(offsetEntity, newOffset); //TODO badRecordsHandler HANDLE ERRORS processingTimer.update(System.currentTimeMillis() - start, TimeUnit.MILLISECONDS); batchesOutput.add(pipeBatch.getSnapshotsOfAllStagesOutput()); } @Override public void destroy( SourcePipe originPipe, List<PipeRunner> pipeRunners, BadRecordsHandler badRecordsHandler, StatsAggregationHandler statsAggregationHandler ) throws StageException, PipelineRuntimeException { // We're not doing any special event propagation during preview destroy phase // Destroy origin on it's own originPipe.destroy(new FullPipeBatch(null,null, batchSize, true)); // And destroy each pipeline instance separately for(PipeRunner pipeRunner: pipeRunners) { final FullPipeBatch pipeBatch = new FullPipeBatch(null,null, batchSize, true); pipeBatch.skipStage(originPipe); pipeRunner.forEach(p -> p.destroy(pipeBatch)); } } @Override public List<List<StageOutput>> getBatchesOutput() { return batchesOutput; } @Override public void setObserver(Observer observer) { } @Override public void registerListener(BatchListener batchListener) { // TODO Auto-generated method stub } public void setPipeContext(PipeContext pipeContext) { } @Override public void setPipelineConfiguration(PipelineConfiguration pipelineConfiguration) { this.pipelineConfiguration = pipelineConfiguration; } @Override public void reportError(String stage, ErrorMessage errorMessage) { } @Override public void setFinished() { LOG.info("PreviewPipelineRunner: setFinished() was called. "); } }