|
114 | 114 | import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; |
115 | 115 | import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; |
116 | 116 | import org.apache.kafka.clients.CommonClientConfigs; |
117 | | -import org.apache.kafka.clients.admin.Admin; |
118 | 117 | import org.apache.kafka.clients.consumer.Consumer; |
119 | 118 | import org.apache.kafka.clients.consumer.ConsumerConfig; |
120 | 119 | import org.apache.kafka.clients.consumer.KafkaConsumer; |
@@ -605,7 +604,6 @@ public static <K, V> Read<K, V> read() { |
605 | 604 | return new AutoValue_KafkaIO_Read.Builder<K, V>() |
606 | 605 | .setTopics(new ArrayList<>()) |
607 | 606 | .setTopicPartitions(new ArrayList<>()) |
608 | | - .setAdminFactoryFn(KafkaIOUtils.KAFKA_ADMIN_FACTORY_FN) |
609 | 607 | .setConsumerFactoryFn(KafkaIOUtils.KAFKA_CONSUMER_FACTORY_FN) |
610 | 608 | .setConsumerConfig(KafkaIOUtils.DEFAULT_CONSUMER_PROPERTIES) |
611 | 609 | .setMaxNumRecords(Long.MAX_VALUE) |
@@ -697,9 +695,6 @@ public abstract static class Read<K, V> |
697 | 695 | @Pure |
698 | 696 | public abstract @Nullable Coder<V> getValueCoder(); |
699 | 697 |
|
700 | | - @Pure |
701 | | - public abstract SerializableFunction<Map<String, Object>, Admin> getAdminFactoryFn(); |
702 | | - |
703 | 698 | @Pure |
704 | 699 | public abstract SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>> |
705 | 700 | getConsumerFactoryFn(); |
@@ -783,9 +778,6 @@ abstract static class Builder<K, V> { |
783 | 778 |
|
784 | 779 | abstract Builder<K, V> setValueCoder(Coder<V> valueCoder); |
785 | 780 |
|
786 | | - abstract Builder<K, V> setAdminFactoryFn( |
787 | | - SerializableFunction<Map<String, Object>, Admin> adminFactoryFn); |
788 | | - |
789 | 781 | abstract Builder<K, V> setConsumerFactoryFn( |
790 | 782 | SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>> consumerFactoryFn); |
791 | 783 |
|
@@ -869,7 +861,6 @@ static <K, V> void setupExternalBuilder( |
869 | 861 |
|
870 | 862 | // Set required defaults |
871 | 863 | builder.setTopicPartitions(Collections.emptyList()); |
872 | | - builder.setAdminFactoryFn(KafkaIOUtils.KAFKA_ADMIN_FACTORY_FN); |
873 | 864 | builder.setConsumerFactoryFn(KafkaIOUtils.KAFKA_CONSUMER_FACTORY_FN); |
874 | 865 | if (config.maxReadTime != null) { |
875 | 866 | builder.setMaxReadTime(Duration.standardSeconds(config.maxReadTime)); |
@@ -1315,15 +1306,6 @@ public Read<K, V> withValueDeserializerProviderAndCoder( |
1315 | 1306 | .build(); |
1316 | 1307 | } |
1317 | 1308 |
|
1318 | | - /** |
1319 | | - * A factory to create Kafka {@link Admin} from offset consumer configuration. This is useful |
1320 | | - * for supporting another version of Kafka admin. Default is {@link Admin#create(Map)}. |
1321 | | - */ |
1322 | | - public Read<K, V> withAdminFactoryFn( |
1323 | | - SerializableFunction<Map<String, Object>, Admin> adminFactoryFn) { |
1324 | | - return toBuilder().setAdminFactoryFn(adminFactoryFn).build(); |
1325 | | - } |
1326 | | - |
1327 | 1309 | /** |
1328 | 1310 | * A factory to create Kafka {@link Consumer} from consumer configuration. This is useful for |
1329 | 1311 | * supporting another version of Kafka consumer. Default is {@link KafkaConsumer}. |
@@ -1981,7 +1963,6 @@ public PCollection<KafkaRecord<K, V>> expand(PBegin input) { |
1981 | 1963 | ReadSourceDescriptors.<K, V>read() |
1982 | 1964 | .withConsumerConfigOverrides(kafkaRead.getConsumerConfig()) |
1983 | 1965 | .withOffsetConsumerConfigOverrides(kafkaRead.getOffsetConsumerConfig()) |
1984 | | - .withAdminFactoryFn(kafkaRead.getAdminFactoryFn()) |
1985 | 1966 | .withConsumerFactoryFn(kafkaRead.getConsumerFactoryFn()) |
1986 | 1967 | .withKeyDeserializerProviderAndCoder( |
1987 | 1968 | kafkaRead.getKeyDeserializerProvider(), keyCoder) |
@@ -2498,9 +2479,6 @@ public abstract static class ReadSourceDescriptors<K, V> |
2498 | 2479 | @Pure |
2499 | 2480 | abstract @Nullable Coder<V> getValueCoder(); |
2500 | 2481 |
|
2501 | | - @Pure |
2502 | | - abstract SerializableFunction<Map<String, Object>, Admin> getAdminFactoryFn(); |
2503 | | - |
2504 | 2482 | @Pure |
2505 | 2483 | abstract SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>> |
2506 | 2484 | getConsumerFactoryFn(); |
@@ -2551,9 +2529,6 @@ abstract static class Builder<K, V> { |
2551 | 2529 | abstract ReadSourceDescriptors.Builder<K, V> setOffsetConsumerConfig( |
2552 | 2530 | @Nullable Map<String, Object> offsetConsumerConfig); |
2553 | 2531 |
|
2554 | | - abstract ReadSourceDescriptors.Builder<K, V> setAdminFactoryFn( |
2555 | | - SerializableFunction<Map<String, Object>, Admin> adminFactoryFn); |
2556 | | - |
2557 | 2532 | abstract ReadSourceDescriptors.Builder<K, V> setConsumerFactoryFn( |
2558 | 2533 | SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>> consumerFactoryFn); |
2559 | 2534 |
|
@@ -2608,7 +2583,6 @@ abstract ReadSourceDescriptors.Builder<K, V> setBadRecordErrorHandler( |
2608 | 2583 |
|
2609 | 2584 | public static <K, V> ReadSourceDescriptors<K, V> read() { |
2610 | 2585 | return new AutoValue_KafkaIO_ReadSourceDescriptors.Builder<K, V>() |
2611 | | - .setAdminFactoryFn(KafkaIOUtils.KAFKA_ADMIN_FACTORY_FN) |
2612 | 2586 | .setConsumerFactoryFn(KafkaIOUtils.KAFKA_CONSUMER_FACTORY_FN) |
2613 | 2587 | .setConsumerConfig(KafkaIOUtils.DEFAULT_CONSUMER_PROPERTIES) |
2614 | 2588 | .setCommitOffsetEnabled(false) |
@@ -2709,15 +2683,6 @@ public ReadSourceDescriptors<K, V> withValueDeserializerProviderAndCoder( |
2709 | 2683 | .build(); |
2710 | 2684 | } |
2711 | 2685 |
|
2712 | | - /** |
2713 | | - * A factory to create Kafka {@link Admin} from offset consumer configuration. This is useful |
2714 | | - * for supporting another version of Kafka admin. Default is {@link Admin#create(Map)}. |
2715 | | - */ |
2716 | | - public ReadSourceDescriptors<K, V> withAdminFactoryFn( |
2717 | | - SerializableFunction<Map<String, Object>, Admin> adminFactoryFn) { |
2718 | | - return toBuilder().setAdminFactoryFn(adminFactoryFn).build(); |
2719 | | - } |
2720 | | - |
2721 | 2686 | /** |
2722 | 2687 | * A factory to create Kafka {@link Consumer} from consumer configuration. This is useful for |
2723 | 2688 | * supporting another version of Kafka consumer. Default is {@link KafkaConsumer}. |
|
0 commit comments