2323import java .util .Map ;
2424import java .util .Optional ;
2525import org .apache .kafka .clients .CommonClientConfigs ;
26- import org .apache .kafka .clients .producer .ProducerConfig ;
2726import org .apache .kafka .common .config .AbstractConfig ;
2827import org .apache .kafka .common .config .ConfigDef ;
2928
30- public class KafkaFileObjectStateBackingStoreConfig extends AbstractConfig {
29+ public final class KafkaFileObjectStateBackingStoreConfig extends AbstractConfig {
3130
3231 private static final String GROUP = "KafkaFileObjectStateBackingStore" ;
3332
34- public static final String TASKS_FILE_STATUS_STORAGE_TOPIC_CONFIG = "tasks.file.status.storage.topic" ;
33+ public static final String TASKS_FILE_STATUS_STORAGE_PREFIX = "tasks.file.status.storage." ;
34+
35+ public static final String TASKS_FILE_STATUS_STORAGE_TOPIC_CONFIG = TASKS_FILE_STATUS_STORAGE_PREFIX + "topic" ;
3536 private static final String TASKS_FILE_STATUS_STORAGE_TOPIC_DOC = "The topic name which is used to report file states." ;
3637 private static final String TASKS_FILE_STATUS_STORAGE_TOPIC_DEFAULT = "connect-file-pulse-status" ;
3738
38- public static final String TASKS_FILE_STATUS_STORAGE_NAME_CONFIG = "tasks.file.status.storage. name" ;
39+ public static final String TASKS_FILE_STATUS_STORAGE_NAME_CONFIG = TASKS_FILE_STATUS_STORAGE_PREFIX + " name" ;
3940 private static final String TASKS_FILE_STATUS_STORAGE_NAME_DOC = "The reporter identifier to be used by tasks and connector to report and monitor file progression." ;
4041
41- public static final String TASKS_FILE_STATUS_STORAGE_BOOTSTRAP_SERVERS_CONFIG = "tasks.file.status.storage.bootstrap.servers" ;
42+ public static final String TASKS_FILE_STATUS_STORAGE_BOOTSTRAP_SERVERS_CONFIG = TASKS_FILE_STATUS_STORAGE_PREFIX + CommonClientConfigs . BOOTSTRAP_SERVERS_CONFIG ;
4243
43- public static final String TASKS_FILE_STATUS_STORAGE_CONSUMER_ENABLED_CONFIG = "tasks.file.status.storage. consumer.enabled" ;
44+ public static final String TASKS_FILE_STATUS_STORAGE_CONSUMER_ENABLED_CONFIG = TASKS_FILE_STATUS_STORAGE_PREFIX + " consumer.enabled" ;
4445 public static final String TASKS_FILE_STATUS_STORAGE_CONSUMER_ENABLED_DOC = "Boolean to indicate if the storage should consume the status topic." ;
4546
46- public static final String TASKS_FILE_STATUS_STORAGE_TOPIC_PARTITIONS_CONFIG = "tasks.file.status.storage. topic.partitions" ;
47+ public static final String TASKS_FILE_STATUS_STORAGE_TOPIC_PARTITIONS_CONFIG = TASKS_FILE_STATUS_STORAGE_PREFIX + " topic.partitions" ;
4748 public static final String TASKS_FILE_STATUS_STORAGE_TOPIC_PARTITIONS_DOC = "The number of partitions to be used for the status storage topic." ;
4849
49- public static final String TASKS_FILE_STATUS_STORAGE_TOPIC_REPLICATION_FACTOR_CONFIG = "tasks.file.status.storage. topic.replication.factor" ;
50+ public static final String TASKS_FILE_STATUS_STORAGE_TOPIC_REPLICATION_FACTOR_CONFIG = TASKS_FILE_STATUS_STORAGE_PREFIX + " topic.replication.factor" ;
5051 public static final String TASKS_FILE_STATUS_STORAGE_TOPIC_REPLICATION_FACTOR_DOC = "The replication factor to be used for the status storage topic." ;
5152
5253 /**
@@ -72,21 +73,21 @@ public String getTaskStorageName() {
7273
7374 public Map <String , Object > getConsumerTaskStorageConfigs () {
7475 final Map <String , Object > configs = new HashMap <>();
75- configs .put (ProducerConfig .BOOTSTRAP_SERVERS_CONFIG , getInternalBootstrapServers ());
76+ configs .put (CommonClientConfigs .BOOTSTRAP_SERVERS_CONFIG , getInternalBootstrapServers ());
7677 configs .putAll (getInternalKafkaConsumerConfigs ());
7778 return configs ;
7879 }
7980
8081 public Map <String , Object > getProducerTaskStorageConfigs () {
8182 final Map <String , Object > configs = new HashMap <>();
82- configs .put (ProducerConfig .BOOTSTRAP_SERVERS_CONFIG , getInternalBootstrapServers ());
83+ configs .put (CommonClientConfigs .BOOTSTRAP_SERVERS_CONFIG , getInternalBootstrapServers ());
8384 configs .putAll (getInternalKafkaProducerConfigs ());
8485 return configs ;
8586 }
8687
8788 public Map <String , Object > getAdminClientTaskStorageConfigs () {
8889 final Map <String , Object > configs = new HashMap <>();
89- configs .put (ProducerConfig .BOOTSTRAP_SERVERS_CONFIG , getInternalBootstrapServers ());
90+ configs .put (CommonClientConfigs .BOOTSTRAP_SERVERS_CONFIG , getInternalBootstrapServers ());
9091 configs .putAll (getInternalKafkaAdminClientConfigs ());
9192 return configs ;
9293 }
@@ -96,21 +97,24 @@ private String getInternalBootstrapServers() {
9697 }
9798
9899 private Map <String , Object > getInternalKafkaAdminClientConfigs () {
99- Map <String , Object > consumerConfigs = KafkaUtils .getAdminClientConfigs (originals ());
100- consumerConfigs .putAll (originalsWithPrefix ("tasks.file.status.storage.admin." ));
101- return consumerConfigs ;
100+ Map <String , Object > originals = originalsWithPrefix (TASKS_FILE_STATUS_STORAGE_PREFIX , true );
101+ Map <String , Object > adminClientConfigs = new HashMap <>(originals );
102+ adminClientConfigs .putAll (originalsWithPrefix (TASKS_FILE_STATUS_STORAGE_PREFIX + "admin." ));
103+ return KafkaUtils .getAdminClientConfigs (originals );
102104 }
103105
104106 private Map <String , Object > getInternalKafkaConsumerConfigs () {
105- Map <String , Object > consumerConfigs = KafkaUtils .getConsumerConfigs (originals ());
106- consumerConfigs .putAll (originalsWithPrefix ("tasks.file.status.storage.consumer." ));
107- return consumerConfigs ;
107+ Map <String , Object > originals = originalsWithPrefix (TASKS_FILE_STATUS_STORAGE_PREFIX , true );
108+ Map <String , Object > consumerConfigs = new HashMap <>(originals );
109+ consumerConfigs .putAll (originalsWithPrefix (TASKS_FILE_STATUS_STORAGE_PREFIX + "consumer." ));
110+ return KafkaUtils .getConsumerConfigs (consumerConfigs );
108111 }
109112
110113 private Map <String , Object > getInternalKafkaProducerConfigs () {
111- Map <String , Object > producerConfigs = KafkaUtils .getProducerConfigs (originals ());
112- producerConfigs .putAll (originalsWithPrefix ("tasks.file.status.storage.producer." ));
113- return producerConfigs ;
114+ Map <String , Object > originals = originalsWithPrefix (TASKS_FILE_STATUS_STORAGE_PREFIX , true );
115+ Map <String , Object > producerConfigs = new HashMap <>(originals );
116+ producerConfigs .putAll (originalsWithPrefix (TASKS_FILE_STATUS_STORAGE_PREFIX + "producer." ));
117+ return KafkaUtils .getProducerConfigs (producerConfigs );
114118 }
115119
116120 Optional <Integer > getTopicPartitions () {
0 commit comments