From f1067de8b937dfb611edd4907e844f880e63624e Mon Sep 17 00:00:00 2001 From: Sangeet Mishra Date: Thu, 22 Aug 2024 18:55:36 +0530 Subject: [PATCH] Release bump 2 4 0 (#63) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * NOSNOW: added utility to upload connector to our nexus for simplified k8s deployment (#874) (cherry picked from commit ee83e22d0a697b8d9e0f69cf12c57fb8adb7f8bf) * SNOW-1514185: Do assign the reopened channel unless kafka offsets are fully reset (#875) (cherry picked from commit 05dcbdf1adac7d87794d3a99e896a35be3efb8d0) * SNOW-1061851: Run SnowflakeSinkServiceV2IT for a single buffer (#876) (cherry picked from commit 7fce0f5c4c25256c3e172b6011b7715456b457b8) * SNOW-1061855 Fix the E2E test for SchemaEvolutionDropTable with a single buffer (#882) (cherry picked from commit fbd15d6498c82ed6d6ca163a0c098729668485ef) * NOSNOW: improve troubleshooting experience for scenario with skipped offsets (#883) (cherry picked from commit 7172481999dfcd3a80fa3b2052cbddf05b2213e8) * Adding logs to track schematization (#884) (cherry picked from commit 9905c7f0ac4ff08a3cc06a7fe7f26b7d86622e5c) * Remove insertRows DEBUG log (called once per row) (#886) (cherry picked from commit 363457d03d20854b7a1831d33445074a9a26cf45) * NO-SNOW Upgrade nexus-staging-maven-plugin version (#887) (cherry picked from commit ac1276ea5cf15061b75eb8c014ee5193d734e07c) * Upgrade com.google.protobuf to 3.24.4 (#892) (cherry picked from commit 3c7e18cd47f055faa4336c88806e58ba785d7194) * SNOW-1229110 Add single buffer usage data to telemetry (#893) (cherry picked from commit 96d3bfac7384bd1311aa800b1a2c4711dfdda1be) * SNOW-1541942 Extend Snowpipe initialization logging at INFO level (#885) (cherry picked from commit eab4c9ca901a92f77a69095a112a6d94013b640c) * SNOW-947731 Remove deprecated avro-python3 package (#889) (cherry picked from commit 47624f5a6dfa98fcc560c403c077e31ddf4a9200) * Bump dev.failsafe:failsafe from 3.2.1 to 3.3.2 (#859) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Michał Bobowski (cherry picked from commit 4a22f5618c3558f335c08eb194f23d0e234abdd4) * Bump org.assertj:assertj-core from 3.25.3 to 3.26.3 (#881) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Michał Bobowski (cherry picked from commit adb896c581d5a7fc07d566d61b3596bcf59e1dd3) * Bump io.dropwizard.metrics:metrics-core from 4.2.3 to 4.2.26 (#863) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Michał Bobowski (cherry picked from commit 258ed2d2a796b64467dca5e73dba37477585f44e) * NO-SNOW Ignore OAuth test (#906) (cherry picked from commit 30d2e8b744874fb8faa41aee03bf27ca8db3e15f) * SNOW-1229108 Add warning about buffer config when single buffer enabled (#896) (cherry picked from commit 9aa0b0bbfa51d0aa74d41cad3f79355a04202a37) * NO_SNOW Update dependencies (#898) Co-authored-by: Michał Bobowski (cherry picked from commit eff23e21856e8f26b9d612f2b72a93b5b23a3603) * SNOW-1618257 - upgrade to 2.2.0 ingest-sdk (#910) (cherry picked from commit d0e97fbdf85565e01add6b31b80be3a9c8dcd645) * NO-SNOW Release v.2.4.0 (#907) Co-authored-by: Xin Huang (cherry picked from commit 05c1148423a4c0bdfd1af5637a5ff99b9f9f406e) --------- Co-authored-by: Greg Jachimko Co-authored-by: Adrian Kowalczyk <85181931+sfc-gh-akowalczyk@users.noreply.github.com> Co-authored-by: Michał Bobowski <145468486+sfc-gh-mbobowski@users.noreply.github.com> Co-authored-by: Xin Huang Co-authored-by: Wojciech Trefon Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/End2EndTestApache.yml | 6 +- .github/workflows/End2EndTestConfluent.yml | 6 +- .github/workflows/IntegrationTest.yml | 4 +- .github/workflows/StressTest.yml | 6 +- pom.xml | 39 +- pom_confluent.xml | 33 +- .../connector/ConnectorConfigValidator.java | 14 + .../DefaultConnectorConfigValidator.java | 280 ++++++++++++++ .../connector/SnowflakeSinkConnector.java | 15 +- .../com/snowflake/kafka/connector/Utils.java | 261 +------------ .../SnowflakeConnectionServiceV1.java | 24 +- .../BufferedTopicPartitionChannel.java | 51 ++- .../DefaultStreamingConfigValidator.java | 220 +++++++++++ .../DirectTopicPartitionChannel.java | 111 ++++-- .../streaming/SnowflakeSinkServiceV2.java | 13 +- .../streaming/StreamingConfigValidator.java | 14 + .../internal/streaming/StreamingUtils.java | 172 --------- .../telemetry/SnowflakeTelemetryService.java | 22 +- .../ConnectorConfigValidatorLogsTest.java | 86 +++++ ...java => ConnectorConfigValidatorTest.java} | 181 ++++----- .../kafka/connector/ConnectorIT.java | 5 + .../kafka/connector/SecurityTest.java | 46 --- .../SnowflakeSinkTaskForStreamingIT.java | 74 ++-- .../snowflake/kafka/connector/UtilsTest.java | 2 + .../internal/ConnectionServiceIT.java | 39 +- .../kafka/connector/internal/TestUtils.java | 40 ++ .../internal/TombstoneRecordIngestionIT.java | 90 +++-- .../CloseTopicPartitionChannelIT.java | 22 +- .../DirectStreamingClientHandlerTest.java | 7 +- .../streaming/SnowflakeSinkServiceV2IT.java | 306 +++++++++------ .../SnowflakeSinkServiceV2StopIT.java | 24 +- .../streaming/StreamingClientProviderIT.java | 2 + .../streaming/TopicPartitionChannelTest.java | 66 ++++ .../SnowflakeTelemetryServiceTest.java | 357 ++++++++++-------- src/test/resources/log4j.properties | 2 +- ...s_correct_schema_evolution_drop_table.json | 2 +- ...hema_evolution_multi_topic_drop_table.json | 2 +- .../test_confluent_protobuf_protobuf.py | 5 +- test/test_suites.py | 1 - upload_jar.sh | 36 ++ 40 files changed, 1598 insertions(+), 1088 deletions(-) create mode 100644 src/main/java/com/snowflake/kafka/connector/ConnectorConfigValidator.java create mode 100644 src/main/java/com/snowflake/kafka/connector/DefaultConnectorConfigValidator.java create mode 100644 src/main/java/com/snowflake/kafka/connector/internal/streaming/DefaultStreamingConfigValidator.java create mode 100644 src/main/java/com/snowflake/kafka/connector/internal/streaming/StreamingConfigValidator.java create mode 100644 src/test/java/com/snowflake/kafka/connector/ConnectorConfigValidatorLogsTest.java rename src/test/java/com/snowflake/kafka/connector/{ConnectorConfigTest.java => ConnectorConfigValidatorTest.java} (86%) delete mode 100644 src/test/java/com/snowflake/kafka/connector/SecurityTest.java create mode 100755 upload_jar.sh diff --git a/.github/workflows/End2EndTestApache.yml b/.github/workflows/End2EndTestApache.yml index 8cb05ac06..3888baa1a 100644 --- a/.github/workflows/End2EndTestApache.yml +++ b/.github/workflows/End2EndTestApache.yml @@ -43,9 +43,9 @@ jobs: run: | pip3 install --upgrade setuptools sudo apt-get install librdkafka-dev - pip3 install requests certifi "confluent-kafka[avro,json,protobuf]==1.7.0" - pip3 install avro-python3 kafka-python - pip3 install --upgrade protobuf==3.20.0 + pip3 install requests certifi "confluent-kafka[avro,json,protobuf]==1.9.2" + pip3 install avro kafka-python + pip3 install --upgrade protobuf==3.20.3 pip3 install --upgrade snowflake-connector-python==2.7.4 curl https://raw.githubusercontent.com/helm/helm/master/scripts/get-helm-3 | bash sudo apt-get -y install jq vim diff --git a/.github/workflows/End2EndTestConfluent.yml b/.github/workflows/End2EndTestConfluent.yml index 4926cb3aa..01e25a852 100644 --- a/.github/workflows/End2EndTestConfluent.yml +++ b/.github/workflows/End2EndTestConfluent.yml @@ -43,9 +43,9 @@ jobs: run: | pip3 install --upgrade setuptools sudo apt-get install librdkafka-dev - pip3 install requests certifi "confluent-kafka[avro,json,protobuf]==1.7.0" - pip3 install avro-python3 kafka-python - pip3 install --upgrade protobuf==3.20.0 + pip3 install requests certifi "confluent-kafka[avro,json,protobuf]==1.9.2" + pip3 install avro kafka-python + pip3 install --upgrade protobuf==3.20.3 pip3 install --upgrade snowflake-connector-python==2.7.4 curl https://raw.githubusercontent.com/helm/helm/master/scripts/get-helm-3 | bash sudo apt-get -y install jq vim diff --git a/.github/workflows/IntegrationTest.yml b/.github/workflows/IntegrationTest.yml index c2fef45e4..693112009 100644 --- a/.github/workflows/IntegrationTest.yml +++ b/.github/workflows/IntegrationTest.yml @@ -39,8 +39,8 @@ jobs: - name: Install Dependency run: | pip3 install --upgrade setuptools - pip3 install requests certifi "confluent-kafka[avro,json,protobuf]==1.7.0" - pip3 install avro-python3 kafka-python + pip3 install requests certifi "confluent-kafka[avro,json,protobuf]==1.9.2" + pip3 install avro kafka-python pip3 install protobuf pip3 install --upgrade snowflake-connector-python==2.7.4 curl https://raw.githubusercontent.com/helm/helm/master/scripts/get-helm-3 | bash diff --git a/.github/workflows/StressTest.yml b/.github/workflows/StressTest.yml index 1bd7a0fcc..81935f380 100644 --- a/.github/workflows/StressTest.yml +++ b/.github/workflows/StressTest.yml @@ -36,9 +36,9 @@ jobs: run: | pip3 install --upgrade setuptools sudo apt-get install librdkafka-dev - pip3 install requests certifi "confluent-kafka[avro,json,protobuf]==1.7.0" - pip3 install avro-python3 kafka-python - pip3 install --upgrade protobuf==3.20.0 + pip3 install requests certifi "confluent-kafka[avro,json,protobuf]==1.9.2" + pip3 install avro kafka-python + pip3 install --upgrade protobuf==3.20.3 pip3 install --upgrade snowflake-connector-python==2.7.4 curl https://raw.githubusercontent.com/helm/helm/master/scripts/get-helm-3 | bash sudo apt-get -y install jq diff --git a/pom.xml b/pom.xml index d0ae62b1d..2521ce5c1 100644 --- a/pom.xml +++ b/pom.xml @@ -12,7 +12,7 @@ com.snowflake snowflake-kafka-connector - 2.3.0 + 2.4.0 jar Snowflake Kafka Connector Snowflake Kafka Connect Sink Connector @@ -52,8 +52,10 @@ 3.7.0 4.0.0 - 3.25.3 - 7.6.0 + 3.26.3 + 7.7.0 + + 3.24.4 @@ -147,7 +149,7 @@ org.sonatype.plugins nexus-staging-maven-plugin - 1.6.7 + 1.7.0 true ossrh @@ -352,14 +354,14 @@ net.snowflake snowflake-jdbc - 3.14.5 + 3.18.0 net.snowflake snowflake-ingest-sdk - 2.1.1 + 2.2.0 net.snowflake @@ -398,18 +400,18 @@ commons-compress 1.26.0 - + com.fasterxml.jackson.core jackson-core - 2.16.1 - + 2.17.2 + com.fasterxml.jackson.core jackson-databind - 2.16.1 + 2.17.2 @@ -441,7 +443,6 @@ io.confluent kafka-avro-serializer ${confluent.version} - provided io.confluent @@ -454,7 +455,6 @@ io.confluent kafka-connect-avro-converter ${confluent.version} - provided @@ -473,7 +473,7 @@ io.dropwizard.metrics metrics-core - 4.2.3 + 4.2.26 @@ -488,11 +488,22 @@ 32.0.1-jre + + com.google.protobuf + protobuf-java + ${protobuf.version} + + + com.google.protobuf + protobuf-java-util + ${protobuf.version} + + dev.failsafe failsafe - 3.2.1 + 3.3.2 diff --git a/pom_confluent.xml b/pom_confluent.xml index afd2164f1..29779dd36 100644 --- a/pom_confluent.xml +++ b/pom_confluent.xml @@ -12,7 +12,7 @@ com.snowflake snowflake-kafka-connector - 2.3.0 + 2.4.0 jar Snowflake Kafka Connector Snowflake Kafka Connect Sink Connector @@ -62,8 +62,10 @@ 3.7.0 4.0.0 - 3.25.3 - 7.6.0 + 3.26.3 + 7.7.0 + + 3.24.4 @@ -305,7 +307,7 @@ org.sonatype.plugins nexus-staging-maven-plugin - 1.6.7 + 1.7.0 true ossrh @@ -482,14 +484,25 @@ net.snowflake snowflake-jdbc - 3.14.5 + 3.18.0 + + + + com.google.protobuf + protobuf-java + ${protobuf.version} + + + com.google.protobuf + protobuf-java-util + ${protobuf.version} net.snowflake snowflake-ingest-sdk - 2.1.1 + 2.2.0 net.snowflake @@ -549,13 +562,13 @@ com.fasterxml.jackson.core jackson-core - 2.16.1 + 2.17.2 com.fasterxml.jackson.core jackson-databind - 2.16.1 + 2.17.2 @@ -619,7 +632,7 @@ io.dropwizard.metrics metrics-core - 4.2.3 + 4.2.26 @@ -638,7 +651,7 @@ dev.failsafe failsafe - 3.2.1 + 3.3.2 diff --git a/src/main/java/com/snowflake/kafka/connector/ConnectorConfigValidator.java b/src/main/java/com/snowflake/kafka/connector/ConnectorConfigValidator.java new file mode 100644 index 000000000..9b882ddde --- /dev/null +++ b/src/main/java/com/snowflake/kafka/connector/ConnectorConfigValidator.java @@ -0,0 +1,14 @@ +package com.snowflake.kafka.connector; + +import java.util.Map; + +public interface ConnectorConfigValidator { + + /** + * Validate input configuration + * + * @param config configuration Map + * @return connector name + */ + String validateConfig(Map config); +} diff --git a/src/main/java/com/snowflake/kafka/connector/DefaultConnectorConfigValidator.java b/src/main/java/com/snowflake/kafka/connector/DefaultConnectorConfigValidator.java new file mode 100644 index 000000000..c40fb4277 --- /dev/null +++ b/src/main/java/com/snowflake/kafka/connector/DefaultConnectorConfigValidator.java @@ -0,0 +1,280 @@ +package com.snowflake.kafka.connector; + +import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG; +import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.BehaviorOnNullValues.VALIDATOR; +import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.JMX_OPT; +import static com.snowflake.kafka.connector.Utils.*; + +import com.google.common.collect.ImmutableMap; +import com.snowflake.kafka.connector.internal.BufferThreshold; +import com.snowflake.kafka.connector.internal.KCLogger; +import com.snowflake.kafka.connector.internal.SnowflakeErrors; +import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; +import com.snowflake.kafka.connector.internal.streaming.StreamingConfigValidator; +import java.util.HashMap; +import java.util.Map; +import org.apache.kafka.common.config.ConfigException; + +public class DefaultConnectorConfigValidator implements ConnectorConfigValidator { + + private static final KCLogger LOGGER = + new KCLogger(DefaultConnectorConfigValidator.class.getName()); + + private final StreamingConfigValidator streamingConfigValidator; + + public DefaultConnectorConfigValidator(StreamingConfigValidator streamingConfigValidator) { + this.streamingConfigValidator = streamingConfigValidator; + } + + /** + * Validate input configuration + * + * @param config configuration Map + * @return connector name + */ + public String validateConfig(Map config) { + Map invalidConfigParams = new HashMap(); + + // define the input parameters / keys in one place as static constants, + // instead of using them directly + // define the thresholds statically in one place as static constants, + // instead of using the values directly + + // unique name of this connector instance + String connectorName = config.getOrDefault(SnowflakeSinkConnectorConfig.NAME, ""); + if (connectorName.isEmpty() || !isValidSnowflakeApplicationName(connectorName)) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.NAME, + Utils.formatString( + "{} is empty or invalid. It should match Snowflake object identifier syntax. Please" + + " see the documentation.", + SnowflakeSinkConnectorConfig.NAME)); + } + + // If config doesnt have ingestion method defined, default is snowpipe or if snowpipe is + // explicitly passed in as ingestion method + // Below checks are just for snowpipe. + if (isSnowpipeIngestion(config)) { + invalidConfigParams.putAll( + BufferThreshold.validateBufferThreshold(config, IngestionMethodConfig.SNOWPIPE)); + + if (config.containsKey(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG) + && Boolean.parseBoolean( + config.get(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG))) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, + Utils.formatString( + "Schematization is only available with {}.", + IngestionMethodConfig.SNOWPIPE_STREAMING.toString())); + } + if (config.containsKey(SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_MAX_CLIENT_LAG)) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_MAX_CLIENT_LAG, + Utils.formatString( + "{} is only available with ingestion type: {}.", + SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_MAX_CLIENT_LAG, + IngestionMethodConfig.SNOWPIPE_STREAMING.toString())); + } + if (config.containsKey( + SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_MAX_MEMORY_LIMIT_IN_BYTES)) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_MAX_MEMORY_LIMIT_IN_BYTES, + Utils.formatString( + "{} is only available with ingestion type: {}.", + SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_MAX_MEMORY_LIMIT_IN_BYTES, + IngestionMethodConfig.SNOWPIPE_STREAMING.toString())); + } + if (config.containsKey( + SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER)) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER, + Utils.formatString( + "{} is only available with ingestion type: {}.", + SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER, + IngestionMethodConfig.SNOWPIPE_STREAMING.toString())); + } + if (config.containsKey( + SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_CLIENT_PROVIDER_OVERRIDE_MAP)) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_CLIENT_PROVIDER_OVERRIDE_MAP, + Utils.formatString( + "{} is only available with ingestion type: {}.", + SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_CLIENT_PROVIDER_OVERRIDE_MAP, + IngestionMethodConfig.SNOWPIPE_STREAMING.toString())); + } + if (config.containsKey( + SnowflakeSinkConnectorConfig.ENABLE_STREAMING_CLIENT_OPTIMIZATION_CONFIG) + && Boolean.parseBoolean( + config.get( + SnowflakeSinkConnectorConfig.ENABLE_STREAMING_CLIENT_OPTIMIZATION_CONFIG))) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.ENABLE_STREAMING_CLIENT_OPTIMIZATION_CONFIG, + Utils.formatString( + "Streaming client optimization is only available with {}.", + IngestionMethodConfig.SNOWPIPE_STREAMING.toString())); + } + if (config.containsKey( + SnowflakeSinkConnectorConfig.ENABLE_CHANNEL_OFFSET_TOKEN_MIGRATION_CONFIG)) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.ENABLE_CHANNEL_OFFSET_TOKEN_MIGRATION_CONFIG, + Utils.formatString( + "Streaming client Channel migration is only available with {}.", + IngestionMethodConfig.SNOWPIPE_STREAMING.toString())); + } + } + + if (config.containsKey(SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP) + && parseTopicToTableMap(config.get(SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP)) + == null) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP, + Utils.formatString( + "Invalid {} config format: {}", + SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP, + config.get(SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP))); + } + + // sanity check + if (!config.containsKey(SnowflakeSinkConnectorConfig.SNOWFLAKE_DATABASE)) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.SNOWFLAKE_DATABASE, + Utils.formatString( + "{} cannot be empty.", SnowflakeSinkConnectorConfig.SNOWFLAKE_DATABASE)); + } + + // sanity check + if (!config.containsKey(SnowflakeSinkConnectorConfig.SNOWFLAKE_SCHEMA)) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.SNOWFLAKE_SCHEMA, + Utils.formatString("{} cannot be empty.", SnowflakeSinkConnectorConfig.SNOWFLAKE_SCHEMA)); + } + + switch (config + .getOrDefault(SnowflakeSinkConnectorConfig.AUTHENTICATOR_TYPE, Utils.SNOWFLAKE_JWT) + .toLowerCase()) { + // TODO: SNOW-889748 change to enum + case Utils.SNOWFLAKE_JWT: + if (!config.containsKey(SnowflakeSinkConnectorConfig.SNOWFLAKE_PRIVATE_KEY)) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.SNOWFLAKE_PRIVATE_KEY, + Utils.formatString( + "{} cannot be empty when using {} authenticator.", + SnowflakeSinkConnectorConfig.SNOWFLAKE_PRIVATE_KEY, + Utils.SNOWFLAKE_JWT)); + } + break; + case Utils.OAUTH: + if (!config.containsKey(SnowflakeSinkConnectorConfig.OAUTH_CLIENT_ID)) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.OAUTH_CLIENT_ID, + Utils.formatString( + "{} cannot be empty when using {} authenticator.", + SnowflakeSinkConnectorConfig.OAUTH_CLIENT_ID, + Utils.OAUTH)); + } + if (!config.containsKey(SnowflakeSinkConnectorConfig.OAUTH_CLIENT_SECRET)) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.OAUTH_CLIENT_SECRET, + Utils.formatString( + "{} cannot be empty when using {} authenticator.", + SnowflakeSinkConnectorConfig.OAUTH_CLIENT_SECRET, + Utils.OAUTH)); + } + if (!config.containsKey(SnowflakeSinkConnectorConfig.OAUTH_REFRESH_TOKEN)) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.OAUTH_REFRESH_TOKEN, + Utils.formatString( + "{} cannot be empty when using {} authenticator.", + SnowflakeSinkConnectorConfig.OAUTH_REFRESH_TOKEN, + Utils.OAUTH)); + } + break; + default: + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.AUTHENTICATOR_TYPE, + Utils.formatString( + "{} should be one of {} or {}.", + SnowflakeSinkConnectorConfig.AUTHENTICATOR_TYPE, + Utils.SNOWFLAKE_JWT, + Utils.OAUTH)); + } + + if (!config.containsKey(SnowflakeSinkConnectorConfig.SNOWFLAKE_USER)) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.SNOWFLAKE_USER, + Utils.formatString("{} cannot be empty.", SnowflakeSinkConnectorConfig.SNOWFLAKE_USER)); + } + + if (!config.containsKey(SnowflakeSinkConnectorConfig.SNOWFLAKE_URL)) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.SNOWFLAKE_URL, + Utils.formatString("{} cannot be empty.", SnowflakeSinkConnectorConfig.SNOWFLAKE_URL)); + } + // jvm proxy settings + invalidConfigParams.putAll(validateProxySettings(config)); + + // set jdbc logging directory + Utils.setJDBCLoggingDirectory(); + + // validate whether kafka provider config is a valid value + if (config.containsKey(SnowflakeSinkConnectorConfig.PROVIDER_CONFIG)) { + try { + SnowflakeSinkConnectorConfig.KafkaProvider.of( + config.get(SnowflakeSinkConnectorConfig.PROVIDER_CONFIG)); + } catch (IllegalArgumentException exception) { + invalidConfigParams.put( + SnowflakeSinkConnectorConfig.PROVIDER_CONFIG, + Utils.formatString("Kafka provider config error:{}", exception.getMessage())); + } + } + + if (config.containsKey(BEHAVIOR_ON_NULL_VALUES_CONFIG)) { + try { + // This throws an exception if config value is invalid. + VALIDATOR.ensureValid( + BEHAVIOR_ON_NULL_VALUES_CONFIG, config.get(BEHAVIOR_ON_NULL_VALUES_CONFIG)); + } catch (ConfigException exception) { + invalidConfigParams.put( + BEHAVIOR_ON_NULL_VALUES_CONFIG, + Utils.formatString( + "Kafka config:{} error:{}", + BEHAVIOR_ON_NULL_VALUES_CONFIG, + exception.getMessage())); + } + } + + if (config.containsKey(JMX_OPT)) { + if (!(config.get(JMX_OPT).equalsIgnoreCase("true") + || config.get(JMX_OPT).equalsIgnoreCase("false"))) { + invalidConfigParams.put( + JMX_OPT, Utils.formatString("Kafka config:{} should either be true or false", JMX_OPT)); + } + } + + // Check all config values for ingestion method == IngestionMethodConfig.SNOWPIPE_STREAMING + invalidConfigParams.putAll(streamingConfigValidator.validate(config)); + + // logs and throws exception if there are invalid params + handleInvalidParameters(ImmutableMap.copyOf(invalidConfigParams)); + + return connectorName; + } + + private void handleInvalidParameters(ImmutableMap invalidConfigParams) { + // log all invalid params and throw exception + if (!invalidConfigParams.isEmpty()) { + String invalidParamsMessage = ""; + + for (String invalidKey : invalidConfigParams.keySet()) { + String invalidValue = invalidConfigParams.get(invalidKey); + String errorMessage = + Utils.formatString( + "Config value '{}' is invalid. Error message: '{}'", invalidKey, invalidValue); + invalidParamsMessage += errorMessage + "\n"; + } + + LOGGER.error("Invalid config: " + invalidParamsMessage); + throw SnowflakeErrors.ERROR_0001.getException(invalidParamsMessage); + } + } +} diff --git a/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkConnector.java b/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkConnector.java index cf7e8b81f..a619a677a 100644 --- a/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkConnector.java +++ b/src/main/java/com/snowflake/kafka/connector/SnowflakeSinkConnector.java @@ -21,6 +21,7 @@ import com.snowflake.kafka.connector.internal.SnowflakeConnectionServiceFactory; import com.snowflake.kafka.connector.internal.SnowflakeErrors; import com.snowflake.kafka.connector.internal.SnowflakeKafkaConnectorException; +import com.snowflake.kafka.connector.internal.streaming.DefaultStreamingConfigValidator; import com.snowflake.kafka.connector.internal.telemetry.SnowflakeTelemetryService; import java.util.ArrayList; import java.util.HashMap; @@ -63,9 +64,8 @@ public class SnowflakeSinkConnector extends SinkConnector { // Using setupComplete to synchronize private boolean setupComplete; - private static final int VALIDATION_NETWORK_TIMEOUT_IN_MS = 45000; - - private static final int VALIDATION_LOGIN_TIMEOUT_IN_SEC = 20; + private final ConnectorConfigValidator connectorConfigValidator = + new DefaultConnectorConfigValidator(new DefaultStreamingConfigValidator()); /** No-Arg constructor. Required by Kafka Connect framework */ public SnowflakeSinkConnector() { @@ -95,7 +95,7 @@ public void start(final Map parsedConfig) { // modify invalid connector name Utils.convertAppName(config); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); // enable mdc logging if needed KCLogger.toggleGlobalMdcLoggingContext( @@ -226,12 +226,7 @@ public Config validate(Map connectorConfigs) { SnowflakeConnectionService testConnection; try { testConnection = - SnowflakeConnectionServiceFactory.builder() - .setNetworkTimeout(VALIDATION_NETWORK_TIMEOUT_IN_MS) - .setLoginTimeOut(VALIDATION_LOGIN_TIMEOUT_IN_SEC) - .setProperties(connectorConfigs) - .build(); - + SnowflakeConnectionServiceFactory.builder().setProperties(connectorConfigs).build(); } catch (SnowflakeKafkaConnectorException e) { LOGGER.error( "Validate: Error connecting to snowflake:{}, errorCode:{}", e.getMessage(), e.getCode()); diff --git a/src/main/java/com/snowflake/kafka/connector/Utils.java b/src/main/java/com/snowflake/kafka/connector/Utils.java index 7cf9db56d..c0d021902 100644 --- a/src/main/java/com/snowflake/kafka/connector/Utils.java +++ b/src/main/java/com/snowflake/kafka/connector/Utils.java @@ -16,13 +16,9 @@ */ package com.snowflake.kafka.connector; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.BehaviorOnNullValues.VALIDATOR; import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.JMX_OPT; import com.google.common.collect.ImmutableMap; -import com.snowflake.kafka.connector.internal.BufferThreshold; import com.snowflake.kafka.connector.internal.InternalUtils; import com.snowflake.kafka.connector.internal.KCLogger; import com.snowflake.kafka.connector.internal.OAuthConstants; @@ -30,7 +26,6 @@ import com.snowflake.kafka.connector.internal.SnowflakeInternalOperations; import com.snowflake.kafka.connector.internal.SnowflakeURL; import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; -import com.snowflake.kafka.connector.internal.streaming.StreamingUtils; import java.io.BufferedReader; import java.io.File; import java.io.InputStream; @@ -64,14 +59,13 @@ import net.snowflake.client.jdbc.internal.google.gson.JsonObject; import net.snowflake.client.jdbc.internal.google.gson.JsonParser; import org.apache.kafka.common.config.Config; -import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.ConfigValue; /** Various arbitrary helper functions */ public class Utils { // Connector version, change every release - public static final String VERSION = "2.3.0"; + public static final String VERSION = "2.4.0"; // connector parameter list public static final String NAME = "name"; @@ -391,241 +385,6 @@ static boolean isValidSnowflakeTableName(String tableName) { return tableName.matches("^([_a-zA-Z]{1}[_$a-zA-Z0-9]+\\.){0,2}[_a-zA-Z]{1}[_$a-zA-Z0-9]+$"); } - /** - * Validate input configuration - * - * @param config configuration Map - * @return connector name - */ - static String validateConfig(Map config) { - Map invalidConfigParams = new HashMap(); // verify all config - - // define the input parameters / keys in one place as static constants, - // instead of using them directly - // define the thresholds statically in one place as static constants, - // instead of using the values directly - - // unique name of this connector instance - String connectorName = config.getOrDefault(SnowflakeSinkConnectorConfig.NAME, ""); - if (connectorName.isEmpty() || !isValidSnowflakeApplicationName(connectorName)) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.NAME, - Utils.formatString( - "{} is empty or invalid. It should match Snowflake object identifier syntax. Please" - + " see the documentation.", - SnowflakeSinkConnectorConfig.NAME)); - } - - // If config doesnt have ingestion method defined, default is snowpipe or if snowpipe is - // explicitly passed in as ingestion method - // Below checks are just for snowpipe. - if (isSnowpipeIngestion(config)) { - invalidConfigParams.putAll( - BufferThreshold.validateBufferThreshold(config, IngestionMethodConfig.SNOWPIPE)); - - if (config.containsKey(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG) - && Boolean.parseBoolean( - config.get(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG))) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, - Utils.formatString( - "Schematization is only available with {}.", - IngestionMethodConfig.SNOWPIPE_STREAMING.toString())); - } - if (config.containsKey(SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_MAX_CLIENT_LAG)) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_MAX_CLIENT_LAG, - Utils.formatString( - "{} is only available with ingestion type: {}.", - SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_MAX_CLIENT_LAG, - IngestionMethodConfig.SNOWPIPE_STREAMING.toString())); - } - if (config.containsKey( - SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_MAX_MEMORY_LIMIT_IN_BYTES)) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_MAX_MEMORY_LIMIT_IN_BYTES, - Utils.formatString( - "{} is only available with ingestion type: {}.", - SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_MAX_MEMORY_LIMIT_IN_BYTES, - IngestionMethodConfig.SNOWPIPE_STREAMING.toString())); - } - if (config.containsKey( - SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER)) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER, - Utils.formatString( - "{} is only available with ingestion type: {}.", - SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER, - IngestionMethodConfig.SNOWPIPE_STREAMING.toString())); - } - if (config.containsKey( - SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_CLIENT_PROVIDER_OVERRIDE_MAP)) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_CLIENT_PROVIDER_OVERRIDE_MAP, - Utils.formatString( - "{} is only available with ingestion type: {}.", - SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_CLIENT_PROVIDER_OVERRIDE_MAP, - IngestionMethodConfig.SNOWPIPE_STREAMING.toString())); - } - if (config.containsKey( - SnowflakeSinkConnectorConfig.ENABLE_STREAMING_CLIENT_OPTIMIZATION_CONFIG) - && Boolean.parseBoolean( - config.get( - SnowflakeSinkConnectorConfig.ENABLE_STREAMING_CLIENT_OPTIMIZATION_CONFIG))) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.ENABLE_STREAMING_CLIENT_OPTIMIZATION_CONFIG, - Utils.formatString( - "Streaming client optimization is only available with {}.", - IngestionMethodConfig.SNOWPIPE_STREAMING.toString())); - } - if (config.containsKey( - SnowflakeSinkConnectorConfig.ENABLE_CHANNEL_OFFSET_TOKEN_MIGRATION_CONFIG)) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.ENABLE_CHANNEL_OFFSET_TOKEN_MIGRATION_CONFIG, - Utils.formatString( - "Streaming client Channel migration is only available with {}.", - IngestionMethodConfig.SNOWPIPE_STREAMING.toString())); - } - } - - if (config.containsKey(SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP) - && !config.get(SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP).isEmpty() - && parseTopicToTableMap(config.get(SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP)) - == null) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP, - Utils.formatString( - "Invalid {} config format: {}", - SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP, - config.get(SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP))); - } - - // sanity check - if (!config.containsKey(SnowflakeSinkConnectorConfig.SNOWFLAKE_DATABASE)) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.SNOWFLAKE_DATABASE, - Utils.formatString( - "{} cannot be empty.", SnowflakeSinkConnectorConfig.SNOWFLAKE_DATABASE)); - } - - // sanity check - if (!config.containsKey(SnowflakeSinkConnectorConfig.SNOWFLAKE_SCHEMA)) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.SNOWFLAKE_SCHEMA, - Utils.formatString("{} cannot be empty.", SnowflakeSinkConnectorConfig.SNOWFLAKE_SCHEMA)); - } - - switch (config - .getOrDefault(SnowflakeSinkConnectorConfig.AUTHENTICATOR_TYPE, Utils.SNOWFLAKE_JWT) - .toLowerCase()) { - // TODO: SNOW-889748 change to enum - case Utils.SNOWFLAKE_JWT: - if (!config.containsKey(SnowflakeSinkConnectorConfig.SNOWFLAKE_PRIVATE_KEY)) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.SNOWFLAKE_PRIVATE_KEY, - Utils.formatString( - "{} cannot be empty when using {} authenticator.", - SnowflakeSinkConnectorConfig.SNOWFLAKE_PRIVATE_KEY, - Utils.SNOWFLAKE_JWT)); - } - break; - case Utils.OAUTH: - if (!config.containsKey(SnowflakeSinkConnectorConfig.OAUTH_CLIENT_ID)) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.OAUTH_CLIENT_ID, - Utils.formatString( - "{} cannot be empty when using {} authenticator.", - SnowflakeSinkConnectorConfig.OAUTH_CLIENT_ID, - Utils.OAUTH)); - } - if (!config.containsKey(SnowflakeSinkConnectorConfig.OAUTH_CLIENT_SECRET)) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.OAUTH_CLIENT_SECRET, - Utils.formatString( - "{} cannot be empty when using {} authenticator.", - SnowflakeSinkConnectorConfig.OAUTH_CLIENT_SECRET, - Utils.OAUTH)); - } - if (!config.containsKey(SnowflakeSinkConnectorConfig.OAUTH_REFRESH_TOKEN)) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.OAUTH_REFRESH_TOKEN, - Utils.formatString( - "{} cannot be empty when using {} authenticator.", - SnowflakeSinkConnectorConfig.OAUTH_REFRESH_TOKEN, - Utils.OAUTH)); - } - break; - default: - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.AUTHENTICATOR_TYPE, - Utils.formatString( - "{} should be one of {} or {}.", - SnowflakeSinkConnectorConfig.AUTHENTICATOR_TYPE, - Utils.SNOWFLAKE_JWT, - Utils.OAUTH)); - } - - if (!config.containsKey(SnowflakeSinkConnectorConfig.SNOWFLAKE_USER)) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.SNOWFLAKE_USER, - Utils.formatString("{} cannot be empty.", SnowflakeSinkConnectorConfig.SNOWFLAKE_USER)); - } - - if (!config.containsKey(SnowflakeSinkConnectorConfig.SNOWFLAKE_URL)) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.SNOWFLAKE_URL, - Utils.formatString("{} cannot be empty.", SnowflakeSinkConnectorConfig.SNOWFLAKE_URL)); - } - // jvm proxy settings - invalidConfigParams.putAll(validateProxySettings(config)); - - // set jdbc logging directory - Utils.setJDBCLoggingDirectory(); - - // validate whether kafka provider config is a valid value - if (config.containsKey(SnowflakeSinkConnectorConfig.PROVIDER_CONFIG)) { - try { - SnowflakeSinkConnectorConfig.KafkaProvider.of( - config.get(SnowflakeSinkConnectorConfig.PROVIDER_CONFIG)); - } catch (IllegalArgumentException exception) { - invalidConfigParams.put( - SnowflakeSinkConnectorConfig.PROVIDER_CONFIG, - Utils.formatString("Kafka provider config error:{}", exception.getMessage())); - } - } - - if (config.containsKey(BEHAVIOR_ON_NULL_VALUES_CONFIG)) { - try { - // This throws an exception if config value is invalid. - VALIDATOR.ensureValid( - BEHAVIOR_ON_NULL_VALUES_CONFIG, config.get(BEHAVIOR_ON_NULL_VALUES_CONFIG)); - } catch (ConfigException exception) { - invalidConfigParams.put( - BEHAVIOR_ON_NULL_VALUES_CONFIG, - Utils.formatString( - "Kafka config:{} error:{}", - BEHAVIOR_ON_NULL_VALUES_CONFIG, - exception.getMessage())); - } - } - - if (config.containsKey(JMX_OPT)) { - if (!(config.get(JMX_OPT).equalsIgnoreCase("true") - || config.get(JMX_OPT).equalsIgnoreCase("false"))) { - invalidConfigParams.put( - JMX_OPT, Utils.formatString("Kafka config:{} should either be true or false", JMX_OPT)); - } - } - - // Check all config values for ingestion method == IngestionMethodConfig.SNOWPIPE_STREAMING - invalidConfigParams.putAll(StreamingUtils.validateStreamingSnowpipeConfig(config)); - - // logs and throws exception if there are invalid params - handleInvalidParameters(ImmutableMap.copyOf(invalidConfigParams)); - - return connectorName; - } - /** * Returns whether INGESTION_METHOD_OPT is set to SNOWPIPE. If INGESTION_METHOD_OPT not specified, * returns true as default. @@ -1014,22 +773,4 @@ public static String getExceptionMessage(String customMessage, Exception ex) { return formatString(GET_EXCEPTION_FORMAT, customMessage, message, cause); } - - private static void handleInvalidParameters(ImmutableMap invalidConfigParams) { - // log all invalid params and throw exception - if (!invalidConfigParams.isEmpty()) { - String invalidParamsMessage = ""; - - for (String invalidKey : invalidConfigParams.keySet()) { - String invalidValue = invalidConfigParams.get(invalidKey); - String errorMessage = - Utils.formatString( - "Config value '{}' is invalid. Error message: '{}'", invalidKey, invalidValue); - invalidParamsMessage += errorMessage + "\n"; - } - - LOGGER.error("Invalid config: " + invalidParamsMessage); - throw SnowflakeErrors.ERROR_0001.getException(invalidParamsMessage); - } - } } diff --git a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeConnectionServiceV1.java b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeConnectionServiceV1.java index cf53840a9..6bedd93e0 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeConnectionServiceV1.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/SnowflakeConnectionServiceV1.java @@ -351,7 +351,7 @@ public boolean isTableCompatible(final String tableName) { } compatible = hasMeta && hasContent && allNullable; } catch (SQLException e) { - LOGGER.debug("table {} doesn't exist", tableName); + LOGGER.debug("Table {} doesn't exist. Exception {}", tableName, e.getStackTrace()); compatible = false; } finally { try { @@ -370,6 +370,7 @@ public boolean isTableCompatible(final String tableName) { e.printStackTrace(); } } + LOGGER.info("Table {} compatibility is {}", tableName, compatible); return compatible; } @@ -425,6 +426,7 @@ public void appendMetaColIfNotExist(final String tableName) { */ @Override public boolean hasSchemaEvolutionPermission(String tableName, String role) { + LOGGER.info("Checking schema evolution permission for table {}", tableName); checkConnection(); InternalUtils.assertNotEmpty("tableName", tableName); String query = "show grants on table identifier(?)"; @@ -576,18 +578,22 @@ public void alterNonNullableColumns(String tableName, List columnNames) public boolean isStageCompatible(final String stageName) { checkConnection(); InternalUtils.assertNotEmpty("stageName", stageName); + boolean isCompatible = true; + if (!stageExist(stageName)) { LOGGER.debug("stage {} doesn't exists", stageName); - return false; - } - List files = listStage(stageName, ""); - for (String name : files) { - if (!FileNameUtils.verifyFileName(name)) { - LOGGER.debug("file name {} in stage {} is not valid", name, stageName); - return false; + isCompatible = false; + } else { + List files = listStage(stageName, ""); + for (String name : files) { + if (!FileNameUtils.verifyFileName(name)) { + LOGGER.info("file name {} in stage {} is not valid", name, stageName); + isCompatible = false; + } } } - return true; + LOGGER.info("Stage {} compatibility is {}", stageName, isCompatible); + return isCompatible; } @Override diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/BufferedTopicPartitionChannel.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/BufferedTopicPartitionChannel.java index d6b2b8213..47d36274c 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/streaming/BufferedTopicPartitionChannel.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/BufferedTopicPartitionChannel.java @@ -402,13 +402,13 @@ public void insertRecord(SinkRecord kafkaSinkRecord, boolean isFirstRowPerPartit insertRecords(copiedStreamingBuffer); } } else { - LOGGER.debug( - "Skip adding offset:{} to buffer for channel:{} because" - + " offsetPersistedInSnowflake:{}, processedOffset:{}", - kafkaSinkRecord.kafkaOffset(), + LOGGER.warn( + "Channel {} - skipping current record - expected offset {} but received {}. The current" + + " offset stored in Snowflake: {}", this.getChannelNameFormatV1(), - currentOffsetPersistedInSnowflake, - currentProcessedOffset); + currentProcessedOffset, + kafkaSinkRecord.kafkaOffset(), + currentOffsetPersistedInSnowflake); } } @@ -669,6 +669,10 @@ public InsertRowsResponse get() throws Throwable { // Simply added to the final response if it's not schema related errors finalResponse.addError(insertError); } else { + LOGGER.info( + "Triggering schema evolution. NonNullableColumns={}, extraColumns={}", + String.join(",", nonNullableColumns), + extraColNames == null ? "null" : String.join(",", extraColNames)); SchematizationUtils.evolveSchemaIfNeeded( this.conn, this.channel.getTableName(), @@ -853,9 +857,17 @@ public long fetchOffsetTokenWithRetry() { */ private long streamingApiFallbackSupplier( final StreamingApiFallbackInvoker streamingApiFallbackInvoker) { - final long offsetRecoveredFromSnowflake = - getRecoveredOffsetFromSnowflake(streamingApiFallbackInvoker); - resetChannelMetadataAfterRecovery(streamingApiFallbackInvoker, offsetRecoveredFromSnowflake); + SnowflakeStreamingIngestChannel newChannel = reopenChannel(streamingApiFallbackInvoker); + + LOGGER.warn( + "{} Fetching offsetToken after re-opening the channel:{}", + streamingApiFallbackInvoker, + this.getChannelNameFormatV1()); + long offsetRecoveredFromSnowflake = fetchLatestOffsetFromChannel(newChannel); + + resetChannelMetadataAfterRecovery( + streamingApiFallbackInvoker, offsetRecoveredFromSnowflake, newChannel); + return offsetRecoveredFromSnowflake; } @@ -871,10 +883,12 @@ private long streamingApiFallbackSupplier( * for logging mainly. * @param offsetRecoveredFromSnowflake offset number found in snowflake for this * channel(partition) + * @param newChannel a channel to assign to the current instance */ private void resetChannelMetadataAfterRecovery( final StreamingApiFallbackInvoker streamingApiFallbackInvoker, - final long offsetRecoveredFromSnowflake) { + final long offsetRecoveredFromSnowflake, + SnowflakeStreamingIngestChannel newChannel) { if (offsetRecoveredFromSnowflake == NO_OFFSET_TOKEN_REGISTERED_IN_SNOWFLAKE) { LOGGER.info( "{} Channel:{}, offset token is NULL, will use the consumer offset managed by the" @@ -915,6 +929,7 @@ private void resetChannelMetadataAfterRecovery( // Set the flag so that any leftover rows in the buffer should be skipped, it will be // re-ingested since the offset in kafka was reset needToSkipCurrentBatch = true; + this.channel = newChannel; } finally { this.bufferLock.unlock(); } @@ -935,16 +950,11 @@ private void resetChannelMetadataAfterRecovery( * @param streamingApiFallbackInvoker Streaming API which invoked this function. * @return offset which was last present in Snowflake */ - private long getRecoveredOffsetFromSnowflake( + private SnowflakeStreamingIngestChannel reopenChannel( final StreamingApiFallbackInvoker streamingApiFallbackInvoker) { LOGGER.warn( "{} Re-opening channel:{}", streamingApiFallbackInvoker, this.getChannelNameFormatV1()); - this.channel = Preconditions.checkNotNull(openChannelForTable()); - LOGGER.warn( - "{} Fetching offsetToken after re-opening the channel:{}", - streamingApiFallbackInvoker, - this.getChannelNameFormatV1()); - return fetchLatestCommittedOffsetFromSnowflake(); + return Preconditions.checkNotNull(openChannelForTable()); } /** @@ -958,11 +968,16 @@ private long getRecoveredOffsetFromSnowflake( * snowflake. */ private long fetchLatestCommittedOffsetFromSnowflake() { + SnowflakeStreamingIngestChannel channelToGetOffset = this.channel; + return fetchLatestOffsetFromChannel(channelToGetOffset); + } + + private long fetchLatestOffsetFromChannel(SnowflakeStreamingIngestChannel channel) { LOGGER.debug( "Fetching last committed offset for partition channel:{}", this.getChannelNameFormatV1()); String offsetToken = null; try { - offsetToken = this.channel.getLatestCommittedOffsetToken(); + offsetToken = channel.getLatestCommittedOffsetToken(); LOGGER.info( "Fetched offsetToken for channelName:{}, offset:{}", this.getChannelNameFormatV1(), diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/DefaultStreamingConfigValidator.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/DefaultStreamingConfigValidator.java new file mode 100644 index 000000000..027a3d775 --- /dev/null +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/DefaultStreamingConfigValidator.java @@ -0,0 +1,220 @@ +package com.snowflake.kafka.connector.internal.streaming; + +import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.*; +import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT; + +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; +import com.snowflake.kafka.connector.Utils; +import com.snowflake.kafka.connector.internal.BufferThreshold; +import com.snowflake.kafka.connector.internal.KCLogger; +import com.snowflake.kafka.connector.internal.parameters.InternalBufferParameters; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.kafka.common.config.ConfigException; + +public class DefaultStreamingConfigValidator implements StreamingConfigValidator { + + private final SingleBufferConfigValidator singleBufferConfigValidator = + new SingleBufferConfigValidator(); + private final DoubleBufferConfigValidator doubleBufferConfigValidator = + new DoubleBufferConfigValidator(); + + private static final Set DISALLOWED_CONVERTERS_STREAMING = CUSTOM_SNOWFLAKE_CONVERTERS; + private static final String STRING_CONVERTER_KEYWORD = "StringConverter"; + private static final String BYTE_ARRAY_CONVERTER_KEYWORD = "ByteArrayConverter"; + + @Override + public ImmutableMap validate(Map inputConfig) { + Map invalidParams = new HashMap<>(); + + // For snowpipe_streaming, role should be non empty + if (inputConfig.containsKey(INGESTION_METHOD_OPT)) { + if (InternalBufferParameters.isSingleBufferEnabled(inputConfig)) { + singleBufferConfigValidator.logDoubleBufferingParametersWarning(inputConfig); + } else { + invalidParams.putAll(doubleBufferConfigValidator.validate(inputConfig)); + } + + try { + // This throws an exception if config value is invalid. + IngestionMethodConfig.VALIDATOR.ensureValid( + INGESTION_METHOD_OPT, inputConfig.get(INGESTION_METHOD_OPT)); + if (inputConfig + .get(INGESTION_METHOD_OPT) + .equalsIgnoreCase(IngestionMethodConfig.SNOWPIPE_STREAMING.toString())) { + invalidParams.putAll(validateConfigConverters(KEY_CONVERTER_CONFIG_FIELD, inputConfig)); + invalidParams.putAll(validateConfigConverters(VALUE_CONVERTER_CONFIG_FIELD, inputConfig)); + + // Validate if snowflake role is present + if (!inputConfig.containsKey(Utils.SF_ROLE) + || Strings.isNullOrEmpty(inputConfig.get(Utils.SF_ROLE))) { + invalidParams.put( + Utils.SF_ROLE, + Utils.formatString( + "Config:{} should be present if ingestionMethod is:{}", + Utils.SF_ROLE, + inputConfig.get(INGESTION_METHOD_OPT))); + } + + /** + * Only checking in streaming since we are utilizing the values before we send it to + * DLQ/output to log file + */ + if (inputConfig.containsKey(ERRORS_TOLERANCE_CONFIG)) { + SnowflakeSinkConnectorConfig.ErrorTolerance.VALIDATOR.ensureValid( + ERRORS_TOLERANCE_CONFIG, inputConfig.get(ERRORS_TOLERANCE_CONFIG)); + } + if (inputConfig.containsKey(ERRORS_LOG_ENABLE_CONFIG)) { + BOOLEAN_VALIDATOR.ensureValid( + ERRORS_LOG_ENABLE_CONFIG, inputConfig.get(ERRORS_LOG_ENABLE_CONFIG)); + } + if (inputConfig.containsKey(ENABLE_CHANNEL_OFFSET_TOKEN_MIGRATION_CONFIG)) { + BOOLEAN_VALIDATOR.ensureValid( + ENABLE_CHANNEL_OFFSET_TOKEN_MIGRATION_CONFIG, + inputConfig.get(ENABLE_CHANNEL_OFFSET_TOKEN_MIGRATION_CONFIG)); + } + + if (inputConfig.containsKey(SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER)) { + BOOLEAN_VALIDATOR.ensureValid( + SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER, + inputConfig.get(SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER)); + } + + if (inputConfig.containsKey(SNOWPIPE_STREAMING_MAX_CLIENT_LAG)) { + ensureValidLong(inputConfig, SNOWPIPE_STREAMING_MAX_CLIENT_LAG, invalidParams); + } + + if (inputConfig.containsKey(SNOWPIPE_STREAMING_MAX_MEMORY_LIMIT_IN_BYTES)) { + ensureValidLong( + inputConfig, SNOWPIPE_STREAMING_MAX_MEMORY_LIMIT_IN_BYTES, invalidParams); + } + + // Valid schematization for Snowpipe Streaming + invalidParams.putAll(validateSchematizationConfig(inputConfig)); + } + } catch (ConfigException exception) { + invalidParams.put( + INGESTION_METHOD_OPT, + Utils.formatString( + "Kafka config:{} error:{}", INGESTION_METHOD_OPT, exception.getMessage())); + } + } + + return ImmutableMap.copyOf(invalidParams); + } + + private static void ensureValidLong( + Map inputConfig, String param, Map invalidParams) { + try { + Long.parseLong(inputConfig.get(param)); + } catch (NumberFormatException exception) { + invalidParams.put( + param, + Utils.formatString( + param + " configuration must be a parsable long. Given configuration" + " was: {}", + inputConfig.get(param))); + } + } + + /** + * Validates if the configs are allowed values when schematization is enabled. + * + *

return a map of invalid params + */ + private static Map validateSchematizationConfig(Map inputConfig) { + Map invalidParams = new HashMap<>(); + + if (inputConfig.containsKey(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG)) { + BOOLEAN_VALIDATOR.ensureValid( + SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, + inputConfig.get(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG)); + + if (Boolean.parseBoolean( + inputConfig.get(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG)) + && inputConfig.get(VALUE_CONVERTER_CONFIG_FIELD) != null + && (inputConfig.get(VALUE_CONVERTER_CONFIG_FIELD).contains(STRING_CONVERTER_KEYWORD) + || inputConfig + .get(VALUE_CONVERTER_CONFIG_FIELD) + .contains(BYTE_ARRAY_CONVERTER_KEYWORD))) { + invalidParams.put( + inputConfig.get(VALUE_CONVERTER_CONFIG_FIELD), + Utils.formatString( + "The value converter:{} is not supported with schematization.", + inputConfig.get(VALUE_CONVERTER_CONFIG_FIELD))); + } + } + + return invalidParams; + } + + /** + * Validates if key and value converters are allowed values if {@link + * IngestionMethodConfig#SNOWPIPE_STREAMING} is used. + * + *

Map if invalid parameters + */ + private static Map validateConfigConverters( + final String inputConfigConverterField, Map inputConfig) { + Map invalidParams = new HashMap<>(); + + if (inputConfig.containsKey(inputConfigConverterField) + && DISALLOWED_CONVERTERS_STREAMING.contains(inputConfig.get(inputConfigConverterField))) { + invalidParams.put( + inputConfigConverterField, + Utils.formatString( + "Config:{} has provided value:{}. If ingestionMethod is:{}, Snowflake Custom" + + " Converters are not allowed. \n" + + "Invalid Converters:{}", + inputConfigConverterField, + inputConfig.get(inputConfigConverterField), + IngestionMethodConfig.SNOWPIPE_STREAMING, + Iterables.toString(DISALLOWED_CONVERTERS_STREAMING))); + } + + return invalidParams; + } + + /** Config validations specific to single buffer architecture */ + private static class SingleBufferConfigValidator { + + private static final KCLogger LOGGER = + new KCLogger(SingleBufferConfigValidator.class.getName()); + + private void logDoubleBufferingParametersWarning(Map config) { + if (InternalBufferParameters.isSingleBufferEnabled(config)) { + List ignoredParameters = + Arrays.asList(BUFFER_FLUSH_TIME_SEC, BUFFER_SIZE_BYTES, BUFFER_COUNT_RECORDS); + ignoredParameters.stream() + .filter(config::containsKey) + .forEach( + param -> + LOGGER.warn( + "{} parameter value is ignored because internal buffer is disabled. To go" + + " back to previous behaviour set " + + SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER + + " to false", + param)); + } + } + } + + /** Config validations specific to double buffer architecture */ + private static class DoubleBufferConfigValidator { + private Map validate(Map inputConfig) { + Map invalidParams = new HashMap<>(); + + // check if buffer thresholds are within permissible range + invalidParams.putAll( + BufferThreshold.validateBufferThreshold( + inputConfig, IngestionMethodConfig.SNOWPIPE_STREAMING)); + + return invalidParams; + } + } +} diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/DirectTopicPartitionChannel.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/DirectTopicPartitionChannel.java index 33577dd0f..3fa7e98e4 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/streaming/DirectTopicPartitionChannel.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/DirectTopicPartitionChannel.java @@ -13,7 +13,10 @@ import com.google.common.base.MoreObjects; import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.Utils; import com.snowflake.kafka.connector.dlq.KafkaRecordErrorReporter; @@ -38,6 +41,7 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; @@ -76,6 +80,12 @@ public class DirectTopicPartitionChannel implements TopicPartitionChannel { private final AtomicLong processedOffset = new AtomicLong(NO_OFFSET_TOKEN_REGISTERED_IN_SNOWFLAKE); + // This offset is would not be required for buffer-less channel, but we add it to keep buffered + // and non-buffered + // channel versions compatible. + private final AtomicLong currentConsumerGroupOffset = + new AtomicLong(NO_OFFSET_TOKEN_REGISTERED_IN_SNOWFLAKE); + // Indicates whether we need to skip and discard any leftover rows in the current batch, this // could happen when the channel gets invalidated and reset, then anything left in the buffer // should be skipped @@ -254,7 +264,7 @@ public DirectTopicPartitionChannel( metricsJmxReporter, this.offsetPersistedInSnowflake, this.processedOffset, - new AtomicLong(0)); + this.currentConsumerGroupOffset); this.telemetryServiceV2.reportKafkaPartitionStart( new SnowflakeTelemetryChannelCreation(this.tableName, this.channelNameFormatV1, startTime)); @@ -299,6 +309,11 @@ public void insertRecord(SinkRecord kafkaSinkRecord, boolean isFirstRowPerPartit final long currentOffsetPersistedInSnowflake = this.offsetPersistedInSnowflake.get(); final long currentProcessedOffset = this.processedOffset.get(); + // for backwards compatibility - set the consumer offset to be the first one received from kafka + if (currentConsumerGroupOffset.get() == NO_OFFSET_TOKEN_REGISTERED_IN_SNOWFLAKE) { + this.currentConsumerGroupOffset.set(kafkaSinkRecord.kafkaOffset()); + } + // Reset the value if it's a new batch if (isFirstRowPerPartitionInBatch) { needToSkipCurrentBatch = false; @@ -320,13 +335,13 @@ public void insertRecord(SinkRecord kafkaSinkRecord, boolean isFirstRowPerPartit || kafkaSinkRecord.kafkaOffset() >= currentProcessedOffset + 1) { transformAndSend(kafkaSinkRecord); } else { - LOGGER.debug( - "Skip adding offset:{} to buffer for channel:{} because" - + " offsetPersistedInSnowflake:{}, processedOffset:{}", - kafkaSinkRecord.kafkaOffset(), + LOGGER.warn( + "Channel {} - skipping current record - expected offset {} but received {}. The current" + + " offset stored in Snowflake: {}", this.getChannelNameFormatV1(), - currentOffsetPersistedInSnowflake, - currentProcessedOffset); + currentProcessedOffset, + kafkaSinkRecord.kafkaOffset(), + currentOffsetPersistedInSnowflake); } } @@ -468,12 +483,7 @@ private InsertValidationResponse insertRowWithFallback( .build(); return Failsafe.with(reopenChannelFallbackExecutorForInsertRows) - .get( - () -> { - LOGGER.debug( - "Invoking insertRows API for channel:{}", this.channel.getFullyQualifiedName()); - return this.channel.insertRow(transformedRecord, Long.toString(offset)); - }); + .get(() -> this.channel.insertRow(transformedRecord, Long.toString(offset))); } /** @@ -510,11 +520,14 @@ private void handleInsertRowFailure( InsertValidationResponse.InsertError insertError = insertErrors.get(0); List extraColNames = insertError.getExtraColNames(); List nonNullableColumns = insertError.getMissingNotNullColNames(); - if (extraColNames != null || nonNullableColumns != null) { + List nullValueForNotNullColNames = insertError.getNullValueForNotNullColNames(); + if (extraColNames != null + || nonNullableColumns != null + || nullValueForNotNullColNames != null) { SchematizationUtils.evolveSchemaIfNeeded( this.conn, this.channel.getTableName(), - nonNullableColumns, + join(nonNullableColumns, nullValueForNotNullColNames), extraColNames, kafkaSinkRecord); streamingApiFallbackSupplier( @@ -559,6 +572,14 @@ private void handleInsertRowFailure( } } + private List join( + List nonNullableColumns, List nullValueForNotNullColNames) { + return Lists.newArrayList( + Iterables.concat( + Optional.ofNullable(nonNullableColumns).orElse(ImmutableList.of()), + Optional.ofNullable(nullValueForNotNullColNames).orElse(ImmutableList.of()))); + } + // TODO: SNOW-529755 POLL committed offsets in background thread @Override @@ -640,9 +661,17 @@ public long fetchOffsetTokenWithRetry() { */ private long streamingApiFallbackSupplier( final StreamingApiFallbackInvoker streamingApiFallbackInvoker) { - final long offsetRecoveredFromSnowflake = - getRecoveredOffsetFromSnowflake(streamingApiFallbackInvoker); - resetChannelMetadataAfterRecovery(streamingApiFallbackInvoker, offsetRecoveredFromSnowflake); + SnowflakeStreamingIngestChannel newChannel = reopenChannel(streamingApiFallbackInvoker); + + LOGGER.warn( + "{} Fetching offsetToken after re-opening the channel:{}", + streamingApiFallbackInvoker, + this.getChannelNameFormatV1()); + long offsetRecoveredFromSnowflake = fetchLatestOffsetFromChannel(newChannel); + + resetChannelMetadataAfterRecovery( + streamingApiFallbackInvoker, offsetRecoveredFromSnowflake, newChannel); + return offsetRecoveredFromSnowflake; } @@ -658,23 +687,26 @@ private long streamingApiFallbackSupplier( * for logging mainly. * @param offsetRecoveredFromSnowflake offset number found in snowflake for this * channel(partition) + * @param newChannel a channel to assign to the current instance */ private void resetChannelMetadataAfterRecovery( final StreamingApiFallbackInvoker streamingApiFallbackInvoker, - final long offsetRecoveredFromSnowflake) { + final long offsetRecoveredFromSnowflake, + SnowflakeStreamingIngestChannel newChannel) { if (offsetRecoveredFromSnowflake == NO_OFFSET_TOKEN_REGISTERED_IN_SNOWFLAKE) { LOGGER.info( - "{} Channel:{}, offset token is NULL", + "{} Channel:{}, offset token is NULL, will attempt to use offset managed by the connector" + + ", consumer offset: {}", streamingApiFallbackInvoker, - this.getChannelNameFormatV1()); + this.getChannelNameFormatV1(), + this.currentConsumerGroupOffset.get()); } - final long offsetToResetInKafka = offsetRecoveredFromSnowflake + 1L; + final long offsetToResetInKafka = + offsetRecoveredFromSnowflake == NO_OFFSET_TOKEN_REGISTERED_IN_SNOWFLAKE + ? currentConsumerGroupOffset.get() + : offsetRecoveredFromSnowflake + 1L; if (offsetToResetInKafka == NO_OFFSET_TOKEN_REGISTERED_IN_SNOWFLAKE) { - LOGGER.info( - "There is no offset registered for {} channel in Snowflake. Stop recovering the channel" - + " metadata.", - this.getChannelNameFormatV1()); return; } @@ -689,6 +721,7 @@ private void resetChannelMetadataAfterRecovery( // Set the flag so that any leftover rows in the buffer should be skipped, it will be // re-ingested since the offset in kafka was reset needToSkipCurrentBatch = true; + this.channel = newChannel; LOGGER.warn( "{} Channel:{}, setting sinkTaskOffset to {}, offsetPersistedInSnowflake to {}," @@ -709,16 +742,11 @@ private void resetChannelMetadataAfterRecovery( * @param streamingApiFallbackInvoker Streaming API which invoked this function. * @return offset which was last present in Snowflake */ - private long getRecoveredOffsetFromSnowflake( + private SnowflakeStreamingIngestChannel reopenChannel( final StreamingApiFallbackInvoker streamingApiFallbackInvoker) { LOGGER.warn( "{} Re-opening channel:{}", streamingApiFallbackInvoker, this.getChannelNameFormatV1()); - this.channel = Preconditions.checkNotNull(openChannelForTable()); - LOGGER.warn( - "{} Fetching offsetToken after re-opening the channel:{}", - streamingApiFallbackInvoker, - this.getChannelNameFormatV1()); - return fetchLatestCommittedOffsetFromSnowflake(); + return Preconditions.checkNotNull(openChannelForTable()); } /** @@ -734,9 +762,14 @@ private long getRecoveredOffsetFromSnowflake( private long fetchLatestCommittedOffsetFromSnowflake() { LOGGER.debug( "Fetching last committed offset for partition channel:{}", this.getChannelNameFormatV1()); + SnowflakeStreamingIngestChannel channelToGetOffset = this.channel; + return fetchLatestOffsetFromChannel(channelToGetOffset); + } + + private long fetchLatestOffsetFromChannel(SnowflakeStreamingIngestChannel channel) { String offsetToken = null; try { - offsetToken = this.channel.getLatestCommittedOffsetToken(); + offsetToken = channel.getLatestCommittedOffsetToken(); LOGGER.info( "Fetched offsetToken for channelName:{}, offset:{}", this.getChannelNameFormatV1(), @@ -891,9 +924,8 @@ public long getProcessedOffset() { @Override @VisibleForTesting - @Deprecated public long getLatestConsumerOffset() { - return 0; + return this.currentConsumerGroupOffset.get(); } @Override @@ -922,8 +954,11 @@ public SnowflakeTelemetryChannelStatus getSnowflakeTelemetryChannelStatus() { } @Override - @Deprecated - public void setLatestConsumerOffset(long consumerOffset) {} + public void setLatestConsumerOffset(long consumerOffset) { + if (consumerOffset > this.currentConsumerGroupOffset.get()) { + this.currentConsumerGroupOffset.set(consumerOffset); + } + } /** * Converts the original kafka sink record into a Json Record. i.e key and values are converted diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java index 1aea76d56..908efe762 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2.java @@ -707,12 +707,19 @@ private void createTableIfNotExists(final String tableName) { private void populateSchemaEvolutionPermissions(String tableName) { if (!tableName2SchemaEvolutionPermission.containsKey(tableName)) { if (enableSchematization) { - tableName2SchemaEvolutionPermission.put( - tableName, + boolean hasSchemaEvolutionPermission = conn != null && conn.hasSchemaEvolutionPermission( - tableName, connectorConfig.get(SNOWFLAKE_ROLE))); + tableName, connectorConfig.get(SNOWFLAKE_ROLE)); + LOGGER.info( + "[SCHEMA_EVOLUTION_CACHE] Setting {} for table {}", + hasSchemaEvolutionPermission, + tableName); + tableName2SchemaEvolutionPermission.put(tableName, hasSchemaEvolutionPermission); } else { + LOGGER.info( + "[SCHEMA_EVOLUTION_CACHE] Schematization disabled. Setting false for table {}", + tableName); tableName2SchemaEvolutionPermission.put(tableName, false); } } diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/StreamingConfigValidator.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/StreamingConfigValidator.java new file mode 100644 index 000000000..1649cf550 --- /dev/null +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/StreamingConfigValidator.java @@ -0,0 +1,14 @@ +package com.snowflake.kafka.connector.internal.streaming; + +import com.google.common.collect.ImmutableMap; +import java.util.Map; + +/** Validates connector config for Snowpipe Streaming */ +public interface StreamingConfigValidator { + + /** + * @param inputConfig connector provided by user + * @return map of invalid parameters + */ + ImmutableMap validate(final Map inputConfig); +} diff --git a/src/main/java/com/snowflake/kafka/connector/internal/streaming/StreamingUtils.java b/src/main/java/com/snowflake/kafka/connector/internal/streaming/StreamingUtils.java index 81931c1a6..42b54ff59 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/streaming/StreamingUtils.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/streaming/StreamingUtils.java @@ -1,33 +1,18 @@ package com.snowflake.kafka.connector.internal.streaming; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.BOOLEAN_VALIDATOR; import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.CUSTOM_SNOWFLAKE_CONVERTERS; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.ENABLE_CHANNEL_OFFSET_TOKEN_MIGRATION_CONFIG; import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG; import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.ERRORS_LOG_ENABLE_CONFIG; import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.ERRORS_TOLERANCE_CONFIG; import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.ErrorTolerance; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_MAX_CLIENT_LAG; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_MAX_MEMORY_LIMIT_IN_BYTES; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD; -import com.google.common.base.Strings; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; -import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.Utils; -import com.snowflake.kafka.connector.internal.BufferThreshold; import java.time.Duration; -import java.util.HashMap; import java.util.Map; import java.util.Properties; import java.util.Set; import net.snowflake.ingest.streaming.OffsetTokenVerificationFunction; import net.snowflake.ingest.utils.Constants; -import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.record.DefaultRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -188,161 +173,4 @@ public static boolean logErrors(Map sfConnectorConfig) { public static String getDlqTopicName(Map sfConnectorConfig) { return sfConnectorConfig.getOrDefault(ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG, ""); } - - /** - * Validate Streaming snowpipe related config provided by config(customer's config) - * - * @param inputConfig given in connector json file - * @return map of invalid parameters - */ - public static ImmutableMap validateStreamingSnowpipeConfig( - final Map inputConfig) { - Map invalidParams = new HashMap<>(); - - // For snowpipe_streaming, role should be non empty - if (inputConfig.containsKey(INGESTION_METHOD_OPT)) { - try { - // This throws an exception if config value is invalid. - IngestionMethodConfig.VALIDATOR.ensureValid( - INGESTION_METHOD_OPT, inputConfig.get(INGESTION_METHOD_OPT)); - if (inputConfig - .get(INGESTION_METHOD_OPT) - .equalsIgnoreCase(IngestionMethodConfig.SNOWPIPE_STREAMING.toString())) { - - // check if buffer thresholds are within permissible range - invalidParams.putAll( - BufferThreshold.validateBufferThreshold( - inputConfig, IngestionMethodConfig.SNOWPIPE_STREAMING)); - - invalidParams.putAll(validateConfigConverters(KEY_CONVERTER_CONFIG_FIELD, inputConfig)); - invalidParams.putAll(validateConfigConverters(VALUE_CONVERTER_CONFIG_FIELD, inputConfig)); - - // Validate if snowflake role is present - if (!inputConfig.containsKey(Utils.SF_ROLE) - || Strings.isNullOrEmpty(inputConfig.get(Utils.SF_ROLE))) { - invalidParams.put( - Utils.SF_ROLE, - Utils.formatString( - "Config:{} should be present if ingestionMethod is:{}", - Utils.SF_ROLE, - inputConfig.get(INGESTION_METHOD_OPT))); - } - - /** - * Only checking in streaming since we are utilizing the values before we send it to - * DLQ/output to log file - */ - if (inputConfig.containsKey(ERRORS_TOLERANCE_CONFIG)) { - SnowflakeSinkConnectorConfig.ErrorTolerance.VALIDATOR.ensureValid( - ERRORS_TOLERANCE_CONFIG, inputConfig.get(ERRORS_TOLERANCE_CONFIG)); - } - if (inputConfig.containsKey(ERRORS_LOG_ENABLE_CONFIG)) { - BOOLEAN_VALIDATOR.ensureValid( - ERRORS_LOG_ENABLE_CONFIG, inputConfig.get(ERRORS_LOG_ENABLE_CONFIG)); - } - if (inputConfig.containsKey(ENABLE_CHANNEL_OFFSET_TOKEN_MIGRATION_CONFIG)) { - BOOLEAN_VALIDATOR.ensureValid( - ENABLE_CHANNEL_OFFSET_TOKEN_MIGRATION_CONFIG, - inputConfig.get(ENABLE_CHANNEL_OFFSET_TOKEN_MIGRATION_CONFIG)); - } - - if (inputConfig.containsKey(SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER)) { - BOOLEAN_VALIDATOR.ensureValid( - SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER, - inputConfig.get(SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER)); - } - - if (inputConfig.containsKey(SNOWPIPE_STREAMING_MAX_CLIENT_LAG)) { - ensureValidLong(inputConfig, SNOWPIPE_STREAMING_MAX_CLIENT_LAG, invalidParams); - } - - if (inputConfig.containsKey(SNOWPIPE_STREAMING_MAX_MEMORY_LIMIT_IN_BYTES)) { - ensureValidLong( - inputConfig, SNOWPIPE_STREAMING_MAX_MEMORY_LIMIT_IN_BYTES, invalidParams); - } - - // Valid schematization for Snowpipe Streaming - invalidParams.putAll(validateSchematizationConfig(inputConfig)); - } - } catch (ConfigException exception) { - invalidParams.put( - INGESTION_METHOD_OPT, - Utils.formatString( - "Kafka config:{} error:{}", INGESTION_METHOD_OPT, exception.getMessage())); - } - } - - return ImmutableMap.copyOf(invalidParams); - } - - private static void ensureValidLong( - Map inputConfig, String param, Map invalidParams) { - try { - Long.parseLong(inputConfig.get(param)); - } catch (NumberFormatException exception) { - invalidParams.put( - param, - Utils.formatString( - param + " configuration must be a parsable long. Given configuration" + " was: {}", - inputConfig.get(param))); - } - } - - /** - * Validates if key and value converters are allowed values if {@link - * IngestionMethodConfig#SNOWPIPE_STREAMING} is used. - * - *

Map if invalid parameters - */ - private static Map validateConfigConverters( - final String inputConfigConverterField, Map inputConfig) { - Map invalidParams = new HashMap<>(); - - if (inputConfig.containsKey(inputConfigConverterField) - && DISALLOWED_CONVERTERS_STREAMING.contains(inputConfig.get(inputConfigConverterField))) { - invalidParams.put( - inputConfigConverterField, - Utils.formatString( - "Config:{} has provided value:{}. If ingestionMethod is:{}, Snowflake Custom" - + " Converters are not allowed. \n" - + "Invalid Converters:{}", - inputConfigConverterField, - inputConfig.get(inputConfigConverterField), - IngestionMethodConfig.SNOWPIPE_STREAMING, - Iterables.toString(DISALLOWED_CONVERTERS_STREAMING))); - } - - return invalidParams; - } - - /** - * Validates if the configs are allowed values when schematization is enabled. - * - *

return a map of invalid params - */ - private static Map validateSchematizationConfig(Map inputConfig) { - Map invalidParams = new HashMap<>(); - - if (inputConfig.containsKey(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG)) { - BOOLEAN_VALIDATOR.ensureValid( - SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, - inputConfig.get(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG)); - - if (Boolean.parseBoolean( - inputConfig.get(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG)) - && inputConfig.get(VALUE_CONVERTER_CONFIG_FIELD) != null - && (inputConfig.get(VALUE_CONVERTER_CONFIG_FIELD).contains(STRING_CONVERTER_KEYWORD) - || inputConfig - .get(VALUE_CONVERTER_CONFIG_FIELD) - .contains(BYTE_ARRAY_CONVERTER_KEYWORD))) { - invalidParams.put( - inputConfig.get(VALUE_CONVERTER_CONFIG_FIELD), - Utils.formatString( - "The value converter:{} is not supported with schematization.", - inputConfig.get(VALUE_CONVERTER_CONFIG_FIELD))); - } - } - - return invalidParams; - } } diff --git a/src/main/java/com/snowflake/kafka/connector/internal/telemetry/SnowflakeTelemetryService.java b/src/main/java/com/snowflake/kafka/connector/internal/telemetry/SnowflakeTelemetryService.java index ff6d82385..11d1054ef 100644 --- a/src/main/java/com/snowflake/kafka/connector/internal/telemetry/SnowflakeTelemetryService.java +++ b/src/main/java/com/snowflake/kafka/connector/internal/telemetry/SnowflakeTelemetryService.java @@ -1,19 +1,6 @@ package com.snowflake.kafka.connector.internal.telemetry; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.BUFFER_COUNT_RECORDS; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.BUFFER_SIZE_BYTES; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.ENABLE_CHANNEL_OFFSET_TOKEN_MIGRATION_CONFIG; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_DEFAULT; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.ENABLE_STREAMING_CLIENT_OPTIMIZATION_CONFIG; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.ENABLE_STREAMING_CLIENT_OPTIMIZATION_DEFAULT; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.INGESTION_METHOD_DEFAULT_SNOWPIPE; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_CLIENT_PROVIDER_OVERRIDE_MAP; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_MAX_CLIENT_LAG; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD; +import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.*; import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.Utils; @@ -291,6 +278,13 @@ protected void addUserConnectorPropertiesToDataNode( SNOWPIPE_STREAMING_CLIENT_PROVIDER_OVERRIDE_MAP, userProvidedConfig.get(SNOWPIPE_STREAMING_CLIENT_PROVIDER_OVERRIDE_MAP)); } + if (userProvidedConfig.containsKey(SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER)) { + dataObjectNode.put( + SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER, + userProvidedConfig.getOrDefault( + SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER, + String.valueOf(SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER_DEFAULT))); + } } public enum TelemetryType { diff --git a/src/test/java/com/snowflake/kafka/connector/ConnectorConfigValidatorLogsTest.java b/src/test/java/com/snowflake/kafka/connector/ConnectorConfigValidatorLogsTest.java new file mode 100644 index 000000000..5669e927a --- /dev/null +++ b/src/test/java/com/snowflake/kafka/connector/ConnectorConfigValidatorLogsTest.java @@ -0,0 +1,86 @@ +package com.snowflake.kafka.connector; + +import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.*; +import static com.snowflake.kafka.connector.internal.TestUtils.getConfig; + +import com.snowflake.kafka.connector.internal.EncryptionUtils; +import com.snowflake.kafka.connector.internal.FIPSTest; +import com.snowflake.kafka.connector.internal.TestUtils; +import com.snowflake.kafka.connector.internal.streaming.DefaultStreamingConfigValidator; +import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; +import java.io.BufferedReader; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Map; +import org.bouncycastle.operator.OperatorCreationException; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class ConnectorConfigValidatorLogsTest { + + private final ConnectorConfigValidator connectorConfigValidator = + new DefaultConnectorConfigValidator(new DefaultStreamingConfigValidator()); + + @Test + public void testRSAPasswordOutput() throws IOException, OperatorCreationException { + // given + String testPasswd = "TestPassword1234!"; + String testKey = FIPSTest.generateAESKey(TestUtils.getPrivateKey(), testPasswd.toCharArray()); + Map testConf = getConfig(); + testConf.remove(SnowflakeSinkConnectorConfig.SNOWFLAKE_PRIVATE_KEY); + testConf.put(SnowflakeSinkConnectorConfig.SNOWFLAKE_PRIVATE_KEY, testKey); + testConf.put(SnowflakeSinkConnectorConfig.SNOWFLAKE_PRIVATE_KEY_PASSPHRASE, testPasswd); + + // when + connectorConfigValidator.validateConfig(testConf); + + // then + EncryptionUtils.parseEncryptedPrivateKey(testKey, testPasswd); + Assertions.assertFalse(logFileContains(testPasswd)); + } + + @ParameterizedTest + @ValueSource(strings = {BUFFER_FLUSH_TIME_SEC, BUFFER_SIZE_BYTES, BUFFER_COUNT_RECORDS}) + public void shouldLogWarningIfBufferingPropertiesDefinedForSingleBuffer(String parameter) + throws IOException { + // given + Map config = TestUtils.getConfForStreaming(); + config.put(INGESTION_METHOD_OPT, IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); + config.put(SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER, "true"); + config.put(parameter, "1000"); + String expectedLog = + parameter + + " parameter value is ignored because internal buffer is disabled. To go back to" + + " previous behaviour set " + + SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER + + " to false"; + + // when + connectorConfigValidator.validateConfig(config); + + // then + Assertions.assertTrue(logFileContains(expectedLog)); + } + + // Note that sf.log accumulates logs between the consecutive test runs + // That's why it's very hard to test many scenarios without hacks like test ordering and deleting + // log file + private boolean logFileContains(String str) throws IOException { + String fileName = "sf.log"; + File log = new File(fileName); + FileReader fileReader = new FileReader(log); + BufferedReader buffer = new BufferedReader(fileReader); + String line; + while ((line = buffer.readLine()) != null) { + if (line.contains(str)) { + return true; + } + } + buffer.close(); + fileReader.close(); + return false; + } +} diff --git a/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java b/src/test/java/com/snowflake/kafka/connector/ConnectorConfigValidatorTest.java similarity index 86% rename from src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java rename to src/test/java/com/snowflake/kafka/connector/ConnectorConfigValidatorTest.java index 52068d571..c27277c09 100644 --- a/src/test/java/com/snowflake/kafka/connector/ConnectorConfigTest.java +++ b/src/test/java/com/snowflake/kafka/connector/ConnectorConfigValidatorTest.java @@ -19,9 +19,9 @@ import static org.assertj.core.api.Assertions.*; import static org.junit.Assert.assertEquals; -import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.TopicToTableValidator; import com.snowflake.kafka.connector.internal.SnowflakeErrors; import com.snowflake.kafka.connector.internal.SnowflakeKafkaConnectorException; +import com.snowflake.kafka.connector.internal.streaming.DefaultStreamingConfigValidator; import com.snowflake.kafka.connector.internal.streaming.IngestionMethodConfig; import com.snowflake.kafka.connector.internal.streaming.StreamingUtils; import java.util.ArrayList; @@ -29,8 +29,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; - -import org.apache.kafka.common.config.ConfigException; import java.util.stream.Stream; import org.apache.kafka.connect.storage.Converter; import org.junit.Assert; @@ -40,9 +38,11 @@ import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.MethodSource; -import static org.junit.Assert.assertThrows; +public class ConnectorConfigValidatorTest { + + private final ConnectorConfigValidator connectorConfigValidator = + new DefaultConnectorConfigValidator(new DefaultStreamingConfigValidator()); -public class ConnectorConfigTest { // subset of valid community converters public static final List COMMUNITY_CONVERTER_SUBSET = Arrays.asList( @@ -64,7 +64,7 @@ private static Stream customSnowflakeConverters() { @Test public void testConfig() { Map config = getConfig(); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test @@ -74,14 +74,14 @@ public void testConfig_ConvertedInvalidAppName() { Utils.convertAppName(config); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test public void testEmptyFlushTime() { Map config = getConfig(); config.remove(SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC); } @@ -92,7 +92,7 @@ public void testFlushTimeSmall() { config.put( SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC, (SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC_MIN - 1) + ""); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC); } @@ -101,7 +101,7 @@ public void testFlushTimeSmall() { public void testFlushTimeNotNumber() { Map config = getConfig(); config.put(SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC, "fdas"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC); } @@ -120,7 +120,7 @@ public void testFlushTimeNotNumber() { public void shouldThrowExForEmptyProperty(String prop) { Map config = getConfig(); config.remove(prop); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(prop); } @@ -130,14 +130,14 @@ public void testCorrectProxyHost() { Map config = getConfig(); config.put(JVM_PROXY_HOST, "127.0.0.1"); config.put(JVM_PROXY_PORT, "3128"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test public void testEmptyPort() { Map config = getConfig(); config.put(JVM_PROXY_HOST, "127.0.0.1"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(JVM_PROXY_HOST); } @@ -146,7 +146,7 @@ public void testEmptyPort() { public void testEmptyHost() { Map config = getConfig(); config.put(JVM_PROXY_PORT, "3128"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(JVM_PROXY_PORT); } @@ -182,7 +182,7 @@ public void testNonProxyHosts() { public void testIllegalTopicMap() { Map config = getConfig(); config.put(SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP, "$@#$#@%^$12312"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP); } @@ -191,7 +191,7 @@ public void testIllegalTopicMap() { public void testIllegalTableName() { Map config = getConfig(); config.put(SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP, "topic1:!@#@!#!@"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .matches( ex -> @@ -204,7 +204,7 @@ public void testIllegalTableName() { public void testDuplicatedTopic() { Map config = getConfig(); config.put(SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP, "topic1:table1,topic1:table2"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .matches( ex -> @@ -217,23 +217,7 @@ public void testDuplicatedTopic() { public void testDuplicatedTableName() { Map config = getConfig(); config.put(SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP, "topic1:table1,topic2:table1"); - Utils.validateConfig(config); - } - - @Test - public void testTopicToTableValidatorOnlyThrowsConfigException() { - assertThrows(ConfigException.class, () -> { - new TopicToTableValidator().ensureValid(SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP, - "$@#$#@%^$12312"); - }); - assertThrows(ConfigException.class, () -> { - new TopicToTableValidator().ensureValid(SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP, - "topic1:!@#@!#!@"); - }); - assertThrows(ConfigException.class, () -> { - new TopicToTableValidator().ensureValid(SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP, - "topic1:table1,topic1:table2"); - }); + connectorConfigValidator.validateConfig(config); } @Test @@ -241,14 +225,14 @@ public void testNameMapCovered() { Map config = getConfig(); config.put(SnowflakeSinkConnectorConfig.TOPICS, "!@#,$%^,test"); config.put(SnowflakeSinkConnectorConfig.TOPICS_TABLES_MAP, "!@#:table1,$%^:table2"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test public void testBufferSizeRange() { Map config = getConfig(); config.put(BUFFER_SIZE_BYTES, SnowflakeSinkConnectorConfig.BUFFER_SIZE_BYTES_MIN - 1 + ""); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(BUFFER_SIZE_BYTES); } @@ -257,7 +241,7 @@ public void testBufferSizeRange() { public void testBufferSizeValue() { Map config = getConfig(); config.put(BUFFER_SIZE_BYTES, "afdsa"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(BUFFER_SIZE_BYTES); } @@ -266,7 +250,7 @@ public void testBufferSizeValue() { public void testEmptyBufferCountNegative() { Map config = getConfig(); config.put(BUFFER_COUNT_RECORDS, "-1"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(BUFFER_COUNT_RECORDS); } @@ -275,7 +259,7 @@ public void testEmptyBufferCountNegative() { public void testBufferCountValue() { Map config = getConfig(); config.put(BUFFER_COUNT_RECORDS, "adssadsa"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(BUFFER_COUNT_RECORDS); } @@ -284,34 +268,34 @@ public void testBufferCountValue() { public void testKafkaProviderConfigValue_valid_null() { Map config = getConfig(); config.put(SnowflakeSinkConnectorConfig.PROVIDER_CONFIG, null); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test public void testKafkaProviderConfigValue_valid_empty() { Map config = getConfig(); config.put(SnowflakeSinkConnectorConfig.PROVIDER_CONFIG, ""); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test public void testKafkaProviderConfigValue_valid_provider() { Map config = getConfig(); config.put(SnowflakeSinkConnectorConfig.PROVIDER_CONFIG, "self_hosted"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); config.put(SnowflakeSinkConnectorConfig.PROVIDER_CONFIG, "CONFLUENT"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); config.put(SnowflakeSinkConnectorConfig.PROVIDER_CONFIG, "UNKNOWN"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test public void testKafkaProviderConfigValue_invalid_value() { Map config = getConfig(); config.put(SnowflakeSinkConnectorConfig.PROVIDER_CONFIG, "Something_which_is_not_supported"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.PROVIDER_CONFIG); } @@ -320,17 +304,17 @@ public void testKafkaProviderConfigValue_invalid_value() { public void testBehaviorOnNullValuesConfig_valid_value() { Map config = getConfig(); config.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, "IGNORE"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); config.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, "DEFAULT"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test public void testBehaviorOnNullValuesConfig_invalid_value() { Map config = getConfig(); config.put(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, "INVALID"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG); } @@ -339,17 +323,17 @@ public void testBehaviorOnNullValuesConfig_invalid_value() { public void testJMX_valid_value() { Map config = getConfig(); config.put(SnowflakeSinkConnectorConfig.JMX_OPT, "true"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); config.put(SnowflakeSinkConnectorConfig.JMX_OPT, "False"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test public void testJMX_invalid_value() { Map config = getConfig(); config.put(SnowflakeSinkConnectorConfig.JMX_OPT, "INVALID"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.JMX_OPT); } @@ -360,7 +344,7 @@ public void testIngestionTypeConfig_valid_value_snowpipe() { config.put( SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT, IngestionMethodConfig.SNOWPIPE.toString()); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test @@ -371,7 +355,7 @@ public void testIngestionTypeConfig_valid_value_snowpipe_streaming() { SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT, IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test @@ -382,7 +366,7 @@ public void testIngestionTypeConfig_invalid_snowpipe_streaming() { SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT, IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); config.put(Utils.SF_ROLE, ""); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(Utils.SF_ROLE); } @@ -391,7 +375,7 @@ public void testIngestionTypeConfig_invalid_snowpipe_streaming() { public void testIngestionTypeConfig_invalid_value() { Map config = getConfig(); config.put(SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT, "INVALID_VALUE"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT); } @@ -442,14 +426,14 @@ public void testErrorTolerance_AllowedValues() { SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT, IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); config.put( ERRORS_TOLERANCE_CONFIG, SnowflakeSinkConnectorConfig.ErrorTolerance.NONE.toString()); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); config.put(ERRORS_TOLERANCE_CONFIG, "all"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test @@ -460,7 +444,7 @@ public void testErrorTolerance_DisallowedValues() { SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT, IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT); } @@ -473,13 +457,13 @@ public void testErrorLog_AllowedValues() { SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT, IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); config.put(ERRORS_LOG_ENABLE_CONFIG, "FALSE"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); config.put(ERRORS_LOG_ENABLE_CONFIG, "TRUE"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test @@ -490,7 +474,7 @@ public void testErrorLog_DisallowedValues() { SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT, IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.ERRORS_LOG_ENABLE_CONFIG); } @@ -504,7 +488,7 @@ public void testStreamingEmptyFlushTime() { IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); config.remove(SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC); } @@ -519,7 +503,7 @@ public void testStreamingFlushTimeSmall() { config.put( SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC, (StreamingUtils.STREAMING_BUFFER_FLUSH_TIME_MINIMUM_SEC - 1) + ""); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC); } @@ -532,7 +516,7 @@ public void testStreamingFlushTimeNotNumber() { IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); config.put(SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC, "fdas"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC); } @@ -545,7 +529,7 @@ public void testStreamingEmptyBufferSize() { IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); config.remove(BUFFER_SIZE_BYTES); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.BUFFER_SIZE_BYTES); } @@ -558,7 +542,7 @@ public void testStreamingEmptyBufferCount() { IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); config.remove(BUFFER_COUNT_RECORDS); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(BUFFER_COUNT_RECORDS); } @@ -571,7 +555,7 @@ public void testStreamingBufferCountNegative() { IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); config.put(BUFFER_COUNT_RECORDS, "-1"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(BUFFER_COUNT_RECORDS); } @@ -584,7 +568,7 @@ public void testStreamingBufferCountValue() { IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); config.put(BUFFER_COUNT_RECORDS, "adssadsa"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(BUFFER_COUNT_RECORDS); } @@ -602,7 +586,7 @@ public void testValidKeyAndValueConvertersForStreamingSnowpipe() { config.put( SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD, converter.getClass().toString()); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); }); COMMUNITY_CONVERTER_SUBSET.forEach( @@ -610,7 +594,7 @@ public void testValidKeyAndValueConvertersForStreamingSnowpipe() { config.put( SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, converter.getClass().toString()); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); }); } @@ -629,7 +613,7 @@ public void testInvalidKeyConvertersForStreamingSnowpipe(Converter converter) { SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, "org.apache.kafka.connect.storage.StringConverter"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD); } @@ -649,7 +633,7 @@ public void testInvalidValueConvertersForStreamingSnowpipe(Converter converter) config.put( SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, converter.getClass().getName()); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD); } @@ -670,7 +654,7 @@ public void shouldNotThrowExceptionForProperStreamingClientPropsValue(String pro config.put(prop, value); // WHEN/THEN - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @ParameterizedTest @@ -692,7 +676,7 @@ public void shouldThrowExceptionForInvalidStreamingClientPropsValue(String prop, config.put(prop, value); // WHEN/THEN - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(prop); } @@ -713,7 +697,7 @@ public void shouldThrowExceptionForStreamingClientPropsWhenSnowpipeStreamingNotE config.put(prop, value); // WHEN/THEN - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(prop); } @@ -726,7 +710,7 @@ public void testInvalidSchematizationForSnowpipe() { IngestionMethodConfig.SNOWPIPE.toString()); config.put(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, "true"); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); } @@ -739,7 +723,7 @@ public void testValidSchematizationForStreamingSnowpipe() { IngestionMethodConfig.SNOWPIPE_STREAMING.toString()); config.put(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, "true"); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test @@ -753,7 +737,7 @@ public void testSchematizationWithUnsupportedConverter() { SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, "org.apache.kafka.connect.storage.StringConverter"); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining("org.apache.kafka.connect.storage.StringConverter"); } @@ -769,7 +753,7 @@ public void testDisabledSchematizationWithUnsupportedConverter() { SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, "org.apache.kafka.connect.storage.StringConverter"); config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test @@ -781,7 +765,7 @@ public void testEnableOptimizeStreamingClientConfig() { config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); config.put(SnowflakeSinkConnectorConfig.ENABLE_STREAMING_CLIENT_OPTIMIZATION_CONFIG, "true"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test @@ -792,7 +776,7 @@ public void testInvalidEnableOptimizeStreamingClientConfig() { IngestionMethodConfig.SNOWPIPE.toString()); config.put(SnowflakeSinkConnectorConfig.ENABLE_STREAMING_CLIENT_OPTIMIZATION_CONFIG, "true"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining( SnowflakeSinkConnectorConfig.ENABLE_STREAMING_CLIENT_OPTIMIZATION_CONFIG); @@ -807,7 +791,7 @@ public void testEnableStreamingChannelMigrationConfig() { config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); config.put(SnowflakeSinkConnectorConfig.ENABLE_CHANNEL_OFFSET_TOKEN_MIGRATION_CONFIG, "true"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test @@ -818,7 +802,7 @@ public void testEnableStreamingChannelMigrationConfig_invalidWithSnowpipe() { IngestionMethodConfig.SNOWPIPE.toString()); config.put(SnowflakeSinkConnectorConfig.ENABLE_CHANNEL_OFFSET_TOKEN_MIGRATION_CONFIG, "true"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining( SnowflakeSinkConnectorConfig.ENABLE_CHANNEL_OFFSET_TOKEN_MIGRATION_CONFIG); @@ -834,7 +818,7 @@ public void testEnableStreamingChannelMigrationConfig_invalidWithSnowpipe() { config.put(Utils.SF_ROLE, "ACCOUNTADMIN"); config.put( SnowflakeSinkConnectorConfig.ENABLE_CHANNEL_OFFSET_TOKEN_MIGRATION_CONFIG, "INVALID"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining( SnowflakeSinkConnectorConfig.ENABLE_CHANNEL_OFFSET_TOKEN_MIGRATION_CONFIG); @@ -849,7 +833,7 @@ public void testStreamingProviderOverrideConfig_invalidWithSnowpipe() { config.put( SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_CLIENT_PROVIDER_OVERRIDE_MAP, "a:b,c:d"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining( SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_CLIENT_PROVIDER_OVERRIDE_MAP); @@ -865,13 +849,13 @@ public void testStreamingProviderOverrideConfig_validWithSnowpipeStreaming() { config.put( SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_CLIENT_PROVIDER_OVERRIDE_MAP, "a:b,c:d,e:100,f:true"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test public void testInvalidEmptyConfig() { Map config = new HashMap<>(); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SNOWFLAKE_DATABASE) .hasMessageContaining(SNOWFLAKE_SCHEMA) @@ -913,14 +897,14 @@ public void testOAuthAuthenticator() { config.put(SnowflakeSinkConnectorConfig.OAUTH_CLIENT_ID, "client_id"); config.put(SnowflakeSinkConnectorConfig.OAUTH_CLIENT_SECRET, "client_secret"); config.put(SnowflakeSinkConnectorConfig.OAUTH_REFRESH_TOKEN, "refresh_token"); - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } @Test public void testInvalidAuthenticator() { Map config = getConfig(); config.put(SnowflakeSinkConnectorConfig.AUTHENTICATOR_TYPE, "invalid_authenticator"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.AUTHENTICATOR_TYPE); } @@ -931,7 +915,7 @@ public void testEmptyClientId() { config.put(SnowflakeSinkConnectorConfig.AUTHENTICATOR_TYPE, Utils.OAUTH); config.put(SnowflakeSinkConnectorConfig.OAUTH_CLIENT_SECRET, "client_secret"); config.put(SnowflakeSinkConnectorConfig.OAUTH_REFRESH_TOKEN, "refresh_token"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.OAUTH_CLIENT_ID); } @@ -942,7 +926,7 @@ public void testEmptyClientSecret() { config.put(SnowflakeSinkConnectorConfig.AUTHENTICATOR_TYPE, Utils.OAUTH); config.put(SnowflakeSinkConnectorConfig.OAUTH_CLIENT_ID, "client_id"); config.put(SnowflakeSinkConnectorConfig.OAUTH_REFRESH_TOKEN, "refresh_token"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.OAUTH_CLIENT_SECRET); } @@ -953,7 +937,7 @@ public void testEmptyRefreshToken() { config.put(SnowflakeSinkConnectorConfig.AUTHENTICATOR_TYPE, Utils.OAUTH); config.put(SnowflakeSinkConnectorConfig.OAUTH_CLIENT_ID, "client_id"); config.put(SnowflakeSinkConnectorConfig.OAUTH_CLIENT_SECRET, "client_secret"); - assertThatThrownBy(() -> Utils.validateConfig(config)) + assertThatThrownBy(() -> connectorConfigValidator.validateConfig(config)) .isInstanceOf(SnowflakeKafkaConnectorException.class) .hasMessageContaining(SnowflakeSinkConnectorConfig.OAUTH_REFRESH_TOKEN); } @@ -965,18 +949,11 @@ private void invalidConfigRunner(List paramsToRemove) { } try { - Utils.validateConfig(config); + connectorConfigValidator.validateConfig(config); } catch (SnowflakeKafkaConnectorException exception) { for (String configParam : paramsToRemove) { assert exception.getMessage().contains(configParam); } } } - - @Test - public void testEmptyTopic2TableMap() { - Map config = getConfig(); - config.put(SnowflakeSinkConnectorConfig.TOPICS, ""); - Utils.validateConfig(config); - } } diff --git a/src/test/java/com/snowflake/kafka/connector/ConnectorIT.java b/src/test/java/com/snowflake/kafka/connector/ConnectorIT.java index 3dcd4e8f1..79fc241fd 100644 --- a/src/test/java/com/snowflake/kafka/connector/ConnectorIT.java +++ b/src/test/java/com/snowflake/kafka/connector/ConnectorIT.java @@ -11,6 +11,7 @@ import java.util.concurrent.Executors; import org.apache.kafka.common.config.Config; import org.apache.kafka.common.config.ConfigValue; +import org.junit.Ignore; import org.junit.Test; public class ConnectorIT { @@ -139,6 +140,7 @@ public void testValidateCorrectConfig() { } @Test + @Ignore("OAuth tests are temporary disabled") public void testValidateCorrectConfigWithOAuth() { Map validateMap = toValidateMap(getCorrectConfigWithOAuth()); assertPropHasError(validateMap, new String[] {}); @@ -209,6 +211,7 @@ public void testValidateNullPasswordConfig() { } @Test + @Ignore("OAuth tests are temporary disabled") public void testValidateNullOAuthClientIdConfig() { Map config = getCorrectConfigWithOAuth(); config.remove(SnowflakeSinkConnectorConfig.OAUTH_CLIENT_ID); @@ -217,6 +220,7 @@ public void testValidateNullOAuthClientIdConfig() { } @Test + @Ignore("OAuth tests are temporary disabled") public void testValidateNullOAuthClientSecretConfig() { Map config = getCorrectConfigWithOAuth(); config.remove(SnowflakeSinkConnectorConfig.OAUTH_CLIENT_SECRET); @@ -226,6 +230,7 @@ public void testValidateNullOAuthClientSecretConfig() { } @Test + @Ignore("OAuth tests are temporary disabled") public void testValidateNullOAuthRefreshTokenConfig() { Map config = getCorrectConfigWithOAuth(); config.remove(SnowflakeSinkConnectorConfig.OAUTH_REFRESH_TOKEN); diff --git a/src/test/java/com/snowflake/kafka/connector/SecurityTest.java b/src/test/java/com/snowflake/kafka/connector/SecurityTest.java deleted file mode 100644 index e85c4b478..000000000 --- a/src/test/java/com/snowflake/kafka/connector/SecurityTest.java +++ /dev/null @@ -1,46 +0,0 @@ -package com.snowflake.kafka.connector; - -import static com.snowflake.kafka.connector.internal.TestUtils.getConfig; - -import com.snowflake.kafka.connector.internal.EncryptionUtils; -import com.snowflake.kafka.connector.internal.FIPSTest; -import com.snowflake.kafka.connector.internal.TestUtils; -import java.io.BufferedReader; -import java.io.File; -import java.io.FileReader; -import java.io.IOException; -import java.util.Map; -import org.bouncycastle.operator.OperatorCreationException; -import org.junit.Test; - -public class SecurityTest { - - @Test - public void testRSAPasswordOutput() throws IOException, OperatorCreationException { - String testPasswd = "TestPassword1234!"; - String testKey = FIPSTest.generateAESKey(TestUtils.getPrivateKey(), testPasswd.toCharArray()); - Map testConf = getConfig(); - testConf.remove(SnowflakeSinkConnectorConfig.SNOWFLAKE_PRIVATE_KEY); - testConf.put(SnowflakeSinkConnectorConfig.SNOWFLAKE_PRIVATE_KEY, testKey); - testConf.put(SnowflakeSinkConnectorConfig.SNOWFLAKE_PRIVATE_KEY_PASSPHRASE, testPasswd); - Utils.validateConfig(testConf); - EncryptionUtils.parseEncryptedPrivateKey(testKey, testPasswd); - assert !searchInLogFile(testPasswd); - } - - static boolean searchInLogFile(String str) throws IOException { - String fileName = "sf.log"; - File log = new File(fileName); - FileReader fileReader = new FileReader(log); - BufferedReader buffer = new BufferedReader(fileReader); - String line; - while ((line = buffer.readLine()) != null) { - if (line.contains(str)) { - return true; - } - } - buffer.close(); - fileReader.close(); - return false; - } -} diff --git a/src/test/java/com/snowflake/kafka/connector/SnowflakeSinkTaskForStreamingIT.java b/src/test/java/com/snowflake/kafka/connector/SnowflakeSinkTaskForStreamingIT.java index a207b53ce..8a4fa1b5d 100644 --- a/src/test/java/com/snowflake/kafka/connector/SnowflakeSinkTaskForStreamingIT.java +++ b/src/test/java/com/snowflake/kafka/connector/SnowflakeSinkTaskForStreamingIT.java @@ -13,7 +13,6 @@ import java.sql.ResultSet; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -21,18 +20,19 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Stream; import net.snowflake.client.jdbc.internal.fasterxml.jackson.core.JsonProcessingException; import net.snowflake.client.jdbc.internal.fasterxml.jackson.databind.JsonNode; import net.snowflake.client.jdbc.internal.fasterxml.jackson.databind.ObjectMapper; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.sink.SinkRecord; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import org.mockito.InjectMocks; import org.mockito.Mock; import org.mockito.Mockito; @@ -43,7 +43,6 @@ * Sink Task IT test which uses {@link * com.snowflake.kafka.connector.internal.streaming.SnowflakeSinkServiceV2} */ -@RunWith(Parameterized.class) public class SnowflakeSinkTaskForStreamingIT { private String topicName; @@ -57,33 +56,27 @@ public class SnowflakeSinkTaskForStreamingIT { @InjectMocks private SnowflakeSinkTask sinkTask1 = new SnowflakeSinkTask(); - // use OAuth as authenticator or not - private boolean useOAuth; - - @Parameterized.Parameters(name = "useOAuth: {0}") - public static Collection input() { - // TODO: Added {true} after SNOW-352846 is released - return Arrays.asList(new Object[][] {{false}}); - } - - public SnowflakeSinkTaskForStreamingIT(boolean useOAuth) { - this.useOAuth = useOAuth; - } - - @Before - public void setup() { + @BeforeEach + public void beforeEach() { topicName = TestUtils.randomTableName(); topicPartition = new TopicPartition(topicName, partition); } - @After - public void after() { + @AfterEach + public void afterEach() { TestUtils.dropTable(topicName); } - @Test - public void testSinkTask() throws Exception { - Map config = getConfig(); + private static Stream oAuthAndSingleBufferParameters() { + return Stream.of(Arguments.of(false, false), Arguments.of(false, true)); + // OAuth tests are temporary disabled + // return TestUtils.nBooleanProduct(2); + } + + @ParameterizedTest(name = "useOAuth: {0}, useSingleBuffer: {1}") + @MethodSource("oAuthAndSingleBufferParameters") + public void testSinkTask(boolean useOAuth, boolean useSingleBuffer) throws Exception { + Map config = getConfig(useOAuth, useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); config.put(BUFFER_COUNT_RECORDS, "1"); // override @@ -120,9 +113,11 @@ public void testSinkTask() throws Exception { sinkTask.stop(); } - @Test - public void testSinkTaskWithMultipleOpenClose() throws Exception { - Map config = getConfig(); + @ParameterizedTest(name = "useOAuth: {0}, useSingleBuffer: {1}") + @MethodSource("oAuthAndSingleBufferParameters") + public void testSinkTaskWithMultipleOpenClose(boolean useOAuth, boolean useSingleBuffer) + throws Exception { + Map config = getConfig(useOAuth, useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); config.put(BUFFER_COUNT_RECORDS, "1"); // override @@ -207,16 +202,17 @@ public void testSinkTaskWithMultipleOpenClose() throws Exception { metadata.get("offset").asText().equals("0"); partitionsInTable.add(metadata.get("partition").asLong()); } catch (JsonProcessingException e) { - Assert.fail(); + Assertions.fail(); } }); assert partitionsInTable.size() == 2; } - @Test - public void testTopicToTableRegex() { - Map config = getConfig(); + @ParameterizedTest(name = "useOAuth: {0}, useSingleBuffer: {1}") + @MethodSource("oAuthAndSingleBufferParameters") + public void testTopicToTableRegex(boolean useOAuth, boolean useSingleBuffer) { + Map config = getConfig(useOAuth, useSingleBuffer); testTopicToTableRegexMain(config); } @@ -340,15 +336,15 @@ private static void testTopicToTableRegexRunner( topic = currTp; } } - Assert.assertNotNull("Expected topic partition was not opened by the tast", topic); + Assertions.assertNotNull(topic, "Expected topic partition was not opened by the tast"); } } - private Map getConfig() { + private Map getConfig(boolean useOAuth, boolean useSingleBuffer) { if (!useOAuth) { - return TestUtils.getConfForStreaming(); + return TestUtils.getConfForStreaming(useSingleBuffer); } else { - return TestUtils.getConfForStreamingWithOAuth(); + return TestUtils.getConfForStreamingWithOAuth(useSingleBuffer); } } } diff --git a/src/test/java/com/snowflake/kafka/connector/UtilsTest.java b/src/test/java/com/snowflake/kafka/connector/UtilsTest.java index 0eea5f58e..4a21ee177 100644 --- a/src/test/java/com/snowflake/kafka/connector/UtilsTest.java +++ b/src/test/java/com/snowflake/kafka/connector/UtilsTest.java @@ -6,6 +6,7 @@ import java.util.HashMap; import java.util.Map; import org.junit.Assert; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.contrib.java.lang.system.EnvironmentVariables; @@ -283,6 +284,7 @@ public void testGetExceptionMessage() throws Exception { } @Test + @Ignore("OAuth tests are temporary disabled") public void testGetSnowflakeOAuthAccessToken() { Map config = TestUtils.getConfForStreamingWithOAuth(); if (config != null) { diff --git a/src/test/java/com/snowflake/kafka/connector/internal/ConnectionServiceIT.java b/src/test/java/com/snowflake/kafka/connector/internal/ConnectionServiceIT.java index 99be58f42..6da25f051 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/ConnectionServiceIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/ConnectionServiceIT.java @@ -32,9 +32,12 @@ import net.snowflake.client.jdbc.internal.apache.http.client.methods.HttpPost; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.sink.SinkRecord; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; public class ConnectionServiceIT { private final SnowflakeConnectionService conn = TestUtils.getConnectionService(); @@ -54,6 +57,7 @@ public void testEncryptedKey() { } @Test + @Disabled("OAuth tests are temporary disabled") public void testOAuthAZ() { Map confWithOAuth = TestUtils.getConfWithOAuth(); assert confWithOAuth.containsKey(Utils.SF_OAUTH_CLIENT_ID); @@ -103,12 +107,12 @@ public void testUserAgentSuffixInIngestionService() { for (Header h : httpPostInsertRequest.getAllHeaders()) { if (h.getName().equalsIgnoreCase(HttpHeaders.USER_AGENT)) { System.out.println(h); - Assert.assertTrue(h.getValue().contains(userAgentExpectedSuffixInHttpHeader)); - Assert.assertTrue(h.getValue().endsWith(userAgentExpectedSuffixInHttpHeader)); + Assertions.assertTrue(h.getValue().contains(userAgentExpectedSuffixInHttpHeader)); + Assertions.assertTrue(h.getValue().endsWith(userAgentExpectedSuffixInHttpHeader)); } } } catch (Exception e) { - Assert.fail("Should not throw an exception:" + e.getMessage()); + Assertions.fail("Should not throw an exception:" + e.getMessage()); } } @@ -191,7 +195,7 @@ public void createConnectionService_SnowpipeStreaming() { .equals("1"); } - @After + @AfterEach public void afterEach() { TestUtils.dropTable(tableName); conn.dropPipe(pipeName); @@ -425,9 +429,10 @@ public void testConnectionFunction() { assert service.isClosed(); } - @Test - public void testStreamingChannelOffsetMigration() { - Map testConfig = TestUtils.getConfForStreaming(); + @ParameterizedTest(name = "useSingleBuffer: {0}") + @ValueSource(booleans = {false, true}) + public void testStreamingChannelOffsetMigration(boolean useSingleBuffer) { + Map testConfig = TestUtils.getConfForStreaming(useSingleBuffer); SnowflakeConnectionService conn = SnowflakeConnectionServiceFactory.builder().setProperties(testConfig).build(); conn.createTable(tableName); @@ -441,8 +446,9 @@ public void testStreamingChannelOffsetMigration() { ChannelMigrateOffsetTokenResponseDTO channelMigrateOffsetTokenResponseDTO = conn.migrateStreamingChannelOffsetToken( tableName, sourceChannelName, destinationChannelName); - Assert.assertTrue(isChannelMigrationResponseSuccessful(channelMigrateOffsetTokenResponseDTO)); - Assert.assertEquals( + Assertions.assertTrue( + isChannelMigrationResponseSuccessful(channelMigrateOffsetTokenResponseDTO)); + Assertions.assertEquals( OFFSET_MIGRATION_SOURCE_CHANNEL_DOES_NOT_EXIST.getStatusCode(), channelMigrateOffsetTokenResponseDTO.getResponseCode()); @@ -459,7 +465,7 @@ public void testStreamingChannelOffsetMigration() { try { // ### TEST 3 - Source Channel (v2 channel doesnt exist) - Map config = TestUtils.getConfForStreaming(); + Map config = TestUtils.getConfForStreaming(useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); TopicPartition topicPartition = new TopicPartition(tableName, 0); @@ -489,8 +495,9 @@ public void testStreamingChannelOffsetMigration() { channelMigrateOffsetTokenResponseDTO = conn.migrateStreamingChannelOffsetToken( tableName, sourceChannelName, destinationChannelName); - Assert.assertTrue(isChannelMigrationResponseSuccessful(channelMigrateOffsetTokenResponseDTO)); - Assert.assertEquals( + Assertions.assertTrue( + isChannelMigrationResponseSuccessful(channelMigrateOffsetTokenResponseDTO)); + Assertions.assertEquals( OFFSET_MIGRATION_SOURCE_CHANNEL_DOES_NOT_EXIST.getStatusCode(), channelMigrateOffsetTokenResponseDTO.getResponseCode()); @@ -531,7 +538,7 @@ public void testStreamingChannelOffsetMigration() { TestUtils.assertWithRetry( () -> service.getOffset(new TopicPartition(tableName, 0)) == (noOfRecords * 2), 5, 5); } catch (Exception e) { - Assert.fail("Should not throw an exception:" + e.getMessage()); + Assertions.fail("Should not throw an exception:" + e.getMessage()); } finally { TestUtils.dropTable(tableName); } diff --git a/src/test/java/com/snowflake/kafka/connector/internal/TestUtils.java b/src/test/java/com/snowflake/kafka/connector/internal/TestUtils.java index b604ec824..66720dea4 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/TestUtils.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/TestUtils.java @@ -33,6 +33,8 @@ import static com.snowflake.kafka.connector.Utils.buildOAuthHttpPostRequest; import static com.snowflake.kafka.connector.Utils.getSnowflakeOAuthToken; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; import com.snowflake.client.jdbc.SnowflakeDriver; import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.Utils; @@ -61,6 +63,9 @@ import java.util.Random; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; import net.snowflake.client.jdbc.internal.apache.http.HttpHeaders; import net.snowflake.client.jdbc.internal.apache.http.client.methods.CloseableHttpResponse; import net.snowflake.client.jdbc.internal.apache.http.client.methods.HttpPost; @@ -82,6 +87,7 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.jupiter.params.provider.Arguments; public class TestUtils { // test profile properties @@ -325,6 +331,18 @@ public static Map getConfForStreaming() { return configuration; } + public static Map getConfForStreaming(boolean useSingleBuffer) { + Map config = getConfForStreaming(); + + if (useSingleBuffer) { + config.put( + SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER, + Boolean.TRUE.toString()); + } + + return config; + } + /* Get configuration map from profile path. Used against prod deployment of Snowflake */ public static Map getConfForStreamingWithOAuth() { Map configuration = getConfWithOAuth(); @@ -340,6 +358,18 @@ public static Map getConfForStreamingWithOAuth() { return configuration; } + public static Map getConfForStreamingWithOAuth(boolean useSingleBuffer) { + Map config = getConfForStreamingWithOAuth(); + + if (useSingleBuffer) { + config.put( + SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER, + Boolean.TRUE.toString()); + } + + return config; + } + /** @return JDBC config with encrypted private key */ static Map getConfWithEncryptedKey() { if (conf == null) { @@ -958,4 +988,14 @@ private static StringEntity buildStringEntity(String payload) { throw new RuntimeException(e); } } + + public static Stream nBooleanProduct(int n) { + return Sets.cartesianProduct( + IntStream.range(0, n) + .mapToObj(i -> ImmutableSet.of(false, true)) + .collect(Collectors.toList())) + .stream() + .map(List::toArray) + .map(Arguments::of); + } } diff --git a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java index 75c7a11a4..71401d067 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/TombstoneRecordIngestionIT.java @@ -1,8 +1,10 @@ package com.snowflake.kafka.connector.internal; -import static com.snowflake.kafka.connector.ConnectorConfigTest.COMMUNITY_CONVERTER_SUBSET; -import static com.snowflake.kafka.connector.ConnectorConfigTest.CUSTOM_SNOWFLAKE_CONVERTERS; +import static com.snowflake.kafka.connector.ConnectorConfigValidatorTest.COMMUNITY_CONVERTER_SUBSET; +import static com.snowflake.kafka.connector.ConnectorConfigValidatorTest.CUSTOM_SNOWFLAKE_CONVERTERS; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.Utils; import com.snowflake.kafka.connector.dlq.InMemoryKafkaRecordErrorReporter; @@ -13,39 +15,33 @@ import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Stream; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.storage.Converter; -import org.junit.After; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; -@RunWith(Parameterized.class) public class TombstoneRecordIngestionIT { - @Parameterized.Parameters(name = "behaviorOnNullValues: {0}") - public static Collection input() { - return Arrays.asList(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.values()); - } - private final int partition = 0; private final String topic = "test"; - private final String table; - private final Converter jsonConverter; - private final Map converterConfig; + private String table; + private Converter jsonConverter; + private Map converterConfig; - private final SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior; - - public TombstoneRecordIngestionIT(SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior) { - this.behavior = behavior; + @BeforeEach + public void beforeEach() { this.table = TestUtils.randomTableName(); this.jsonConverter = new JsonConverter(); @@ -54,15 +50,27 @@ public TombstoneRecordIngestionIT(SnowflakeSinkConnectorConfig.BehaviorOnNullVal this.jsonConverter.configure(this.converterConfig, false); } - @After + @AfterEach public void afterEach() { TestUtils.dropTable(table); } - @Test - public void testStreamingTombstoneBehavior() throws Exception { + private static Stream behaviorAndSingleBufferParameters() { + return Sets.cartesianProduct( + ImmutableSet.copyOf(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.values()), + ImmutableSet.of(false, true)) + .stream() + .map(List::toArray) + .map(Arguments::of); + } + + @ParameterizedTest(name = "behavior: {0}, useSingleBuffer: {1}") + @MethodSource("behaviorAndSingleBufferParameters") + public void testStreamingTombstoneBehavior( + SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior, boolean useSingleBuffer) + throws Exception { // setup - Map connectorConfig = TestUtils.getConfForStreaming(); + Map connectorConfig = TestUtils.getConfForStreaming(useSingleBuffer); TopicPartition topicPartition = new TopicPartition(topic, partition); SnowflakeSinkService service = SnowflakeSinkServiceFactory.builder( @@ -73,7 +81,7 @@ public void testStreamingTombstoneBehavior() throws Exception { .setErrorReporter(new InMemoryKafkaRecordErrorReporter()) .setSinkTaskContext(new InMemorySinkTaskContext(Collections.singleton(topicPartition))) .addTask(table, topicPartition) - .setBehaviorOnNullValuesConfig(this.behavior) + .setBehaviorOnNullValuesConfig(behavior) .build(); Map converterConfig = new HashMap<>(); converterConfig.put("schemas.enable", "false"); @@ -82,16 +90,19 @@ public void testStreamingTombstoneBehavior() throws Exception { SinkRecord normalRecord = TestUtils.createNativeJsonSinkRecords(0, 1, topic, partition).get(0); // test - this.testIngestTombstoneRunner(normalRecord, COMMUNITY_CONVERTER_SUBSET, service); + this.testIngestTombstoneRunner(normalRecord, COMMUNITY_CONVERTER_SUBSET, service, behavior); // cleanup service.closeAll(); } - @Test - public void testStreamingTombstoneBehaviorWithSchematization() throws Exception { + @ParameterizedTest(name = "behavior: {0}, useSingleBuffer: {1}") + @MethodSource("behaviorAndSingleBufferParameters") + public void testStreamingTombstoneBehaviorWithSchematization( + SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior, boolean useSingleBuffer) + throws Exception { // setup - Map connectorConfig = TestUtils.getConfForStreaming(); + Map connectorConfig = TestUtils.getConfForStreaming(useSingleBuffer); connectorConfig.put(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, "true"); TopicPartition topicPartition = new TopicPartition(topic, partition); SnowflakeSinkService service = @@ -103,7 +114,7 @@ public void testStreamingTombstoneBehaviorWithSchematization() throws Exception .setErrorReporter(new InMemoryKafkaRecordErrorReporter()) .setSinkTaskContext(new InMemorySinkTaskContext(Collections.singleton(topicPartition))) .addTask(table, topicPartition) - .setBehaviorOnNullValuesConfig(this.behavior) + .setBehaviorOnNullValuesConfig(behavior) .build(); Map converterConfig = new HashMap<>(); converterConfig.put("schemas.enable", "false"); @@ -113,14 +124,16 @@ public void testStreamingTombstoneBehaviorWithSchematization() throws Exception service.insert(normalRecord); // schematization needs first insert for evolution // test - this.testIngestTombstoneRunner(normalRecord, COMMUNITY_CONVERTER_SUBSET, service); + this.testIngestTombstoneRunner(normalRecord, COMMUNITY_CONVERTER_SUBSET, service, behavior); // cleanup service.closeAll(); } - @Test - public void testSnowpipeTombstoneBehavior() throws Exception { + @ParameterizedTest + @EnumSource(SnowflakeSinkConnectorConfig.BehaviorOnNullValues.class) + public void testSnowpipeTombstoneBehavior( + SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior) throws Exception { // setup SnowflakeConnectionService conn = TestUtils.getConnectionService(); Map connectorConfig = TestUtils.getConfig(); @@ -133,7 +146,7 @@ public void testSnowpipeTombstoneBehavior() throws Exception { .setErrorReporter(new InMemoryKafkaRecordErrorReporter()) .setSinkTaskContext(new InMemorySinkTaskContext(Collections.singleton(topicPartition))) .addTask(table, topicPartition) - .setBehaviorOnNullValuesConfig(this.behavior) + .setBehaviorOnNullValuesConfig(behavior) .build(); Map converterConfig = new HashMap<>(); converterConfig.put("schemas.enable", "false"); @@ -144,7 +157,7 @@ public void testSnowpipeTombstoneBehavior() throws Exception { // test List converters = new ArrayList<>(COMMUNITY_CONVERTER_SUBSET); converters.addAll(CUSTOM_SNOWFLAKE_CONVERTERS); - this.testIngestTombstoneRunner(normalRecord, converters, service); + this.testIngestTombstoneRunner(normalRecord, converters, service, behavior); // cleanup service.closeAll(); @@ -154,7 +167,10 @@ public void testSnowpipeTombstoneBehavior() throws Exception { // all ingestion methods should have the same behavior for tombstone records private void testIngestTombstoneRunner( - SinkRecord normalRecord, List converters, SnowflakeSinkService service) + SinkRecord normalRecord, + List converters, + SnowflakeSinkService service, + SnowflakeSinkConnectorConfig.BehaviorOnNullValues behavior) throws Exception { int offset = 1; // normalRecord should be offset 0 List sinkRecords = new ArrayList<>(); @@ -217,7 +233,7 @@ private void testIngestTombstoneRunner( // verify inserted int expectedOffset = - this.behavior == SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT + behavior == SnowflakeSinkConnectorConfig.BehaviorOnNullValues.DEFAULT ? sinkRecords.size() : 1; TestUtils.assertWithRetry(() -> TestUtils.tableSize(table) == expectedOffset, 10, 20); diff --git a/src/test/java/com/snowflake/kafka/connector/internal/streaming/CloseTopicPartitionChannelIT.java b/src/test/java/com/snowflake/kafka/connector/internal/streaming/CloseTopicPartitionChannelIT.java index df5962389..7bb941a85 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/streaming/CloseTopicPartitionChannelIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/streaming/CloseTopicPartitionChannelIT.java @@ -1,6 +1,7 @@ package com.snowflake.kafka.connector.internal.streaming; import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_CLOSE_CHANNELS_IN_PARALLEL; +import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER; import static org.awaitility.Awaitility.await; import com.snowflake.kafka.connector.ConnectClusterBaseIT; @@ -8,11 +9,13 @@ import java.time.Duration; import java.util.Map; import java.util.stream.IntStream; +import java.util.stream.Stream; import net.snowflake.ingest.streaming.SnowflakeStreamingIngestChannel; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; class CloseTopicPartitionChannelIT extends ConnectClusterBaseIT { @@ -44,11 +47,16 @@ void tearDown() { connectCluster.kafka().deleteTopic(topicName); } - @ParameterizedTest - @ValueSource(booleans = {false, true}) - void closeChannels(boolean closeInParallel) { + private static Stream closeInParallelAndSingleBufferParams() { + return TestUtils.nBooleanProduct(2); + } + + @ParameterizedTest(name = "closeInParallel: {0}, useSingleBuffer: {1}") + @MethodSource("closeInParallelAndSingleBufferParams") + void closeChannels(boolean closeInParallel, boolean useSingleBuffer) { // given - connectCluster.configureConnector(connectorName, connectorProperties(closeInParallel)); + connectCluster.configureConnector( + connectorName, connectorProperties(closeInParallel, useSingleBuffer)); waitForConnectorRunning(connectorName); await("channelsCreated").atMost(Duration.ofSeconds(30)).until(this::channelsCreated); @@ -75,10 +83,12 @@ private boolean channelsClosed() { return PARTITIONS_NUMBER == channelsCount; } - private Map connectorProperties(boolean closeInParallel) { + private Map connectorProperties( + boolean closeInParallel, boolean useSingleBuffer) { Map config = defaultProperties(topicName, connectorName); config.put(SNOWPIPE_STREAMING_CLOSE_CHANNELS_IN_PARALLEL, Boolean.toString(closeInParallel)); + config.put(SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER, Boolean.toString(useSingleBuffer)); return config; } diff --git a/src/test/java/com/snowflake/kafka/connector/internal/streaming/DirectStreamingClientHandlerTest.java b/src/test/java/com/snowflake/kafka/connector/internal/streaming/DirectStreamingClientHandlerTest.java index cc07aa020..8c7db64f3 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/streaming/DirectStreamingClientHandlerTest.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/streaming/DirectStreamingClientHandlerTest.java @@ -31,13 +31,11 @@ public class DirectStreamingClientHandlerTest { private StreamingClientHandler streamingClientHandler; private Map connectorConfig; - private Map connectorConfigWithOAuth; @Before public void setup() { this.streamingClientHandler = new DirectStreamingClientHandler(); this.connectorConfig = TestUtils.getConfForStreaming(); - this.connectorConfigWithOAuth = TestUtils.getConfForStreamingWithOAuth(); } @Test @@ -67,9 +65,10 @@ public void testCreateClient() throws Exception { @Test @Ignore // TODO: Remove ignore after SNOW-859929 is released public void testCreateOAuthClient() { - if (this.connectorConfigWithOAuth != null) { + Map connectorConfigWithOAuth = TestUtils.getConfForStreamingWithOAuth(); + if (connectorConfigWithOAuth != null) { this.streamingClientHandler.createClient( - new StreamingClientProperties(this.connectorConfigWithOAuth)); + new StreamingClientProperties(connectorConfigWithOAuth)); } } diff --git a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java index 9c2d3eb05..726b0ac29 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2IT.java @@ -31,11 +31,11 @@ import java.sql.SQLException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Stream; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; @@ -43,14 +43,13 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.sink.SinkRecord; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import org.mockito.Mockito; -@RunWith(Parameterized.class) public class SnowflakeSinkServiceV2IT { private SnowflakeConnectionService conn; @@ -62,32 +61,34 @@ public class SnowflakeSinkServiceV2IT { private String topic = table; private TopicPartition topicPartition = new TopicPartition(topic, partition); - // use OAuth as authenticator or not - private boolean useOAuth; - - @Parameterized.Parameters(name = "useOAuth: {0}") - public static Collection input() { - // TODO: Added {true} after SNOW-352846 is released - return Arrays.asList(new Object[][] {{false}}); - } - - public SnowflakeSinkServiceV2IT(boolean useOAuth) { - this.useOAuth = useOAuth; - if (!useOAuth) { - conn = TestUtils.getConnectionServiceForStreaming(); + private SnowflakeConnectionService getConn(boolean useOAuth) { + if (useOAuth) { + return TestUtils.getOAuthConnectionServiceForStreaming(); } else { - conn = TestUtils.getOAuthConnectionServiceForStreaming(); + return TestUtils.getConnectionServiceForStreaming(); } } - @After + @AfterEach public void afterEach() { TestUtils.dropTable(table); } - @Test - public void testSinkServiceV2Builder() { - Map config = getConfig(); + private static Stream singleBufferParameters() { + return Stream.of(Arguments.of(false), Arguments.of(true)); + } + + private static Stream oAuthAndSingleBufferParameters() { + return Stream.of(Arguments.of(false, false), Arguments.of(false, true)); + // OAuth tests are temporary disabled + // return TestUtils.nBooleanProduct(2); + } + + @ParameterizedTest(name = "useOAuth: {0}, useSingleBuffer: {1}") + @MethodSource("oAuthAndSingleBufferParameters") + public void testSinkServiceV2Builder(boolean useOAuth, boolean useSingleBuffer) { + conn = getConn(useOAuth); + Map config = getConfig(useOAuth, useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); @@ -119,9 +120,12 @@ public void testSinkServiceV2Builder() { }); } - @Test - public void testChannelCloseIngestion() throws Exception { - Map config = getConfig(); + @ParameterizedTest(name = "useOAuth: {0}, useSingleBuffer: {1}") + @MethodSource("oAuthAndSingleBufferParameters") + public void testChannelCloseIngestion(boolean useOAuth, boolean useSingleBuffer) + throws Exception { + conn = getConn(useOAuth); + Map config = getConfig(useOAuth, useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); conn.createTable(table); @@ -167,10 +171,12 @@ public void testChannelCloseIngestion() throws Exception { // just before rebalance, there is data in buffer for other partition, // Send data again for both partitions. // Successfully able to ingest all records - @Test - public void testStreamingIngest_multipleChannelPartitions_closeSubsetOfPartitionsAssigned() - throws Exception { - Map config = TestUtils.getConfForStreaming(); + @ParameterizedTest(name = "useSingleBuffer: {0}") + @MethodSource("singleBufferParameters") + public void testStreamingIngest_multipleChannelPartitions_closeSubsetOfPartitionsAssigned( + boolean useSingleBuffer) throws Exception { + conn = getConn(false); + Map config = TestUtils.getConfForStreaming(useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); conn.createTable(table); TopicPartition tp1 = new TopicPartition(table, partition); @@ -222,7 +228,7 @@ public void testStreamingIngest_multipleChannelPartitions_closeSubsetOfPartition // remaining partition should be present in the map SnowflakeSinkServiceV2 snowflakeSinkServiceV2 = (SnowflakeSinkServiceV2) service; - Assert.assertTrue( + Assertions.assertTrue( snowflakeSinkServiceV2 .getTopicPartitionChannelFromCacheKey(partitionChannelKey(tp2.topic(), tp2.partition())) .isPresent()); @@ -246,9 +252,12 @@ public void testStreamingIngest_multipleChannelPartitions_closeSubsetOfPartition 20); } - @Test - public void testRebalanceOpenCloseIngestion() throws Exception { - Map config = getConfig(); + @ParameterizedTest(name = "useOAuth: {0}, useSingleBuffer: {1}") + @MethodSource("oAuthAndSingleBufferParameters") + public void testRebalanceOpenCloseIngestion(boolean useOAuth, boolean useSingleBuffer) + throws Exception { + conn = getConn(useOAuth); + Map config = getConfig(useOAuth, useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); conn.createTable(table); @@ -291,9 +300,11 @@ public void testRebalanceOpenCloseIngestion() throws Exception { service.closeAll(); } - @Test - public void testStreamingIngestion() throws Exception { - Map config = getConfig(); + @ParameterizedTest(name = "useOAuth: {0}, useSingleBuffer: {1}") + @MethodSource("oAuthAndSingleBufferParameters") + public void testStreamingIngestion(boolean useOAuth, boolean useSingleBuffer) throws Exception { + conn = getConn(useOAuth); + Map config = getConfig(useOAuth, useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); conn.createTable(table); @@ -355,9 +366,12 @@ public void testStreamingIngestion() throws Exception { service.closeAll(); } - @Test - public void testStreamingIngest_multipleChannelPartitions_withMetrics() throws Exception { - Map config = getConfig(); + @ParameterizedTest(name = "useOAuth: {0}, useSingleBuffer: {1}") + @MethodSource("oAuthAndSingleBufferParameters") + public void testStreamingIngest_multipleChannelPartitions_withMetrics( + boolean useOAuth, boolean useSingleBuffer) throws Exception { + conn = getConn(useOAuth); + Map config = getConfig(useOAuth, useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); // set up telemetry service spy @@ -428,7 +442,8 @@ public void testStreamingIngest_multipleChannelPartitions_withMetrics() throws E recordsInPartition1 - 1, recordsInPartition1, 1, - this.conn.getConnectorName()); + this.conn.getConnectorName(), + useSingleBuffer); this.verifyPartitionMetrics( metricRegistry, partitionChannelKey(topic, partition2), @@ -436,7 +451,8 @@ public void testStreamingIngest_multipleChannelPartitions_withMetrics() throws E recordsInPartition2 - 1, recordsInPartition2, 1, - this.conn.getConnectorName()); + this.conn.getConnectorName(), + useSingleBuffer); // verify telemetry Mockito.verify(telemetryService, Mockito.times(2)) @@ -461,7 +477,8 @@ private void verifyPartitionMetrics( long processedOffset, long latestConsumerOffset, long currentTpChannelOpenCount, - String connectorName) { + String connectorName, + boolean useSingleBuffer) { // offsets assert (long) metricRegistry @@ -481,24 +498,29 @@ private void verifyPartitionMetrics( MetricsUtil.PROCESSED_OFFSET)) .getValue() == processedOffset; - assert (long) - metricRegistry - .get( - MetricsUtil.constructMetricName( - partitionChannelKey, - MetricsUtil.OFFSET_SUB_DOMAIN, - MetricsUtil.LATEST_CONSUMER_OFFSET)) - .getValue() - == latestConsumerOffset; + if (!useSingleBuffer) { + assert (long) + metricRegistry + .get( + MetricsUtil.constructMetricName( + partitionChannelKey, + MetricsUtil.OFFSET_SUB_DOMAIN, + MetricsUtil.LATEST_CONSUMER_OFFSET)) + .getValue() + == latestConsumerOffset; + } } - @Test - public void testStreamingIngest_multipleChannelPartitionsWithTopic2Table() throws Exception { + @ParameterizedTest(name = "useSingleBuffer: {0}") + @MethodSource("singleBufferParameters") + public void testStreamingIngest_multipleChannelPartitionsWithTopic2Table(boolean useSingleBuffer) + throws Exception { + conn = getConn(false); final int partitionCount = 3; final int recordsInEachPartition = 2; final int topicCount = 3; - Map config = TestUtils.getConfForStreaming(); + Map config = TestUtils.getConfForStreaming(useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); ArrayList topics = new ArrayList<>(); @@ -559,12 +581,15 @@ public void testStreamingIngest_multipleChannelPartitionsWithTopic2Table() throw service.closeAll(); } - @Test - public void testStreamingIngest_startPartitionsWithMultipleChannelPartitions() throws Exception { + @ParameterizedTest(name = "useSingleBuffer: {0}") + @MethodSource("singleBufferParameters") + public void testStreamingIngest_startPartitionsWithMultipleChannelPartitions( + boolean useSingleBuffer) throws Exception { + conn = getConn(false); final int partitionCount = 5; final int recordsInEachPartition = 2; - Map config = TestUtils.getConfForStreaming(); + Map config = TestUtils.getConfForStreaming(useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); SnowflakeSinkService service = @@ -612,9 +637,12 @@ public void testStreamingIngest_startPartitionsWithMultipleChannelPartitions() t service.closeAll(); } - @Test - public void testStreamingIngestion_timeBased() throws Exception { - Map config = getConfig(); + @ParameterizedTest(name = "useOAuth: {0}, useSingleBuffer: {1}") + @MethodSource("oAuthAndSingleBufferParameters") + public void testStreamingIngestion_timeBased(boolean useOAuth, boolean useSingleBuffer) + throws Exception { + conn = getConn(useOAuth); + Map config = getConfig(useOAuth, useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); conn.createTable(table); @@ -649,9 +677,12 @@ public void testStreamingIngestion_timeBased() throws Exception { service.closeAll(); } - @Test - public void testNativeJsonInputIngestion() throws Exception { - Map config = getConfig(); + @ParameterizedTest(name = "useOAuth: {0}, useSingleBuffer: {1}") + @MethodSource("oAuthAndSingleBufferParameters") + public void testNativeJsonInputIngestion(boolean useOAuth, boolean useSingleBuffer) + throws Exception { + conn = getConn(useOAuth); + Map config = getConfig(useOAuth, useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); conn.createTable(table); @@ -748,9 +779,12 @@ public void testNativeJsonInputIngestion() throws Exception { service.closeAll(); } - @Test - public void testNativeAvroInputIngestion() throws Exception { - Map config = getConfig(); + @ParameterizedTest(name = "useOAuth: {0}, useSingleBuffer: {1}") + @MethodSource("oAuthAndSingleBufferParameters") + public void testNativeAvroInputIngestion(boolean useOAuth, boolean useSingleBuffer) + throws Exception { + conn = getConn(useOAuth); + Map config = getConfig(useOAuth, useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); // avro SchemaBuilder schemaBuilder = @@ -909,9 +943,11 @@ public void testNativeAvroInputIngestion() throws Exception { service.closeAll(); } - @Test - public void testBrokenIngestion() throws Exception { - Map config = getConfig(); + @ParameterizedTest(name = "useOAuth: {0}, useSingleBuffer: {1}") + @MethodSource("oAuthAndSingleBufferParameters") + public void testBrokenIngestion(boolean useOAuth, boolean useSingleBuffer) throws Exception { + conn = getConn(useOAuth); + Map config = getConfig(useOAuth, useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); conn.createTable(table); @@ -978,9 +1014,12 @@ public void testBrokenIngestion() throws Exception { : "expected: " + 0 + " actual: " + TestUtils.tableSize(table); } - @Test - public void testBrokenRecordIngestionFollowedUpByValidRecord() throws Exception { - Map config = getConfig(); + @ParameterizedTest(name = "useOAuth: {0}, useSingleBuffer: {1}") + @MethodSource("oAuthAndSingleBufferParameters") + public void testBrokenRecordIngestionFollowedUpByValidRecord( + boolean useOAuth, boolean useSingleBuffer) throws Exception { + conn = getConn(useOAuth); + Map config = getConfig(useOAuth, useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); conn.createTable(table); @@ -1034,9 +1073,12 @@ public void testBrokenRecordIngestionFollowedUpByValidRecord() throws Exception * broken records (Non valid JSON) followed by another good record with max buffer record size * being 2 */ - @Test - public void testBrokenRecordIngestionAfterValidRecord() throws Exception { - Map config = getConfig(); + @ParameterizedTest(name = "useOAuth: {0}, useSingleBuffer: {1}") + @MethodSource("oAuthAndSingleBufferParameters") + public void testBrokenRecordIngestionAfterValidRecord(boolean useOAuth, boolean useSingleBuffer) + throws Exception { + conn = getConn(useOAuth); + Map config = getConfig(useOAuth, useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); conn.createTable(table); @@ -1090,11 +1132,14 @@ public void testBrokenRecordIngestionAfterValidRecord() throws Exception { /* Service start -> Insert -> Close. service start -> fetch the offsetToken, compare and ingest check data */ - @Test - public void testStreamingIngestionWithExactlyOnceSemanticsNoOverlappingOffsets() - throws Exception { + @ParameterizedTest(name = "useSingleBuffer: {0}") + @MethodSource("singleBufferParameters") + public void testStreamingIngestionWithExactlyOnceSemanticsNoOverlappingOffsets( + boolean useSingleBuffer) throws Exception { + conn = getConn(false); + conn.createTable(table); - Map config = TestUtils.getConfForStreaming(); + Map config = TestUtils.getConfForStreaming(useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); SnowflakeSinkService service = SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE_STREAMING, config) @@ -1149,10 +1194,13 @@ public void testStreamingIngestionWithExactlyOnceSemanticsNoOverlappingOffsets() /* Service start -> Insert -> Close. service start -> fetch the offsetToken, compare and ingest check data */ - @Test - public void testStreamingIngestionWithExactlyOnceSemanticsOverlappingOffsets() throws Exception { + @ParameterizedTest(name = "useSingleBuffer: {0}") + @MethodSource("singleBufferParameters") + public void testStreamingIngestionWithExactlyOnceSemanticsOverlappingOffsets( + boolean useSingleBuffer) throws Exception { + conn = getConn(false); conn.createTable(table); - Map config = TestUtils.getConfForStreaming(); + Map config = TestUtils.getConfForStreaming(useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); SnowflakeSinkService service = SnowflakeSinkServiceFactory.builder(conn, IngestionMethodConfig.SNOWPIPE_STREAMING, config) @@ -1208,9 +1256,12 @@ public void testStreamingIngestionWithExactlyOnceSemanticsOverlappingOffsets() t service2.closeAll(); } - @Test - public void testSchematizationWithTableCreationAndAvroInput() throws Exception { - Map config = TestUtils.getConfForStreaming(); + @ParameterizedTest(name = "useSingleBuffer: {0}") + @MethodSource("singleBufferParameters") + public void testSchematizationWithTableCreationAndAvroInput(boolean useSingleBuffer) + throws Exception { + conn = getConn(false); + Map config = TestUtils.getConfForStreaming(useSingleBuffer); config.put(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, "true"); config.put( SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, @@ -1304,9 +1355,12 @@ public void testSchematizationWithTableCreationAndAvroInput() throws Exception { service.closeAll(); } - @Test - public void testSchematizationWithTableCreationAndJsonInput() throws Exception { - Map config = TestUtils.getConfForStreaming(); + @ParameterizedTest(name = "useSingleBuffer: {0}") + @MethodSource("singleBufferParameters") + public void testSchematizationWithTableCreationAndJsonInput(boolean useSingleBuffer) + throws Exception { + conn = getConn(false); + Map config = TestUtils.getConfForStreaming(useSingleBuffer); config.put(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, "true"); config.put( SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, @@ -1392,9 +1446,12 @@ public void testSchematizationWithTableCreationAndJsonInput() throws Exception { service.closeAll(); } - @Test - public void testSchematizationSchemaEvolutionWithNonNullableColumn() throws Exception { - Map config = TestUtils.getConfForStreaming(); + @ParameterizedTest(name = "useSingleBuffer: {0}") + @MethodSource("singleBufferParameters") + public void testSchematizationSchemaEvolutionWithNonNullableColumn(boolean useSingleBuffer) + throws Exception { + conn = getConn(false); + Map config = TestUtils.getConfForStreaming(useSingleBuffer); config.put(SnowflakeSinkConnectorConfig.ENABLE_SCHEMATIZATION_CONFIG, "true"); config.put( SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, @@ -1478,9 +1535,12 @@ public void testSchematizationSchemaEvolutionWithNonNullableColumn() throws Exce service.closeAll(); } - @Test - public void testStreamingIngestionValidClientLag() throws Exception { - Map config = getConfig(); + @ParameterizedTest(name = "useOAuth: {0}, useSingleBuffer: {1}") + @MethodSource("oAuthAndSingleBufferParameters") + public void testStreamingIngestionValidClientLag(boolean useOAuth, boolean useSingleBuffer) + throws Exception { + conn = getConn(useOAuth); + Map config = getConfig(useOAuth, useSingleBuffer); config.put(SNOWPIPE_STREAMING_MAX_CLIENT_LAG, "30"); SnowflakeSinkConnectorConfig.setDefaultValues(config); conn.createTable(table); @@ -1506,7 +1566,7 @@ public void testStreamingIngestionValidClientLag() throws Exception { // Wait 20 seconds here and no flush should happen since the max client lag is 30 seconds TestUtils.assertWithRetry( () -> service.getOffset(new TopicPartition(topic, partition)) == noOfRecords, 5, 4); - Assert.fail("The rows should not be flushed"); + Assertions.fail("The rows should not be flushed"); } catch (Exception e) { // do nothing } @@ -1518,9 +1578,11 @@ public void testStreamingIngestionValidClientLag() throws Exception { service.closeAll(); } - @Test - public void testStreamingIngestionInvalidClientLag() { - Map config = TestUtils.getConfForStreaming(); + @ParameterizedTest(name = "useSingleBuffer: {0}") + @MethodSource("singleBufferParameters") + public void testStreamingIngestionInvalidClientLag(boolean useSingleBuffer) { + conn = getConn(false); + Map config = TestUtils.getConfForStreaming(useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); Map overriddenConfig = new HashMap<>(config); overriddenConfig.put( @@ -1538,13 +1600,16 @@ public void testStreamingIngestionInvalidClientLag() { .addTask(table, new TopicPartition(topic, partition)) // Internally calls startTask .build(); } catch (IllegalArgumentException ex) { - Assert.assertEquals(NumberFormatException.class, ex.getCause().getClass()); + Assertions.assertEquals(NumberFormatException.class, ex.getCause().getClass()); } } - @Test - public void testStreamingIngestionValidClientPropertiesOverride() throws Exception { - Map config = new HashMap<>(getConfig()); + @ParameterizedTest(name = "useOAuth: {0}, useSingleBuffer: {1}") + @MethodSource("oAuthAndSingleBufferParameters") + public void testStreamingIngestionValidClientPropertiesOverride( + boolean useOAuth, boolean useSingleBuffer) throws Exception { + conn = getConn(useOAuth); + Map config = new HashMap<>(getConfig(useOAuth, useSingleBuffer)); config.put( SNOWPIPE_STREAMING_CLIENT_PROVIDER_OVERRIDE_MAP, "MAX_CHANNEL_SIZE_IN_BYTES:10000000,ENABLE_SNOWPIPE_STREAMING_JMX_METRICS:false"); @@ -1582,9 +1647,12 @@ public void testStreamingIngestionValidClientPropertiesOverride() throws Excepti * Even if override key is invalid, we will still create a client since we dont verify key and * values, only format. */ - @Test - public void testStreamingIngestion_invalidClientPropertiesOverride() throws Exception { - Map config = new HashMap<>(getConfig()); + @ParameterizedTest(name = "useOAuth: {0}, useSingleBuffer: {1}") + @MethodSource("oAuthAndSingleBufferParameters") + public void testStreamingIngestion_invalidClientPropertiesOverride( + boolean useOAuth, boolean useSingleBuffer) throws Exception { + conn = getConn(useOAuth); + Map config = new HashMap<>(getConfig(useOAuth, useSingleBuffer)); config.put(SNOWPIPE_STREAMING_CLIENT_PROVIDER_OVERRIDE_MAP, "MAX_SOMETHING_SOMETHING:1"); SnowflakeSinkConnectorConfig.setDefaultValues(config); conn.createTable(table); @@ -1630,36 +1698,42 @@ private void createNonNullableColumn(String tableName, String colName) { } } - private Map getConfig() { + private Map getConfig(boolean useOAuth, boolean useSingleBuffer) { if (!useOAuth) { - return TestUtils.getConfForStreaming(); + return TestUtils.getConfForStreaming(useSingleBuffer); } else { - return TestUtils.getConfForStreamingWithOAuth(); + return TestUtils.getConfForStreamingWithOAuth(useSingleBuffer); } } // note this test relies on testrole_kafka and testrole_kafka_1 roles being granted to test_kafka // user - @Test - public void testStreamingIngest_multipleChannel_distinctClients() throws Exception { + // todo SNOW-1528892: This test does not pass for oAuth turned on - investigate it and fix + @ParameterizedTest(name = "useSingleBuffer: {0}") + @MethodSource("singleBufferParameters") + public void testStreamingIngest_multipleChannel_distinctClients(boolean useSingleBuffer) + throws Exception { + boolean useOAuth = false; + + conn = getConn(useOAuth); // create cat and dog configs and partitions // one client is enabled but two clients should be created because different roles in config String catTopic = "catTopic_" + TestUtils.randomTableName(); - Map catConfig = getConfig(); + Map catConfig = getConfig(useOAuth, useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(catConfig); catConfig.put(SnowflakeSinkConnectorConfig.ENABLE_STREAMING_CLIENT_OPTIMIZATION_CONFIG, "true"); catConfig.put(Utils.SF_OAUTH_CLIENT_ID, "1"); catConfig.put(Utils.NAME, catTopic); String dogTopic = "dogTopic_" + TestUtils.randomTableName(); - Map dogConfig = getConfig(); + Map dogConfig = getConfig(useOAuth, useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(dogConfig); dogConfig.put(SnowflakeSinkConnectorConfig.ENABLE_STREAMING_CLIENT_OPTIMIZATION_CONFIG, "true"); dogConfig.put(Utils.SF_OAUTH_CLIENT_ID, "2"); dogConfig.put(Utils.NAME, dogTopic); String fishTopic = "fishTopic_" + TestUtils.randomTableName(); - Map fishConfig = getConfig(); + Map fishConfig = getConfig(useOAuth, useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(fishConfig); fishConfig.put( SnowflakeSinkConnectorConfig.ENABLE_STREAMING_CLIENT_OPTIMIZATION_CONFIG, "true"); diff --git a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2StopIT.java b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2StopIT.java index 81840b061..bfe36a484 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2StopIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/streaming/SnowflakeSinkServiceV2StopIT.java @@ -31,17 +31,16 @@ public void afterEach() { TestUtils.dropTable(topicAndTableName); } - @ParameterizedTest + @ParameterizedTest(name = "useSingleBuffer: {0}, optimizationEnabled: {1}") @MethodSource("singleServiceTestCases") public void stop_forSingleService_closesClientDependingOnOptimization( - boolean optimizationEnabled, boolean clientClosed) { + boolean useSingleBuffer, boolean optimizationEnabled) { + final boolean clientClosed = !optimizationEnabled; // given SnowflakeConnectionService conn = TestUtils.getConnectionServiceForStreaming(); - Map config = TestUtils.getConfForStreaming(); + Map config = getConfig(optimizationEnabled, useSingleBuffer); SnowflakeSinkConnectorConfig.setDefaultValues(config); - config.put( - SnowflakeSinkConnectorConfig.ENABLE_STREAMING_CLIENT_OPTIMIZATION_CONFIG, - String.valueOf(optimizationEnabled)); + conn.createTable(topicAndTableName); int partition = 0; TopicPartition topicPartition = new TopicPartition(topicAndTableName, partition); @@ -66,7 +65,18 @@ public void stop_forSingleService_closesClientDependingOnOptimization( Assertions.assertEquals(clientClosed, client.isClosed()); } + private Map getConfig(boolean optimizationEnabled, boolean useSingleBuffer) { + Map config = TestUtils.getConfForStreaming(); + config.put( + SnowflakeSinkConnectorConfig.ENABLE_STREAMING_CLIENT_OPTIMIZATION_CONFIG, + String.valueOf(optimizationEnabled)); + config.put( + SnowflakeSinkConnectorConfig.SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER, + String.valueOf(useSingleBuffer)); + return config; + } + public static Stream singleServiceTestCases() { - return Stream.of(Arguments.of(true, false), Arguments.of(false, true)); + return TestUtils.nBooleanProduct(2); } } diff --git a/src/test/java/com/snowflake/kafka/connector/internal/streaming/StreamingClientProviderIT.java b/src/test/java/com/snowflake/kafka/connector/internal/streaming/StreamingClientProviderIT.java index 79fd44118..8e2da4947 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/streaming/StreamingClientProviderIT.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/streaming/StreamingClientProviderIT.java @@ -22,6 +22,7 @@ import java.util.Map; import net.snowflake.ingest.streaming.SnowflakeStreamingIngestClient; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -29,6 +30,7 @@ public class StreamingClientProviderIT { @Test + @Disabled("CI flaky") public void getClient_forOptimizationEnabled_returnSameClient() { // given Map clientConfig = getClientConfig(true); diff --git a/src/test/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannelTest.java b/src/test/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannelTest.java index 9ab661530..c7584f02a 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannelTest.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/streaming/TopicPartitionChannelTest.java @@ -10,6 +10,8 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyIterable; import static org.mockito.ArgumentMatchers.anyMap; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; @@ -1245,4 +1247,68 @@ public void testOffsetTokenVerificationFunction() { Assert.assertFalse(StreamingUtils.offsetTokenVerificationFunction.verify("1", "3", "4", 3)); Assert.assertFalse(StreamingUtils.offsetTokenVerificationFunction.verify("2", "1", "4", 3)); } + + @Test + public void assignANewChannelAfterTheSetupIsFullyDone() throws Exception { + // given + + SnowflakeStreamingIngestChannel channel1 = Mockito.mock(SnowflakeStreamingIngestChannel.class); + Mockito.when(channel1.getLatestCommittedOffsetToken()) + .thenReturn("0") + .thenThrow(new SFException(ErrorCode.CHANNEL_STATUS_INVALID)); + + Mockito.when(channel1.insertRow(anyMap(), anyString())) + .thenThrow(new SFException(ErrorCode.CHANNEL_STATUS_INVALID)); + Mockito.when(channel1.insertRows(anyIterable(), anyString(), anyString())) + .thenThrow(new SFException(ErrorCode.CHANNEL_STATUS_INVALID)); + + SnowflakeStreamingIngestChannel channel2 = Mockito.mock(SnowflakeStreamingIngestChannel.class); + Mockito.when(channel2.getLatestCommittedOffsetToken()) + .thenThrow(new SFException(ErrorCode.IO_ERROR)); + Mockito.when(channel2.insertRow(anyMap(), anyString())) + .thenReturn(new InsertValidationResponse()); + Mockito.when(channel2.insertRows(anyIterable(), anyString(), anyString())) + .thenReturn(new InsertValidationResponse()); + + Mockito.when(mockStreamingClient.openChannel(any(OpenChannelRequest.class))) + .thenReturn(channel1, channel2); + + TopicPartitionChannel topicPartitionChannel = + createTopicPartitionChannel( + this.mockStreamingClient, + this.topicPartition, + TEST_CHANNEL_NAME, + TEST_TABLE_NAME, + this.enableSchematization, + this.streamingBufferThreshold, + this.sfConnectorConfig, + this.mockKafkaRecordErrorReporter, + this.mockSinkTaskContext, + this.mockSnowflakeConnectionService, + new RecordService(), + this.mockTelemetryService, + true, + null); + + // expect + Assert.assertThrows( + SFException.class, () -> topicPartitionChannel.getOffsetSafeToCommitToKafka()); + + // when + List records = TestUtils.createJsonStringSinkRecords(0, 2, TOPIC, PARTITION); + + // expect + Assert.assertThrows(SFException.class, () -> insertAndFlush(topicPartitionChannel, records)); + } + + private void insertAndFlush(TopicPartitionChannel channel, List records) + throws InterruptedException { + for (int idx = 0; idx < records.size(); idx++) { + channel.insertRecord(records.get(idx), idx == 0); + } + + // expect + Thread.sleep(this.streamingBufferThreshold.getFlushTimeThresholdSeconds() + 1); + channel.insertBufferedRecordsIfFlushTimeThresholdReached(); + } } diff --git a/src/test/java/com/snowflake/kafka/connector/internal/telemetry/SnowflakeTelemetryServiceTest.java b/src/test/java/com/snowflake/kafka/connector/internal/telemetry/SnowflakeTelemetryServiceTest.java index 88cb4edd5..e44b52ef9 100644 --- a/src/test/java/com/snowflake/kafka/connector/internal/telemetry/SnowflakeTelemetryServiceTest.java +++ b/src/test/java/com/snowflake/kafka/connector/internal/telemetry/SnowflakeTelemetryServiceTest.java @@ -1,11 +1,7 @@ package com.snowflake.kafka.connector.internal.telemetry; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.BUFFER_COUNT_RECORDS; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.BUFFER_SIZE_BYTES; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.INGESTION_METHOD_OPT; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.KEY_CONVERTER_CONFIG_FIELD; -import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD; +import static com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig.*; +import static org.junit.jupiter.api.Assertions.*; import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig; import com.snowflake.kafka.connector.Utils; @@ -15,9 +11,7 @@ import com.snowflake.kafka.connector.internal.streaming.telemetry.SnowflakeTelemetryChannelCreation; import com.snowflake.kafka.connector.internal.streaming.telemetry.SnowflakeTelemetryChannelStatus; import com.snowflake.kafka.connector.internal.streaming.telemetry.SnowflakeTelemetryServiceV2; -import java.util.Arrays; import java.util.LinkedList; -import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -26,22 +20,15 @@ import net.snowflake.client.jdbc.internal.fasterxml.jackson.databind.JsonNode; import net.snowflake.client.jdbc.telemetry.Telemetry; import net.snowflake.client.jdbc.telemetry.TelemetryData; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.ValueSource; -@RunWith(Parameterized.class) public class SnowflakeTelemetryServiceTest { - @Parameterized.Parameters(name = "ingestionMethod: {0}") - public static List input() { - return Arrays.asList(IngestionMethodConfig.SNOWPIPE, IngestionMethodConfig.SNOWPIPE_STREAMING); - } - private final IngestionMethodConfig ingestionMethodConfig; - private final SnowflakeTelemetryService snowflakeTelemetryService; - private final long startTime; - private final Map config; + private long startTime; private MockTelemetryClient mockTelemetryClient; public static final String KAFKA_STRING_CONVERTER = @@ -49,51 +36,78 @@ public static List input() { public static final String KAFKA_CONFLUENT_AVRO_CONVERTER = "io.confluent.connect.avro.AvroConverter"; - public SnowflakeTelemetryServiceTest(IngestionMethodConfig ingestionMethodConfig) { + @BeforeEach + void setUp() { this.startTime = System.currentTimeMillis(); - this.ingestionMethodConfig = ingestionMethodConfig; this.mockTelemetryClient = new MockTelemetryClient(); + } - if (this.ingestionMethodConfig == IngestionMethodConfig.SNOWPIPE) { - this.snowflakeTelemetryService = new SnowflakeTelemetryServiceV1(this.mockTelemetryClient); - this.config = TestUtils.getConfig(); - } else { - this.snowflakeTelemetryService = new SnowflakeTelemetryServiceV2(this.mockTelemetryClient); - this.config = TestUtils.getConfForStreaming(); - SnowflakeSinkConnectorConfig.setDefaultValues(this.config); - } - - this.snowflakeTelemetryService.setAppName("TEST_APP"); - this.snowflakeTelemetryService.setTaskID("1"); + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void shouldReportSingleBufferUsageForStreaming(boolean singleBufferEnabled) { + // given + Map connectorConfig = + createConnectorConfig(IngestionMethodConfig.SNOWPIPE_STREAMING); + connectorConfig.put( + SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER, String.valueOf(singleBufferEnabled)); + SnowflakeTelemetryService snowflakeTelemetryService = + createSnowflakeTelemetryService(IngestionMethodConfig.SNOWPIPE_STREAMING, connectorConfig); + + // when + snowflakeTelemetryService.reportKafkaConnectStart(System.currentTimeMillis(), connectorConfig); + + // then + assertEquals( + String.valueOf(singleBufferEnabled), + sentTelemetryDataField(SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER)); } @Test - public void testReportKafkaConnectStart() { - addKeyAndValueConvertersToConfigMap(this.config); + public void shouldReportSingleBufferUsageDefaultValue() { + // given + Map connectorConfig = + createConnectorConfig(IngestionMethodConfig.SNOWPIPE_STREAMING); + SnowflakeTelemetryService snowflakeTelemetryService = + createSnowflakeTelemetryService(IngestionMethodConfig.SNOWPIPE_STREAMING, connectorConfig); + + // when + snowflakeTelemetryService.reportKafkaConnectStart(System.currentTimeMillis(), connectorConfig); + + // then + assertEquals( + String.valueOf(SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER_DEFAULT), + sentTelemetryDataField(SNOWPIPE_STREAMING_ENABLE_SINGLE_BUFFER)); + } - // test report start - this.snowflakeTelemetryService.reportKafkaConnectStart(System.currentTimeMillis(), this.config); + @ParameterizedTest + @EnumSource(value = IngestionMethodConfig.class) + public void testReportKafkaConnectStart(IngestionMethodConfig ingestionMethodConfig) { + // given + Map connectorConfig = createConnectorConfig(ingestionMethodConfig); + connectorConfig.put(KEY_CONVERTER_CONFIG_FIELD, KAFKA_STRING_CONVERTER); + connectorConfig.put( + SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, KAFKA_CONFLUENT_AVRO_CONVERTER); + SnowflakeTelemetryService snowflakeTelemetryService = + createSnowflakeTelemetryService(ingestionMethodConfig, connectorConfig); - // validate data sent + // when + snowflakeTelemetryService.reportKafkaConnectStart(System.currentTimeMillis(), connectorConfig); + + // then LinkedList sentData = this.mockTelemetryClient.getSentTelemetryData(); - Assert.assertEquals(1, sentData.size()); + assertEquals(1, sentData.size()); JsonNode allNode = sentData.get(0).getMessage(); - Assert.assertEquals( + assertEquals( SnowflakeTelemetryService.TelemetryType.KAFKA_START.toString(), allNode.get("type").asText()); - Assert.assertEquals("kafka_connector", allNode.get("source").asText()); - Assert.assertEquals(Utils.VERSION, allNode.get("version").asText()); + assertEquals("kafka_connector", allNode.get("source").asText()); + assertEquals(Utils.VERSION, allNode.get("version").asText()); + + assertEquals(sentTelemetryDataField(INGESTION_METHOD_OPT), ingestionMethodConfig.toString()); JsonNode dataNode = allNode.get("data"); - Assert.assertNotNull(dataNode); - Assert.assertTrue(dataNode.has(INGESTION_METHOD_OPT)); - Assert.assertTrue( - dataNode - .get(INGESTION_METHOD_OPT) - .asText() - .equalsIgnoreCase(this.ingestionMethodConfig.toString())); - Assert.assertTrue( + assertTrue( dataNode.get(TelemetryConstants.START_TIME).asLong() <= System.currentTimeMillis() && dataNode.get(TelemetryConstants.START_TIME).asLong() >= this.startTime); @@ -101,64 +115,79 @@ public void testReportKafkaConnectStart() { validateKeyAndValueConverter(dataNode); } - @Test - public void testReportKafkaConnectStop() { - // test report start - this.snowflakeTelemetryService.reportKafkaConnectStop(System.currentTimeMillis()); + @ParameterizedTest + @EnumSource(value = IngestionMethodConfig.class) + public void testReportKafkaConnectStop(IngestionMethodConfig ingestionMethodConfig) { + // given + Map connectorConfig = createConnectorConfig(ingestionMethodConfig); + SnowflakeTelemetryService snowflakeTelemetryService = + createSnowflakeTelemetryService(ingestionMethodConfig, connectorConfig); - // validate data sent + // when + snowflakeTelemetryService.reportKafkaConnectStop(System.currentTimeMillis()); + + // then LinkedList sentData = this.mockTelemetryClient.getSentTelemetryData(); - Assert.assertEquals(1, sentData.size()); + assertEquals(1, sentData.size()); JsonNode allNode = sentData.get(0).getMessage(); - Assert.assertEquals( + assertEquals( SnowflakeTelemetryService.TelemetryType.KAFKA_STOP.toString(), allNode.get("type").asText()); - Assert.assertEquals("kafka_connector", allNode.get("source").asText()); - Assert.assertEquals(Utils.VERSION, allNode.get("version").asText()); + assertEquals("kafka_connector", allNode.get("source").asText()); + assertEquals(Utils.VERSION, allNode.get("version").asText()); JsonNode dataNode = allNode.get("data"); - Assert.assertNotNull(dataNode); - Assert.assertTrue(dataNode.has(INGESTION_METHOD_OPT)); - Assert.assertTrue( - dataNode.get(INGESTION_METHOD_OPT).asInt() == this.ingestionMethodConfig.ordinal()); - Assert.assertTrue( + assertNotNull(dataNode); + assertTrue(dataNode.has(INGESTION_METHOD_OPT)); + assertTrue(dataNode.get(INGESTION_METHOD_OPT).asInt() == ingestionMethodConfig.ordinal()); + assertTrue( dataNode.get(TelemetryConstants.START_TIME).asLong() <= System.currentTimeMillis() && dataNode.get(TelemetryConstants.START_TIME).asLong() >= this.startTime); } - @Test - public void testReportKafkaConnectFatalError() { - final String expectedException = + @ParameterizedTest + @EnumSource(value = IngestionMethodConfig.class) + public void testReportKafkaConnectFatalError(IngestionMethodConfig ingestionMethodConfig) { + // given + Map connectorConfig = createConnectorConfig(ingestionMethodConfig); + SnowflakeTelemetryService snowflakeTelemetryService = + createSnowflakeTelemetryService(ingestionMethodConfig, connectorConfig); + String expectedException = SnowflakeErrors.ERROR_0003.getException("test exception").getMessage(); - // test report start - this.snowflakeTelemetryService.reportKafkaConnectFatalError(expectedException); + // when + snowflakeTelemetryService.reportKafkaConnectFatalError(expectedException); // validate data sent LinkedList sentData = this.mockTelemetryClient.getSentTelemetryData(); - Assert.assertEquals(1, sentData.size()); + assertEquals(1, sentData.size()); JsonNode allNode = sentData.get(0).getMessage(); - Assert.assertEquals( + assertEquals( SnowflakeTelemetryService.TelemetryType.KAFKA_FATAL_ERROR.toString(), allNode.get("type").asText()); - Assert.assertEquals("kafka_connector", allNode.get("source").asText()); - Assert.assertEquals(Utils.VERSION, allNode.get("version").asText()); + assertEquals("kafka_connector", allNode.get("source").asText()); + assertEquals(Utils.VERSION, allNode.get("version").asText()); JsonNode dataNode = allNode.get("data"); - Assert.assertNotNull(dataNode); - Assert.assertTrue(dataNode.has(INGESTION_METHOD_OPT)); - Assert.assertTrue( - dataNode.get(INGESTION_METHOD_OPT).asInt() == this.ingestionMethodConfig.ordinal()); - Assert.assertTrue( + assertNotNull(dataNode); + assertTrue(dataNode.has(INGESTION_METHOD_OPT)); + assertTrue(dataNode.get(INGESTION_METHOD_OPT).asInt() == ingestionMethodConfig.ordinal()); + assertTrue( dataNode.get("time").asLong() <= System.currentTimeMillis() && dataNode.get("time").asLong() >= this.startTime); - Assert.assertEquals(dataNode.get("error_number").asText(), expectedException); + assertEquals(dataNode.get("error_number").asText(), expectedException); } - @Test - public void testReportKafkaPartitionUsage() { + @ParameterizedTest + @EnumSource(value = IngestionMethodConfig.class) + public void testReportKafkaPartitionUsage(IngestionMethodConfig ingestionMethodConfig) { + // given + Map connectorConfig = createConnectorConfig(ingestionMethodConfig); + SnowflakeTelemetryService snowflakeTelemetryService = + createSnowflakeTelemetryService(ingestionMethodConfig, connectorConfig); + // expected values final String expectedTableName = "tableName"; final String expectedStageName = "stageName"; @@ -174,7 +203,7 @@ public void testReportKafkaPartitionUsage() { SnowflakeTelemetryBasicInfo partitionUsage; - if (this.ingestionMethodConfig == IngestionMethodConfig.SNOWPIPE) { + if (ingestionMethodConfig == IngestionMethodConfig.SNOWPIPE) { SnowflakeTelemetryPipeStatus pipeStatus = new SnowflakeTelemetryPipeStatus( expectedTableName, expectedStageName, expectedPipeName, false, null); @@ -199,67 +228,70 @@ public void testReportKafkaPartitionUsage() { partitionUsage = channelStatus; } - // test report start - this.snowflakeTelemetryService.reportKafkaPartitionUsage(partitionUsage, false); + // when + snowflakeTelemetryService.reportKafkaPartitionUsage(partitionUsage, false); - // validate data sent + // then LinkedList sentData = this.mockTelemetryClient.getSentTelemetryData(); - Assert.assertEquals(1, sentData.size()); + assertEquals(1, sentData.size()); JsonNode allNode = sentData.get(0).getMessage(); - Assert.assertEquals("kafka_connector", allNode.get("source").asText()); - Assert.assertEquals(Utils.VERSION, allNode.get("version").asText()); + assertEquals("kafka_connector", allNode.get("source").asText()); + assertEquals(Utils.VERSION, allNode.get("version").asText()); JsonNode dataNode = allNode.get("data"); - Assert.assertNotNull(dataNode); - Assert.assertTrue(dataNode.has(INGESTION_METHOD_OPT)); - Assert.assertTrue( - dataNode.get(INGESTION_METHOD_OPT).asInt() == this.ingestionMethodConfig.ordinal()); - Assert.assertEquals( + assertNotNull(dataNode); + assertTrue(dataNode.has(INGESTION_METHOD_OPT)); + assertTrue(dataNode.get(INGESTION_METHOD_OPT).asInt() == ingestionMethodConfig.ordinal()); + assertEquals( expectedProcessedOffset, dataNode.get(TelemetryConstants.PROCESSED_OFFSET).asLong()); - Assert.assertEquals(expectedTableName, dataNode.get(TelemetryConstants.TABLE_NAME).asText()); + assertEquals(expectedTableName, dataNode.get(TelemetryConstants.TABLE_NAME).asText()); if (ingestionMethodConfig == IngestionMethodConfig.SNOWPIPE) { - Assert.assertTrue( + assertTrue( dataNode.get(TelemetryConstants.START_TIME).asLong() <= System.currentTimeMillis() && dataNode.get(TelemetryConstants.START_TIME).asLong() >= this.startTime); - Assert.assertEquals( + assertEquals( SnowflakeTelemetryService.TelemetryType.KAFKA_PIPE_USAGE.toString(), allNode.get("type").asText()); - Assert.assertEquals( - expectedFlushedOffset, dataNode.get(TelemetryConstants.FLUSHED_OFFSET).asLong()); - Assert.assertEquals( + assertEquals(expectedFlushedOffset, dataNode.get(TelemetryConstants.FLUSHED_OFFSET).asLong()); + assertEquals( expectedCommittedOffset, dataNode.get(TelemetryConstants.COMMITTED_OFFSET).asLong()); - Assert.assertEquals(expectedPipeName, dataNode.get(TelemetryConstants.PIPE_NAME).asText()); - Assert.assertEquals(expectedStageName, dataNode.get(TelemetryConstants.STAGE_NAME).asText()); + assertEquals(expectedPipeName, dataNode.get(TelemetryConstants.PIPE_NAME).asText()); + assertEquals(expectedStageName, dataNode.get(TelemetryConstants.STAGE_NAME).asText()); } else { - Assert.assertTrue( + assertTrue( dataNode.get(TelemetryConstants.TOPIC_PARTITION_CHANNEL_CREATION_TIME).asLong() == expectedTpChannelCreationTime); - Assert.assertTrue( + assertTrue( dataNode.get(TelemetryConstants.TOPIC_PARTITION_CHANNEL_CLOSE_TIME).asLong() <= System.currentTimeMillis() && dataNode.get(TelemetryConstants.TOPIC_PARTITION_CHANNEL_CLOSE_TIME).asLong() >= this.startTime); - Assert.assertEquals( + assertEquals( SnowflakeTelemetryService.TelemetryType.KAFKA_CHANNEL_USAGE.toString(), allNode.get("type").asText()); - Assert.assertEquals( + assertEquals( expectedLatestConsumerOffset, dataNode.get(TelemetryConstants.LATEST_CONSUMER_OFFSET).asLong()); - Assert.assertEquals( + assertEquals( expectedOffsetPersistedInSnowflake, dataNode.get(TelemetryConstants.OFFSET_PERSISTED_IN_SNOWFLAKE).asLong()); - Assert.assertEquals( + assertEquals( expectedTpChannelName, dataNode.get(TelemetryConstants.TOPIC_PARTITION_CHANNEL_NAME).asText()); - Assert.assertEquals( - expectedConnectorName, dataNode.get(TelemetryConstants.CONNECTOR_NAME).asText()); + assertEquals(expectedConnectorName, dataNode.get(TelemetryConstants.CONNECTOR_NAME).asText()); } } - @Test - public void testReportKafkaPartitionStart() { + @ParameterizedTest + @EnumSource(value = IngestionMethodConfig.class) + public void testReportKafkaPartitionStart(IngestionMethodConfig ingestionMethodConfig) { + // given + Map connectorConfig = createConnectorConfig(ingestionMethodConfig); + SnowflakeTelemetryService snowflakeTelemetryService = + createSnowflakeTelemetryService(ingestionMethodConfig, connectorConfig); + SnowflakeTelemetryBasicInfo partitionCreation; final String expectedTableName = "tableName"; final String expectedStageName = "stageName"; @@ -267,7 +299,7 @@ public void testReportKafkaPartitionStart() { final String expectedChannelName = "channelName"; final long expectedChannelCreationTime = 1234; - if (this.ingestionMethodConfig == IngestionMethodConfig.SNOWPIPE) { + if (ingestionMethodConfig == IngestionMethodConfig.SNOWPIPE) { SnowflakeTelemetryPipeCreation pipeCreation = new SnowflakeTelemetryPipeCreation( expectedTableName, expectedStageName, expectedPipeName); @@ -287,64 +319,89 @@ public void testReportKafkaPartitionStart() { partitionCreation = channelCreation; } - // test report start - this.snowflakeTelemetryService.reportKafkaPartitionStart(partitionCreation); + // when + snowflakeTelemetryService.reportKafkaPartitionStart(partitionCreation); - // validate data sent + // then LinkedList sentData = this.mockTelemetryClient.getSentTelemetryData(); - Assert.assertEquals(1, sentData.size()); + assertEquals(1, sentData.size()); JsonNode allNode = sentData.get(0).getMessage(); - Assert.assertEquals("kafka_connector", allNode.get("source").asText()); - Assert.assertEquals(Utils.VERSION, allNode.get("version").asText()); + assertEquals("kafka_connector", allNode.get("source").asText()); + assertEquals(Utils.VERSION, allNode.get("version").asText()); JsonNode dataNode = allNode.get("data"); - Assert.assertNotNull(dataNode); - Assert.assertTrue(dataNode.has(INGESTION_METHOD_OPT)); - Assert.assertTrue( - dataNode.get(INGESTION_METHOD_OPT).asInt() == this.ingestionMethodConfig.ordinal()); - Assert.assertTrue(dataNode.get(TelemetryConstants.IS_REUSE_TABLE).asBoolean()); - Assert.assertEquals(expectedTableName, dataNode.get(TelemetryConstants.TABLE_NAME).asText()); + assertNotNull(dataNode); + assertTrue(dataNode.has(INGESTION_METHOD_OPT)); + assertTrue(dataNode.get(INGESTION_METHOD_OPT).asInt() == ingestionMethodConfig.ordinal()); + assertTrue(dataNode.get(TelemetryConstants.IS_REUSE_TABLE).asBoolean()); + assertEquals(expectedTableName, dataNode.get(TelemetryConstants.TABLE_NAME).asText()); if (ingestionMethodConfig == IngestionMethodConfig.SNOWPIPE) { - Assert.assertTrue( + assertTrue( dataNode.get(TelemetryConstants.START_TIME).asLong() <= System.currentTimeMillis() && dataNode.get(TelemetryConstants.START_TIME).asLong() >= this.startTime); - Assert.assertEquals( + assertEquals( SnowflakeTelemetryService.TelemetryType.KAFKA_PIPE_START.toString(), allNode.get("type").asText()); - Assert.assertTrue(dataNode.get(TelemetryConstants.IS_REUSE_PIPE).asBoolean()); - Assert.assertTrue(dataNode.get(TelemetryConstants.IS_REUSE_STAGE).asBoolean()); - Assert.assertEquals(10, dataNode.get(TelemetryConstants.FILE_COUNT_REPROCESS_PURGE).asInt()); - Assert.assertEquals(11, dataNode.get(TelemetryConstants.FILE_COUNT_RESTART).asInt()); - Assert.assertEquals(expectedStageName, dataNode.get(TelemetryConstants.STAGE_NAME).asText()); - Assert.assertEquals(expectedPipeName, dataNode.get(TelemetryConstants.PIPE_NAME).asText()); + assertTrue(dataNode.get(TelemetryConstants.IS_REUSE_PIPE).asBoolean()); + assertTrue(dataNode.get(TelemetryConstants.IS_REUSE_STAGE).asBoolean()); + assertEquals(10, dataNode.get(TelemetryConstants.FILE_COUNT_REPROCESS_PURGE).asInt()); + assertEquals(11, dataNode.get(TelemetryConstants.FILE_COUNT_RESTART).asInt()); + assertEquals(expectedStageName, dataNode.get(TelemetryConstants.STAGE_NAME).asText()); + assertEquals(expectedPipeName, dataNode.get(TelemetryConstants.PIPE_NAME).asText()); } else { - Assert.assertTrue( + assertTrue( dataNode.get(TelemetryConstants.TOPIC_PARTITION_CHANNEL_CREATION_TIME).asLong() == expectedChannelCreationTime); - Assert.assertEquals( + assertEquals( SnowflakeTelemetryService.TelemetryType.KAFKA_CHANNEL_START.toString(), allNode.get("type").asText()); - Assert.assertEquals( + assertEquals( expectedChannelName, dataNode.get(TelemetryConstants.TOPIC_PARTITION_CHANNEL_NAME).asText()); } } - private void addKeyAndValueConvertersToConfigMap(Map userProvidedConfig) { - userProvidedConfig.put(KEY_CONVERTER_CONFIG_FIELD, KAFKA_STRING_CONVERTER); - userProvidedConfig.put( - SnowflakeSinkConnectorConfig.VALUE_CONVERTER_CONFIG_FIELD, KAFKA_CONFLUENT_AVRO_CONVERTER); + private Map createConnectorConfig(IngestionMethodConfig ingestionMethodConfig) { + if (ingestionMethodConfig == IngestionMethodConfig.SNOWPIPE) { + return TestUtils.getConfig(); + } else { + return TestUtils.getConfForStreaming(); + } + } + + private SnowflakeTelemetryService createSnowflakeTelemetryService( + IngestionMethodConfig ingestionMethodConfig, Map connectorConfig) { + SnowflakeTelemetryService snowflakeTelemetryService; + + if (ingestionMethodConfig == IngestionMethodConfig.SNOWPIPE) { + snowflakeTelemetryService = new SnowflakeTelemetryServiceV1(mockTelemetryClient); + } else { + snowflakeTelemetryService = new SnowflakeTelemetryServiceV2(mockTelemetryClient); + SnowflakeSinkConnectorConfig.setDefaultValues(connectorConfig); + } + + snowflakeTelemetryService.setAppName("TEST_APP"); + snowflakeTelemetryService.setTaskID("1"); + + return snowflakeTelemetryService; + } + + private String sentTelemetryDataField(String field) { + LinkedList sentData = this.mockTelemetryClient.getSentTelemetryData(); + assertEquals(1, sentData.size()); + JsonNode allNode = sentData.get(0).getMessage(); + return allNode.get("data").get(field).asText(); } private void validateKeyAndValueConverter(JsonNode dataNode) { - Assert.assertTrue(dataNode.has(KEY_CONVERTER_CONFIG_FIELD)); - Assert.assertTrue( + assertTrue(dataNode.has(KEY_CONVERTER_CONFIG_FIELD)); + assertTrue( dataNode.get(KEY_CONVERTER_CONFIG_FIELD).asText().equalsIgnoreCase(KAFKA_STRING_CONVERTER)); - Assert.assertTrue(dataNode.has(VALUE_CONVERTER_CONFIG_FIELD)); - Assert.assertTrue( + assertTrue(dataNode.has(VALUE_CONVERTER_CONFIG_FIELD)); + assertTrue( dataNode .get(VALUE_CONVERTER_CONFIG_FIELD) .asText() @@ -352,14 +409,14 @@ private void validateKeyAndValueConverter(JsonNode dataNode) { } private void validateBufferProperties(JsonNode dataNode) { - Assert.assertTrue(dataNode.has(BUFFER_SIZE_BYTES)); - Assert.assertTrue(isNumeric(dataNode.get(BUFFER_SIZE_BYTES).asText())); + assertTrue(dataNode.has(BUFFER_SIZE_BYTES)); + assertTrue(isNumeric(dataNode.get(BUFFER_SIZE_BYTES).asText())); - Assert.assertTrue(dataNode.has(BUFFER_COUNT_RECORDS)); - Assert.assertTrue(isNumeric(dataNode.get(BUFFER_COUNT_RECORDS).asText())); + assertTrue(dataNode.has(BUFFER_COUNT_RECORDS)); + assertTrue(isNumeric(dataNode.get(BUFFER_COUNT_RECORDS).asText())); - Assert.assertTrue(dataNode.has(BUFFER_FLUSH_TIME_SEC)); - Assert.assertTrue(isNumeric(dataNode.get(BUFFER_FLUSH_TIME_SEC).asText())); + assertTrue(dataNode.has(BUFFER_FLUSH_TIME_SEC)); + assertTrue(isNumeric(dataNode.get(BUFFER_FLUSH_TIME_SEC).asText())); } private static boolean isNumeric(String strNum) { diff --git a/src/test/resources/log4j.properties b/src/test/resources/log4j.properties index ad7e53809..73752bc41 100644 --- a/src/test/resources/log4j.properties +++ b/src/test/resources/log4j.properties @@ -1,4 +1,4 @@ -log4j.rootLogger=ALL, STDOUT, file +log4j.rootLogger=INFO, STDOUT, file log4j.logger.deng=INFO log4j.appender.STDOUT=org.apache.log4j.ConsoleAppender log4j.appender.STDOUT.layout=org.apache.log4j.PatternLayout diff --git a/test/rest_request_template/travis_correct_schema_evolution_drop_table.json b/test/rest_request_template/travis_correct_schema_evolution_drop_table.json index 0c6523573..6c2ea6ea6 100644 --- a/test/rest_request_template/travis_correct_schema_evolution_drop_table.json +++ b/test/rest_request_template/travis_correct_schema_evolution_drop_table.json @@ -15,7 +15,7 @@ "snowflake.schema.name": "SNOWFLAKE_SCHEMA", "snowflake.role.name": "SNOWFLAKE_ROLE", "snowflake.ingestion.method": "SNOWPIPE_STREAMING", - "snowflake.streaming.enable.single.buffer": "false", + "snowflake.streaming.enable.single.buffer": "$SNOWFLAKE_STREAMING_ENABLE_SINGLE_BUFFER", "snowflake.streaming.closeChannelsInParallel.enabled": true, "key.converter": "org.apache.kafka.connect.storage.StringConverter", "value.converter": "org.apache.kafka.connect.json.JsonConverter", diff --git a/test/rest_request_template/travis_correct_schema_evolution_multi_topic_drop_table.json b/test/rest_request_template/travis_correct_schema_evolution_multi_topic_drop_table.json index ad9fdc941..c3c516d72 100644 --- a/test/rest_request_template/travis_correct_schema_evolution_multi_topic_drop_table.json +++ b/test/rest_request_template/travis_correct_schema_evolution_multi_topic_drop_table.json @@ -15,7 +15,7 @@ "snowflake.schema.name": "SNOWFLAKE_SCHEMA", "snowflake.role.name": "SNOWFLAKE_ROLE", "snowflake.ingestion.method": "SNOWPIPE_STREAMING", - "snowflake.streaming.enable.single.buffer": "false", + "snowflake.streaming.enable.single.buffer": "$SNOWFLAKE_STREAMING_ENABLE_SINGLE_BUFFER", "snowflake.streaming.closeChannelsInParallel.enabled": true, "key.converter": "org.apache.kafka.connect.storage.StringConverter", "value.converter": "org.apache.kafka.connect.json.JsonConverter", diff --git a/test/test_suit/test_confluent_protobuf_protobuf.py b/test/test_suit/test_confluent_protobuf_protobuf.py index fe168063f..0ad6ff508 100644 --- a/test/test_suit/test_confluent_protobuf_protobuf.py +++ b/test/test_suit/test_confluent_protobuf_protobuf.py @@ -29,8 +29,9 @@ def __init__(self, driver, nameSalt): self.sensor.uint64_val = (1 << 64) - 1 self.schema_registry_client = SchemaRegistryClient({'url': driver.schemaRegistryAddress}) - self.keyProtobufSerializer = ProtobufSerializer(sensor_pb2.SensorReading, self.schema_registry_client) - self.valueProtobufSerializer = ProtobufSerializer(sensor_pb2.SensorReading, self.schema_registry_client) + self.keyProtobufSerializer = ProtobufSerializer(sensor_pb2.SensorReading, self.schema_registry_client, {'use.deprecated.format': True}) + self.valueProtobufSerializer = ProtobufSerializer(sensor_pb2.SensorReading, self.schema_registry_client, {'use.deprecated.format': True}) + producer_conf = { 'bootstrap.servers': driver.kafkaAddress, 'key.serializer': self.keyProtobufSerializer, diff --git a/test/test_suites.py b/test/test_suites.py index bc15ed6ac..eae8d0264 100644 --- a/test/test_suites.py +++ b/test/test_suites.py @@ -203,7 +203,6 @@ def create_end_to_end_test_suites(driver, nameSalt, schemaRegistryAddress, testS test_instance=TestSchemaEvolutionAvroSR(driver, nameSalt), clean=True, run_in_confluent=True, run_in_apache=False )), - # SNOW-947731: Re-enable after avro-python3 package is updated in merge gate ("TestSchemaEvolutionAvroSRLogicalTypes", EndToEndTestSuite( test_instance=TestSchemaEvolutionAvroSRLogicalTypes(driver, nameSalt), clean=True, run_in_confluent=False, run_in_apache=False diff --git a/upload_jar.sh b/upload_jar.sh new file mode 100755 index 000000000..873834d21 --- /dev/null +++ b/upload_jar.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash + + +if ! VERSION=$(xmllint --xpath '/*[local-name()="project"]/*[local-name()="version"]/text()' pom.xml) +then + echo "failed to read version from pom.xml" + exit 1 +fi +echo "version to upload: $VERSION" + +if ! API_KEY_SECRET_ID=$(op item list --tags "connectors-nexus-api-key" --format json | jq -r '.[].id') +then + echo "failed to find required api key in 1password" + exit 1 +fi + +if ! USER_PASS=$(op item get $API_KEY_SECRET_ID --format json | jq -r '.fields[] | select(.type=="CONCEALED") | .value') +then + echo 'failed to read user:password from 1password' + exit 1 +fi + +FILE="https://nexus.int.snowflakecomputing.com/repository/connectors/snowflake-kafka-connector-$USER-$VERSION.jar" + +echo trying to delete $FILE.... +curl -X DELETE \ + -u $USER_PASS \ + $FILE + +echo uploading new file to $FILE... +curl --fail \ + --upload-file ./target/snowflake-kafka-connector-$VERSION.jar \ + -u $USER_PASS \ + -w "\nHTTP Status: %{http_code}\n" \ + $FILE +