@@ -55,8 +55,7 @@ public class FlinkKafkaInternalProducer<K, V> extends KafkaProducer<K, V> {
55
55
private static final String PRODUCER_ID_AND_EPOCH_FIELD_NAME = "producerIdAndEpoch" ;
56
56
57
57
@ Nullable private String transactionalId ;
58
- private volatile boolean inTransaction ;
59
- private volatile boolean hasRecordsInTransaction ;
58
+ private volatile TransactionState transactionState = TransactionState .NOT_IN_TRANSACTION ;
60
59
private volatile boolean closed ;
61
60
62
61
public FlinkKafkaInternalProducer (Properties properties ) {
@@ -79,16 +78,16 @@ private static Properties withTransactionalId(Properties properties, String tran
79
78
80
79
@ Override
81
80
public Future <RecordMetadata > send (ProducerRecord <K , V > record , Callback callback ) {
82
- if (inTransaction ) {
83
- hasRecordsInTransaction = true ;
81
+ if (isInTransaction () ) {
82
+ transactionState = TransactionState . DATA_IN_TRANSACTION ;
84
83
}
85
84
return super .send (record , callback );
86
85
}
87
86
88
87
@ Override
89
88
public void flush () {
90
89
super .flush ();
91
- if (inTransaction ) {
90
+ if (isInTransaction () ) {
92
91
flushNewPartitions ();
93
92
}
94
93
}
@@ -97,33 +96,40 @@ public void flush() {
97
96
public void beginTransaction () throws ProducerFencedException {
98
97
super .beginTransaction ();
99
98
LOG .debug ("beginTransaction {}" , transactionalId );
100
- inTransaction = true ;
99
+ transactionState = TransactionState . IN_TRANSACTION ;
101
100
}
102
101
103
102
@ Override
104
103
public void abortTransaction () throws ProducerFencedException {
105
104
LOG .debug ("abortTransaction {}" , transactionalId );
106
- checkState (inTransaction , "Transaction was not started" );
107
- inTransaction = false ;
108
- hasRecordsInTransaction = false ;
105
+ checkState (isInTransaction (), "Transaction was not started" );
106
+ transactionState = TransactionState .NOT_IN_TRANSACTION ;
109
107
super .abortTransaction ();
110
108
}
111
109
112
110
@ Override
113
111
public void commitTransaction () throws ProducerFencedException {
114
112
LOG .debug ("commitTransaction {}" , transactionalId );
115
- checkState (inTransaction , "Transaction was not started" );
116
- inTransaction = false ;
117
- hasRecordsInTransaction = false ;
113
+ checkState (isInTransaction (), "Transaction was not started" );
114
+ transactionState = TransactionState .NOT_IN_TRANSACTION ;
118
115
super .commitTransaction ();
119
116
}
120
117
121
118
public boolean isInTransaction () {
122
- return inTransaction ;
119
+ return transactionState != TransactionState . NOT_IN_TRANSACTION ;
123
120
}
124
121
125
122
public boolean hasRecordsInTransaction () {
126
- return hasRecordsInTransaction ;
123
+ return transactionState == TransactionState .DATA_IN_TRANSACTION ;
124
+ }
125
+
126
+ public boolean isPrecommitted () {
127
+ return transactionState == TransactionState .PRECOMMITTED ;
128
+ }
129
+
130
+ public void precommitTransaction () {
131
+ checkState (hasRecordsInTransaction (), "Transaction was not started" );
132
+ transactionState = TransactionState .PRECOMMITTED ;
127
133
}
128
134
129
135
@ Override
@@ -172,7 +178,7 @@ public long getProducerId() {
172
178
*/
173
179
public void setTransactionId (String transactionalId ) {
174
180
checkState (
175
- !inTransaction ,
181
+ !isInTransaction () ,
176
182
String .format ("Another transaction %s is still open." , transactionalId ));
177
183
LOG .debug ("Change transaction id from {} to {}" , this .transactionalId , transactionalId );
178
184
this .transactionalId = transactionalId ;
@@ -292,7 +298,7 @@ private static Object getField(Object object, Class<?> clazz, String fieldName)
292
298
* https://github.com/apache/kafka/commit/5d2422258cb975a137a42a4e08f03573c49a387e#diff-f4ef1afd8792cd2a2e9069cd7ddea630
293
299
*/
294
300
public void resumeTransaction (long producerId , short epoch ) {
295
- checkState (!inTransaction , "Already in transaction %s" , transactionalId );
301
+ checkState (!isInTransaction () , "Already in transaction %s" , transactionalId );
296
302
checkState (
297
303
producerId >= 0 && epoch >= 0 ,
298
304
"Incorrect values for producerId %s and epoch %s" ,
@@ -329,9 +335,8 @@ public void resumeTransaction(long producerId, short epoch) {
329
335
// when we create recovery producers to resume transactions and commit
330
336
// them, we should always set this flag.
331
337
setField (transactionManager , "transactionStarted" , true );
332
- this .inTransaction = true ;
333
- this .hasRecordsInTransaction = true ;
334
338
}
339
+ this .transactionState = TransactionState .PRECOMMITTED ;
335
340
}
336
341
337
342
private static Object createProducerIdAndEpoch (long producerId , short epoch ) {
@@ -391,7 +396,14 @@ private static void transitionTransactionManagerStateTo(
391
396
@ Override
392
397
public String toString () {
393
398
return String .format (
394
- "FlinkKafkaInternalProducer@%d{transactionalId='%s', inTransaction=%s, closed=%s}" ,
395
- System .identityHashCode (this ), transactionalId , inTransaction , closed );
399
+ "FlinkKafkaInternalProducer@%d{transactionalId='%s', transactionState=%s, closed=%s}" ,
400
+ System .identityHashCode (this ), transactionalId , transactionState , closed );
401
+ }
402
+
403
+ enum TransactionState {
404
+ NOT_IN_TRANSACTION ,
405
+ IN_TRANSACTION ,
406
+ DATA_IN_TRANSACTION ,
407
+ PRECOMMITTED ,
396
408
}
397
409
}
0 commit comments