Interface BoundedOneInput
-
- All Known Implementing Classes:
AbstractArrowPythonAggregateFunctionOperator
,AbstractEmbeddedStatelessFunctionOperator
,AbstractExternalOneInputPythonFunctionOperator
,AbstractOneInputEmbeddedPythonFunctionOperator
,AbstractOneInputPythonFunctionOperator
,AbstractPythonScalarFunctionOperator
,AbstractPythonStreamAggregateOperator
,AbstractPythonStreamGroupAggregateOperator
,AbstractStatelessFunctionOperator
,AbstractStreamArrowPythonBoundedRangeOperator
,AbstractStreamArrowPythonBoundedRowsOperator
,AbstractStreamArrowPythonOverWindowAggregateFunctionOperator
,AbstractStreamingWriter
,ArrowPythonScalarFunctionOperator
,AsyncWaitOperator
,BatchArrowPythonGroupAggregateFunctionOperator
,BatchArrowPythonGroupWindowAggregateFunctionOperator
,BatchArrowPythonOverWindowAggregateFunctionOperator
,BatchCompactCoordinator
,BatchCompactOperator
,BatchFileWriter
,BootstrapStreamTaskRunner
,BroadcastStateBootstrapOperator
,BufferDataOverWindowOperator
,CompactCoordinator
,CompactCoordinatorStateHandler
,CompactFileWriter
,CompactOperator
,CompactorOperator
,CompactorOperatorStateHandler
,EmbeddedPythonKeyedProcessOperator
,EmbeddedPythonProcessOperator
,EmbeddedPythonScalarFunctionOperator
,EmbeddedPythonTableFunctionOperator
,EmbeddedPythonWindowOperator
,ExternalPythonKeyedProcessOperator
,ExternalPythonProcessOperator
,GlobalRuntimeFilterBuilderOperator
,KeyedProcessOperator
,KeyedSortPartitionOperator
,KeyedStateBootstrapOperator
,KeyedTwoOutputProcessOperator
,LocalRuntimeFilterBuilderOperator
,MapPartitionOperator
,PartitionAggregateOperator
,PartitionReduceOperator
,ProcessOperator
,PythonScalarFunctionOperator
,PythonStreamGroupAggregateOperator
,PythonStreamGroupTableAggregateOperator
,PythonStreamGroupWindowAggregateOperator
,PythonTableFunctionOperator
,SortLimitOperator
,SortOperator
,SortPartitionOperator
,StateBootstrapOperator
,StateBootstrapWrapperOperator
,StreamArrowPythonGroupWindowAggregateFunctionOperator
,StreamArrowPythonProcTimeBoundedRangeOperator
,StreamArrowPythonProcTimeBoundedRowsOperator
,StreamArrowPythonRowTimeBoundedRangeOperator
,StreamArrowPythonRowTimeBoundedRowsOperator
,StreamingFileWriter
,TwoOutputProcessOperator
@PublicEvolving public interface BoundedOneInput
Interface for one-input operators that need to be notified about the logical/semantical end of input.NOTE: Classes should not implement both
BoundedOneInput
andBoundedMultiInput
at the same time!- See Also:
BoundedMultiInput
,StreamOperator.finish()
-
-
Method Summary
All Methods Instance Methods Abstract Methods Modifier and Type Method Description void
endInput()
It is notified that no more data will arrive from the input.
-
-
-
Method Detail
-
endInput
void endInput() throws Exception
It is notified that no more data will arrive from the input.WARNING: It is not safe to use this method to commit any transactions or other side effects! You can use this method to flush any buffered data that can later on be committed e.g. in a
CheckpointListener.notifyCheckpointComplete(long)
.NOTE: Given it is semantically very similar to the
StreamOperator.finish()
method. It might be dropped in favour of the other method at some point in time.- Throws:
Exception
-
-