forked from apache/flink-connector-kafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
FlinkKafkaInternalProducer.java
404 lines (358 loc) · 15.8 KB
/
FlinkKafkaInternalProducer.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.connector.kafka.sink;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.clients.producer.internals.TransactionManager;
import org.apache.kafka.clients.producer.internals.TransactionalRequestResult;
import org.apache.kafka.common.errors.ProducerFencedException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.Nullable;
import java.lang.reflect.Constructor;
import java.lang.reflect.Field;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.time.Duration;
import java.util.Properties;
import java.util.concurrent.Future;
import static org.apache.flink.util.Preconditions.checkState;
/**
* A {@link KafkaProducer} that exposes private fields to allow resume producing from a given state.
*/
class FlinkKafkaInternalProducer<K, V> extends KafkaProducer<K, V> {
private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaInternalProducer.class);
private static final String TRANSACTION_MANAGER_FIELD_NAME = "transactionManager";
private static final String TRANSACTION_MANAGER_STATE_ENUM =
"org.apache.kafka.clients.producer.internals.TransactionManager$State";
private static final String PRODUCER_ID_AND_EPOCH_FIELD_NAME = "producerIdAndEpoch";
@Nullable private String transactionalId;
private volatile boolean inTransaction;
private volatile boolean hasRecordsInTransaction;
private volatile boolean closed;
public FlinkKafkaInternalProducer(Properties properties, @Nullable String transactionalId) {
super(withTransactionalId(properties, transactionalId));
this.transactionalId = transactionalId;
}
private static Properties withTransactionalId(
Properties properties, @Nullable String transactionalId) {
if (transactionalId == null) {
return properties;
}
Properties props = new Properties();
props.putAll(properties);
props.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId);
return props;
}
@Override
public Future<RecordMetadata> send(ProducerRecord<K, V> record, Callback callback) {
if (inTransaction) {
hasRecordsInTransaction = true;
}
return super.send(record, callback);
}
@Override
public void flush() {
super.flush();
if (inTransaction) {
flushNewPartitions();
}
}
@Override
public void beginTransaction() throws ProducerFencedException {
super.beginTransaction();
inTransaction = true;
}
@Override
public void abortTransaction() throws ProducerFencedException {
LOG.debug("abortTransaction {}", transactionalId);
checkState(inTransaction, "Transaction was not started");
inTransaction = false;
hasRecordsInTransaction = false;
super.abortTransaction();
}
@Override
public void commitTransaction() throws ProducerFencedException {
LOG.debug("commitTransaction {}", transactionalId);
checkState(inTransaction, "Transaction was not started");
inTransaction = false;
hasRecordsInTransaction = false;
super.commitTransaction();
}
public boolean isInTransaction() {
return inTransaction;
}
public boolean hasRecordsInTransaction() {
return hasRecordsInTransaction;
}
@Override
public void close() {
closed = true;
if (inTransaction) {
// This is state is most likely reached in case of a failure.
// If this producer is still in transaction, it should be committing.
// However, at this point, we cannot decide that and we shouldn't prolong cancellation.
// So hard kill this producer with all resources.
super.close(Duration.ZERO);
} else {
// If this is outside of a transaction, we should be able to cleanly shutdown.
super.close(Duration.ofHours(1));
}
}
@Override
public void close(Duration timeout) {
closed = true;
super.close(timeout);
}
public boolean isClosed() {
return closed;
}
@Nullable
public String getTransactionalId() {
return transactionalId;
}
public short getEpoch() {
Object transactionManager = getTransactionManager();
Object producerIdAndEpoch = getField(transactionManager, PRODUCER_ID_AND_EPOCH_FIELD_NAME);
return (short) getField(producerIdAndEpoch, "epoch");
}
public long getProducerId() {
Object transactionManager = getTransactionManager();
Object producerIdAndEpoch = getField(transactionManager, PRODUCER_ID_AND_EPOCH_FIELD_NAME);
return (long) getField(producerIdAndEpoch, "producerId");
}
public void initTransactionId(String transactionalId) {
if (!transactionalId.equals(this.transactionalId)) {
setTransactionId(transactionalId);
initTransactions();
}
}
public void setTransactionId(String transactionalId) {
if (!transactionalId.equals(this.transactionalId)) {
checkState(
!inTransaction,
String.format("Another transaction %s is still open.", transactionalId));
LOG.debug("Change transaction id from {} to {}", this.transactionalId, transactionalId);
Object transactionManager = getTransactionManager();
synchronized (transactionManager) {
setField(transactionManager, "transactionalId", transactionalId);
setField(
transactionManager,
"currentState",
getTransactionManagerState("UNINITIALIZED"));
this.transactionalId = transactionalId;
}
}
}
/**
* Besides committing {@link org.apache.kafka.clients.producer.KafkaProducer#commitTransaction}
* is also adding new partitions to the transaction. flushNewPartitions method is moving this
* logic to pre-commit/flush, to make resumeTransaction simpler. Otherwise resumeTransaction
* would require to restore state of the not yet added/"in-flight" partitions.
*/
private void flushNewPartitions() {
LOG.info("Flushing new partitions");
TransactionalRequestResult result = enqueueNewPartitions();
Object sender = getField("sender");
invoke(sender, "wakeup");
result.await();
}
/**
* Enqueues new transactions at the transaction manager and returns a {@link
* TransactionalRequestResult} that allows waiting on them.
*
* <p>If there are no new transactions we return a {@link TransactionalRequestResult} that is
* already done.
*/
private TransactionalRequestResult enqueueNewPartitions() {
Object transactionManager = getTransactionManager();
synchronized (transactionManager) {
Object newPartitionsInTransaction =
getField(transactionManager, "newPartitionsInTransaction");
Object newPartitionsInTransactionIsEmpty =
invoke(newPartitionsInTransaction, "isEmpty");
TransactionalRequestResult result;
if (newPartitionsInTransactionIsEmpty instanceof Boolean
&& !((Boolean) newPartitionsInTransactionIsEmpty)) {
Object txnRequestHandler =
invoke(transactionManager, "addPartitionsToTransactionHandler");
invoke(
transactionManager,
"enqueueRequest",
new Class[] {txnRequestHandler.getClass().getSuperclass()},
new Object[] {txnRequestHandler});
result =
(TransactionalRequestResult)
getField(
txnRequestHandler,
txnRequestHandler.getClass().getSuperclass(),
"result");
} else {
// we don't have an operation but this operation string is also used in
// addPartitionsToTransactionHandler.
result = new TransactionalRequestResult("AddPartitionsToTxn");
result.done();
}
return result;
}
}
private static Object invoke(Object object, String methodName, Object... args) {
Class<?>[] argTypes = new Class[args.length];
for (int i = 0; i < args.length; i++) {
argTypes[i] = args[i].getClass();
}
return invoke(object, methodName, argTypes, args);
}
private static Object invoke(
Object object, String methodName, Class<?>[] argTypes, Object[] args) {
try {
Method method = object.getClass().getDeclaredMethod(methodName, argTypes);
method.setAccessible(true);
return method.invoke(object, args);
} catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
throw new RuntimeException("Incompatible KafkaProducer version", e);
}
}
private Object getField(String fieldName) {
return getField(this, KafkaProducer.class, fieldName);
}
/**
* Gets and returns the field {@code fieldName} from the given Object {@code object} using
* reflection.
*/
private static Object getField(Object object, String fieldName) {
return getField(object, object.getClass(), fieldName);
}
/**
* Gets and returns the field {@code fieldName} from the given Object {@code object} using
* reflection.
*/
private static Object getField(Object object, Class<?> clazz, String fieldName) {
try {
Field field = clazz.getDeclaredField(fieldName);
field.setAccessible(true);
return field.get(object);
} catch (NoSuchFieldException | IllegalAccessException e) {
throw new RuntimeException("Incompatible KafkaProducer version", e);
}
}
/**
* Instead of obtaining producerId and epoch from the transaction coordinator, re-use previously
* obtained ones, so that we can resume transaction after a restart. Implementation of this
* method is based on {@link KafkaProducer#initTransactions}.
* https://github.com/apache/kafka/commit/5d2422258cb975a137a42a4e08f03573c49a387e#diff-f4ef1afd8792cd2a2e9069cd7ddea630
*/
public void resumeTransaction(long producerId, short epoch) {
checkState(!inTransaction, "Already in transaction %s", transactionalId);
checkState(
producerId >= 0 && epoch >= 0,
"Incorrect values for producerId %s and epoch %s",
producerId,
epoch);
LOG.info(
"Attempting to resume transaction {} with producerId {} and epoch {}",
transactionalId,
producerId,
epoch);
Object transactionManager = getTransactionManager();
synchronized (transactionManager) {
Object topicPartitionBookkeeper =
getField(transactionManager, "topicPartitionBookkeeper");
transitionTransactionManagerStateTo(transactionManager, "INITIALIZING");
invoke(topicPartitionBookkeeper, "reset");
setField(
transactionManager,
PRODUCER_ID_AND_EPOCH_FIELD_NAME,
createProducerIdAndEpoch(producerId, epoch));
transitionTransactionManagerStateTo(transactionManager, "READY");
transitionTransactionManagerStateTo(transactionManager, "IN_TRANSACTION");
// the transactionStarted flag in the KafkaProducer controls whether
// an EndTxnRequest will actually be sent to Kafka for a commit
// or abort API call. This flag is set only after the first send (i.e.
// only if data is actually written to some partition).
// In checkpoints, we only ever store metadata of pre-committed
// transactions that actually have records; therefore, on restore
// when we create recovery producers to resume transactions and commit
// them, we should always set this flag.
setField(transactionManager, "transactionStarted", true);
this.inTransaction = true;
this.hasRecordsInTransaction = true;
}
}
private static Object createProducerIdAndEpoch(long producerId, short epoch) {
try {
Field field =
TransactionManager.class.getDeclaredField(PRODUCER_ID_AND_EPOCH_FIELD_NAME);
Class<?> clazz = field.getType();
Constructor<?> constructor = clazz.getDeclaredConstructor(Long.TYPE, Short.TYPE);
constructor.setAccessible(true);
return constructor.newInstance(producerId, epoch);
} catch (InvocationTargetException
| InstantiationException
| IllegalAccessException
| NoSuchFieldException
| NoSuchMethodException e) {
throw new RuntimeException("Incompatible KafkaProducer version", e);
}
}
/**
* Sets the field {@code fieldName} on the given Object {@code object} to {@code value} using
* reflection.
*/
private static void setField(Object object, String fieldName, Object value) {
setField(object, object.getClass(), fieldName, value);
}
private static void setField(Object object, Class<?> clazz, String fieldName, Object value) {
try {
Field field = clazz.getDeclaredField(fieldName);
field.setAccessible(true);
field.set(object, value);
} catch (NoSuchFieldException | IllegalAccessException e) {
throw new RuntimeException("Incompatible KafkaProducer version", e);
}
}
@SuppressWarnings({"unchecked", "rawtypes"})
private static Enum<?> getTransactionManagerState(String enumName) {
try {
Class<Enum> cl = (Class<Enum>) Class.forName(TRANSACTION_MANAGER_STATE_ENUM);
return Enum.valueOf(cl, enumName);
} catch (ClassNotFoundException e) {
throw new RuntimeException("Incompatible KafkaProducer version", e);
}
}
private Object getTransactionManager() {
return getField(TRANSACTION_MANAGER_FIELD_NAME);
}
private static void transitionTransactionManagerStateTo(
Object transactionManager, String state) {
invoke(transactionManager, "transitionTo", getTransactionManagerState(state));
}
@Override
public String toString() {
return "FlinkKafkaInternalProducer{"
+ "transactionalId='"
+ transactionalId
+ "', inTransaction="
+ inTransaction
+ ", closed="
+ closed
+ '}';
}
}