: new StreamConfig(new Configuration()); config.setChainStart(); config.setChainIndex(0); config.setOperatorName(streamGraph.getStreamNode(currentNodeId).getOperatorName()); config.setOutEdgesInOrder(transitiveOutEdges); config.setOutEdges(streamGraph.getStreamNode(currentNodeId).getOutEdges()); config.setTransitiveChainedTaskConfigs(chainedConfigs.get(startNodeId)); chainedConfigs.put(startNodeId, new HashMap<Integer, StreamConfig>()); config.setChainIndex(chainIndex); StreamNode node = streamGraph.getStreamNode(currentNodeId); config.setOperatorName(node.getOperatorName()); chainedConfigs.get(startNodeId).put(currentNodeId, config); config.setOperatorID(new OperatorID(primaryHashBytes)); config.setChainEnd();
ClassLoader userClassLoader = getUserCodeClassLoader(); TypeSerializer<IN1> inputDeserializer1 = configuration.getTypeSerializerIn1(userClassLoader); TypeSerializer<IN2> inputDeserializer2 = configuration.getTypeSerializerIn2(userClassLoader); int numberOfInputs = configuration.getNumberOfInputs(); List<StreamEdge> inEdges = configuration.getInPhysicalEdges(userClassLoader); inputDeserializer1, inputDeserializer2, this, configuration.getCheckpointMode(), getCheckpointLock(), getEnvironment().getIOManager(),
builder.append("Stream Config"); builder.append("======================="); builder.append("\nNumber of non-chained inputs: ").append(getNumberOfInputs()); builder.append("\nNumber of non-chained outputs: ").append(getNumberOfOutputs()); builder.append("\nOutput names: ").append(getNonChainedOutputs(cl)); builder.append("\nPartitioning:"); for (StreamEdge output : getNonChainedOutputs(cl)) { int outputname = output.getTargetId(); builder.append("\n\t").append(outputname).append(": ").append(output.getPartitioner()); builder.append("\nChained subtasks: ").append(getChainedOutputs(cl)); builder.append("\nOperator: ").append(getStreamOperator(cl).getClass().getSimpleName()); builder.append("\nBuffer timeout: ").append(getBufferTimeout()); builder.append("\nState Monitoring: ").append(isCheckpointingEnabled()); if (isChainStart() && getChainedOutputs(cl).size() > 0) { builder.append("\n\n\n---------------------\nChained task configs\n---------------------\n"); builder.append(getTransitiveChainedTaskConfigs(cl));
public Map<Integer, StreamConfig> getTransitiveChainedTaskConfigsWithSelf(ClassLoader cl) { //TODO: could this logic be moved to the user of #setTransitiveChainedTaskConfigs() ? Map<Integer, StreamConfig> chainedTaskConfigs = getTransitiveChainedTaskConfigs(cl); chainedTaskConfigs.put(getVertexID(), this); return chainedTaskConfigs; }
private void connect(Integer headOfChain, StreamEdge edge) { physicalEdgesInOrder.add(edge); Integer downStreamvertexID = edge.getTargetId(); JobVertex headVertex = jobVertices.get(headOfChain); JobVertex downStreamVertex = jobVertices.get(downStreamvertexID); StreamConfig downStreamConfig = new StreamConfig(downStreamVertex.getConfiguration()); downStreamConfig.setNumberOfInputs(downStreamConfig.getNumberOfInputs() + 1); StreamPartitioner<?> partitioner = edge.getPartitioner(); JobEdge jobEdge; if (partitioner instanceof ForwardPartitioner || partitioner instanceof RescalePartitioner) { jobEdge = downStreamVertex.connectNewDataSetAsInput( headVertex, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED_BOUNDED); } else { jobEdge = downStreamVertex.connectNewDataSetAsInput( headVertex, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED_BOUNDED); } // set strategy name so that web interface can show it. jobEdge.setShipStrategyName(partitioner.toString()); if (LOG.isDebugEnabled()) { LOG.debug("CONNECTED: {} - {} -> {}", partitioner.getClass().getSimpleName(), headOfChain, downStreamvertexID); } }
@Test public void testLifeCycleFull() throws Exception { ACTUAL_ORDER_TRACKING.clear(); Configuration taskManagerConfig = new Configuration(); StreamConfig cfg = new StreamConfig(new Configuration()); MockSourceFunction srcFun = new MockSourceFunction(); cfg.setStreamOperator(new LifecycleTrackingStreamSource<>(srcFun, true)); cfg.setOperatorID(new OperatorID()); cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); Task task = StreamTaskTest.createTask(SourceStreamTask.class, cfg, taskManagerConfig); task.startTaskThread(); LifecycleTrackingStreamSource.runStarted.await(); // wait for clean termination task.getExecutingThread().join(); assertEquals(ExecutionState.FINISHED, task.getExecutionState()); assertEquals(EXPECTED_CALL_ORDER_FULL, ACTUAL_ORDER_TRACKING); }
public <IN, OUT> StreamConfigChainer chain( OperatorID operatorID, OneInputStreamOperator<IN, OUT> operator, TypeSerializer<IN> inputSerializer, TypeSerializer<OUT> outputSerializer) { chainIndex++; tailConfig.setChainedOutputs(Collections.singletonList( new StreamEdge( new StreamNode(null, tailConfig.getChainIndex(), null, null, null, null, null, null), new StreamNode(null, chainIndex, null, null, null, null, null, null), 0, Collections.<String>emptyList(), null, null))); tailConfig = new StreamConfig(new Configuration()); tailConfig.setStreamOperator(checkNotNull(operator)); tailConfig.setOperatorID(checkNotNull(operatorID)); tailConfig.setTypeSerializerIn1(inputSerializer); tailConfig.setTypeSerializerOut(outputSerializer); tailConfig.setChainIndex(chainIndex); chainedConfigs.put(chainIndex, tailConfig); return this; }
this.config = config; try { OperatorMetricGroup operatorMetricGroup = environment.getMetricGroup().getOrAddOperator(config.getOperatorID(), config.getOperatorName()); this.output = new CountingOutput(output, operatorMetricGroup.getIOMetricGroup().getNumRecordsOutCounter()); if (config.isChainStart()) { operatorMetricGroup.getIOMetricGroup().reuseInputMetricsForTask(); if (config.isChainEnd()) { operatorMetricGroup.getIOMetricGroup().reuseOutputMetricsForTask(); stateKeySelector1 = config.getStatePartitioner(0, getUserCodeClassloader()); stateKeySelector2 = config.getStatePartitioner(1, getUserCodeClassloader());
streamConfig.setStreamOperator(previousOperator); streamConfig.setOperatorID(new OperatorID(0L, 0L)); StreamConfig chainedConfig = new StreamConfig(new Configuration()); chainedConfig.setStreamOperator(chainedOperator); chainedConfig.setOperatorID(new OperatorID(0L, chainedIndex)); chainedTaskConfigs.put(chainedIndex, chainedConfig); streamConfig.setChainedOutputs(outputEdges); streamConfig.setTransitiveChainedTaskConfigs(chainedTaskConfigs);
@Override public void init() throws Exception { StreamConfig configuration = getConfiguration(); TypeSerializer<IN> inSerializer = configuration.getTypeSerializerIn1(getUserCodeClassLoader()); int numberOfInputs = configuration.getNumberOfInputs(); if (numberOfInputs > 0) { InputGate[] inputGates = getEnvironment().getAllInputGates(); inputProcessor = new StreamInputProcessor<>( inputGates, inSerializer, this, configuration.getCheckpointMode(), getCheckpointLock(), getEnvironment().getIOManager(), getEnvironment().getTaskManagerInfo().getConfiguration(), getStreamStatusMaintainer(), this.headOperator, getEnvironment().getMetricGroup().getIOMetricGroup(), inputWatermarkGauge); } headOperator.getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_WATERMARK, this.inputWatermarkGauge); // wrap watermark gauge since registered metrics must be unique getEnvironment().getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_WATERMARK, this.inputWatermarkGauge::getValue); }
private static <T> void setupSourceOperator( StreamSource<T, ?> operator, ExecutionConfig executionConfig, Environment env, ProcessingTimeService timeProvider) { StreamConfig cfg = new StreamConfig(new Configuration()); cfg.setStateBackend(new MemoryStateBackend()); cfg.setTimeCharacteristic(TimeCharacteristic.EventTime); cfg.setOperatorID(new OperatorID()); StreamStatusMaintainer streamStatusMaintainer = mock(StreamStatusMaintainer.class); when(streamStatusMaintainer.getStreamStatus()).thenReturn(StreamStatus.ACTIVE); StreamTask<?, ?> mockTask = mock(StreamTask.class); when(mockTask.getName()).thenReturn("Mock Task"); when(mockTask.getCheckpointLock()).thenReturn(new Object()); when(mockTask.getConfiguration()).thenReturn(cfg); when(mockTask.getEnvironment()).thenReturn(env); when(mockTask.getExecutionConfig()).thenReturn(executionConfig); when(mockTask.getAccumulatorMap()).thenReturn(Collections.<String, Accumulator<?, ?>>emptyMap()); when(mockTask.getStreamStatusMaintainer()).thenReturn(streamStatusMaintainer); doAnswer(new Answer<ProcessingTimeService>() { @Override public ProcessingTimeService answer(InvocationOnMock invocation) throws Throwable { if (timeProvider == null) { throw new RuntimeException("The time provider is null."); } return timeProvider; } }).when(mockTask).getProcessingTimeService(); operator.setup(mockTask, cfg, (Output<StreamRecord<T>>) mock(Output.class)); }
StreamConfig cfg = new StreamConfig(taskConfig); cfg.setStreamOperator(op); cfg.setOperatorID(new OperatorID()); cfg.setStateBackend(backend);
assertEquals(ResultPartitionType.PIPELINED_BOUNDED, mapPrintVertex.getInputs().get(0).getSource().getResultType()); StreamConfig sourceConfig = new StreamConfig(sourceVertex.getConfiguration()); StreamConfig mapConfig = new StreamConfig(mapPrintVertex.getConfiguration()); Map<Integer, StreamConfig> chainedConfigs = mapConfig.getTransitiveChainedTaskConfigs(getClass().getClassLoader()); StreamConfig printConfig = chainedConfigs.values().iterator().next(); assertTrue(sourceConfig.isChainStart()); assertTrue(sourceConfig.isChainEnd()); assertTrue(mapConfig.isChainStart()); assertFalse(mapConfig.isChainEnd()); assertFalse(printConfig.isChainStart()); assertTrue(printConfig.isChainEnd());
this.rightBuffer = new HashMap<>(); inputSer1 = (AbstractRowSerializer) (lTypeSerializer == null ? config.getTypeSerializerIn1(getUserCodeClassloader()) : lTypeSerializer); inputSer2 = (AbstractRowSerializer) (rTypeSerializer == null ? config.getTypeSerializerIn2(getUserCodeClassloader()) : rTypeSerializer);
/** * Constructor for initialization, possibly with initial state (recovery / savepoint / etc). * * <p>This constructor accepts a special {@link ProcessingTimeService}. By default (and if * null is passes for the time provider) a {@link SystemProcessingTimeService DefaultTimerService} * will be used. * * @param environment The task environment for this task. * @param timeProvider Optionally, a specific time provider to use. */ protected StreamTask( Environment environment, @Nullable ProcessingTimeService timeProvider) { super(environment); this.timerService = timeProvider; this.configuration = new StreamConfig(getTaskConfiguration()); this.accumulatorMap = getEnvironment().getAccumulatorRegistry().getUserMap(); this.recordWriters = createRecordWriters(configuration, environment); }
OneInputStreamOperator<IN, OUT> chainedOperator = operatorConfig.getStreamOperator(userCodeClassloader); TypeSerializer<IN> inSerializer = operatorConfig.getTypeSerializerIn1(userCodeClassloader); currentOperatorOutput = new CopyingChainingOutput<>(chainedOperator, inSerializer, outputTag, this);
final StreamConfig configuration = containingTask.getConfiguration(); headOperator = configuration.getStreamOperator(userCodeClassloader); Map<Integer, StreamConfig> chainedConfigs = configuration.getTransitiveChainedTaskConfigsWithSelf(userCodeClassloader); List<StreamEdge> outEdgesInOrder = configuration.getOutEdgesInOrder(userCodeClassloader); Map<StreamEdge, RecordWriterOutput<?>> streamOutputMap = new HashMap<>(outEdgesInOrder.size()); this.streamOutputs = new RecordWriterOutput<?>[outEdgesInOrder.size()];
final StreamConfig configuration = containingTask.getConfiguration(); headOperator = configuration.getStreamOperator(userCodeClassloader); Map<Integer, StreamConfig> chainedConfigs = configuration.getTransitiveChainedTaskConfigs(userCodeClassloader); chainedConfigs.put(configuration.getVertexID(), configuration); List<StreamEdge> outEdgesInOrder = configuration.getOutEdgesInOrder(userCodeClassloader); Map<StreamEdge, RecordWriterOutput<?>> streamOutputMap = new HashMap<>(outEdgesInOrder.size()); this.streamOutputs = new RecordWriterOutput<?>[outEdgesInOrder.size()];
boolean isSerializingTimestamps() { TimeCharacteristic tc = configuration.getTimeCharacteristic(); return tc == TimeCharacteristic.EventTime | tc == TimeCharacteristic.IngestionTime; }
final StreamConfig operatorChainStreamConfig = new StreamConfig(chainedVertex.getConfiguration()); final AsyncWaitOperator<Integer, Integer> headOperator = operatorChainStreamConfig.getStreamOperator(AsyncWaitOperatorTest.class.getClassLoader()); streamConfig.setStreamOperator(headOperator);