Skip to content

[WIP] KAFKA-19112 Unifying LIST-Type Configuration Validation and Default Values #20334

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 42 commits into
base: trunk
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
42 commits
Select commit Hold shift + click to select a range
304c11e
add new list validator method
m1a2st Aug 10, 2025
68f281f
sasl.oauthbearer.expected.audience
m1a2st Aug 10, 2025
247bca3
ssl.cipher.suites, ssl.enabled.protocols
m1a2st Aug 10, 2025
6cbf14b
process.roles
m1a2st Aug 10, 2025
8e196f1
StreamsConfig
m1a2st Aug 10, 2025
17bbcc3
GroupCoordinatorConfig
m1a2st Aug 10, 2025
92f9b95
fix stream test
m1a2st Aug 10, 2025
a2aef18
MirrorClientConfig, AdminClientConfig, BrokerApiVersionsCommand$Admin…
m1a2st Aug 10, 2025
a58c3c3
MirrorSourceTaskConfig, MirrorCheckpointTaskConfig,
m1a2st Aug 10, 2025
3171385
RestServerConfig, ServerConfigs
m1a2st Aug 10, 2025
61ae799
listeners
m1a2st Aug 10, 2025
7fe1fb3
log.dirs
m1a2st Aug 10, 2025
a3b3920
ClientMetricsConfigs, MetricConfigs, ReplaceField, MaskField
m1a2st Aug 10, 2025
5d3f504
MaskField, HeaderFrom, DropHeaders, SinkConnectorConfig, WorkerConfig
m1a2st Aug 10, 2025
30cf8ee
MirrorMakerConfig,
m1a2st Aug 10, 2025
f603b75
remove NonEmptyListValidator.java
m1a2st Aug 10, 2025
322a991
spotlessApply
m1a2st Aug 10, 2025
adc5c95
Revert "remove NonEmptyListValidator.java"
m1a2st Aug 10, 2025
c98d261
update HeaderFrom config
m1a2st Aug 10, 2025
6b3bb4d
fix consumerConfig BOOTSTRAP_SERVERS_CONFIG
m1a2st Aug 10, 2025
053957b
fix ConsumerConfig
m1a2st Aug 11, 2025
4952999
update bootstrap.server config
m1a2st Aug 11, 2025
8f067b3
fix some build fail
m1a2st Aug 11, 2025
a986cb9
spotlessApply
m1a2st Aug 11, 2025
9402ef1
fix MirrorClientConfig
m1a2st Aug 11, 2025
bd6f2fa
fix LogConfigTest and KRaftClusterTest
m1a2st Aug 11, 2025
780e964
fix MirrorClientConfig and test
m1a2st Aug 11, 2025
27b3091
fix CommonNameLoggingSslEngineFactoryTest add null check back
m1a2st Aug 11, 2025
15718a7
fix KafkaConsumerTest
m1a2st Aug 11, 2025
b395ed2
fix KafkaProducerTest, AdminBootstrapAddressesTest
m1a2st Aug 12, 2025
583955a
update BOOTSTRAP_SERVERS_CONFIG related config
m1a2st Aug 12, 2025
4c7ac76
update MockProcessorContext related test
m1a2st Aug 12, 2025
d80d02a
MirrorClientConfig and test
m1a2st Aug 12, 2025
8f6e3b7
fix StateDirectoryTest and TimeOrderedKeyValueBufferTest
m1a2st Aug 12, 2025
29a1dd3
fix ConsumerConfigTest, ConsumerPartitionAssignorTest, GroupRebalance…
m1a2st Aug 12, 2025
6cfad77
fix other tests
m1a2st Aug 12, 2025
4b7cd4e
fix NetworkClientDelegateTest
m1a2st Aug 12, 2025
c66ec10
update WorkerConfig
m1a2st Aug 12, 2025
d2a048b
spotlessApply
m1a2st Aug 12, 2025
9c7fcee
fix the WorkerConfig related test
m1a2st Aug 13, 2025
58cea0f
update the null checker for config.providers and consumer assignors
m1a2st Aug 13, 2025
b281ee2
revert config.providers
m1a2st Aug 13, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -155,12 +155,14 @@ public class AdminClientConfig extends AbstractConfig {
static {
CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG,
Type.LIST,
"",
List.of(),
ConfigDef.ValidList.anyNonDuplicateValues(true, false),
Importance.HIGH,
BOOTSTRAP_SERVERS_DOC).
define(BOOTSTRAP_CONTROLLERS_CONFIG,
Type.LIST,
"",
List.of(),
ConfigDef.ValidList.anyNonDuplicateValues(true, false),
Importance.HIGH,
BOOTSTRAP_CONTROLLERS_DOC)
.define(CLIENT_ID_CONFIG, Type.STRING, "", Importance.MEDIUM, CLIENT_ID_DOC)
Expand Down Expand Up @@ -239,6 +241,7 @@ public class AdminClientConfig extends AbstractConfig {
.define(METRIC_REPORTER_CLASSES_CONFIG,
Type.LIST,
JmxReporter.class.getName(),
ConfigDef.ValidList.anyNonDuplicateValues(true, false),
Importance.LOW,
METRIC_REPORTER_CLASSES_DOC)
.define(METRICS_RECORDING_LEVEL_CONFIG,
Expand Down Expand Up @@ -284,7 +287,8 @@ public class AdminClientConfig extends AbstractConfig {
METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_DOC)
.define(CONFIG_PROVIDERS_CONFIG,
ConfigDef.Type.LIST,
List.of(),
List.of(),
ConfigDef.ValidList.anyNonDuplicateValues(true, false),
ConfigDef.Importance.LOW,
CONFIG_PROVIDERS_DOC);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,9 +63,9 @@ public class ConsumerConfig extends AbstractConfig {
// a list contains all the assignor names that only assign subscribed topics to consumer. Should be updated when new assignor added.
// This is to help optimize ConsumerCoordinator#performAssignment method
public static final List<String> ASSIGN_FROM_SUBSCRIBED_ASSIGNORS = List.of(
RANGE_ASSIGNOR_NAME,
ROUNDROBIN_ASSIGNOR_NAME,
STICKY_ASSIGNOR_NAME,
RANGE_ASSIGNOR_NAME,
ROUNDROBIN_ASSIGNOR_NAME,
STICKY_ASSIGNOR_NAME,
COOPERATIVE_STICKY_ASSIGNOR_NAME
);

Expand Down Expand Up @@ -406,17 +406,17 @@ public class ConsumerConfig extends AbstractConfig {
* A list of configuration keys not supported for CONSUMER protocol.
*/
private static final List<String> CONSUMER_PROTOCOL_UNSUPPORTED_CONFIGS = List.of(
PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
HEARTBEAT_INTERVAL_MS_CONFIG,
PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
HEARTBEAT_INTERVAL_MS_CONFIG,
SESSION_TIMEOUT_MS_CONFIG,
SHARE_ACKNOWLEDGEMENT_MODE_CONFIG
);

static {
CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG,
Type.LIST,
Collections.emptyList(),
new ConfigDef.NonNullValidator(),
ConfigDef.NO_DEFAULT_VALUE,
ConfigDef.ValidList.anyNonDuplicateValues(false, false),
Importance.HIGH,
CommonClientConfigs.BOOTSTRAP_SERVERS_DOC)
.define(CLIENT_DNS_LOOKUP_CONFIG,
Expand Down Expand Up @@ -446,7 +446,7 @@ public class ConsumerConfig extends AbstractConfig {
.define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
Type.LIST,
List.of(RangeAssignor.class, CooperativeStickyAssignor.class),
new ConfigDef.NonNullValidator(),
ConfigDef.ValidList.anyNonDuplicateValues(true, false),
Importance.MEDIUM,
PARTITION_ASSIGNMENT_STRATEGY_DOC)
.define(METADATA_MAX_AGE_CONFIG,
Expand Down Expand Up @@ -573,7 +573,7 @@ public class ConsumerConfig extends AbstractConfig {
.define(METRIC_REPORTER_CLASSES_CONFIG,
Type.LIST,
JmxReporter.class.getName(),
new ConfigDef.NonNullValidator(),
ConfigDef.ValidList.anyNonDuplicateValues(true, false),
Importance.LOW,
CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC)
.define(KEY_DESERIALIZER_CLASS_CONFIG,
Expand Down Expand Up @@ -615,7 +615,7 @@ public class ConsumerConfig extends AbstractConfig {
.define(INTERCEPTOR_CLASSES_CONFIG,
Type.LIST,
Collections.emptyList(),
new ConfigDef.NonNullValidator(),
ConfigDef.ValidList.anyNonDuplicateValues(true, false),
Importance.LOW,
INTERCEPTOR_CLASSES_DOC)
.define(MAX_POLL_RECORDS_CONFIG,
Expand Down Expand Up @@ -702,6 +702,7 @@ public class ConsumerConfig extends AbstractConfig {
.define(CONFIG_PROVIDERS_CONFIG,
ConfigDef.Type.LIST,
List.of(),
ConfigDef.ValidList.anyNonDuplicateValues(true, false),
ConfigDef.Importance.LOW,
CONFIG_PROVIDERS_DOC);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -291,9 +291,6 @@ static List<ConsumerPartitionAssignor> getAssignorInstances(List<String> assigno
// a map to store assignor name -> assignor class name
Map<String, String> assignorNameMap = new HashMap<>();

if (assignorClasses == null)
return assignors;

for (Object klass : assignorClasses) {
// first try to get the class if passed in as a string
if (klass instanceof String) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;

import static org.apache.kafka.common.config.ConfigDef.NO_DEFAULT_VALUE;
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;
Expand Down Expand Up @@ -373,7 +374,12 @@ public class ProducerConfig extends AbstractConfig {
private static final AtomicInteger PRODUCER_CLIENT_ID_SEQUENCE = new AtomicInteger(1);

static {
CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Collections.emptyList(), new ConfigDef.NonNullValidator(), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC)
CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG,
Type.LIST,
NO_DEFAULT_VALUE,
ConfigDef.ValidList.anyNonDuplicateValues(false, false),
Importance.HIGH,
CommonClientConfigs.BOOTSTRAP_SERVERS_DOC)
.define(CLIENT_DNS_LOOKUP_CONFIG,
Type.STRING,
ClientDnsLookup.USE_ALL_DNS_IPS.toString(),
Expand Down Expand Up @@ -462,7 +468,7 @@ public class ProducerConfig extends AbstractConfig {
.define(METRIC_REPORTER_CLASSES_CONFIG,
Type.LIST,
JmxReporter.class.getName(),
new ConfigDef.NonNullValidator(),
ConfigDef.ValidList.anyNonDuplicateValues(true, false),
Importance.LOW,
CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC)
.define(MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION,
Expand Down Expand Up @@ -502,7 +508,7 @@ public class ProducerConfig extends AbstractConfig {
.define(INTERCEPTOR_CLASSES_CONFIG,
Type.LIST,
Collections.emptyList(),
new ConfigDef.NonNullValidator(),
ConfigDef.ValidList.anyNonDuplicateValues(true, false),
Importance.LOW,
INTERCEPTOR_CLASSES_DOC)
.define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG,
Expand Down Expand Up @@ -553,10 +559,11 @@ public class ProducerConfig extends AbstractConfig {
atLeast(0),
Importance.LOW,
CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_DOC)
.define(CONFIG_PROVIDERS_CONFIG,
.define(CONFIG_PROVIDERS_CONFIG,
ConfigDef.Type.LIST,
List.of(),
ConfigDef.Importance.LOW,
List.of(),
ConfigDef.ValidList.anyNonDuplicateValues(true, false),
ConfigDef.Importance.LOW,
CONFIG_PROVIDERS_DOC);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1006,26 +1006,58 @@ else if (max == null)
public static class ValidList implements Validator {

final ValidString validString;
final boolean isEmptyAllowed;
final boolean isNullAllowed;

private ValidList(List<String> validStrings) {
private ValidList(List<String> validStrings, boolean isEmptyAllowed, boolean isNullAllowed) {
this.validString = new ValidString(validStrings);
this.isEmptyAllowed = isEmptyAllowed;
this.isNullAllowed = isNullAllowed;
}

public static ValidList anyNonDuplicateValues(boolean isEmptyAllowed, boolean isNullAllowed) {
return new ValidList(List.of(), isEmptyAllowed, isNullAllowed);
}

public static ValidList in(String... validStrings) {
return new ValidList(Arrays.asList(validStrings));
return new ValidList(List.of(validStrings), true, false);
}

public static ValidList in(boolean isEmptyAllowed, String... validStrings) {
if (validStrings.length == 0) {
throw new IllegalArgumentException("Valid strings list cannot be empty for inNonEmpty validator");
}
return new ValidList(List.of(validStrings), isEmptyAllowed, false);
}

@Override
public void ensureValid(final String name, final Object value) {
if (value == null && isNullAllowed) {
return;
} else if (value == null) {
throw new ConfigException("Configuration '" + name + "' values must not be null.");
}

@SuppressWarnings("unchecked")
List<String> values = (List<String>) value;
if (!isEmptyAllowed && values.isEmpty()) {
String validString = this.validString.validStrings.isEmpty() ? "any non-empty value" : this.validString.toString();
throw new ConfigException("Configuration '" + name + "' must not be empty. Valid values include: " + validString);
}
if (Set.copyOf(values).size() != values.size()) {
throw new ConfigException("Configuration '" + name + "' values must not be duplicated.");
}
if (validString.validStrings.isEmpty()) {
return;
}
for (String string : values) {
validString.ensureValid(name, string);
}
}

public String toString() {
return validString.toString();
return validString + (isEmptyAllowed ? " (empty config allowed)" : " (empty not allowed)") +
(isNullAllowed ? " (null config allowed)" : " (null not allowed)");
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@
import org.apache.kafka.common.config.ConfigDef.CaseInsensitiveValidString;
import org.apache.kafka.common.config.ConfigDef.Range;

import java.util.List;

public class SaslConfigs {

private static final String OAUTHBEARER_NOTE = " Currently applies only to OAUTHBEARER.";
Expand Down Expand Up @@ -407,7 +409,7 @@ public static void addClientSaslSupport(ConfigDef config) {
.define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, ConfigDef.Type.LONG, DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DOC)
.define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, ConfigDef.Type.LONG, DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DOC)
.define(SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, ConfigDef.Type.INT, DEFAULT_SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC)
.define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE, ConfigDef.Type.LIST, null, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC)
.define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE, ConfigDef.Type.LIST, List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC)
.define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER, ConfigDef.Type.STRING, null, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC)
.define(SaslConfigs.SASL_OAUTHBEARER_HEADER_URLENCODE, ConfigDef.Type.BOOLEAN, DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_HEADER_URLENCODE_DOC);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;

import java.util.List;
import java.util.Set;

import javax.net.ssl.KeyManagerFactory;
Expand Down Expand Up @@ -123,8 +124,8 @@ public class SslConfigs {
public static void addClientSslSupport(ConfigDef config) {
config.define(SslConfigs.SSL_PROTOCOL_CONFIG, ConfigDef.Type.STRING, SslConfigs.DEFAULT_SSL_PROTOCOL, ConfigDef.Importance.MEDIUM, SslConfigs.SSL_PROTOCOL_DOC)
.define(SslConfigs.SSL_PROVIDER_CONFIG, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, SslConfigs.SSL_PROVIDER_DOC)
.define(SslConfigs.SSL_CIPHER_SUITES_CONFIG, ConfigDef.Type.LIST, null, ConfigDef.Importance.LOW, SslConfigs.SSL_CIPHER_SUITES_DOC)
.define(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, ConfigDef.Type.LIST, SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS, ConfigDef.Importance.MEDIUM, SslConfigs.SSL_ENABLED_PROTOCOLS_DOC)
.define(SslConfigs.SSL_CIPHER_SUITES_CONFIG, ConfigDef.Type.LIST, List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, SslConfigs.SSL_CIPHER_SUITES_DOC)
.define(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, ConfigDef.Type.LIST, SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS, ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.MEDIUM, SslConfigs.SSL_ENABLED_PROTOCOLS_DOC)
.define(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, ConfigDef.Type.STRING, SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, ConfigDef.Importance.MEDIUM, SslConfigs.SSL_KEYSTORE_TYPE_DOC)
.define(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, ConfigDef.Type.STRING, null, ConfigDef.Importance.HIGH, SslConfigs.SSL_KEYSTORE_LOCATION_DOC)
.define(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.HIGH, SslConfigs.SSL_KEYSTORE_PASSWORD_DOC)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -154,7 +154,7 @@ public class BrokerSecurityConfigs {
.define(BrokerSecurityConfigs.SSL_ALLOW_SAN_CHANGES_CONFIG, BOOLEAN, BrokerSecurityConfigs.DEFAULT_SSL_ALLOW_SAN_CHANGES_VALUE, LOW, BrokerSecurityConfigs.SSL_ALLOW_SAN_CHANGES_DOC)
.define(SslConfigs.SSL_PROTOCOL_CONFIG, STRING, SslConfigs.DEFAULT_SSL_PROTOCOL, MEDIUM, SslConfigs.SSL_PROTOCOL_DOC)
.define(SslConfigs.SSL_PROVIDER_CONFIG, STRING, null, MEDIUM, SslConfigs.SSL_PROVIDER_DOC)
.define(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, LIST, SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS, MEDIUM, SslConfigs.SSL_ENABLED_PROTOCOLS_DOC)
.define(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, LIST, SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS, ConfigDef.ValidList.anyNonDuplicateValues(true, false), MEDIUM, SslConfigs.SSL_ENABLED_PROTOCOLS_DOC)
.define(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, STRING, SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, MEDIUM, SslConfigs.SSL_KEYSTORE_TYPE_DOC)
.define(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, STRING, null, MEDIUM, SslConfigs.SSL_KEYSTORE_LOCATION_DOC)
.define(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, PASSWORD, null, MEDIUM, SslConfigs.SSL_KEYSTORE_PASSWORD_DOC)
Expand Down Expand Up @@ -218,6 +218,6 @@ public class BrokerSecurityConfigs {
.define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, LONG, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, LOW, SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DOC)
.define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, LONG, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, LOW, SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DOC)
.define(SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, INT, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, LOW, SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC)
.define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE, LIST, null, LOW, SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC)
.define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE, LIST, List.of(), LOW, SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC)
.define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER, STRING, null, LOW, SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC);
}
Loading