KafkaSink
.@Deprecated @PublicEvolving public class FlinkKafkaProducer<IN> extends TwoPhaseCommitSinkFunction<IN,FlinkKafkaProducer.KafkaTransactionState,FlinkKafkaProducer.KafkaTransactionContext>
FlinkKafkaProducer.Semantic.AT_LEAST_ONCE
semantic. Before using FlinkKafkaProducer.Semantic.EXACTLY_ONCE
please refer to Flink's Kafka connector documentation.Modifier and Type | Class and Description |
---|---|
static class |
FlinkKafkaProducer.ContextStateSerializer
Deprecated.
|
static class |
FlinkKafkaProducer.KafkaTransactionContext
Deprecated.
Context associated to this instance of the
FlinkKafkaProducer . |
static class |
FlinkKafkaProducer.KafkaTransactionState
Deprecated.
State for handling transactions.
|
static class |
FlinkKafkaProducer.NextTransactionalIdHint
Deprecated.
Keep information required to deduce next safe to use transactional id.
|
static class |
FlinkKafkaProducer.NextTransactionalIdHintSerializer
Deprecated.
|
static class |
FlinkKafkaProducer.Semantic
Deprecated.
Semantics that can be chosen.
|
static class |
FlinkKafkaProducer.TransactionStateSerializer
Deprecated.
|
TwoPhaseCommitSinkFunction.State<TXN,CONTEXT>, TwoPhaseCommitSinkFunction.StateSerializer<TXN,CONTEXT>, TwoPhaseCommitSinkFunction.StateSerializerSnapshot<TXN,CONTEXT>, TwoPhaseCommitSinkFunction.TransactionHolder<TXN>
SinkFunction.Context
Modifier and Type | Field and Description |
---|---|
protected Exception |
asyncException
Deprecated.
Errors encountered in the async producer are stored here.
|
protected org.apache.kafka.clients.producer.Callback |
callback
Deprecated.
The callback than handles error propagation or logging callbacks.
|
static int |
DEFAULT_KAFKA_PRODUCERS_POOL_SIZE
Deprecated.
Default number of KafkaProducers in the pool.
|
static Time |
DEFAULT_KAFKA_TRANSACTION_TIMEOUT
Deprecated.
Default value for kafka transaction timeout.
|
protected String |
defaultTopicId
Deprecated.
The name of the default topic this producer is writing data to.
|
static String |
KEY_DISABLE_METRICS
Deprecated.
Configuration key for disabling the metrics reporting.
|
protected AtomicLong |
pendingRecords
Deprecated.
Number of unacknowledged records.
|
protected Properties |
producerConfig
Deprecated.
User defined properties for the Producer.
|
static int |
SAFE_SCALE_DOWN_FACTOR
Deprecated.
This coefficient determines what is the safe scale down factor.
|
protected FlinkKafkaProducer.Semantic |
semantic
Deprecated.
Semantic chosen for this instance.
|
protected Map<String,int[]> |
topicPartitionsMap
Deprecated.
Partitions of each topic.
|
protected boolean |
writeTimestampToKafka
Deprecated.
Flag controlling whether we are writing the Flink record's timestamp into Kafka.
|
pendingCommitTransactions, state, userContext
Constructor and Description |
---|
FlinkKafkaProducer(String defaultTopic,
KafkaSerializationSchema<IN> serializationSchema,
Properties producerConfig,
FlinkKafkaProducer.Semantic semantic)
Deprecated.
Creates a
FlinkKafkaProducer for a given topic. |
FlinkKafkaProducer(String defaultTopic,
KafkaSerializationSchema<IN> serializationSchema,
Properties producerConfig,
FlinkKafkaProducer.Semantic semantic,
int kafkaProducersPoolSize)
Deprecated.
Creates a FlinkKafkaProducer for a given topic.
|
FlinkKafkaProducer(String topicId,
KeyedSerializationSchema<IN> serializationSchema,
Properties producerConfig)
|
FlinkKafkaProducer(String topicId,
KeyedSerializationSchema<IN> serializationSchema,
Properties producerConfig,
FlinkKafkaProducer.Semantic semantic)
|
FlinkKafkaProducer(String defaultTopicId,
KeyedSerializationSchema<IN> serializationSchema,
Properties producerConfig,
Optional<FlinkKafkaPartitioner<IN>> customPartitioner)
|
FlinkKafkaProducer(String defaultTopicId,
KeyedSerializationSchema<IN> serializationSchema,
Properties producerConfig,
Optional<FlinkKafkaPartitioner<IN>> customPartitioner,
FlinkKafkaProducer.Semantic semantic,
int kafkaProducersPoolSize)
|
FlinkKafkaProducer(String topicId,
SerializationSchema<IN> serializationSchema,
Properties producerConfig)
Deprecated.
Creates a FlinkKafkaProducer for a given topic.
|
FlinkKafkaProducer(String topicId,
SerializationSchema<IN> serializationSchema,
Properties producerConfig,
FlinkKafkaPartitioner<IN> customPartitioner,
FlinkKafkaProducer.Semantic semantic,
int kafkaProducersPoolSize)
Deprecated.
Creates a FlinkKafkaProducer for a given topic.
|
FlinkKafkaProducer(String topicId,
SerializationSchema<IN> serializationSchema,
Properties producerConfig,
Optional<FlinkKafkaPartitioner<IN>> customPartitioner)
Deprecated.
Creates a FlinkKafkaProducer for a given topic.
|
FlinkKafkaProducer(String brokerList,
String topicId,
KeyedSerializationSchema<IN> serializationSchema)
|
FlinkKafkaProducer(String brokerList,
String topicId,
SerializationSchema<IN> serializationSchema)
Deprecated.
Creates a FlinkKafkaProducer for a given topic.
|
Modifier and Type | Method and Description |
---|---|
protected void |
abort(FlinkKafkaProducer.KafkaTransactionState transaction)
Deprecated.
Abort a transaction.
|
protected void |
acknowledgeMessage()
Deprecated.
ATTENTION to subclass implementors: When overriding this method, please always call
super.acknowledgeMessage() to keep the invariants of the internal bookkeeping of the
producer. |
protected FlinkKafkaProducer.KafkaTransactionState |
beginTransaction()
Deprecated.
Method that starts a new transaction.
|
protected void |
checkErroneous()
Deprecated.
|
void |
close()
Deprecated.
Tear-down method for the user code.
|
protected void |
commit(FlinkKafkaProducer.KafkaTransactionState transaction)
Deprecated.
Commit a pre-committed transaction.
|
protected FlinkKafkaInternalProducer<byte[],byte[]> |
createProducer()
Deprecated.
|
protected void |
finishRecoveringContext(Collection<FlinkKafkaProducer.KafkaTransactionState> handledTransactions)
Deprecated.
Callback for subclasses which is called after restoring (each) user context.
|
protected static int[] |
getPartitionsByTopic(String topic,
org.apache.kafka.clients.producer.Producer<byte[],byte[]> producer)
Deprecated.
|
static long |
getTransactionTimeout(Properties producerConfig)
Deprecated.
|
FlinkKafkaProducer<IN> |
ignoreFailuresAfterTransactionTimeout()
Deprecated.
Disables the propagation of exceptions thrown when committing presumably timed out Kafka
transactions during recovery of the job.
|
void |
initializeState(FunctionInitializationContext context)
Deprecated.
This method is called when the parallel function instance is created during distributed
execution.
|
protected Optional<FlinkKafkaProducer.KafkaTransactionContext> |
initializeUserContext()
Deprecated.
|
void |
invoke(FlinkKafkaProducer.KafkaTransactionState transaction,
IN next,
SinkFunction.Context context)
Deprecated.
Write value within a transaction.
|
void |
open(Configuration configuration)
Deprecated.
Initializes the connection to Kafka.
|
protected void |
preCommit(FlinkKafkaProducer.KafkaTransactionState transaction)
Deprecated.
Pre commit previously created transaction.
|
protected void |
recoverAndAbort(FlinkKafkaProducer.KafkaTransactionState transaction)
Deprecated.
Abort a transaction that was rejected by a coordinator after a failure.
|
protected void |
recoverAndCommit(FlinkKafkaProducer.KafkaTransactionState transaction)
Deprecated.
Invoked on recovered transactions after a failure.
|
void |
setLogFailuresOnly(boolean logFailuresOnly)
Deprecated.
Defines whether the producer should fail on errors, or only log them.
|
void |
setTransactionalIdPrefix(String transactionalIdPrefix)
Deprecated.
Specifies the prefix of the transactional.id property to be used by the producers when
communicating with Kafka.
|
void |
setWriteTimestampToKafka(boolean writeTimestampToKafka)
Deprecated.
If set to true, Flink will write the (event time) timestamp attached to each record into
Kafka.
|
void |
snapshotState(FunctionSnapshotContext context)
Deprecated.
This method is called when a snapshot for a checkpoint is requested.
|
currentTransaction, enableTransactionTimeoutWarnings, finish, finishProcessing, getUserContext, invoke, invoke, notifyCheckpointAborted, notifyCheckpointComplete, pendingTransactions, setTransactionTimeout
getIterationRuntimeContext, getRuntimeContext, setRuntimeContext
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
writeWatermark
public static final int SAFE_SCALE_DOWN_FACTOR
If the Flink application previously failed before first checkpoint completed or we are
starting new batch of FlinkKafkaProducer
from scratch without clean shutdown of the
previous one, FlinkKafkaProducer
doesn't know what was the set of previously used
Kafka's transactionalId's. In that case, it will try to play safe and abort all of the
possible transactionalIds from the range of: [0, getNumberOfParallelSubtasks() *
kafkaProducersPoolSize * SAFE_SCALE_DOWN_FACTOR)
The range of available to use transactional ids is: [0,
getNumberOfParallelSubtasks() * kafkaProducersPoolSize)
This means that if we decrease getNumberOfParallelSubtasks()
by a factor larger
than SAFE_SCALE_DOWN_FACTOR
we can have a left some lingering transaction.
public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE
FlinkKafkaProducer.Semantic.EXACTLY_ONCE
.public static final Time DEFAULT_KAFKA_TRANSACTION_TIMEOUT
public static final String KEY_DISABLE_METRICS
protected final Properties producerConfig
protected final String defaultTopicId
protected final Map<String,int[]> topicPartitionsMap
protected boolean writeTimestampToKafka
protected FlinkKafkaProducer.Semantic semantic
@Nullable protected transient org.apache.kafka.clients.producer.Callback callback
@Nullable protected transient volatile Exception asyncException
protected final AtomicLong pendingRecords
public FlinkKafkaProducer(String brokerList, String topicId, SerializationSchema<IN> serializationSchema)
brokerList
- Comma separated addresses of the brokerstopicId
- ID of the Kafka topic.serializationSchema
- User defined (keyless) serialization schema.public FlinkKafkaProducer(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig)
Using this constructor, the default FlinkFixedPartitioner
will be used as the
partitioner. This default partitioner maps each sink subtask to a single Kafka partition
(i.e. all records received by a sink subtask will end up in the same Kafka partition).
To use a custom partitioner, please use FlinkKafkaProducer(String,
SerializationSchema, Properties, Optional)
instead.
topicId
- ID of the Kafka topic.serializationSchema
- User defined key-less serialization schema.producerConfig
- Properties with the producer configuration.public FlinkKafkaProducer(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, Optional<FlinkKafkaPartitioner<IN>> customPartitioner)
SerializationSchema
and possibly a custom FlinkKafkaPartitioner
.
Since a key-less SerializationSchema
is used, all records sent to Kafka will not
have an attached key. Therefore, if a partitioner is also not provided, records will be
distributed to Kafka partitions in a round-robin fashion.
topicId
- The topic to write data toserializationSchema
- A key-less serializable serialization schema for turning user
objects into a kafka-consumable byte[]producerConfig
- Configuration properties for the KafkaProducer. 'bootstrap.servers.' is
the only required argument.customPartitioner
- A serializable partitioner for assigning messages to Kafka
partitions. If a partitioner is not provided, records will be distributed to Kafka
partitions in a round-robin fashion.public FlinkKafkaProducer(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, @Nullable FlinkKafkaPartitioner<IN> customPartitioner, FlinkKafkaProducer.Semantic semantic, int kafkaProducersPoolSize)
SerializationSchema
and possibly a custom FlinkKafkaPartitioner
.
Since a key-less SerializationSchema
is used, all records sent to Kafka will not
have an attached key. Therefore, if a partitioner is also not provided, records will be
distributed to Kafka partitions in a round-robin fashion.
topicId
- The topic to write data toserializationSchema
- A key-less serializable serialization schema for turning user
objects into a kafka-consumable byte[]producerConfig
- Configuration properties for the KafkaProducer. 'bootstrap.servers.' is
the only required argument.customPartitioner
- A serializable partitioner for assigning messages to Kafka
partitions. If a partitioner is not provided, records will be distributed to Kafka
partitions in a round-robin fashion.semantic
- Defines semantic that will be used by this producer (see FlinkKafkaProducer.Semantic
).kafkaProducersPoolSize
- Overwrite default KafkaProducers pool size (see FlinkKafkaProducer.Semantic.EXACTLY_ONCE
).@Deprecated public FlinkKafkaProducer(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema)
FlinkKafkaProducer(String, KafkaSerializationSchema, Properties,
FlinkKafkaProducer.Semantic)
Using this constructor, the default FlinkFixedPartitioner
will be used as the
partitioner. This default partitioner maps each sink subtask to a single Kafka partition
(i.e. all records received by a sink subtask will end up in the same Kafka partition).
To use a custom partitioner, please use FlinkKafkaProducer(String,
KeyedSerializationSchema, Properties, Optional)
instead.
brokerList
- Comma separated addresses of the brokerstopicId
- ID of the Kafka topic.serializationSchema
- User defined serialization schema supporting key/value messages@Deprecated public FlinkKafkaProducer(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig)
FlinkKafkaProducer(String, KafkaSerializationSchema, Properties,
FlinkKafkaProducer.Semantic)
Using this constructor, the default FlinkFixedPartitioner
will be used as the
partitioner. This default partitioner maps each sink subtask to a single Kafka partition
(i.e. all records received by a sink subtask will end up in the same Kafka partition).
To use a custom partitioner, please use FlinkKafkaProducer(String,
KeyedSerializationSchema, Properties, Optional)
instead.
topicId
- ID of the Kafka topic.serializationSchema
- User defined serialization schema supporting key/value messagesproducerConfig
- Properties with the producer configuration.@Deprecated public FlinkKafkaProducer(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaProducer.Semantic semantic)
FlinkKafkaProducer(String, KafkaSerializationSchema, Properties,
FlinkKafkaProducer.Semantic)
Using this constructor, the default FlinkFixedPartitioner
will be used as the
partitioner. This default partitioner maps each sink subtask to a single Kafka partition
(i.e. all records received by a sink subtask will end up in the same Kafka partition).
topicId
- ID of the Kafka topic.serializationSchema
- User defined serialization schema supporting key/value messagesproducerConfig
- Properties with the producer configuration.semantic
- Defines semantic that will be used by this producer (see FlinkKafkaProducer.Semantic
).@Deprecated public FlinkKafkaProducer(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, Optional<FlinkKafkaPartitioner<IN>> customPartitioner)
FlinkKafkaProducer(String, KafkaSerializationSchema, Properties,
FlinkKafkaProducer.Semantic)
KeyedSerializationSchema
and possibly a custom FlinkKafkaPartitioner
.
If a partitioner is not provided, written records will be partitioned by the attached key
of each record (as determined by KeyedSerializationSchema.serializeKey(Object)
). If
written records do not have a key (i.e., KeyedSerializationSchema.serializeKey(Object)
returns null
), they will be
distributed to Kafka partitions in a round-robin fashion.
defaultTopicId
- The default topic to write data toserializationSchema
- A serializable serialization schema for turning user objects into
a kafka-consumable byte[] supporting key/value messagesproducerConfig
- Configuration properties for the KafkaProducer. 'bootstrap.servers.' is
the only required argument.customPartitioner
- A serializable partitioner for assigning messages to Kafka
partitions. If a partitioner is not provided, records will be partitioned by the key of
each record (determined by KeyedSerializationSchema.serializeKey(Object)
). If the
keys are null
, then records will be distributed to Kafka partitions in a
round-robin fashion.@Deprecated public FlinkKafkaProducer(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, Optional<FlinkKafkaPartitioner<IN>> customPartitioner, FlinkKafkaProducer.Semantic semantic, int kafkaProducersPoolSize)
FlinkKafkaProducer(String, KafkaSerializationSchema, Properties,
FlinkKafkaProducer.Semantic)
KeyedSerializationSchema
and possibly a custom FlinkKafkaPartitioner
.
If a partitioner is not provided, written records will be partitioned by the attached key
of each record (as determined by KeyedSerializationSchema.serializeKey(Object)
). If
written records do not have a key (i.e., KeyedSerializationSchema.serializeKey(Object)
returns null
), they will be
distributed to Kafka partitions in a round-robin fashion.
defaultTopicId
- The default topic to write data toserializationSchema
- A serializable serialization schema for turning user objects into
a kafka-consumable byte[] supporting key/value messagesproducerConfig
- Configuration properties for the KafkaProducer. 'bootstrap.servers.' is
the only required argument.customPartitioner
- A serializable partitioner for assigning messages to Kafka
partitions. If a partitioner is not provided, records will be partitioned by the key of
each record (determined by KeyedSerializationSchema.serializeKey(Object)
). If the
keys are null
, then records will be distributed to Kafka partitions in a
round-robin fashion.semantic
- Defines semantic that will be used by this producer (see FlinkKafkaProducer.Semantic
).kafkaProducersPoolSize
- Overwrite default KafkaProducers pool size (see FlinkKafkaProducer.Semantic.EXACTLY_ONCE
).public FlinkKafkaProducer(String defaultTopic, KafkaSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaProducer.Semantic semantic)
FlinkKafkaProducer
for a given topic. The sink produces its input to the
topic. It accepts a KafkaSerializationSchema
for serializing records to a ProducerRecord
, including partitioning information.defaultTopic
- The default topic to write data toserializationSchema
- A serializable serialization schema for turning user objects into
a kafka-consumable byte[] supporting key/value messagesproducerConfig
- Configuration properties for the KafkaProducer. 'bootstrap.servers.' is
the only required argument.semantic
- Defines semantic that will be used by this producer (see FlinkKafkaProducer.Semantic
).public FlinkKafkaProducer(String defaultTopic, KafkaSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaProducer.Semantic semantic, int kafkaProducersPoolSize)
KafkaSerializationSchema
and possibly a custom FlinkKafkaPartitioner
.defaultTopic
- The default topic to write data toserializationSchema
- A serializable serialization schema for turning user objects into
a kafka-consumable byte[] supporting key/value messagesproducerConfig
- Configuration properties for the KafkaProducer. 'bootstrap.servers.' is
the only required argument.semantic
- Defines semantic that will be used by this producer (see FlinkKafkaProducer.Semantic
).kafkaProducersPoolSize
- Overwrite default KafkaProducers pool size (see FlinkKafkaProducer.Semantic.EXACTLY_ONCE
).public void setWriteTimestampToKafka(boolean writeTimestampToKafka)
writeTimestampToKafka
- Flag indicating if Flink's internal timestamps are written to
Kafka.public void setLogFailuresOnly(boolean logFailuresOnly)
logFailuresOnly
- The flag to indicate logging-only on exceptions.public void setTransactionalIdPrefix(String transactionalIdPrefix)
taskName + "-" + operatorUid
.
Note that, if we change the prefix when the Flink application previously failed before
first checkpoint completed or we are starting new batch of FlinkKafkaProducer
from
scratch without clean shutdown of the previous one, since we don't know what was the
previously used transactional.id prefix, there will be some lingering transactions left.
transactionalIdPrefix
- the transactional.id prefixNullPointerException
- Thrown, if the transactionalIdPrefix was null.public FlinkKafkaProducer<IN> ignoreFailuresAfterTransactionTimeout()
Note that we use System.currentTimeMillis()
to track the age of a transaction.
Moreover, only exceptions thrown during the recovery are caught, i.e., the producer will
attempt at least one commit of the transaction before giving up.
public void open(Configuration configuration) throws Exception
open
in interface RichFunction
open
in class AbstractRichFunction
configuration
- 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 invoke(FlinkKafkaProducer.KafkaTransactionState transaction, IN next, SinkFunction.Context context) throws FlinkKafkaException
TwoPhaseCommitSinkFunction
invoke
in class TwoPhaseCommitSinkFunction<IN,FlinkKafkaProducer.KafkaTransactionState,FlinkKafkaProducer.KafkaTransactionContext>
FlinkKafkaException
public void close() throws FlinkKafkaException
RichFunction
This method can be used for clean up work.
close
in interface RichFunction
close
in class TwoPhaseCommitSinkFunction<IN,FlinkKafkaProducer.KafkaTransactionState,FlinkKafkaProducer.KafkaTransactionContext>
FlinkKafkaException
protected FlinkKafkaProducer.KafkaTransactionState beginTransaction() throws FlinkKafkaException
TwoPhaseCommitSinkFunction
beginTransaction
in class TwoPhaseCommitSinkFunction<IN,FlinkKafkaProducer.KafkaTransactionState,FlinkKafkaProducer.KafkaTransactionContext>
FlinkKafkaException
protected void preCommit(FlinkKafkaProducer.KafkaTransactionState transaction) throws FlinkKafkaException
TwoPhaseCommitSinkFunction
Usually implementation involves flushing the data.
preCommit
in class TwoPhaseCommitSinkFunction<IN,FlinkKafkaProducer.KafkaTransactionState,FlinkKafkaProducer.KafkaTransactionContext>
FlinkKafkaException
protected void commit(FlinkKafkaProducer.KafkaTransactionState transaction)
TwoPhaseCommitSinkFunction
TwoPhaseCommitSinkFunction.recoverAndCommit(Object)
will be called again for the
same transaction.protected void recoverAndCommit(FlinkKafkaProducer.KafkaTransactionState transaction)
TwoPhaseCommitSinkFunction
protected void abort(FlinkKafkaProducer.KafkaTransactionState transaction)
TwoPhaseCommitSinkFunction
protected void recoverAndAbort(FlinkKafkaProducer.KafkaTransactionState transaction)
TwoPhaseCommitSinkFunction
protected void acknowledgeMessage()
super.acknowledgeMessage()
to keep the invariants of the internal bookkeeping of the
producer. If not, be sure to know what you are doing.public void snapshotState(FunctionSnapshotContext context) throws Exception
CheckpointedFunction
FunctionInitializationContext
when the Function was initialized, or offered now by FunctionSnapshotContext
itself.snapshotState
in interface CheckpointedFunction
snapshotState
in class TwoPhaseCommitSinkFunction<IN,FlinkKafkaProducer.KafkaTransactionState,FlinkKafkaProducer.KafkaTransactionContext>
context
- the context for drawing a snapshot of the operatorException
- Thrown, if state could not be created ot restored.public void initializeState(FunctionInitializationContext context) throws Exception
CheckpointedFunction
initializeState
in interface CheckpointedFunction
initializeState
in class TwoPhaseCommitSinkFunction<IN,FlinkKafkaProducer.KafkaTransactionState,FlinkKafkaProducer.KafkaTransactionContext>
context
- the context for initializing the operatorException
- Thrown, if state could not be created ot restored.protected Optional<FlinkKafkaProducer.KafkaTransactionContext> initializeUserContext()
protected void finishRecoveringContext(Collection<FlinkKafkaProducer.KafkaTransactionState> handledTransactions)
TwoPhaseCommitSinkFunction
finishRecoveringContext
in class TwoPhaseCommitSinkFunction<IN,FlinkKafkaProducer.KafkaTransactionState,FlinkKafkaProducer.KafkaTransactionContext>
handledTransactions
- transactions which were already committed or aborted and do not
need further handlingprotected FlinkKafkaInternalProducer<byte[],byte[]> createProducer()
protected void checkErroneous() throws FlinkKafkaException
FlinkKafkaException
protected static int[] getPartitionsByTopic(String topic, org.apache.kafka.clients.producer.Producer<byte[],byte[]> producer)
public static long getTransactionTimeout(Properties producerConfig)
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.