@Internal public class ContinuousFileMonitoringFunction<OUT> extends RichSourceFunction<TimestampedFileInputSplit> implements CheckpointedFunction
FileInputFormat
and,
depending on the FileProcessingMode
and the FilePathFilter
, it is responsible
for:
splits
corresponding to those files.
The splits to be read are forwarded to the downstream ContinuousFileReaderOperator
which can have parallelism greater than one.
IMPORTANT NOTE: Splits are forwarded downstream for reading in ascending modification time order, based on the modification time of the files they belong to.
SourceFunction.SourceContext<T>
Modifier and Type | Field and Description |
---|---|
static long |
MIN_MONITORING_INTERVAL
The minimum interval allowed between consecutive path scans.
|
Constructor and Description |
---|
ContinuousFileMonitoringFunction(FileInputFormat<OUT> format,
FileProcessingMode watchType,
int readerParallelism,
long interval) |
ContinuousFileMonitoringFunction(FileInputFormat<OUT> format,
FileProcessingMode watchType,
int readerParallelism,
long interval,
long globalModificationTime) |
Modifier and Type | Method and Description |
---|---|
void |
cancel()
Cancels the source.
|
void |
close()
Tear-down method for the user code.
|
long |
getGlobalModificationTime() |
void |
initializeState(FunctionInitializationContext context)
This method is called when the parallel function instance is created during distributed
execution.
|
void |
open(Configuration parameters)
Initialization method for the function.
|
void |
run(SourceFunction.SourceContext<TimestampedFileInputSplit> context)
Starts the source.
|
void |
snapshotState(FunctionSnapshotContext context)
This method is called when a snapshot for a checkpoint is requested.
|
getIterationRuntimeContext, getRuntimeContext, setRuntimeContext
public static final long MIN_MONITORING_INTERVAL
NOTE: Only applicable to the PROCESS_CONTINUOUSLY
mode.
public ContinuousFileMonitoringFunction(FileInputFormat<OUT> format, FileProcessingMode watchType, int readerParallelism, long interval)
public ContinuousFileMonitoringFunction(FileInputFormat<OUT> format, FileProcessingMode watchType, int readerParallelism, long interval, long globalModificationTime)
@VisibleForTesting public long getGlobalModificationTime()
public void initializeState(FunctionInitializationContext context) throws Exception
CheckpointedFunction
initializeState
in interface CheckpointedFunction
context
- the context for initializing the operatorException
- Thrown, if state could not be created ot restored.public void open(Configuration parameters) throws Exception
RichFunction
The configuration object passed to the function can be used for configuration and initialization. The configuration contains all parameters that were configured on the function in the program composition.
public class MyFilter extends RichFilterFunction<String> {
private String searchString;
public void open(Configuration parameters) {
this.searchString = parameters.getString("foo");
}
public boolean filter(String value) {
return value.equals(searchString);
}
}
By default, this method does nothing.
open
in interface RichFunction
open
in class AbstractRichFunction
parameters
- The configuration containing the parameters attached to the contract.Exception
- Implementations may forward exceptions, which are caught by the runtime.
When the runtime catches an exception, it aborts the task and lets the fail-over logic
decide whether to retry the task execution.Configuration
public void run(SourceFunction.SourceContext<TimestampedFileInputSplit> context) throws Exception
SourceFunction
SourceFunction.SourceContext
emit elements.
Sources that implement CheckpointedFunction
must lock on the checkpoint
lock (using a synchronized block) before updating internal state and emitting elements, to
make both an atomic operation:
public class ExampleCountSource implements SourceFunction<Long>, CheckpointedFunction {
private long count = 0L;
private volatile boolean isRunning = true;
private transient ListState<Long> checkpointedCount;
public void run(SourceContext<T> ctx) {
while (isRunning && count < 1000) {
// this synchronized block ensures that state checkpointing,
// internal state updates and emission of elements are an atomic operation
synchronized (ctx.getCheckpointLock()) {
ctx.collect(count);
count++;
}
}
}
public void cancel() {
isRunning = false;
}
public void initializeState(FunctionInitializationContext context) {
this.checkpointedCount = context
.getOperatorStateStore()
.getListState(new ListStateDescriptor<>("count", Long.class));
if (context.isRestored()) {
for (Long count : this.checkpointedCount.get()) {
this.count = count;
}
}
}
public void snapshotState(FunctionSnapshotContext context) {
this.checkpointedCount.clear();
this.checkpointedCount.add(count);
}
}
run
in interface SourceFunction<TimestampedFileInputSplit>
context
- The context to emit elements to and for accessing locks.Exception
public void close() throws Exception
RichFunction
This method can be used for clean up work.
close
in interface RichFunction
close
in class AbstractRichFunction
Exception
- Implementations may forward exceptions, which are caught by the runtime.
When the runtime catches an exception, it aborts the task and lets the fail-over logic
decide whether to retry the task execution.public void cancel()
SourceFunction
SourceFunction.run(SourceContext)
method. The implementation needs to ensure that the source will break
out of that loop after this method is called.
A typical pattern is to have an "volatile boolean isRunning"
flag that is set to
false
in this method. That flag is checked in the loop condition.
When a source is canceled, the executing thread will also be interrupted (via Thread.interrupt()
). The interruption happens strictly after this method has been called, so
any interruption handler can rely on the fact that this method has completed. It is good
practice to make any flags altered by this method "volatile", in order to guarantee the
visibility of the effects of this method to any interruption handler.
cancel
in interface SourceFunction<TimestampedFileInputSplit>
public void snapshotState(FunctionSnapshotContext context) throws Exception
CheckpointedFunction
FunctionInitializationContext
when the Function was initialized, or offered now by FunctionSnapshotContext
itself.snapshotState
in interface CheckpointedFunction
context
- the context for drawing a snapshot of the operatorException
- Thrown, if state could not be created ot restored.Copyright © 2014–2021 The Apache Software Foundation. All rights reserved.