/
StreamsConfig.java
1820 lines (1612 loc) · 92.1 KB
/
StreamsConfig.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
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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.kafka.streams;
import java.util.Arrays;
import java.util.HashSet;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.admin.AdminClientConfig;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Importance;
import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.config.TopicConfig;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.metrics.Sensor.RecordingLevel;
import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler;
import org.apache.kafka.streams.errors.DeserializationExceptionHandler;
import org.apache.kafka.streams.errors.LogAndFailExceptionHandler;
import org.apache.kafka.streams.errors.ProductionExceptionHandler;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.internals.StreamsConfigUtils;
import org.apache.kafka.streams.processor.FailOnInvalidTimestamp;
import org.apache.kafka.streams.processor.TimestampExtractor;
import org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.time.Duration;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Properties;
import java.util.Set;
import java.util.stream.Collectors;
import static org.apache.kafka.common.IsolationLevel.READ_COMMITTED;
import static org.apache.kafka.common.config.ConfigDef.ListSize.atMostOfSize;
import static org.apache.kafka.common.config.ConfigDef.Range.atLeast;
import static org.apache.kafka.common.config.ConfigDef.Range.between;
import static org.apache.kafka.common.config.ConfigDef.ValidString.in;
import static org.apache.kafka.common.config.ConfigDef.parseType;
/**
* Configuration for a {@link KafkaStreams} instance.
* Can also be used to configure the Kafka Streams internal {@link KafkaConsumer}, {@link KafkaProducer} and {@link Admin}.
* To avoid consumer/producer/admin property conflicts, you should prefix those properties using
* {@link #consumerPrefix(String)}, {@link #producerPrefix(String)} and {@link #adminClientPrefix(String)}, respectively.
* <p>
* Example:
* <pre>{@code
* // potentially wrong: sets "metadata.max.age.ms" to 1 minute for producer AND consumer
* Properties streamsProperties = new Properties();
* streamsProperties.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, 60000);
* // or
* streamsProperties.put(ProducerConfig.METADATA_MAX_AGE_CONFIG, 60000);
*
* // suggested:
* Properties streamsProperties = new Properties();
* // sets "metadata.max.age.ms" to 1 minute for consumer only
* streamsProperties.put(StreamsConfig.consumerPrefix(ConsumerConfig.METADATA_MAX_AGE_CONFIG), 60000);
* // sets "metadata.max.age.ms" to 1 minute for producer only
* streamsProperties.put(StreamsConfig.producerPrefix(ProducerConfig.METADATA_MAX_AGE_CONFIG), 60000);
*
* StreamsConfig streamsConfig = new StreamsConfig(streamsProperties);
* }</pre>
*
* This instance can also be used to pass in custom configurations to different modules (e.g. passing a special config in your customized serde class).
* The consumer/producer/admin prefix can also be used to distinguish these custom config values passed to different clients with the same config name.
* * Example:
* <pre>{@code
* Properties streamsProperties = new Properties();
* // sets "my.custom.config" to "foo" for consumer only
* streamsProperties.put(StreamsConfig.consumerPrefix("my.custom.config"), "foo");
* // sets "my.custom.config" to "bar" for producer only
* streamsProperties.put(StreamsConfig.producerPrefix("my.custom.config"), "bar");
* // sets "my.custom.config2" to "boom" for all clients universally
* streamsProperties.put("my.custom.config2", "boom");
*
* // as a result, inside producer's serde class configure(..) function,
* // users can now read both key-value pairs "my.custom.config" -> "foo"
* // and "my.custom.config2" -> "boom" from the config map
* StreamsConfig streamsConfig = new StreamsConfig(streamsProperties);
* }</pre>
*
*
* When increasing {@link ProducerConfig#MAX_BLOCK_MS_CONFIG} to be more resilient to non-available brokers you should also
* increase {@link ConsumerConfig#MAX_POLL_INTERVAL_MS_CONFIG} using the following guidance:
* <pre>
* max.poll.interval.ms > max.block.ms
* </pre>
*
*
* Kafka Streams requires at least the following properties to be set:
* <ul>
* <li>{@link #APPLICATION_ID_CONFIG "application.id"}</li>
* <li>{@link #BOOTSTRAP_SERVERS_CONFIG "bootstrap.servers"}</li>
* </ul>
*
* By default, Kafka Streams does not allow users to overwrite the following properties (Streams setting shown in parentheses):
* <ul>
* <li>{@link ConsumerConfig#GROUP_ID_CONFIG "group.id"} (<application.id>) - Streams client will always use the application ID a consumer group ID</li>
* <li>{@link ConsumerConfig#ENABLE_AUTO_COMMIT_CONFIG "enable.auto.commit"} (false) - Streams client will always disable/turn off auto committing</li>
* <li>{@link ConsumerConfig#PARTITION_ASSIGNMENT_STRATEGY_CONFIG "partition.assignment.strategy"} (<code>StreamsPartitionAssignor</code>) - Streams client will always use its own partition assignor</li>
* </ul>
*
* If {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} is set to {@link #EXACTLY_ONCE_V2 "exactly_once_v2"},
* {@link #EXACTLY_ONCE "exactly_once"} (deprecated), or {@link #EXACTLY_ONCE_BETA "exactly_once_beta"} (deprecated), Kafka Streams does not
* allow users to overwrite the following properties (Streams setting shown in parentheses):
* <ul>
* <li>{@link ConsumerConfig#ISOLATION_LEVEL_CONFIG "isolation.level"} (read_committed) - Consumers will always read committed data only</li>
* <li>{@link ProducerConfig#ENABLE_IDEMPOTENCE_CONFIG "enable.idempotence"} (true) - Producer will always have idempotency enabled</li>
* </ul>
*
* @see KafkaStreams#KafkaStreams(org.apache.kafka.streams.Topology, Properties)
* @see ConsumerConfig
* @see ProducerConfig
*/
public class StreamsConfig extends AbstractConfig {
private static final Logger log = LoggerFactory.getLogger(StreamsConfig.class);
private static final ConfigDef CONFIG;
private final boolean eosEnabled;
private static final long DEFAULT_COMMIT_INTERVAL_MS = 30000L;
private static final long EOS_DEFAULT_COMMIT_INTERVAL_MS = 100L;
private static final int DEFAULT_TRANSACTION_TIMEOUT = 10000;
public static final int DUMMY_THREAD_INDEX = 1;
public static final long MAX_TASK_IDLE_MS_DISABLED = -1;
// We impose these limitations because client tags are encoded into the subscription info,
// which is part of the group metadata message that is persisted into the internal topic.
public static final int MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE = 5;
public static final int MAX_RACK_AWARE_ASSIGNMENT_TAG_KEY_LENGTH = 20;
public static final int MAX_RACK_AWARE_ASSIGNMENT_TAG_VALUE_LENGTH = 30;
/**
* Prefix used to provide default topic configs to be applied when creating internal topics.
* These should be valid properties from {@link org.apache.kafka.common.config.TopicConfig TopicConfig}.
* It is recommended to use {@link #topicPrefix(String)}.
*/
// TODO: currently we cannot get the full topic configurations and hence cannot allow topic configs without the prefix,
// this can be lifted once kafka.log.LogConfig is completely deprecated by org.apache.kafka.common.config.TopicConfig
@SuppressWarnings("WeakerAccess")
public static final String TOPIC_PREFIX = "topic.";
/**
* Prefix used to isolate {@link KafkaConsumer consumer} configs from other client configs.
* It is recommended to use {@link #consumerPrefix(String)} to add this prefix to {@link ConsumerConfig consumer
* properties}.
*/
@SuppressWarnings("WeakerAccess")
public static final String CONSUMER_PREFIX = "consumer.";
/**
* Prefix used to override {@link KafkaConsumer consumer} configs for the main consumer client from
* the general consumer client configs. The override precedence is the following (from highest to lowest precedence):
* 1. main.consumer.[config-name]
* 2. consumer.[config-name]
* 3. [config-name]
*/
@SuppressWarnings("WeakerAccess")
public static final String MAIN_CONSUMER_PREFIX = "main.consumer.";
/**
* Prefix used to override {@link KafkaConsumer consumer} configs for the restore consumer client from
* the general consumer client configs. The override precedence is the following (from highest to lowest precedence):
* 1. restore.consumer.[config-name]
* 2. consumer.[config-name]
* 3. [config-name]
*/
@SuppressWarnings("WeakerAccess")
public static final String RESTORE_CONSUMER_PREFIX = "restore.consumer.";
/**
* Prefix used to override {@link KafkaConsumer consumer} configs for the global consumer client from
* the general consumer client configs. The override precedence is the following (from highest to lowest precedence):
* 1. global.consumer.[config-name]
* 2. consumer.[config-name]
* 3. [config-name]
*/
@SuppressWarnings("WeakerAccess")
public static final String GLOBAL_CONSUMER_PREFIX = "global.consumer.";
/**
* Prefix used to isolate {@link KafkaProducer producer} configs from other client configs.
* It is recommended to use {@link #producerPrefix(String)} to add this prefix to {@link ProducerConfig producer
* properties}.
*/
@SuppressWarnings("WeakerAccess")
public static final String PRODUCER_PREFIX = "producer.";
/**
* Prefix used to isolate {@link Admin admin} configs from other client configs.
* It is recommended to use {@link #adminClientPrefix(String)} to add this prefix to {@link AdminClientConfig admin
* client properties}.
*/
@SuppressWarnings("WeakerAccess")
public static final String ADMIN_CLIENT_PREFIX = "admin.";
/**
* Prefix used to add arbitrary tags to a Kafka Stream's instance as key-value pairs.
* Example:
* client.tag.zone=zone1
* client.tag.cluster=cluster1
*/
@SuppressWarnings("WeakerAccess")
public static final String CLIENT_TAG_PREFIX = "client.tag.";
/** {@code topology.optimization} */
private static final String CONFIG_ERROR_MSG = "Acceptable values are:"
+ " \"+NO_OPTIMIZATION+\", \"+OPTIMIZE+\", "
+ "or a comma separated list of specific optimizations: "
+ "(\"+REUSE_KTABLE_SOURCE_TOPICS+\", \"+MERGE_REPARTITION_TOPICS+\" + "
+ "\"SINGLE_STORE_SELF_JOIN+\").";
public static final String TOPOLOGY_OPTIMIZATION_CONFIG = "topology.optimization";
private static final String TOPOLOGY_OPTIMIZATION_DOC = "A configuration telling Kafka "
+ "Streams if it should optimize the topology and what optimizations to apply. "
+ CONFIG_ERROR_MSG
+ "\"NO_OPTIMIZATION\" by default.";
/**
* Config value for parameter {@link #TOPOLOGY_OPTIMIZATION_CONFIG "topology.optimization"} for disabling topology optimization
*/
public static final String NO_OPTIMIZATION = "none";
/**
* Config value for parameter {@link #TOPOLOGY_OPTIMIZATION_CONFIG "topology.optimization"} for enabling topology optimization
*/
public static final String OPTIMIZE = "all";
/**
* Config value for parameter {@link #TOPOLOGY_OPTIMIZATION_CONFIG "topology.optimization"}
* for enabling the specific optimization that reuses source topic as changelog topic
* for KTables.
*/
public static final String REUSE_KTABLE_SOURCE_TOPICS = "reuse.ktable.source.topics";
/**
* Config value for parameter {@link #TOPOLOGY_OPTIMIZATION_CONFIG "topology.optimization"}
* for enabling the specific optimization that merges duplicated repartition topics.
*/
public static final String MERGE_REPARTITION_TOPICS = "merge.repartition.topics";
/**
* Config value for parameter {@link #TOPOLOGY_OPTIMIZATION_CONFIG "topology.optimization"}
* for enabling the optimization that optimizes inner stream-stream joins into self-joins when
* both arguments are the same stream.
*/
public static final String SINGLE_STORE_SELF_JOIN = "single.store.self.join";
private static final List<String> TOPOLOGY_OPTIMIZATION_CONFIGS = Arrays.asList(
OPTIMIZE, NO_OPTIMIZATION, REUSE_KTABLE_SOURCE_TOPICS, MERGE_REPARTITION_TOPICS,
SINGLE_STORE_SELF_JOIN);
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 0.10.0.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_0100 = "0.10.0";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 0.10.1.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_0101 = "0.10.1";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 0.10.2.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_0102 = "0.10.2";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 0.11.0.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_0110 = "0.11.0";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 1.0.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_10 = "1.0";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 1.1.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_11 = "1.1";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 2.0.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_20 = "2.0";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 2.1.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_21 = "2.1";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 2.2.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_22 = "2.2";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 2.3.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_23 = "2.3";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 2.4.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_24 = "2.4";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 2.5.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_25 = "2.5";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 2.6.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_26 = "2.6";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 2.7.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_27 = "2.7";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 2.8.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_28 = "2.8";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 3.0.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_30 = "3.0";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 3.1.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_31 = "3.1";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 3.2.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_32 = "3.2";
/**
* Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 3.3.x}.
*/
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_33 = "3.3";
/**
* Config value for parameter {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} for at-least-once processing guarantees.
*/
@SuppressWarnings("WeakerAccess")
public static final String AT_LEAST_ONCE = "at_least_once";
/**
* Config value for parameter {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} for exactly-once processing guarantees.
* <p>
* Enabling exactly-once processing semantics requires broker version 0.11.0 or higher.
* If you enable this feature Kafka Streams will use more resources (like broker connections)
* compared to {@link #AT_LEAST_ONCE "at_least_once"} and {@link #EXACTLY_ONCE_V2 "exactly_once_v2"}.
*
* @deprecated Since 3.0.0, will be removed in 4.0. Use {@link #EXACTLY_ONCE_V2 "exactly_once_v2"} instead.
*/
@SuppressWarnings("WeakerAccess")
@Deprecated
public static final String EXACTLY_ONCE = "exactly_once";
/**
* Config value for parameter {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} for exactly-once processing guarantees.
* <p>
* Enabling exactly-once (beta) requires broker version 2.5 or higher.
* If you enable this feature Kafka Streams will use fewer resources (like broker connections)
* compared to the {@link #EXACTLY_ONCE} (deprecated) case.
*
* @deprecated Since 3.0.0, will be removed in 4.0. Use {@link #EXACTLY_ONCE_V2 "exactly_once_v2"} instead.
*/
@SuppressWarnings("WeakerAccess")
@Deprecated
public static final String EXACTLY_ONCE_BETA = "exactly_once_beta";
/**
* Config value for parameter {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} for exactly-once processing guarantees.
* <p>
* Enabling exactly-once-v2 requires broker version 2.5 or higher.
*/
@SuppressWarnings("WeakerAccess")
public static final String EXACTLY_ONCE_V2 = "exactly_once_v2";
/**
* Config value for parameter {@link #BUILT_IN_METRICS_VERSION_CONFIG "built.in.metrics.version"} for the latest built-in metrics version.
*/
public static final String METRICS_LATEST = "latest";
/** {@code acceptable.recovery.lag} */
public static final String ACCEPTABLE_RECOVERY_LAG_CONFIG = "acceptable.recovery.lag";
private static final String ACCEPTABLE_RECOVERY_LAG_DOC = "The maximum acceptable lag (number of offsets to catch up) for a client to be considered caught-up enough" +
" to receive an active task assignment. Upon assignment, it will still restore the rest of the changelog" +
" before processing. To avoid a pause in processing during rebalances, this config" +
" should correspond to a recovery time of well under a minute for a given workload. Must be at least 0.";
/** {@code application.id} */
@SuppressWarnings("WeakerAccess")
public static final String APPLICATION_ID_CONFIG = "application.id";
private static final String APPLICATION_ID_DOC = "An identifier for the stream processing application. Must be unique within the Kafka cluster. It is used as 1) the default client-id prefix, 2) the group-id for membership management, 3) the changelog topic prefix.";
/**{@code application.server} */
@SuppressWarnings("WeakerAccess")
public static final String APPLICATION_SERVER_CONFIG = "application.server";
private static final String APPLICATION_SERVER_DOC = "A host:port pair pointing to a user-defined endpoint that can be used for state store discovery and interactive queries on this KafkaStreams instance.";
/** {@code bootstrap.servers} */
@SuppressWarnings("WeakerAccess")
public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG;
/** {@code buffered.records.per.partition} */
@SuppressWarnings("WeakerAccess")
public static final String BUFFERED_RECORDS_PER_PARTITION_CONFIG = "buffered.records.per.partition";
public static final String BUFFERED_RECORDS_PER_PARTITION_DOC = "Maximum number of records to buffer per partition.";
/** {@code built.in.metrics.version} */
public static final String BUILT_IN_METRICS_VERSION_CONFIG = "built.in.metrics.version";
private static final String BUILT_IN_METRICS_VERSION_DOC = "Version of the built-in metrics to use.";
/** {@code cache.max.bytes.buffering} */
@SuppressWarnings("WeakerAccess")
@Deprecated
public static final String CACHE_MAX_BYTES_BUFFERING_CONFIG = "cache.max.bytes.buffering";
public static final String CACHE_MAX_BYTES_BUFFERING_DOC = "Maximum number of memory bytes to be used for buffering across all threads";
/** {@code statestore.cache.max.bytes} */
@SuppressWarnings("WeakerAccess")
public static final String STATESTORE_CACHE_MAX_BYTES_CONFIG = "statestore.cache.max.bytes";
public static final String STATESTORE_CACHE_MAX_BYTES_DOC = "Maximum number of memory bytes to be used for statestore cache across all threads";
/** {@code client.id} */
@SuppressWarnings("WeakerAccess")
public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG;
private static final String CLIENT_ID_DOC = "An ID prefix string used for the client IDs of internal consumer, producer and restore-consumer," +
" with pattern <code><client.id>-StreamThread-<threadSequenceNumber$gt;-<consumer|producer|restore-consumer></code>.";
/** {@code commit.interval.ms} */
@SuppressWarnings("WeakerAccess")
public static final String COMMIT_INTERVAL_MS_CONFIG = "commit.interval.ms";
private static final String COMMIT_INTERVAL_MS_DOC = "The frequency in milliseconds with which to commit processing progress." +
" For at-least-once processing, committing means to save the position (ie, offsets) of the processor." +
" For exactly-once processing, it means to commit the transaction which includes to save the position and to make the committed data in the output topic visible to consumers with isolation level read_committed." +
" (Note, if <code>processing.guarantee</code> is set to <code>" + EXACTLY_ONCE_V2 + "</code>, <code>" + EXACTLY_ONCE + "</code>,the default value is <code>" + EOS_DEFAULT_COMMIT_INTERVAL_MS + "</code>," +
" otherwise the default value is <code>" + DEFAULT_COMMIT_INTERVAL_MS + "</code>.";
/** {@code repartition.purge.interval.ms} */
@SuppressWarnings("WeakerAccess")
public static final String REPARTITION_PURGE_INTERVAL_MS_CONFIG = "repartition.purge.interval.ms";
private static final String REPARTITION_PURGE_INTERVAL_MS_DOC = "The frequency in milliseconds with which to delete fully consumed records from repartition topics." +
" Purging will occur after at least this value since the last purge, but may be delayed until later." +
" (Note, unlike <code>commit.interval.ms</code>, the default for this value remains unchanged when <code>processing.guarantee</code> is set to <code>" + EXACTLY_ONCE_V2 + "</code>).";
/** {@code connections.max.idle.ms} */
@SuppressWarnings("WeakerAccess")
public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG;
/** {@code default.deserialization.exception.handler} */
@SuppressWarnings("WeakerAccess")
public static final String DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG = "default.deserialization.exception.handler";
public static final String DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the <code>org.apache.kafka.streams.errors.DeserializationExceptionHandler</code> interface.";
/** {@code default.production.exception.handler} */
@SuppressWarnings("WeakerAccess")
public static final String DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG = "default.production.exception.handler";
private static final String DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the <code>org.apache.kafka.streams.errors.ProductionExceptionHandler</code> interface.";
/** {@code default.dsl.store} */
@SuppressWarnings("WeakerAccess")
public static final String DEFAULT_DSL_STORE_CONFIG = "default.dsl.store";
public static final String DEFAULT_DSL_STORE_DOC = "The default state store type used by DSL operators.";
public static final String ROCKS_DB = "rocksDB";
public static final String IN_MEMORY = "in_memory";
/** {@code default.windowed.key.serde.inner} */
@SuppressWarnings("WeakerAccess")
@Deprecated
public static final String DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS = "default.windowed.key.serde.inner";
private static final String DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS_DOC = "Default serializer / deserializer for the inner class of a windowed key. Must implement the " +
"<code>org.apache.kafka.common.serialization.Serde</code> interface.";
/** {@code default.windowed.value.serde.inner} */
@SuppressWarnings("WeakerAccess")
@Deprecated
public static final String DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS = "default.windowed.value.serde.inner";
private static final String DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS_DOC = "Default serializer / deserializer for the inner class of a windowed value. Must implement the " +
"<code>org.apache.kafka.common.serialization.Serde</code> interface.";
public static final String WINDOWED_INNER_CLASS_SERDE = "windowed.inner.class.serde";
private static final String WINDOWED_INNER_CLASS_SERDE_DOC = " Default serializer / deserializer for the inner class of a windowed record. Must implement the " +
"<code>org.apache.kafka.common.serialization.Serde</code> interface. Note that setting this config in KafkaStreams application would result " +
"in an error as it is meant to be used only from Plain consumer client.";
/** {@code default key.serde} */
@SuppressWarnings("WeakerAccess")
public static final String DEFAULT_KEY_SERDE_CLASS_CONFIG = "default.key.serde";
private static final String DEFAULT_KEY_SERDE_CLASS_DOC = "Default serializer / deserializer class for key that implements the <code>org.apache.kafka.common.serialization.Serde</code> interface. "
+ "Note when windowed serde class is used, one needs to set the inner serde class that implements the <code>org.apache.kafka.common.serialization.Serde</code> interface via '"
+ DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS + "' as well";
/** {@code default value.serde} */
@SuppressWarnings("WeakerAccess")
public static final String DEFAULT_VALUE_SERDE_CLASS_CONFIG = "default.value.serde";
private static final String DEFAULT_VALUE_SERDE_CLASS_DOC = "Default serializer / deserializer class for value that implements the <code>org.apache.kafka.common.serialization.Serde</code> interface. "
+ "Note when windowed serde class is used, one needs to set the inner serde class that implements the <code>org.apache.kafka.common.serialization.Serde</code> interface via '"
+ DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS + "' as well";
/** {@code default.timestamp.extractor} */
@SuppressWarnings("WeakerAccess")
public static final String DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG = "default.timestamp.extractor";
public static final String DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC = "Default timestamp extractor class that implements the <code>org.apache.kafka.streams.processor.TimestampExtractor</code> interface.";
/** {@code max.task.idle.ms} */
public static final String MAX_TASK_IDLE_MS_CONFIG = "max.task.idle.ms";
public static final String MAX_TASK_IDLE_MS_DOC = "This config controls whether joins and merges"
+ " may produce out-of-order results."
+ " The config value is the maximum amount of time in milliseconds a stream task will stay idle"
+ " when it is fully caught up on some (but not all) input partitions"
+ " to wait for producers to send additional records and avoid potential"
+ " out-of-order record processing across multiple input streams."
+ " The default (zero) does not wait for producers to send more records,"
+ " but it does wait to fetch data that is already present on the brokers."
+ " This default means that for records that are already present on the brokers,"
+ " Streams will process them in timestamp order."
+ " Set to -1 to disable idling entirely and process any locally available data,"
+ " even though doing so may produce out-of-order processing.";
/** {@code max.warmup.replicas} */
public static final String MAX_WARMUP_REPLICAS_CONFIG = "max.warmup.replicas";
private static final String MAX_WARMUP_REPLICAS_DOC = "The maximum number of warmup replicas (extra standbys beyond the configured num.standbys) that can be assigned at once for the purpose of keeping " +
" the task available on one instance while it is warming up on another instance it has been reassigned to. Used to throttle how much extra broker " +
" traffic and cluster state can be used for high availability. Must be at least 1.";
/** {@code metadata.max.age.ms} */
@SuppressWarnings("WeakerAccess")
public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG;
/** {@code metrics.num.samples} */
@SuppressWarnings("WeakerAccess")
public static final String METRICS_NUM_SAMPLES_CONFIG = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG;
/** {@code metrics.record.level} */
@SuppressWarnings("WeakerAccess")
public static final String METRICS_RECORDING_LEVEL_CONFIG = CommonClientConfigs.METRICS_RECORDING_LEVEL_CONFIG;
/** {@code metric.reporters} */
@SuppressWarnings("WeakerAccess")
public static final String METRIC_REPORTER_CLASSES_CONFIG = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG;
/** {@code metrics.sample.window.ms} */
@SuppressWarnings("WeakerAccess")
public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG;
/** {@code auto.include.jmx.reporter} */
@Deprecated
public static final String AUTO_INCLUDE_JMX_REPORTER_CONFIG = CommonClientConfigs.AUTO_INCLUDE_JMX_REPORTER_CONFIG;
/** {@code num.standby.replicas} */
@SuppressWarnings("WeakerAccess")
public static final String NUM_STANDBY_REPLICAS_CONFIG = "num.standby.replicas";
private static final String NUM_STANDBY_REPLICAS_DOC = "The number of standby replicas for each task.";
/** {@code num.stream.threads} */
@SuppressWarnings("WeakerAccess")
public static final String NUM_STREAM_THREADS_CONFIG = "num.stream.threads";
private static final String NUM_STREAM_THREADS_DOC = "The number of threads to execute stream processing.";
/** {@code poll.ms} */
@SuppressWarnings("WeakerAccess")
public static final String POLL_MS_CONFIG = "poll.ms";
private static final String POLL_MS_DOC = "The amount of time in milliseconds to block waiting for input.";
/** {@code probing.rebalance.interval.ms} */
public static final String PROBING_REBALANCE_INTERVAL_MS_CONFIG = "probing.rebalance.interval.ms";
private static final String PROBING_REBALANCE_INTERVAL_MS_DOC = "The maximum time in milliseconds to wait before triggering a rebalance to probe for warmup replicas that have finished warming up and are ready to become active." +
" Probing rebalances will continue to be triggered until the assignment is balanced. Must be at least 1 minute.";
/** {@code processing.guarantee} */
@SuppressWarnings("WeakerAccess")
public static final String PROCESSING_GUARANTEE_CONFIG = "processing.guarantee";
private static final String PROCESSING_GUARANTEE_DOC = "The processing guarantee that should be used. " +
"Possible values are <code>" + AT_LEAST_ONCE + "</code> (default) " +
"and <code>" + EXACTLY_ONCE_V2 + "</code> (requires brokers version 2.5 or higher). " +
"Deprecated options are <code>" + EXACTLY_ONCE + "</code> (requires brokers version 0.11.0 or higher) " +
"and <code>" + EXACTLY_ONCE_BETA + "</code> (requires brokers version 2.5 or higher). " +
"Note that exactly-once processing requires a cluster of at least three brokers by default what is the " +
"recommended setting for production; for development you can change this, by adjusting broker setting " +
"<code>transaction.state.log.replication.factor</code> and <code>transaction.state.log.min.isr</code>.";
/** {@code receive.buffer.bytes} */
@SuppressWarnings("WeakerAccess")
public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG;
/** {@code rack.aware.assignment.tags} */
@SuppressWarnings("WeakerAccess")
public static final String RACK_AWARE_ASSIGNMENT_TAGS_CONFIG = "rack.aware.assignment.tags";
private static final String RACK_AWARE_ASSIGNMENT_TAGS_DOC = "List of client tag keys used to distribute standby replicas across Kafka Streams instances." +
" When configured, Kafka Streams will make a best-effort to distribute" +
" the standby tasks over each client tag dimension.";
/** {@code reconnect.backoff.ms} */
@SuppressWarnings("WeakerAccess")
public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG;
/** {@code reconnect.backoff.max} */
@SuppressWarnings("WeakerAccess")
public static final String RECONNECT_BACKOFF_MAX_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_CONFIG;
/** {@code replication.factor} */
@SuppressWarnings("WeakerAccess")
public static final String REPLICATION_FACTOR_CONFIG = "replication.factor";
private static final String REPLICATION_FACTOR_DOC = "The replication factor for change log topics and repartition topics created by the stream processing application."
+ " The default of <code>-1</code> (meaning: use broker default replication factor) requires broker version 2.4 or newer";
/** {@code request.timeout.ms} */
@SuppressWarnings("WeakerAccess")
public static final String REQUEST_TIMEOUT_MS_CONFIG = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG;
/**
* {@code retries}
* <p>
* This config is ignored by Kafka Streams. Note, that the internal clients (producer, admin) are still impacted by this config.
*
* @deprecated since 2.7
*/
@SuppressWarnings("WeakerAccess")
@Deprecated
public static final String RETRIES_CONFIG = CommonClientConfigs.RETRIES_CONFIG;
/** {@code retry.backoff.ms} */
@SuppressWarnings("WeakerAccess")
public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG;
/** {@code rocksdb.config.setter} */
@SuppressWarnings("WeakerAccess")
public static final String ROCKSDB_CONFIG_SETTER_CLASS_CONFIG = "rocksdb.config.setter";
private static final String ROCKSDB_CONFIG_SETTER_CLASS_DOC = "A Rocks DB config setter class or class name that implements the <code>org.apache.kafka.streams.state.RocksDBConfigSetter</code> interface";
/** {@code security.protocol} */
@SuppressWarnings("WeakerAccess")
public static final String SECURITY_PROTOCOL_CONFIG = CommonClientConfigs.SECURITY_PROTOCOL_CONFIG;
/** {@code send.buffer.bytes} */
@SuppressWarnings("WeakerAccess")
public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG;
/** {@code state.cleanup.delay} */
@SuppressWarnings("WeakerAccess")
public static final String STATE_CLEANUP_DELAY_MS_CONFIG = "state.cleanup.delay.ms";
private static final String STATE_CLEANUP_DELAY_MS_DOC = "The amount of time in milliseconds to wait before deleting state when a partition has migrated. Only state directories that have not been modified for at least <code>state.cleanup.delay.ms</code> will be removed";
/** {@code state.dir} */
@SuppressWarnings("WeakerAccess")
public static final String STATE_DIR_CONFIG = "state.dir";
private static final String STATE_DIR_DOC = "Directory location for state store. This path must be unique for each streams instance sharing the same underlying filesystem.";
/** {@code task.timeout.ms} */
public static final String TASK_TIMEOUT_MS_CONFIG = "task.timeout.ms";
public static final String TASK_TIMEOUT_MS_DOC = "The maximum amount of time in milliseconds a task might stall due to internal errors and retries until an error is raised. " +
"For a timeout of 0ms, a task would raise an error for the first internal error. " +
"For any timeout larger than 0ms, a task will retry at least once before an error is raised.";
/** {@code window.size.ms} */
public static final String WINDOW_SIZE_MS_CONFIG = "window.size.ms";
private static final String WINDOW_SIZE_MS_DOC = "Sets window size for the deserializer in order to calculate window end times.";
/** {@code upgrade.from} */
@SuppressWarnings("WeakerAccess")
public static final String UPGRADE_FROM_CONFIG = "upgrade.from";
private static final String UPGRADE_FROM_DOC = "Allows upgrading in a backward compatible way. " +
"This is needed when upgrading from [0.10.0, 1.1] to 2.0+, or when upgrading from [2.0, 2.3] to 2.4+. " +
"When upgrading from 3.3 to a newer version it is not required to specify this config. Default is `null`. " +
"Accepted values are \"" + UPGRADE_FROM_0100 + "\", \"" + UPGRADE_FROM_0101 + "\", \"" +
UPGRADE_FROM_0102 + "\", \"" + UPGRADE_FROM_0110 + "\", \"" + UPGRADE_FROM_10 + "\", \"" +
UPGRADE_FROM_11 + "\", \"" + UPGRADE_FROM_20 + "\", \"" + UPGRADE_FROM_21 + "\", \"" +
UPGRADE_FROM_22 + "\", \"" + UPGRADE_FROM_23 + "\", \"" + UPGRADE_FROM_24 + "\", \"" +
UPGRADE_FROM_25 + "\", \"" + UPGRADE_FROM_26 + "\", \"" + UPGRADE_FROM_27 + "\", \"" +
UPGRADE_FROM_28 + "\", \"" + UPGRADE_FROM_30 + "\", \"" + UPGRADE_FROM_31 + "\", \"" +
UPGRADE_FROM_32 + "\", \"" + UPGRADE_FROM_33 + "\" (for upgrading from the corresponding old version).";
/** {@code windowstore.changelog.additional.retention.ms} */
@SuppressWarnings("WeakerAccess")
public static final String WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG = "windowstore.changelog.additional.retention.ms";
private static final String WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_DOC = "Added to a windows maintainMs to ensure data is not deleted from the log prematurely. Allows for clock drift. Default is 1 day";
/**
* {@code topology.optimization}
* @deprecated since 2.7; use {@link #TOPOLOGY_OPTIMIZATION_CONFIG} instead
*/
@Deprecated
public static final String TOPOLOGY_OPTIMIZATION = TOPOLOGY_OPTIMIZATION_CONFIG;
private static final String[] NON_CONFIGURABLE_CONSUMER_DEFAULT_CONFIGS =
new String[] {ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG};
private static final String[] NON_CONFIGURABLE_CONSUMER_EOS_CONFIGS =
new String[] {ConsumerConfig.ISOLATION_LEVEL_CONFIG};
private static final String[] NON_CONFIGURABLE_PRODUCER_EOS_CONFIGS =
new String[] {
ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG,
ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION,
ProducerConfig.TRANSACTIONAL_ID_CONFIG
};
static {
CONFIG = new ConfigDef()
// HIGH
.define(APPLICATION_ID_CONFIG, // required with no default value
Type.STRING,
Importance.HIGH,
APPLICATION_ID_DOC)
.define(BOOTSTRAP_SERVERS_CONFIG, // required with no default value
Type.LIST,
Importance.HIGH,
CommonClientConfigs.BOOTSTRAP_SERVERS_DOC)
.define(NUM_STANDBY_REPLICAS_CONFIG,
Type.INT,
0,
Importance.HIGH,
NUM_STANDBY_REPLICAS_DOC)
.define(STATE_DIR_CONFIG,
Type.STRING,
System.getProperty("java.io.tmpdir") + File.separator + "kafka-streams",
Importance.HIGH,
STATE_DIR_DOC)
// MEDIUM
.define(ACCEPTABLE_RECOVERY_LAG_CONFIG,
Type.LONG,
10_000L,
atLeast(0),
Importance.MEDIUM,
ACCEPTABLE_RECOVERY_LAG_DOC)
.define(CACHE_MAX_BYTES_BUFFERING_CONFIG,
Type.LONG,
10 * 1024 * 1024L,
atLeast(0),
Importance.MEDIUM,
CACHE_MAX_BYTES_BUFFERING_DOC)
.define(STATESTORE_CACHE_MAX_BYTES_CONFIG,
Type.LONG,
10 * 1024 * 1024L,
atLeast(0),
Importance.MEDIUM,
STATESTORE_CACHE_MAX_BYTES_DOC)
.define(CLIENT_ID_CONFIG,
Type.STRING,
"",
Importance.MEDIUM,
CLIENT_ID_DOC)
.define(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG,
Type.CLASS,
LogAndFailExceptionHandler.class.getName(),
Importance.MEDIUM,
DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC)
.define(DEFAULT_KEY_SERDE_CLASS_CONFIG,
Type.CLASS,
null,
Importance.MEDIUM,
DEFAULT_KEY_SERDE_CLASS_DOC)
.define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS,
Type.CLASS,
null,
Importance.MEDIUM,
CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS_DOC)
.define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS,
Type.CLASS,
null,
Importance.MEDIUM,
CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS_DOC)
.define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS,
Type.CLASS,
null,
Importance.MEDIUM,
CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC)
.define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS,
Type.CLASS,
null,
Importance.MEDIUM,
CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC)
.define(DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG,
Type.CLASS,
DefaultProductionExceptionHandler.class.getName(),
Importance.MEDIUM,
DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC)
.define(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG,
Type.CLASS,
FailOnInvalidTimestamp.class.getName(),
Importance.MEDIUM,
DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC)
.define(DEFAULT_VALUE_SERDE_CLASS_CONFIG,
Type.CLASS,
null,
Importance.MEDIUM,
DEFAULT_VALUE_SERDE_CLASS_DOC)
.define(MAX_TASK_IDLE_MS_CONFIG,
Type.LONG,
0L,
Importance.MEDIUM,
MAX_TASK_IDLE_MS_DOC)
.define(MAX_WARMUP_REPLICAS_CONFIG,
Type.INT,
2,
atLeast(1),
Importance.MEDIUM,
MAX_WARMUP_REPLICAS_DOC)
.define(NUM_STREAM_THREADS_CONFIG,
Type.INT,
1,
Importance.MEDIUM,
NUM_STREAM_THREADS_DOC)
.define(PROCESSING_GUARANTEE_CONFIG,
Type.STRING,
AT_LEAST_ONCE,
in(AT_LEAST_ONCE, EXACTLY_ONCE, EXACTLY_ONCE_BETA, EXACTLY_ONCE_V2),
Importance.MEDIUM,
PROCESSING_GUARANTEE_DOC)
.define(RACK_AWARE_ASSIGNMENT_TAGS_CONFIG,
Type.LIST,
Collections.emptyList(),
atMostOfSize(MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE),
Importance.MEDIUM,
RACK_AWARE_ASSIGNMENT_TAGS_DOC)
.define(REPLICATION_FACTOR_CONFIG,
Type.INT,
-1,
Importance.MEDIUM,
REPLICATION_FACTOR_DOC)
.define(SECURITY_PROTOCOL_CONFIG,
Type.STRING,
CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL,
in(Utils.enumOptions(SecurityProtocol.class)),
Importance.MEDIUM,
CommonClientConfigs.SECURITY_PROTOCOL_DOC)
.define(TASK_TIMEOUT_MS_CONFIG,
Type.LONG,
Duration.ofMinutes(5L).toMillis(),
atLeast(0L),
Importance.MEDIUM,
TASK_TIMEOUT_MS_DOC)
.define(TOPOLOGY_OPTIMIZATION_CONFIG,
Type.STRING,
NO_OPTIMIZATION,
(name, value) -> verifyTopologyOptimizationConfigs((String) value),
Importance.MEDIUM,
TOPOLOGY_OPTIMIZATION_DOC)
// LOW
.define(APPLICATION_SERVER_CONFIG,
Type.STRING,
"",
Importance.LOW,
APPLICATION_SERVER_DOC)
.define(BUFFERED_RECORDS_PER_PARTITION_CONFIG,
Type.INT,
1000,
Importance.LOW,
BUFFERED_RECORDS_PER_PARTITION_DOC)
.define(BUILT_IN_METRICS_VERSION_CONFIG,
Type.STRING,
METRICS_LATEST,
in(
METRICS_LATEST
),
Importance.LOW,
BUILT_IN_METRICS_VERSION_DOC)
.define(COMMIT_INTERVAL_MS_CONFIG,
Type.LONG,
DEFAULT_COMMIT_INTERVAL_MS,
atLeast(0),
Importance.LOW,
COMMIT_INTERVAL_MS_DOC)
.define(REPARTITION_PURGE_INTERVAL_MS_CONFIG,
Type.LONG,
DEFAULT_COMMIT_INTERVAL_MS,
atLeast(0),
Importance.LOW,
REPARTITION_PURGE_INTERVAL_MS_DOC)
.define(CONNECTIONS_MAX_IDLE_MS_CONFIG,
ConfigDef.Type.LONG,
9 * 60 * 1000L,
ConfigDef.Importance.LOW,
CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC)
.define(DEFAULT_DSL_STORE_CONFIG,
Type.STRING,
ROCKS_DB,
in(ROCKS_DB, IN_MEMORY),
Importance.LOW,
DEFAULT_DSL_STORE_DOC)
.define(METADATA_MAX_AGE_CONFIG,
ConfigDef.Type.LONG,
5 * 60 * 1000L,
atLeast(0),
ConfigDef.Importance.LOW,
CommonClientConfigs.METADATA_MAX_AGE_DOC)
.define(METRICS_NUM_SAMPLES_CONFIG,
Type.INT,
2,
atLeast(1),
Importance.LOW,
CommonClientConfigs.METRICS_NUM_SAMPLES_DOC)
.define(METRIC_REPORTER_CLASSES_CONFIG,
Type.LIST,
"",
Importance.LOW,
CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC)
.define(METRICS_RECORDING_LEVEL_CONFIG,
Type.STRING,
Sensor.RecordingLevel.INFO.toString(),
in(Sensor.RecordingLevel.INFO.toString(), Sensor.RecordingLevel.DEBUG.toString(), RecordingLevel.TRACE.toString()),
Importance.LOW,
CommonClientConfigs.METRICS_RECORDING_LEVEL_DOC)
.define(METRICS_SAMPLE_WINDOW_MS_CONFIG,
Type.LONG,
30000L,
atLeast(0),
Importance.LOW,
CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC)
.define(AUTO_INCLUDE_JMX_REPORTER_CONFIG,
Type.BOOLEAN,
true,
Importance.LOW,
CommonClientConfigs.AUTO_INCLUDE_JMX_REPORTER_DOC)
.define(POLL_MS_CONFIG,
Type.LONG,
100L,
Importance.LOW,
POLL_MS_DOC)
.define(PROBING_REBALANCE_INTERVAL_MS_CONFIG,
Type.LONG,
10 * 60 * 1000L,
atLeast(60 * 1000L),
Importance.LOW,
PROBING_REBALANCE_INTERVAL_MS_DOC)
.define(RECEIVE_BUFFER_CONFIG,
Type.INT,
32 * 1024,
atLeast(CommonClientConfigs.RECEIVE_BUFFER_LOWER_BOUND),
Importance.LOW,
CommonClientConfigs.RECEIVE_BUFFER_DOC)