IN
- type of results to be written into the sink.@Internal public class CollectSinkFunction<IN> extends RichSinkFunction<IN> implements CheckpointedFunction, CheckpointListener
This sink works by limiting the number of results buffered in it (can be configured) so that when the buffer is full, it back-pressures the job until the client consumes some results.
NOTE: When using this sink, make sure that its parallelism is 1, and make sure that it is used
in a StreamTask
.
We maintain the following variables in this communication protocol
offset
when the
checkpoint happens. This value will be restored from the checkpoint and set back to
offset
when the sink restarts. Clients who need exactly-once semantics need to rely
on this value for the position to revert when a failover happens.
Client will put version
and offset
into the request, indicating that
it thinks what the current version is and it has received this much results.
Sink will check the validity of the request. If version
mismatches or
offset
is smaller than expected, sink will send back the current version
and
lastCheckpointedOffset
with an empty result list.
If the request is valid, sink prepares some results starting from offset
and
sends them back to the client with lastCheckpointedOffset
. If there is currently no
results starting from offset
, sink will not wait but will instead send back an empty
result list.
For client who wants exactly-once semantics, when receiving the response, the client will check for the following conditions:
lastCheckpointedOffset
.
lastCheckpointedOffset
increases, client knows that a checkpoint happens.
It can now move all results before this offset to a user-visible buffer.
Note that
lastCheckpointedOffset
, and
lastCheckpointedOffset
when sink restarts,
client will never throw away results in user-visible buffer. So this communication protocol achieves exactly-once semantics.
In order not to block job finishing/cancelling, if there are still results in sink's buffer when job terminates, these results will be sent back to client through accumulators.
SinkFunction.Context
Constructor and Description |
---|
CollectSinkFunction(TypeSerializer<IN> serializer,
long maxBytesPerBatch,
String accumulatorName) |
Modifier and Type | Method and Description |
---|---|
void |
accumulateFinalResults() |
void |
close()
Tear-down method for the user code.
|
static Tuple2<Long,CollectCoordinationResponse> |
deserializeAccumulatorResult(byte[] serializedAccResults) |
void |
initializeState(FunctionInitializationContext context)
This method is called when the parallel function instance is created during distributed
execution.
|
void |
invoke(IN value,
SinkFunction.Context context)
Writes the given value to the sink.
|
void |
notifyCheckpointAborted(long checkpointId)
This method is called as a notification once a distributed checkpoint has been aborted.
|
void |
notifyCheckpointComplete(long checkpointId)
Notifies the listener that the checkpoint with the given
checkpointId completed and
was committed. |
void |
open(OpenContext openContext)
Initialization method for the function.
|
static byte[] |
serializeAccumulatorResult(long offset,
String version,
long lastCheckpointedOffset,
List<byte[]> buffer) |
void |
setOperatorEventGateway(OperatorEventGateway eventGateway) |
void |
snapshotState(FunctionSnapshotContext context)
This method is called when a snapshot for a checkpoint is requested.
|
getIterationRuntimeContext, getRuntimeContext, open, setRuntimeContext
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
finish, invoke, writeWatermark
public CollectSinkFunction(TypeSerializer<IN> serializer, long maxBytesPerBatch, String accumulatorName)
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 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.public void open(OpenContext openContext) throws Exception
RichFunction
The openContext object passed to the function can be used for configuration and initialization. The openContext contains some necessary information that were configured on the function in the program composition.
public class MyFilter extends RichFilterFunction<String> {
private String searchString;
public void open(OpenContext openContext) {
// initialize the value of searchString
}
public boolean filter(String value) {
return value.equals(searchString);
}
}
By default, this method does nothing.
1. If you implement open(OpenContext openContext)
, the open(OpenContext
openContext)
will be invoked and the open(Configuration parameters)
won't be
invoked. 2. If you don't implement open(OpenContext openContext)
, the open(Configuration parameters)
will be invoked in the default implementation of the open(OpenContext openContext)
.
open
in interface RichFunction
openContext
- The context containing information about the context in which the function
is opened.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 invoke(IN value, SinkFunction.Context context) throws Exception
SinkFunction
You have to override this method when implementing a SinkFunction
, this is a
default
method for backward compatibility with the old-style method only.
invoke
in interface SinkFunction<IN>
value
- The input record.context
- Additional context about the input record.Exception
- This method may throw exceptions. Throwing an exception will cause the
operation to fail and may trigger recovery.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 notifyCheckpointComplete(long checkpointId)
CheckpointListener
checkpointId
completed and
was committed.
These notifications are "best effort", meaning they can sometimes be skipped. To behave
properly, implementers need to follow the "Checkpoint Subsuming Contract". Please see the
class-level JavaDocs
for details.
Please note that checkpoints may generally overlap, so you cannot assume that the notifyCheckpointComplete()
call is always for the latest prior checkpoint (or snapshot) that
was taken on the function/operator implementing this interface. It might be for a checkpoint
that was triggered earlier. Implementing the "Checkpoint Subsuming Contract" (see above)
properly handles this situation correctly as well.
Please note that throwing exceptions from this method will not cause the completed checkpoint to be revoked. Throwing exceptions will typically cause task/job failure and trigger recovery.
notifyCheckpointComplete
in interface CheckpointListener
checkpointId
- The ID of the checkpoint that has been completed.public void notifyCheckpointAborted(long checkpointId)
CheckpointListener
Important: The fact that a checkpoint has been aborted does NOT mean that the data
and artifacts produced between the previous checkpoint and the aborted checkpoint are to be
discarded. The expected behavior is as if this checkpoint was never triggered in the first
place, and the next successful checkpoint simply covers a longer time span. See the
"Checkpoint Subsuming Contract" in the class-level JavaDocs
for
details.
These notifications are "best effort", meaning they can sometimes be skipped.
This method is very rarely necessary to implement. The "best effort" guarantee, together with the fact that this method should not result in discarding any data (per the "Checkpoint Subsuming Contract") means it is mainly useful for earlier cleanups of auxiliary resources. One example is to pro-actively clear a local per-checkpoint state cache upon checkpoint failure.
notifyCheckpointAborted
in interface CheckpointListener
checkpointId
- The ID of the checkpoint that has been aborted.public void setOperatorEventGateway(OperatorEventGateway eventGateway)
@VisibleForTesting public static byte[] serializeAccumulatorResult(long offset, String version, long lastCheckpointedOffset, List<byte[]> buffer) throws IOException
IOException
public static Tuple2<Long,CollectCoordinationResponse> deserializeAccumulatorResult(byte[] serializedAccResults) throws IOException
IOException
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.