-
Notifications
You must be signed in to change notification settings - Fork 1.5k
/
DefaultKafkaProducerFactory.java
366 lines (310 loc) · 10.5 KB
/
DefaultKafkaProducerFactory.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
/*
* Copyright 2016-2018 the original author or authors.
*
* Licensed 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.springframework.kafka.core;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.Producer;
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.common.Metric;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.ProducerFencedException;
import org.apache.kafka.common.serialization.Serializer;
import org.springframework.beans.factory.DisposableBean;
import org.springframework.context.Lifecycle;
import org.springframework.util.Assert;
/**
* The {@link ProducerFactory} implementation for the {@code singleton} shared {@link Producer}
* instance.
* <p>
* This implementation will produce a new {@link Producer} instance (if transactions are not enabled).
* for provided {@link Map} {@code configs} and optional {@link Serializer} {@code keySerializer},
* {@code valueSerializer} implementations on each {@link #createProducer()}
* invocation.
* <p>
* The {@link Producer} instance is freed from the external {@link Producer#close()} invocation
* with the internal wrapper. The real {@link Producer#close()} is called on the target
* {@link Producer} during the {@link Lifecycle#stop()} or {@link DisposableBean#destroy()}.
* <p>
* Setting {@link #setTransactionIdPrefix(String)} enables transactions; in which case, a cache
* of producers is maintained; closing the producer returns it to the cache.
*
* @param <K> the key type.
* @param <V> the value type.
*
* @author Gary Russell
* @author Murali Reddy
* @author Nakul Mishra
*/
public class DefaultKafkaProducerFactory<K, V> implements ProducerFactory<K, V>, Lifecycle, DisposableBean {
private static final int DEFAULT_PHYSICAL_CLOSE_TIMEOUT = 30;
private static final Log logger = LogFactory.getLog(DefaultKafkaProducerFactory.class);
private final Map<String, Object> configs;
private final AtomicInteger transactionIdSuffix = new AtomicInteger();
private final BlockingQueue<CloseSafeProducer<K, V>> cache = new LinkedBlockingQueue<>();
private volatile CloseSafeProducer<K, V> producer;
private Serializer<K> keySerializer;
private Serializer<V> valueSerializer;
private int physicalCloseTimeout = DEFAULT_PHYSICAL_CLOSE_TIMEOUT;
private String transactionIdPrefix;
private volatile boolean running;
public DefaultKafkaProducerFactory(Map<String, Object> configs) {
this(configs, null, null);
}
public DefaultKafkaProducerFactory(Map<String, Object> configs, Serializer<K> keySerializer,
Serializer<V> valueSerializer) {
this.configs = new HashMap<>(configs);
this.keySerializer = keySerializer;
this.valueSerializer = valueSerializer;
}
public void setKeySerializer(Serializer<K> keySerializer) {
this.keySerializer = keySerializer;
}
public void setValueSerializer(Serializer<V> valueSerializer) {
this.valueSerializer = valueSerializer;
}
/**
* The time to wait when physically closing the producer (when {@link #stop()} or {@link #destroy()} is invoked).
* Specified in seconds; default {@value #DEFAULT_PHYSICAL_CLOSE_TIMEOUT}.
* @param physicalCloseTimeout the timeout in seconds.
* @since 1.0.7
*/
public void setPhysicalCloseTimeout(int physicalCloseTimeout) {
this.physicalCloseTimeout = physicalCloseTimeout;
}
/**
* Set the transactional.id prefix.
* @param transactionIdPrefix the prefix.
* @since 1.3
*/
public void setTransactionIdPrefix(String transactionIdPrefix) {
Assert.notNull(transactionIdPrefix, "'transactionIdPrefix' cannot be null");
this.transactionIdPrefix = transactionIdPrefix;
enableIdempotentBehaviour();
}
/**
* When set to 'true', the producer will ensure that exactly one copy of each message is written in the stream.
*/
private void enableIdempotentBehaviour() {
Object previousValue = this.configs.putIfAbsent(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, true);
if (logger.isDebugEnabled() && Boolean.FALSE.equals(previousValue)) {
logger.debug("The '" + ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG +
"' is set to false, may result in duplicate messages");
}
}
/**
* Return an unmodifiable reference to the configuration map for this factory.
* Useful for cloning to make a similar factory.
* @return the configs.
* @since 1.3
*/
public Map<String, Object> getConfigurationProperties() {
return Collections.unmodifiableMap(this.configs);
}
@Override
public boolean transactionCapable() {
return this.transactionIdPrefix != null;
}
@SuppressWarnings("resource")
@Override
public void destroy() throws Exception { //NOSONAR
CloseSafeProducer<K, V> producer = this.producer;
this.producer = null;
if (producer != null) {
producer.delegate.close(this.physicalCloseTimeout, TimeUnit.SECONDS);
}
producer = this.cache.poll();
while (producer != null) {
try {
producer.delegate.close(this.physicalCloseTimeout, TimeUnit.SECONDS);
}
catch (Exception e) {
logger.error("Exception while closing producer", e);
}
producer = this.cache.poll();
}
}
@Override
public void start() {
this.running = true;
}
@Override
public void stop() {
try {
destroy();
}
catch (Exception e) {
logger.error("Exception while closing producer", e);
}
}
@Override
public boolean isRunning() {
return this.running;
}
@Override
public Producer<K, V> createProducer() {
if (this.transactionIdPrefix != null) {
return createTransactionalProducer();
}
if (this.producer == null) {
synchronized (this) {
if (this.producer == null) {
this.producer = new CloseSafeProducer<K, V>(createKafkaProducer());
}
}
}
return this.producer;
}
/**
* Subclasses must return a raw producer which will be wrapped in a
* {@link CloseSafeProducer}.
* @return the producer.
*/
protected Producer<K, V> createKafkaProducer() {
return new KafkaProducer<K, V>(this.configs, this.keySerializer, this.valueSerializer);
}
/**
* Subclasses must return a producer from the {@link #getCache()} or a
* new raw producer wrapped in a {@link CloseSafeProducer}.
* @return the producer - cannot be null.
* @since 1.3
*/
protected Producer<K, V> createTransactionalProducer() {
Producer<K, V> producer = this.cache.poll();
if (producer == null) {
Map<String, Object> configs = new HashMap<>(this.configs);
configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG,
this.transactionIdPrefix + this.transactionIdSuffix.getAndIncrement());
producer = new KafkaProducer<K, V>(configs, this.keySerializer, this.valueSerializer);
producer.initTransactions();
return new CloseSafeProducer<K, V>(producer, this.cache);
}
else {
return producer;
}
}
protected BlockingQueue<CloseSafeProducer<K, V>> getCache() {
return this.cache;
}
/**
* A wrapper class for the delegate.
*
* @param <K> the key type.
* @param <V> the value type.
*
*/
protected static class CloseSafeProducer<K, V> implements Producer<K, V> {
private final Producer<K, V> delegate;
private final BlockingQueue<CloseSafeProducer<K, V>> cache;
private volatile boolean txFailed;
CloseSafeProducer(Producer<K, V> delegate) {
this(delegate, null);
Assert.isTrue(!(delegate instanceof CloseSafeProducer), "Cannot double-wrap a producer");
}
CloseSafeProducer(Producer<K, V> delegate, BlockingQueue<CloseSafeProducer<K, V>> cache) {
this.delegate = delegate;
this.cache = cache;
}
@Override
public Future<RecordMetadata> send(ProducerRecord<K, V> record) {
return this.delegate.send(record);
}
@Override
public Future<RecordMetadata> send(ProducerRecord<K, V> record, Callback callback) {
return this.delegate.send(record, callback);
}
@Override
public void flush() {
this.delegate.flush();
}
@Override
public List<PartitionInfo> partitionsFor(String topic) {
return this.delegate.partitionsFor(topic);
}
@Override
public Map<MetricName, ? extends Metric> metrics() {
return this.delegate.metrics();
}
@Override
public void initTransactions() {
this.delegate.initTransactions();
}
@Override
public void beginTransaction() throws ProducerFencedException {
try {
this.delegate.beginTransaction();
}
catch (RuntimeException e) {
this.txFailed = true;
logger.error("Illegal transaction state; producer removed from cache; possible cause: "
+ "broker restarted during transaction", e);
try {
this.delegate.close();
}
catch (Exception ee) {
// empty
}
throw e;
}
}
@Override
public void sendOffsetsToTransaction(Map<TopicPartition, OffsetAndMetadata> offsets, String consumerGroupId)
throws ProducerFencedException {
this.delegate.sendOffsetsToTransaction(offsets, consumerGroupId);
}
@Override
public void commitTransaction() throws ProducerFencedException {
this.delegate.commitTransaction();
}
@Override
public void abortTransaction() throws ProducerFencedException {
this.delegate.abortTransaction();
}
@Override
public void close() {
if (this.cache != null && !this.txFailed) {
synchronized (this) {
if (!this.cache.contains(this)) {
this.cache.offer(this);
}
}
}
}
@Override
public void close(long timeout, TimeUnit unit) {
close();
}
@Override
public String toString() {
return "CloseSafeProducer [delegate=" + this.delegate + "]";
}
}
}