|
24 | 24 | import java.math.MathContext; |
25 | 25 | import java.time.Duration; |
26 | 26 | import java.util.Collections; |
| 27 | +import java.util.HashMap; |
27 | 28 | import java.util.List; |
28 | 29 | import java.util.Map; |
29 | 30 | import java.util.Optional; |
@@ -200,6 +201,15 @@ private ReadFromKafkaDoFn( |
200 | 201 | transform.getAdminFactoryFn(); |
201 | 202 | final SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>> consumerFactoryFn = |
202 | 203 | transform.getConsumerFactoryFn(); |
| 204 | + final @Nullable Map<String, Object> offsetConsumerConfigOverrides = |
| 205 | + transform.getOffsetConsumerConfig(); |
| 206 | + final Map<String, Object> offsetConsumerConfig; |
| 207 | + if (offsetConsumerConfigOverrides == null) { |
| 208 | + offsetConsumerConfig = transform.getConsumerConfig(); |
| 209 | + } else { |
| 210 | + offsetConsumerConfig = new HashMap<>(transform.getConsumerConfig()); |
| 211 | + offsetConsumerConfig.putAll(offsetConsumerConfigOverrides); |
| 212 | + } |
203 | 213 | this.consumerConfig = transform.getConsumerConfig(); |
204 | 214 | this.keyDeserializerProvider = |
205 | 215 | Preconditions.checkArgumentNotNull(transform.getKeyDeserializerProvider()); |
@@ -250,7 +260,7 @@ public KafkaLatestOffsetEstimator load( |
250 | 260 | sourceDescriptor); |
251 | 261 | final Map<String, Object> config = |
252 | 262 | KafkaIOUtils.overrideBootstrapServersConfig( |
253 | | - consumerConfig, sourceDescriptor); |
| 263 | + offsetConsumerConfig, sourceDescriptor); |
254 | 264 | final Admin admin = adminFactoryFn.apply(config); |
255 | 265 | return new KafkaLatestOffsetEstimator( |
256 | 266 | admin, sourceDescriptor.getTopicPartition()); |
|
0 commit comments