public String getStageName() { return stageSpec.getName(); }
public StageMetrics createStageMetrics() { return new DefaultStageMetrics(metrics, stageSpec.getName()); }
@Override public void run() { JavaPairRDD<Object, Object> sinkRDD = rdd.flatMapToPair(sinkFunction); sinkFactory.writeFromRDD(sinkRDD, sec, stageSpec.getName(), Object.class, Object.class); } };
private boolean shouldCache(PipelinePhase pipelinePhase, StageSpec stageSpec) { // cache this RDD if it has multiple outputs, // otherwise computation of each output may trigger recomputing this stage Set<String> outputs = pipelinePhase.getStageOutputs(stageSpec.getName()); if (outputs.size() > 1) { return true; } // cache this stage if it is an input to a stage that has multiple inputs. // otherwise the union computation may trigger recomputing this stage for (String outputStageName : outputs) { StageSpec outputStage = pipelinePhase.getStage(outputStageName); //noinspection ConstantConditions if (pipelinePhase.getStageInputs(outputStageName).size() > 1) { return true; } } return false; }
public SparkBatchSinkContext(SparkBatchSinkFactory sinkFactory, JavaSparkExecutionContext sec, DatasetContext datasetContext, PipelineRuntime pipelineRuntime, StageSpec stageSpec) { super(pipelineRuntime, stageSpec, datasetContext, sec.getAdmin()); this.sinkFactory = sinkFactory; this.isPreviewEnabled = sec.getDataTracer(stageSpec.getName()).isEnabled(); }
public SparkBatchSinkContext(SparkBatchSinkFactory sinkFactory, SparkClientContext sparkContext, PipelineRuntime pipelineRuntime, DatasetContext datasetContext, StageSpec stageSpec) { super(pipelineRuntime, stageSpec, datasetContext, sparkContext.getAdmin()); this.sinkFactory = sinkFactory; this.isPreviewEnabled = sparkContext.getDataTracer(stageSpec.getName()).isEnabled(); }
public SparkBatchSourceContext(SparkBatchSourceFactory sourceFactory, SparkClientContext sparkContext, PipelineRuntime pipelineRuntime, DatasetContext datasetContext, StageSpec stageSpec) { super(pipelineRuntime, stageSpec, datasetContext, sparkContext.getAdmin()); this.sparkContext = sparkContext; this.sourceFactory = sourceFactory; this.isPreviewEnabled = sparkContext.getDataTracer(stageSpec.getName()).isEnabled(); }
/** * Instantiates and initializes the plugin for the stage. * * @param stageInfo the stage info * @return the initialized Transformation * @throws InstantiationException if the plugin for the stage could not be instantiated * @throws Exception if there was a problem initializing the plugin */ private <T extends Transformation & StageLifecycle<BatchRuntimeContext>> Transformation getInitializedTransformation(StageSpec stageInfo) throws Exception { BatchRuntimeContext runtimeContext = createRuntimeContext(stageInfo); T plugin = pluginInstantiator.newPluginInstance(stageInfo.getName(), macroEvaluator); plugin.initialize(runtimeContext); return plugin; }
public <T> T createPlugin() throws Exception { if (Constants.Connector.PLUGIN_TYPE.equals(stageSpec.getPluginType())) { String connectorType = stageSpec.getPlugin().getProperties().get(Constants.Connector.TYPE); // ok to pass in null to constructors here since we are only going to use the transform method if (connectorType.equals(Constants.Connector.SOURCE_TYPE)) { return (T) new SingleConnectorSource(null, null); } else { return (T) new SingleConnectorSink(null, null); } } MacroEvaluator macroEvaluator = new DefaultMacroEvaluator(arguments, logicalStartTime, secureStore, namespace); return getPluginContext().newPluginInstance(stageSpec.getName(), macroEvaluator); }
@Override public void run() { String stageName = stageSpec.getName(); PipelineRuntime pipelineRuntime = new SparkPipelineRuntime(sec); SparkExecutionPluginContext sparkPluginContext = new BasicSparkExecutionPluginContext(sec, jsc, datasetContext, pipelineRuntime, stageSpec); JavaRDD<T> countedRDD = rdd.map(new CountingFunction<T>(stageName, sec.getMetrics(), "records.in", null)).cache(); try { sink.run(sparkPluginContext, countedRDD); } catch (Exception e) { Throwables.propagate(e); } } };
} else if (pluginTypes.contains(Constants.SPARK_PROGRAM_PLUGIN_TYPE)) { String stageName = phase.getStagesOfType(Constants.SPARK_PROGRAM_PLUGIN_TYPE).iterator().next().getName(); StageSpec stageSpec = stageSpecs.get(stageName); applicationConfigurer.addSpark(new ExternalSparkProgram(batchPhaseSpec, stageSpec));
@Override protected SparkCollection<RecordInfo<Object>> getSource(StageSpec stageSpec, StageStatisticsCollector collector) { PluginFunctionContext pluginFunctionContext = new PluginFunctionContext(stageSpec, sec, collector); return new RDDCollection<>(sec, jsc, datasetContext, sinkFactory, sourceFactory.createRDD(sec, jsc, stageSpec.getName(), Object.class, Object.class) .flatMap(Compat.convert(new BatchSourceFunction(pluginFunctionContext, numOfRecordsPreview)))); }
@Override public void run() throws Exception { CustomActionContext context = getContext(); Map<String, String> properties = context.getSpecification().getProperties(); BatchPhaseSpec phaseSpec = GSON.fromJson(properties.get(Constants.PIPELINEID), BatchPhaseSpec.class); PipelinePhase phase = phaseSpec.getPhase(); StageSpec stageSpec = phase.iterator().next(); PluginContext pluginContext = new PipelinePluginContext(context, metrics, phaseSpec.isStageLoggingEnabled(), phaseSpec.isProcessTimingEnabled()); PipelineRuntime pipelineRuntime = new PipelineRuntime(context, metrics); Action action = pluginContext.newPluginInstance(stageSpec.getName(), new DefaultMacroEvaluator(pipelineRuntime.getArguments(), context.getLogicalStartTime(), context, context.getNamespace())); ActionContext actionContext = new BasicActionContext(context, pipelineRuntime, stageSpec); if (!context.getDataTracer(stageSpec.getName()).isEnabled()) { action.run(actionContext); } WorkflowToken token = context.getWorkflowToken(); if (token == null) { throw new IllegalStateException("WorkflowToken cannot be null when action is executed through Workflow."); } for (Map.Entry<String, String> entry : pipelineRuntime.getArguments().getAddedArguments().entrySet()) { token.put(entry.getKey(), entry.getValue()); } } }
@Override public void publishAlerts(StageSpec stageSpec, StageStatisticsCollector collector) throws Exception { PluginFunctionContext pluginFunctionContext = new PluginFunctionContext(stageSpec, sec, collector); AlertPublisher alertPublisher = pluginFunctionContext.createPlugin(); PipelineRuntime pipelineRuntime = new SparkPipelineRuntime(sec); AlertPublisherContext alertPublisherContext = new DefaultAlertPublisherContext(pipelineRuntime, stageSpec, sec.getMessagingContext(), sec.getAdmin()); alertPublisher.initialize(alertPublisherContext); StageMetrics stageMetrics = new DefaultStageMetrics(sec.getMetrics(), stageSpec.getName()); TrackedIterator<Alert> trackedAlerts = new TrackedIterator<>(((JavaRDD<Alert>) rdd).collect().iterator(), stageMetrics, Constants.Metrics.RECORDS_IN); alertPublisher.publish(trackedAlerts); alertPublisher.destroy(); }
public PluginFunctionContext(StageSpec stageSpec, JavaSparkExecutionContext sec, Map<String, String> arguments, long logicalStartTime, StageStatisticsCollector collector) { this.namespace = sec.getNamespace(); this.pipelineName = sec.getApplicationSpecification().getName(); this.stageSpec = stageSpec; this.logicalStartTime = logicalStartTime; this.arguments = new BasicArguments(sec); this.pluginContext = sec.getPluginContext(); this.serviceDiscoverer = sec.getServiceDiscoverer(); this.metrics = sec.getMetrics(); this.secureStore = sec.getSecureStore(); this.dataTracer = sec.getDataTracer(stageSpec.getName()); this.pipelinePluginContext = getPluginContext(); this.collector = collector; }
@Override public JavaRDD<U> call(JavaRDD<T> data, Time batchTime) throws Exception { SparkExecutionPluginContext sparkPluginContext = new SparkStreamingExecutionContext(sec, JavaSparkContext.fromSparkContext(data.context()), batchTime.milliseconds(), stageSpec); String stageName = stageSpec.getName(); data = data.map(new CountingFunction<T>(stageName, sec.getMetrics(), "records.in", null)); return compute.transform(sparkPluginContext, data) .map(new CountingFunction<U>(stageName, sec.getMetrics(), "records.out", sec.getDataTracer(stageName))); } }
@Override public <U> SparkCollection<U> compute(StageSpec stageSpec, SparkCompute<T, U> compute) throws Exception { String stageName = stageSpec.getName(); PipelineRuntime pipelineRuntime = new SparkPipelineRuntime(sec); SparkExecutionPluginContext sparkPluginContext = new BasicSparkExecutionPluginContext(sec, jsc, datasetContext, pipelineRuntime, stageSpec); compute.initialize(sparkPluginContext); JavaRDD<T> countedInput = rdd.map(new CountingFunction<T>(stageName, sec.getMetrics(), "records.in", null)).cache(); return wrap(compute.transform(sparkPluginContext, countedInput) .map(new CountingFunction<U>(stageName, sec.getMetrics(), "records.out", sec.getDataTracer(stageName)))); }
@Override public SparkCollection<T> window(StageSpec stageSpec, Windower windower) { String stageName = stageSpec.getName(); return wrap(stream.transform(new CountingTransformFunction<T>(stageName, sec.getMetrics(), "records.in", null)) .window(Durations.seconds(windower.getWidth()), Durations.seconds(windower.getSlideInterval())) .transform(new CountingTransformFunction<T>(stageName, sec.getMetrics(), "records.out", sec.getDataTracer(stageName)))); }
private <IN, ERROR> TrackedMultiOutputTransform<IN, ERROR> getMultiOutputTransform(StageSpec stageSpec) throws Exception { String stageName = stageSpec.getName(); DefaultMacroEvaluator macroEvaluator = new DefaultMacroEvaluator(arguments, taskContext.getLogicalStartTime(), taskContext, taskContext.getNamespace()); SplitterTransform<IN, ERROR> splitterTransform = pluginInstantiator.newPluginInstance(stageName, macroEvaluator); TransformContext transformContext = createRuntimeContext(stageSpec); splitterTransform.initialize(transformContext); StageMetrics stageMetrics = new DefaultStageMetrics(metrics, stageName); TaskAttemptContext taskAttemptContext = (TaskAttemptContext) taskContext.getHadoopContext(); StageStatisticsCollector collector = isPipelineContainsCondition ? new MapReduceStageStatisticsCollector(stageName, taskAttemptContext) : new NoopStageStatisticsCollector(); return new TrackedMultiOutputTransform<>(splitterTransform, stageMetrics, taskContext.getDataTracer(stageName), collector); }
@Override public Void call(JavaRDD<Alert> data, Time batchTime) throws Exception { MacroEvaluator evaluator = new DefaultMacroEvaluator(new BasicArguments(sec), batchTime.milliseconds(), sec.getSecureStore(), sec.getNamespace()); PluginContext pluginContext = new SparkPipelinePluginContext(sec.getPluginContext(), sec.getMetrics(), stageSpec.isStageLoggingEnabled(), stageSpec.isProcessTimingEnabled()); String stageName = stageSpec.getName(); AlertPublisher alertPublisher = pluginContext.newPluginInstance(stageName, evaluator); PipelineRuntime pipelineRuntime = new SparkPipelineRuntime(sec, batchTime.milliseconds()); AlertPublisherContext alertPublisherContext = new DefaultAlertPublisherContext(pipelineRuntime, stageSpec, sec.getMessagingContext(), sec.getAdmin()); alertPublisher.initialize(alertPublisherContext); StageMetrics stageMetrics = new DefaultStageMetrics(sec.getMetrics(), stageName); TrackedIterator<Alert> trackedAlerts = new TrackedIterator<>(data.collect().iterator(), stageMetrics, Constants.Metrics.RECORDS_IN); alertPublisher.publish(trackedAlerts); alertPublisher.destroy(); return null; } }