Type
- The type of the messages created by the source.UId
- The type of unique IDs which may be used to acknowledge elements.@PublicEvolving public abstract class MessageAcknowledgingSourceBase<Type,UId> extends RichSourceFunction<Type> implements Checkpointed<SerializedCheckpointData[]>, CheckpointListener
The mechanism for this source assumes that messages are identified by a unique ID. When messages are taken from the message queue, the message must not be dropped immediately, but must be retained until acknowledged. Messages that are not acknowledged within a certain time interval will be served again (to a different connection, established by the recovered source).
Note that this source can give no guarantees about message order in the case of failures, because messages that were retrieved but not yet acknowledged will be returned later again, after a set of messages that was not retrieved before the failure.
Internally, this source gathers the IDs of elements it emits. Per checkpoint, the IDs are stored and acknowledged when the checkpoint is complete. That way, no message is acknowledged unless it is certain that it has been successfully processed throughout the topology and the updates to any state caused by that message are persistent.
All messages that are emitted and successfully processed by the streaming program will eventually be acknowledged. In corner cases, the source may receive certain IDs multiple times, if a failure occurs while acknowledging. To cope with this situation, an additional Set stores all processed IDs. IDs are only removed after they have been acknowledged.
A typical way to use this base in a source function is by implementing a run() method as follows:
public void run(SourceContext<Type> ctx) throws Exception {
while (running) {
Message msg = queue.retrieve();
synchronized (ctx.getCheckpointLock()) {
ctx.collect(msg.getMessageData());
addId(msg.getMessageId());
}
}
}
SourceFunction.SourceContext<T>
Modifier and Type | Field and Description |
---|---|
protected int |
numCheckpointsToKeep |
Modifier | Constructor and Description |
---|---|
protected |
MessageAcknowledgingSourceBase(Class<UId> idClass)
Creates a new MessageAcknowledgingSourceBase for IDs of the given type.
|
protected |
MessageAcknowledgingSourceBase(TypeInformation<UId> idTypeInfo)
Creates a new MessageAcknowledgingSourceBase for IDs of the given type.
|
Modifier and Type | Method and Description |
---|---|
protected abstract void |
acknowledgeIDs(long checkpointId,
List<UId> UIds)
This method must be implemented to acknowledge the given set of IDs back to the message queue.
|
protected boolean |
addId(UId uid)
Adds an ID to be stored with the current checkpoint.
|
void |
close()
Tear-down method for the user code.
|
void |
notifyCheckpointComplete(long checkpointId)
This method is called as a notification once a distributed checkpoint has been completed.
|
void |
open(Configuration parameters)
Initialization method for the function.
|
void |
restoreState(SerializedCheckpointData[] state)
Restores the state of the function or operator to that of a previous checkpoint.
|
SerializedCheckpointData[] |
snapshotState(long checkpointId,
long checkpointTimestamp)
Gets the current state of the function of operator.
|
getIterationRuntimeContext, getRuntimeContext, setRuntimeContext
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
cancel, run
protected MessageAcknowledgingSourceBase(Class<UId> idClass)
idClass
- The class of the message ID type, used to create a serializer for the message IDs.protected MessageAcknowledgingSourceBase(TypeInformation<UId> idTypeInfo)
idTypeInfo
- The type information of the message ID type, used to create a serializer for the message IDs.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 MyMapper extends FilterFunction<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 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.protected abstract void acknowledgeIDs(long checkpointId, List<UId> UIds)
UIds
- The list od IDs to acknowledge.protected boolean addId(UId uid)
uid
- The ID to add.public SerializedCheckpointData[] snapshotState(long checkpointId, long checkpointTimestamp) throws Exception
Checkpointed
snapshotState
in interface Checkpointed<SerializedCheckpointData[]>
checkpointId
- The ID of the checkpoint.checkpointTimestamp
- The timestamp of the checkpoint, as derived by
System.currentTimeMillis() on the JobManager.Exception
- Thrown if the creation of the state object failed. This causes the
checkpoint to fail. The system may decide to fail the operation (and trigger
recovery), or to discard this checkpoint attempt and to continue running
and to try again with the next checkpoint attempt.public void restoreState(SerializedCheckpointData[] state) throws Exception
Checkpointed
restoreState
in interface Checkpointed<SerializedCheckpointData[]>
state
- The state to be restored.Exception
public void notifyCheckpointComplete(long checkpointId) throws Exception
CheckpointListener
notifyCheckpointComplete
in interface CheckpointListener
checkpointId
- The ID of the checkpoint that has been completed.Exception
Copyright © 2014–2017 The Apache Software Foundation. All rights reserved.