OUT
- The type of the data read from RabbitMQ.public class RMQSource<OUT> extends MultipleIdsMessageAcknowledgingSourceBase<OUT,String,Long> implements ResultTypeQueryable<OUT>
RabbitMQ requires messages to be acknowledged. On failures, RabbitMQ will re-resend all
messages which have not been acknowledged previously. When a failure occurs directly after a
completed checkpoint, all messages part of this checkpoint might be processed again because they
couldn't be acknowledged before failure. This case is handled by the MessageAcknowledgingSourceBase
base class which deduplicates the messages using the correlation
id.
RabbitMQ's Delivery Tags do NOT represent unique ids / offsets. That's why the source uses the Correlation ID in the message properties to check for duplicate messages. Note that the correlation id has to be set at the producer. If the correlation id is not set, messages may be produced more than once in corner cases.
This source can be operated in three different modes:
1) Exactly-once (when checkpointed) with RabbitMQ transactions and messages with unique correlation IDs. 2) At-least-once (when checkpointed) with RabbitMQ transactions but no deduplication mechanism (correlation id is not set). 3) No strong delivery guarantees (without checkpointing) with RabbitMQ auto-commit mode.
Users may overwrite the setupConnectionFactory() method to pass their setup their own ConnectionFactory in case the constructor parameters are not sufficient.
SourceFunction.SourceContext<T>
Modifier and Type | Field and Description |
---|---|
protected boolean |
autoAck |
protected com.rabbitmq.client.Channel |
channel |
protected com.rabbitmq.client.Connection |
connection |
protected com.rabbitmq.client.QueueingConsumer |
consumer |
protected String |
queueName |
protected DeserializationSchema<OUT> |
schema |
sessionIds, sessionIdsPerSnapshot
pendingCheckpoints
Constructor and Description |
---|
RMQSource(RMQConnectionConfig rmqConnectionConfig,
String queueName,
boolean usesCorrelationId,
DeserializationSchema<OUT> deserializationSchema)
Creates a new RabbitMQ source.
|
RMQSource(RMQConnectionConfig rmqConnectionConfig,
String queueName,
DeserializationSchema<OUT> deserializationSchema)
Creates a new RabbitMQ source with at-least-once message processing guarantee when
checkpointing is enabled.
|
Modifier and Type | Method and Description |
---|---|
protected void |
acknowledgeSessionIDs(List<Long> sessionIds)
Acknowledges the session ids.
|
void |
cancel()
Cancels the source.
|
void |
close()
Tear-down method for the user code.
|
TypeInformation<OUT> |
getProducedType()
Gets the data type (as a
TypeInformation ) produced by this function or input format. |
void |
open(Configuration config)
Initialization method for the function.
|
void |
run(SourceFunction.SourceContext<OUT> ctx)
Starts the source.
|
protected com.rabbitmq.client.Connection |
setupConnection()
Initializes the connection to RMQ using the default connection factory from
setupConnectionFactory() . |
protected com.rabbitmq.client.ConnectionFactory |
setupConnectionFactory()
Initializes the connection to RMQ with a default connection factory.
|
protected void |
setupQueue()
Sets up the queue.
|
acknowledgeIDs, snapshotState
addId, initializeState, notifyCheckpointAborted, notifyCheckpointComplete
getIterationRuntimeContext, getRuntimeContext, setRuntimeContext
protected final String queueName
protected DeserializationSchema<OUT> schema
protected transient com.rabbitmq.client.Connection connection
protected transient com.rabbitmq.client.Channel channel
protected transient com.rabbitmq.client.QueueingConsumer consumer
protected transient boolean autoAck
public RMQSource(RMQConnectionConfig rmqConnectionConfig, String queueName, DeserializationSchema<OUT> deserializationSchema)
For exactly-once, please use the constructor RMQSource(RMQConnectionConfig, String, boolean, DeserializationSchema)
.
rmqConnectionConfig
- The RabbiMQ connection configuration RMQConnectionConfig
.queueName
- The queue to receive messages from.deserializationSchema
- A DeserializationSchema
for turning the bytes received
into Java objects.public RMQSource(RMQConnectionConfig rmqConnectionConfig, String queueName, boolean usesCorrelationId, DeserializationSchema<OUT> deserializationSchema)
rmqConnectionConfig
- The RabbiMQ connection configuration RMQConnectionConfig
.queueName
- The queue to receive messages from.usesCorrelationId
- Whether the messages received are supplied with a unique id
to deduplicate messages (in case of failed acknowledgments). Only used when checkpointing
is enabled.deserializationSchema
- A DeserializationSchema
for turning the bytes received
into Java objects.protected com.rabbitmq.client.ConnectionFactory setupConnectionFactory() throws Exception
ConnectionFactory
.Exception
protected com.rabbitmq.client.Connection setupConnection() throws Exception
setupConnectionFactory()
. The user may override this method to setup and configure their
own Connection
.Exception
protected void setupQueue() throws IOException
IOException
public void open(Configuration config) 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 MultipleIdsMessageAcknowledgingSourceBase<OUT,String,Long>
config
- 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 MultipleIdsMessageAcknowledgingSourceBase<OUT,String,Long>
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 run(SourceFunction.SourceContext<OUT> ctx) 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<OUT>
ctx
- The context to emit elements to and for accessing locks.Exception
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<OUT>
protected void acknowledgeSessionIDs(List<Long> sessionIds)
MultipleIdsMessageAcknowledgingSourceBase
acknowledgeSessionIDs
in class MultipleIdsMessageAcknowledgingSourceBase<OUT,String,Long>
sessionIds
- The message ids for this session.public TypeInformation<OUT> getProducedType()
ResultTypeQueryable
TypeInformation
) produced by this function or input format.getProducedType
in interface ResultTypeQueryable<OUT>
Copyright © 2014–2021 The Apache Software Foundation. All rights reserved.