From b25db8b9e4c73f95a3cf51bb6eab7114f12ca0ac Mon Sep 17 00:00:00 2001 From: nilaysundarkar <3598371+nilaysundarkar@users.noreply.github.com> Date: Wed, 8 May 2024 10:10:50 -0500 Subject: [PATCH 1/2] Custom Auth Provider Plug point changes --- README.md | 23 ++++ build.gradle.kts | 2 +- .../kafka/connect/sink/MongoSinkConfig.java | 12 ++ .../kafka/connect/sink/MongoSinkTask.java | 4 + .../connect/source/MongoSourceConfig.java | 11 ++ .../kafka/connect/source/MongoSourceTask.java | 8 ++ .../connect/util/ConnectionValidator.java | 16 +++ .../credentials/CustomCredentialProvider.java | 13 +++ .../CustomCredentialProviderConstants.java | 10 ++ ...mCredentialProviderGenericInitializer.java | 76 ++++++++++++ ...dentialProviderGenericInitializerTest.java | 110 ++++++++++++++++++ .../TestCustomCredentialProvider.java | 34 ++++++ .../TestInvalidCustomCredentialProvider.java | 3 + 13 files changed, 321 insertions(+), 1 deletion(-) create mode 100644 src/main/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProvider.java create mode 100644 src/main/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProviderConstants.java create mode 100644 src/main/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProviderGenericInitializer.java create mode 100644 src/test/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProviderGenericInitializerTest.java create mode 100644 src/test/java/com/mongodb/kafka/connect/util/custom/credentials/TestCustomCredentialProvider.java create mode 100644 src/test/java/com/mongodb/kafka/connect/util/custom/credentials/TestInvalidCustomCredentialProvider.java diff --git a/README.md b/README.md index f5019976..c86724ed 100644 --- a/README.md +++ b/README.md @@ -76,3 +76,26 @@ A couple of manual configuration steps are required to run the code in IntelliJ: - Run the `compileBuildConfig` task: eg: `./gradlew compileBuildConfig` or via Gradle > mongo-kafka > Tasks > other > compileBuildConfig - Set `compileBuildConfig` to execute Before Build. via Gradle > Tasks > other > right click compileBuildConfig - click on "Execute Before Build" - Delegate all build actions to Gradle: Settings > Build, Execution, Deployment > Build Tools > Gradle > Runner - tick "Delegate IDE build/run actions to gradle" + +## Custom Auth Provider Interface + +The `com.mongodb.kafka.connect.util.custom.credentials.CustomCredentialProvider` interface can be implemented to provide an object of type `com.mongodb.MongoCredential` which gets wrapped in the MongoClient that is constructed for the sink and source connector. +The following properties need to be set - + +``` +mongo.custom.auth.mechanism.enable - set to true. +mongo.custom.auth.mechanism.providerClass - qualified class name of the implementation class +``` +Additional properties and can be set as required within the implementation class. +The init and validate methods of the implementation class get called when the connector initializes. + +### Example +When using MONGODB-AWS authentication mechanism for atlas, one can specify the following configuration - + +``` +"connection.uri": "mongodb+srv:///?authMechanism=MONGODB-AWS" +"mongo.custom.auth.mechanism.enable": true, +"mongo.custom.auth.mechanism.providerClass": "sample.AwsAssumeRoleCredentialProvider" +"mongodbaws.auth.mechanism.roleArn": "arn:aws:iam:::role/" +``` +Here the `sample.AwsAssumeRoleCredentialProvider` must be available on the classpath. `mongodbaws.auth.mechanism.roleArn` is an example of custom properties that can be read by `sample.AwsAssumeRoleCredentialProvider`. diff --git a/build.gradle.kts b/build.gradle.kts index 2a0e0c8c..28f7637a 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -49,7 +49,7 @@ repositories { } extra.apply { - set("mongodbDriverVersion", "[4.7,4.7.99)") + set("mongodbDriverVersion", "4.9.1") set("kafkaVersion", "2.6.0") set("avroVersion", "1.9.2") diff --git a/src/main/java/com/mongodb/kafka/connect/sink/MongoSinkConfig.java b/src/main/java/com/mongodb/kafka/connect/sink/MongoSinkConfig.java index 28f2de00..0a42b7b6 100644 --- a/src/main/java/com/mongodb/kafka/connect/sink/MongoSinkConfig.java +++ b/src/main/java/com/mongodb/kafka/connect/sink/MongoSinkConfig.java @@ -25,6 +25,8 @@ import static com.mongodb.kafka.connect.util.ServerApiConfig.addServerApiConfig; import static com.mongodb.kafka.connect.util.SslConfigs.addSslConfigDef; import static com.mongodb.kafka.connect.util.Validators.errorCheckingPasswordValueValidator; +import static com.mongodb.kafka.connect.util.custom.credentials.CustomCredentialProviderConstants.*; +import static com.mongodb.kafka.connect.util.custom.credentials.CustomCredentialProviderGenericInitializer.initializeCustomProvider; import static java.lang.String.format; import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; @@ -50,6 +52,7 @@ import com.mongodb.kafka.connect.MongoSinkConnector; import com.mongodb.kafka.connect.util.Validators; +import com.mongodb.kafka.connect.util.custom.credentials.CustomCredentialProvider; public class MongoSinkConfig extends AbstractConfig { private static final String EMPTY_STRING = ""; @@ -100,6 +103,7 @@ public class MongoSinkConfig extends AbstractConfig { private final Optional topicsRegex; private Map topicSinkConnectorConfigMap; private ConnectionString connectionString; + private CustomCredentialProvider customCredentialProvider; public MongoSinkConfig(final Map originals) { super(CONFIG, originals, false); @@ -146,6 +150,10 @@ public MongoSinkConfig(final Map originals) { } }); } + // Initialize CustomCredentialProvider if mongo.custom.auth.mechanism.enable is set to true + if (Boolean.parseBoolean(originals.get(CUSTOM_AUTH_ENABLE_CONFIG))) { + customCredentialProvider = initializeCustomProvider(originals); + } } public static final ConfigDef CONFIG = createConfigDef(); @@ -157,6 +165,10 @@ static String createOverrideKey(final String topic, final String config) { return format(TOPIC_OVERRIDE_CONFIG, topic, config); } + public CustomCredentialProvider getCustomCredentialProvider() { + return customCredentialProvider; + } + public ConnectionString getConnectionString() { return connectionString; } diff --git a/src/main/java/com/mongodb/kafka/connect/sink/MongoSinkTask.java b/src/main/java/com/mongodb/kafka/connect/sink/MongoSinkTask.java index 3b2bfa3a..43ff6d7b 100644 --- a/src/main/java/com/mongodb/kafka/connect/sink/MongoSinkTask.java +++ b/src/main/java/com/mongodb/kafka/connect/sink/MongoSinkTask.java @@ -155,6 +155,10 @@ private static MongoClient createMongoClient(final MongoSinkConfig sinkConfig) { MongoClientSettings.builder() .applyConnectionString(sinkConfig.getConnectionString()) .applyToSslSettings(sslBuilder -> setupSsl(sslBuilder, sinkConfig)); + if (sinkConfig.getCustomCredentialProvider() != null) { + builder.credential( + sinkConfig.getCustomCredentialProvider().getCustomCredential(sinkConfig.getOriginals())); + } setServerApi(builder, sinkConfig); return MongoClients.create( diff --git a/src/main/java/com/mongodb/kafka/connect/source/MongoSourceConfig.java b/src/main/java/com/mongodb/kafka/connect/source/MongoSourceConfig.java index 31efbab2..4a4aeb9c 100644 --- a/src/main/java/com/mongodb/kafka/connect/source/MongoSourceConfig.java +++ b/src/main/java/com/mongodb/kafka/connect/source/MongoSourceConfig.java @@ -35,6 +35,8 @@ import static com.mongodb.kafka.connect.util.Validators.errorCheckingPasswordValueValidator; import static com.mongodb.kafka.connect.util.Validators.errorCheckingValueValidator; import static com.mongodb.kafka.connect.util.VisibleForTesting.AccessModifier.PACKAGE; +import static com.mongodb.kafka.connect.util.custom.credentials.CustomCredentialProviderConstants.CUSTOM_AUTH_ENABLE_CONFIG; +import static com.mongodb.kafka.connect.util.custom.credentials.CustomCredentialProviderGenericInitializer.initializeCustomProvider; import static java.lang.String.format; import static java.util.Arrays.asList; import static java.util.Collections.emptyList; @@ -78,6 +80,7 @@ import com.mongodb.kafka.connect.util.Validators; import com.mongodb.kafka.connect.util.VisibleForTesting; import com.mongodb.kafka.connect.util.config.BsonTimestampParser; +import com.mongodb.kafka.connect.util.custom.credentials.CustomCredentialProvider; public class MongoSourceConfig extends AbstractConfig { @@ -583,6 +586,11 @@ public class MongoSourceConfig extends AbstractConfig { + "connection details will be used."; static final String PROVIDER_CONFIG = "provider"; + private CustomCredentialProvider customCredentialProvider; + + public CustomCredentialProvider getCustomCredentialProvider() { + return customCredentialProvider; + } public static final ConfigDef CONFIG = createConfigDef(); private static final List> INITIALIZERS = @@ -745,6 +753,9 @@ public String value() { public MongoSourceConfig(final Map originals) { this(originals, true); + if (Boolean.parseBoolean((String) originals.get(CUSTOM_AUTH_ENABLE_CONFIG))) { + customCredentialProvider = initializeCustomProvider(originals); + } } private MongoSourceConfig(final Map originals, final boolean validateAll) { diff --git a/src/main/java/com/mongodb/kafka/connect/source/MongoSourceTask.java b/src/main/java/com/mongodb/kafka/connect/source/MongoSourceTask.java index 7812a510..eed35aff 100644 --- a/src/main/java/com/mongodb/kafka/connect/source/MongoSourceTask.java +++ b/src/main/java/com/mongodb/kafka/connect/source/MongoSourceTask.java @@ -133,6 +133,14 @@ public void commandFailed(final CommandFailedEvent event) { .applyConnectionString(sourceConfig.getConnectionString()) .addCommandListener(statisticsCommandListener) .applyToSslSettings(sslBuilder -> setupSsl(sslBuilder, sourceConfig)); + + if (sourceConfig.getCustomCredentialProvider() != null) { + builder.credential( + sourceConfig + .getCustomCredentialProvider() + .getCustomCredential(sourceConfig.originals())); + } + setServerApi(builder, sourceConfig); mongoClient = diff --git a/src/main/java/com/mongodb/kafka/connect/util/ConnectionValidator.java b/src/main/java/com/mongodb/kafka/connect/util/ConnectionValidator.java index e54a13c6..9a972078 100644 --- a/src/main/java/com/mongodb/kafka/connect/util/ConnectionValidator.java +++ b/src/main/java/com/mongodb/kafka/connect/util/ConnectionValidator.java @@ -49,6 +49,10 @@ import com.mongodb.event.ClusterListener; import com.mongodb.event.ClusterOpeningEvent; +import com.mongodb.kafka.connect.sink.MongoSinkConfig; +import com.mongodb.kafka.connect.source.MongoSourceConfig; +import com.mongodb.kafka.connect.util.custom.credentials.CustomCredentialProvider; + public final class ConnectionValidator { private static final Logger LOGGER = LoggerFactory.getLogger(ConnectionValidator.class); @@ -77,6 +81,18 @@ public static Optional validateCanConnect( new ConnectionString(((Password) configValue.value()).value()); MongoClientSettings.Builder mongoClientSettingsBuilder = MongoClientSettings.builder().applyConnectionString(connectionString); + CustomCredentialProvider customCredentialProvider = null; + if (connectorProperties instanceof MongoSinkConfig) { + customCredentialProvider = + ((MongoSinkConfig) connectorProperties).getCustomCredentialProvider(); + } else if (connectorProperties instanceof MongoSourceConfig) { + customCredentialProvider = + ((MongoSourceConfig) connectorProperties).getCustomCredentialProvider(); + } + if (customCredentialProvider != null) { + mongoClientSettingsBuilder.credential( + customCredentialProvider.getCustomCredential(connectorProperties.originals())); + } setServerApi(mongoClientSettingsBuilder, config); MongoClientSettings mongoClientSettings = diff --git a/src/main/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProvider.java b/src/main/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProvider.java new file mode 100644 index 00000000..83332333 --- /dev/null +++ b/src/main/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProvider.java @@ -0,0 +1,13 @@ +package com.mongodb.kafka.connect.util.custom.credentials; + +import java.util.Map; + +import com.mongodb.MongoCredential; + +public interface CustomCredentialProvider { + MongoCredential getCustomCredential(Map configs); + + void validate(Map configs); + + void init(Map configs); +} diff --git a/src/main/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProviderConstants.java b/src/main/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProviderConstants.java new file mode 100644 index 00000000..5202ceb8 --- /dev/null +++ b/src/main/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProviderConstants.java @@ -0,0 +1,10 @@ +package com.mongodb.kafka.connect.util.custom.credentials; + +public final class CustomCredentialProviderConstants { + private CustomCredentialProviderConstants() {} + + public static final String CUSTOM_AUTH_ENABLE_CONFIG = "mongo.custom.auth.mechanism.enable"; + + public static final String CUSTOM_AUTH_PROVIDER_CLASS = + "mongo.custom.auth.mechanism.providerClass"; +} diff --git a/src/main/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProviderGenericInitializer.java b/src/main/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProviderGenericInitializer.java new file mode 100644 index 00000000..6be44ed4 --- /dev/null +++ b/src/main/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProviderGenericInitializer.java @@ -0,0 +1,76 @@ +package com.mongodb.kafka.connect.util.custom.credentials; + +import java.lang.reflect.InvocationTargetException; +import java.util.Map; + +import org.apache.kafka.common.config.ConfigException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CustomCredentialProviderGenericInitializer { + + static final Logger LOGGER = + LoggerFactory.getLogger(CustomCredentialProviderGenericInitializer.class); + + public static CustomCredentialProvider initializeCustomProvider(Map originals) + throws ConfigException { + // Validate if CUSTOM_AUTH_ENABLE_CONFIG is set to true + String customAuthMechanismEnabled = + String.valueOf(originals.get(CustomCredentialProviderConstants.CUSTOM_AUTH_ENABLE_CONFIG)); + if (customAuthMechanismEnabled == null + || customAuthMechanismEnabled.equals("null") + || customAuthMechanismEnabled.isEmpty()) { + throw new ConfigException( + CustomCredentialProviderConstants.CUSTOM_AUTH_ENABLE_CONFIG + + " is not set to true. " + + "CustomCredentialProvider should not be used."); + } + // Validate if CUSTOM_AUTH_PROVIDER_CLASS is provided + String qualifiedAuthProviderClassName = + String.valueOf(originals.get(CustomCredentialProviderConstants.CUSTOM_AUTH_PROVIDER_CLASS)); + if (qualifiedAuthProviderClassName == null + || qualifiedAuthProviderClassName.equals("null") + || qualifiedAuthProviderClassName.isEmpty()) { + throw new ConfigException( + CustomCredentialProviderConstants.CUSTOM_AUTH_PROVIDER_CLASS + + " is required when " + + CustomCredentialProviderConstants.CUSTOM_AUTH_ENABLE_CONFIG + + " is set to true."); + } + try { + // Validate if qualifiedAuthProviderClassName is on the class path. + Class authProviderClass = + Class.forName( + qualifiedAuthProviderClassName, + false, + CustomCredentialProviderGenericInitializer.class.getClassLoader()); + // Validate if qualifiedAuthProviderClassName implements CustomCredentialProvider interface. + if (!CustomCredentialProvider.class.isAssignableFrom(authProviderClass)) { + throw new ConfigException( + "Provided Class does not implement CustomCredentialProvider interface."); + } + CustomCredentialProvider customCredentialProvider = + initializeCustomProvider(authProviderClass); + // Perform config validations specific to CustomCredentialProvider impl provided + customCredentialProvider.validate(originals); + // Initialize custom variables required by implementation of CustomCredentialProvider + customCredentialProvider.init(originals); + return customCredentialProvider; + } catch (ClassNotFoundException e) { + throw new ConfigException( + "Unable to find " + qualifiedAuthProviderClassName + " on the classpath."); + } + } + + private static CustomCredentialProvider initializeCustomProvider(Class authProviderClass) { + try { + return (CustomCredentialProvider) authProviderClass.getDeclaredConstructor().newInstance(); + } catch (InstantiationException + | IllegalAccessException + | InvocationTargetException + | NoSuchMethodException e) { + LOGGER.error("Error while instantiating " + authProviderClass + " class"); + throw new RuntimeException(e); + } + } +} diff --git a/src/test/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProviderGenericInitializerTest.java b/src/test/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProviderGenericInitializerTest.java new file mode 100644 index 00000000..a637d30c --- /dev/null +++ b/src/test/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProviderGenericInitializerTest.java @@ -0,0 +1,110 @@ +package com.mongodb.kafka.connect.util.custom.credentials; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.kafka.common.config.ConfigException; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; + +public class CustomCredentialProviderGenericInitializerTest { + + @Test + @DisplayName("Test Exception scenarios") + void testExceptions() { + Map props = new HashMap<>(); + ConfigException configException = + assertThrows( + ConfigException.class, + () -> CustomCredentialProviderGenericInitializer.initializeCustomProvider(props), + "Expected initializeCustomProvider() to throw, but it didn't"); + assertEquals( + CustomCredentialProviderConstants.CUSTOM_AUTH_ENABLE_CONFIG + + " is not set to true. " + + "CustomCredentialProvider should not be used.", + configException.getMessage()); + props.put(CustomCredentialProviderConstants.CUSTOM_AUTH_ENABLE_CONFIG, true); + configException = + assertThrows( + ConfigException.class, + () -> CustomCredentialProviderGenericInitializer.initializeCustomProvider(props), + "Expected initializeCustomProvider() to throw, but it didn't"); + assertEquals( + CustomCredentialProviderConstants.CUSTOM_AUTH_PROVIDER_CLASS + + " is required when " + + CustomCredentialProviderConstants.CUSTOM_AUTH_ENABLE_CONFIG + + " is set to true.", + configException.getMessage()); + String qualifiedAuthProviderClassName = "com.nonexistant.package.Test"; + props.put( + CustomCredentialProviderConstants.CUSTOM_AUTH_PROVIDER_CLASS, + qualifiedAuthProviderClassName); + configException = + assertThrows( + ConfigException.class, + () -> CustomCredentialProviderGenericInitializer.initializeCustomProvider(props), + "Expected initializeCustomProvider() to throw, but it didn't"); + assertEquals( + "Unable to find " + qualifiedAuthProviderClassName + " on the classpath.", + configException.getMessage()); + qualifiedAuthProviderClassName = + "com.mongodb.kafka.connect.util.custom.credentials.TestInvalidCustomCredentialProvider"; + props.put( + CustomCredentialProviderConstants.CUSTOM_AUTH_PROVIDER_CLASS, + qualifiedAuthProviderClassName); + configException = + assertThrows( + ConfigException.class, + () -> CustomCredentialProviderGenericInitializer.initializeCustomProvider(props), + "Expected initializeCustomProvider() to throw, but it didn't"); + assertEquals( + "Provided Class does not implement CustomCredentialProvider interface.", + configException.getMessage()); + } + + @Test + void testInitializeCustomCredentialProvider() { + Map props = new HashMap<>(); + props.put(CustomCredentialProviderConstants.CUSTOM_AUTH_ENABLE_CONFIG, true); + props.put( + CustomCredentialProviderConstants.CUSTOM_AUTH_PROVIDER_CLASS, + "com.mongodb.kafka.connect.util.custom.credentials.TestCustomCredentialProvider"); + props.put("customProperty", "customValue"); + CustomCredentialProvider customCredentialProvider = + CustomCredentialProviderGenericInitializer.initializeCustomProvider(props); + assertEquals(TestCustomCredentialProvider.class, customCredentialProvider.getClass()); + } + + @Test + void testCustomPropsInit() { + Map props = new HashMap<>(); + props.put(CustomCredentialProviderConstants.CUSTOM_AUTH_ENABLE_CONFIG, true); + props.put( + CustomCredentialProviderConstants.CUSTOM_AUTH_PROVIDER_CLASS, + "com.mongodb.kafka.connect.util.custom.credentials.TestCustomCredentialProvider"); + props.put("customProperty", "customValue"); + TestCustomCredentialProvider customCredentialProvider = + (TestCustomCredentialProvider) + CustomCredentialProviderGenericInitializer.initializeCustomProvider(props); + assertEquals("customValue", customCredentialProvider.getCustomProperty()); + } + + @Test + void testCustomPropsValidate() { + Map props = new HashMap<>(); + props.put(CustomCredentialProviderConstants.CUSTOM_AUTH_ENABLE_CONFIG, true); + props.put( + CustomCredentialProviderConstants.CUSTOM_AUTH_PROVIDER_CLASS, + "com.mongodb.kafka.connect.util.custom.credentials.TestCustomCredentialProvider"); + props.put("customProperty", "invalidValue"); + ConfigException configException = + assertThrows( + ConfigException.class, + () -> CustomCredentialProviderGenericInitializer.initializeCustomProvider(props), + "Expected initializeCustomProvider() to throw, but it didn't"); + assertEquals("Invalid value set for customProperty", configException.getMessage()); + } +} diff --git a/src/test/java/com/mongodb/kafka/connect/util/custom/credentials/TestCustomCredentialProvider.java b/src/test/java/com/mongodb/kafka/connect/util/custom/credentials/TestCustomCredentialProvider.java new file mode 100644 index 00000000..e41a10b5 --- /dev/null +++ b/src/test/java/com/mongodb/kafka/connect/util/custom/credentials/TestCustomCredentialProvider.java @@ -0,0 +1,34 @@ +package com.mongodb.kafka.connect.util.custom.credentials; + +import java.util.Map; + +import org.apache.kafka.common.config.ConfigException; + +import com.mongodb.MongoCredential; + +public class TestCustomCredentialProvider implements CustomCredentialProvider { + + private String customProperty; + + public String getCustomProperty() { + return customProperty; + } + + @Override + public MongoCredential getCustomCredential(Map configs) { + return MongoCredential.createAwsCredential("userName", "password".toCharArray()); + } + + @Override + public void validate(Map configs) { + String customProperty = (String) configs.get("customProperty"); + if (customProperty.equals("invalidValue")) { + throw new ConfigException("Invalid value set for customProperty"); + } + } + + @Override + public void init(Map configs) { + customProperty = (String) configs.get("customProperty"); + } +} diff --git a/src/test/java/com/mongodb/kafka/connect/util/custom/credentials/TestInvalidCustomCredentialProvider.java b/src/test/java/com/mongodb/kafka/connect/util/custom/credentials/TestInvalidCustomCredentialProvider.java new file mode 100644 index 00000000..0a5eb1ad --- /dev/null +++ b/src/test/java/com/mongodb/kafka/connect/util/custom/credentials/TestInvalidCustomCredentialProvider.java @@ -0,0 +1,3 @@ +package com.mongodb.kafka.connect.util.custom.credentials; + +public class TestInvalidCustomCredentialProvider {} From bae395e695f660aa958235b0319611714db6d677 Mon Sep 17 00:00:00 2001 From: nilaysundarkar <3598371+nilaysundarkar@users.noreply.github.com> Date: Mon, 13 May 2024 09:09:36 -0500 Subject: [PATCH 2/2] added display names to tests --- .../CustomCredentialProviderGenericInitializerTest.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/test/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProviderGenericInitializerTest.java b/src/test/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProviderGenericInitializerTest.java index a637d30c..220f4c07 100644 --- a/src/test/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProviderGenericInitializerTest.java +++ b/src/test/java/com/mongodb/kafka/connect/util/custom/credentials/CustomCredentialProviderGenericInitializerTest.java @@ -66,6 +66,7 @@ void testExceptions() { } @Test + @DisplayName("Test CustomCredentialProvider initialization") void testInitializeCustomCredentialProvider() { Map props = new HashMap<>(); props.put(CustomCredentialProviderConstants.CUSTOM_AUTH_ENABLE_CONFIG, true); @@ -79,6 +80,7 @@ void testInitializeCustomCredentialProvider() { } @Test + @DisplayName("Test CustomCredentialProvider custom props initialization") void testCustomPropsInit() { Map props = new HashMap<>(); props.put(CustomCredentialProviderConstants.CUSTOM_AUTH_ENABLE_CONFIG, true); @@ -93,6 +95,7 @@ void testCustomPropsInit() { } @Test + @DisplayName("Test CustomCredentialProvider custom props validation") void testCustomPropsValidate() { Map props = new HashMap<>(); props.put(CustomCredentialProviderConstants.CUSTOM_AUTH_ENABLE_CONFIG, true);