From 047b9530823bbbdd638f16d7843c8fbd093641a6 Mon Sep 17 00:00:00 2001 From: David Sloan <33483659+davidsloan@users.noreply.github.com> Date: Fri, 23 Feb 2024 14:48:08 +0000 Subject: [PATCH] Refactoring to enable additional sources (#1032) * GCP Storage Source Refactoring * Refactors to allow reusability of components in both sink and source and reduce code existing code. * Remove LazyLogging from CloudConfigDef subclasses * Reduce implicit usage of the validator on Source and Sink tasks --- .../aws/s3/sink/S3AvroWriterManagerTest.scala | 2 +- .../aws/s3/sink/S3JsonWriterManagerTest.scala | 6 +- .../s3/sink/S3ParquetWriterManagerTest.scala | 2 +- .../connect/aws/s3/sink/S3SinkTaskTest.scala | 5 +- .../s3/source/S3SourceAvroEnvelopeTest.scala | 4 +- ...urceAvroWithValueAsArrayEnvelopeTest.scala | 4 +- ...WithValueAsOptionalArrayEnvelopeTest.scala | 6 +- .../s3/source/S3SourceJsonEnvelopeTest.scala | 4 +- .../source/S3SourceParquetEnvelopeTest.scala | 4 +- .../source/S3SourceTaskBucketRootTest.scala | 4 +- .../aws/s3/source/S3SourceTaskTest.scala | 4 +- .../s3/source/S3SourceTaskXmlReaderTest.scala | 4 +- .../aws/s3/utils/S3ProxyContainerTest.scala | 7 +- .../sink/CloudPlatformEmulatorSuite.scala | 10 +- .../common/sink/CoreSinkTaskTestCases.scala | 14 +- .../aws/s3/auth/AwsS3ClientCreator.scala | 10 +- .../aws/s3/config/S3ConfigSettings.scala | 7 - ...3Config.scala => S3ConnectionConfig.scala} | 13 +- .../connect/aws/s3/sink/S3SinkTask.scala | 44 ++--- .../config/S3ConsumerGroupsSinkConfig.scala | 4 +- .../aws/s3/sink/config/S3SinkConfig.scala | 31 ++-- .../aws/s3/sink/config/S3SinkConfigDef.scala | 43 +---- .../connect/aws/s3/source/S3SourceTask.scala | 120 +++++-------- .../aws/s3/source/config/S3SourceConfig.scala | 121 +++---------- .../s3/source/config/S3SourceConfigDef.scala | 89 +--------- .../config/S3SourceConfigDefBuilder.scala | 12 +- .../SourcePartitionSearcherSettings.scala | 92 ---------- ...archer.scala => S3PartitionSearcher.scala} | 21 ++- .../aws/s3/source/state/S3SourceBuilder.scala | 89 ---------- .../connect/aws/s3/config/S3ConfigTest.scala | 7 +- .../S3ConsumerGroupsSinkConfigTest.scala | 5 +- .../S3ConsumerGroupsSinkConfigTest.scala | 8 +- .../config/S3SinkConfigDefBuilderTest.scala | 20 ++- .../aws/s3/sink/config/S3SinkConfigTest.scala | 12 +- .../s3/source/config/S3SourceConfigTest.scala | 3 +- .../source/config/S3SourceConfigTests.scala | 13 +- .../reader/S3PartitionDiscoveryTest.scala | 50 +++--- .../state/ReaderManagerBuilderTest.scala | 13 +- .../datalake/auth/DatalakeClientCreator.scala | 14 +- ...nfig.scala => AzureConnectionConfig.scala} | 13 +- .../datalake/sink/DatalakeSinkTask.scala | 44 ++--- .../sink/config/DatalakeSinkConfig.scala | 21 +-- .../sink/config/DatalakeSinkConfigDef.scala | 43 +---- .../cloud/common/auth/ClientCreator.scala | 4 +- .../cloud/common/config/CloudConfigDef.scala | 62 +++++++ .../{CloudConfig.scala => RetryConfig.scala} | 2 +- .../common/config/traits/CloudConfig.scala | 101 +++++++++++ .../config/traits/CloudConnectionConfig.scala | 40 +++++ .../traits/PropsToConfigConverter.scala | 51 ++++++ .../cloud/common/sink/CloudSinkTask.scala | 51 +++++- .../common/sink/WriterManagerCreator.scala | 2 +- .../sink/config/CloudSinkBucketOptions.scala | 6 +- .../cloud/common/source/CloudSourceTask.scala | 163 ++++++++++++++++++ .../config/CloudSourceBucketOptions.scala | 90 ++++++++++ .../config/CloudSourceConfigDefBuilder.scala | 38 ++++ .../source/config/CloudSourceSettings.scala | 57 ++++++ .../config/CloudSourceSettingsKeys.scala | 120 +++++++++++++ .../source/config/PartitionExtractor.scala | 1 + .../source/config/S3SourceBucketOptions.scala | 15 ++ .../source/state/CloudSourceTaskState.scala | 9 +- .../source/state/PartitionSearcher.scala | 36 ++++ .../source/state/ReaderManagerBuilder.scala | 18 +- .../PropsParser.scala} | 11 +- .../sink/WriterManagerCreatorTest.scala | 13 +- .../sink/CloudPlatformEmulatorSuite.scala | 10 +- .../common/sink/CoreSinkTaskTestCases.scala | 14 +- .../storage/sink/GCPStorageSinkTaskTest.scala | 9 +- .../storage/utils/GCPProxyContainerTest.scala | 13 +- .../auth/GCPStorageClientCreator.scala | 8 +- .../storage/config/GCPConfigSettings.scala | 4 + ...Config.scala => GCPConnectionConfig.scala} | 13 +- .../gcp/storage/sink/GCPStorageSinkTask.scala | 46 ++--- .../sink/config/GCPStorageSinkConfig.scala | 21 +-- .../sink/config/GCPStorageSinkConfigDef.scala | 43 +---- .../config/GCPStorageSourceConfig.scala | 68 ++++++++ .../config/GCPStorageSourceConfigDef.scala | 37 ++++ .../GCPStorageSourceConfigDefBuilder.scala | 30 ++++ .../auth/GCPStorageClientCreatorTest.scala | 6 +- .../gcp/storage/config/GCPConfigTest.scala | 23 +-- ...geGCPStorageSinkConfigDefBuilderTest.scala | 20 ++- .../config/GCPStorageSinkConfigTest.scala | 12 +- .../config/GCPStorageSourceConfigTest.scala | 91 ++++++++++ 82 files changed, 1506 insertions(+), 894 deletions(-) rename kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/config/{S3Config.scala => S3ConnectionConfig.scala} (92%) delete mode 100644 kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/config/SourcePartitionSearcherSettings.scala rename kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/distribution/{PartitionSearcher.scala => S3PartitionSearcher.scala} (78%) delete mode 100644 kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/state/S3SourceBuilder.scala rename kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/config/{AzureConfig.scala => AzureConnectionConfig.scala} (91%) create mode 100644 kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/CloudConfigDef.scala rename kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/{CloudConfig.scala => RetryConfig.scala} (90%) create mode 100644 kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/traits/CloudConfig.scala create mode 100644 kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/traits/CloudConnectionConfig.scala create mode 100644 kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/traits/PropsToConfigConverter.scala create mode 100644 kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/CloudSourceTask.scala create mode 100644 kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/CloudSourceBucketOptions.scala create mode 100644 kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/CloudSourceConfigDefBuilder.scala create mode 100644 kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/CloudSourceSettings.scala create mode 100644 kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/CloudSourceSettingsKeys.scala create mode 100644 kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/S3SourceBucketOptions.scala create mode 100644 kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/state/PartitionSearcher.scala rename {kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3 => kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common}/source/state/ReaderManagerBuilder.scala (82%) rename kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/{sink/config/CloudSinkConfig.scala => traits/PropsParser.scala} (66%) rename kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/config/{GCPConfig.scala => GCPConnectionConfig.scala} (90%) create mode 100644 kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/source/config/GCPStorageSourceConfig.scala create mode 100644 kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/source/config/GCPStorageSourceConfigDef.scala create mode 100644 kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/source/config/GCPStorageSourceConfigDefBuilder.scala create mode 100644 kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/source/config/GCPStorageSourceConfigTest.scala diff --git a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3AvroWriterManagerTest.scala b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3AvroWriterManagerTest.scala index feaf14260..39100e7d3 100644 --- a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3AvroWriterManagerTest.scala +++ b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3AvroWriterManagerTest.scala @@ -73,7 +73,7 @@ class S3AvroWriterManagerTest extends AnyFlatSpec with Matchers with S3ProxyCont private implicit val cloudLocationValidator = S3LocationValidator private val bucketAndPrefix = CloudLocation(BucketName, PathPrefix.some) private def avroConfig = S3SinkConfig( - S3Config( + S3ConnectionConfig( None, Some(s3Container.identity.identity), Some(s3Container.identity.credential), diff --git a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3JsonWriterManagerTest.scala b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3JsonWriterManagerTest.scala index 9402908b7..84e82f199 100644 --- a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3JsonWriterManagerTest.scala +++ b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3JsonWriterManagerTest.scala @@ -69,7 +69,7 @@ class S3JsonWriterManagerTest extends AnyFlatSpec with Matchers with S3ProxyCont val bucketAndPrefix = CloudLocation(BucketName, PathPrefix.some) val config = S3SinkConfig( - S3Config( + S3ConnectionConfig( None, Some(s3Container.identity.identity), Some(s3Container.identity.credential), @@ -123,7 +123,7 @@ class S3JsonWriterManagerTest extends AnyFlatSpec with Matchers with S3ProxyCont val bucketAndPrefix = CloudLocation(BucketName, PathPrefix.some) val config = S3SinkConfig( - S3Config( + S3ConnectionConfig( None, Some(s3Container.identity.identity), Some(s3Container.identity.credential), @@ -181,7 +181,7 @@ class S3JsonWriterManagerTest extends AnyFlatSpec with Matchers with S3ProxyCont val bucketAndPrefix = CloudLocation(BucketName, PathPrefix.some) val config = S3SinkConfig( - S3Config( + S3ConnectionConfig( None, Some(s3Container.identity.identity), Some(s3Container.identity.credential), diff --git a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3ParquetWriterManagerTest.scala b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3ParquetWriterManagerTest.scala index 100a9b610..548dda544 100644 --- a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3ParquetWriterManagerTest.scala +++ b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3ParquetWriterManagerTest.scala @@ -69,7 +69,7 @@ class S3ParquetWriterManagerTest extends AnyFlatSpec with Matchers with S3ProxyC private val bucketAndPrefix = CloudLocation(BucketName, PathPrefix.some) private def parquetConfig = S3SinkConfig( - S3Config( + S3ConnectionConfig( None, Some(s3Container.identity.identity), Some(s3Container.identity.credential), diff --git a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3SinkTaskTest.scala b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3SinkTaskTest.scala index ce9ec9cbc..ee8cefb4d 100644 --- a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3SinkTaskTest.scala +++ b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3SinkTaskTest.scala @@ -18,6 +18,7 @@ package io.lenses.streamreactor.connect.aws.s3.sink import cats.effect.IO import cats.effect.kernel.Resource import cats.effect.unsafe.implicits.global +import io.lenses.streamreactor.connect.aws.s3.sink.config.S3SinkConfig import io.lenses.streamreactor.connect.aws.s3.storage.AwsS3StorageInterface import io.lenses.streamreactor.connect.aws.s3.storage.S3FileMetadata import io.lenses.streamreactor.connect.aws.s3.utils.S3ProxyContainerTest @@ -27,7 +28,9 @@ import software.amazon.awssdk.services.s3.S3Client import scala.jdk.CollectionConverters.MapHasAsJava class S3SinkTaskTest - extends CoreSinkTaskTestCases[S3FileMetadata, AwsS3StorageInterface, S3SinkTask, S3Client]("S3SinkTask") + extends CoreSinkTaskTestCases[S3FileMetadata, AwsS3StorageInterface, S3SinkConfig, S3Client, S3SinkTask]( + "S3SinkTask", + ) with S3ProxyContainerTest { "S3SinkTask" should "fail with message when deprecated properties are used" in { diff --git a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceAvroEnvelopeTest.scala b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceAvroEnvelopeTest.scala index 28f169208..5e79b6b9b 100644 --- a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceAvroEnvelopeTest.scala +++ b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceAvroEnvelopeTest.scala @@ -1,10 +1,10 @@ package io.lenses.streamreactor.connect.aws.s3.source import cats.implicits.catsSyntaxEitherId -import io.lenses.streamreactor.connect.aws.s3.source.config.SourcePartitionSearcherSettingsKeys import io.lenses.streamreactor.connect.aws.s3.storage.AwsS3StorageInterface import io.lenses.streamreactor.connect.aws.s3.utils.S3ProxyContainerTest import io.lenses.streamreactor.connect.cloud.common.model.UploadableFile +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceSettingsKeys import org.apache.avro.SchemaBuilder import org.apache.avro.file.CodecFactory import org.apache.avro.file.DataFileWriter @@ -26,7 +26,7 @@ class S3SourceAvroEnvelopeTest with AnyFlatSpecLike with Matchers with EitherValues - with SourcePartitionSearcherSettingsKeys { + with CloudSourceSettingsKeys { def DefaultProps: Map[String, String] = defaultProps + ( SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS -> "1000", diff --git a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceAvroWithValueAsArrayEnvelopeTest.scala b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceAvroWithValueAsArrayEnvelopeTest.scala index fa18f19ed..ab3e64dcc 100644 --- a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceAvroWithValueAsArrayEnvelopeTest.scala +++ b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceAvroWithValueAsArrayEnvelopeTest.scala @@ -1,10 +1,10 @@ package io.lenses.streamreactor.connect.aws.s3.source import cats.implicits.catsSyntaxEitherId -import io.lenses.streamreactor.connect.aws.s3.source.config.SourcePartitionSearcherSettingsKeys import io.lenses.streamreactor.connect.aws.s3.storage.AwsS3StorageInterface import io.lenses.streamreactor.connect.aws.s3.utils.S3ProxyContainerTest import io.lenses.streamreactor.connect.cloud.common.model.UploadableFile +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceSettingsKeys import org.apache.avro.SchemaBuilder import org.apache.avro.file.CodecFactory import org.apache.avro.file.DataFileWriter @@ -28,7 +28,7 @@ class S3SourceAvroWithValueAsArrayEnvelopeTest with AnyFlatSpecLike with Matchers with EitherValues - with SourcePartitionSearcherSettingsKeys { + with CloudSourceSettingsKeys { def DefaultProps: Map[String, String] = defaultProps + ( SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS -> "1000", diff --git a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceAvroWithValueAsOptionalArrayEnvelopeTest.scala b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceAvroWithValueAsOptionalArrayEnvelopeTest.scala index 1100731be..f7fb7f7c1 100644 --- a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceAvroWithValueAsOptionalArrayEnvelopeTest.scala +++ b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceAvroWithValueAsOptionalArrayEnvelopeTest.scala @@ -1,10 +1,10 @@ package io.lenses.streamreactor.connect.aws.s3.source import cats.implicits.catsSyntaxEitherId -import io.lenses.streamreactor.connect.aws.s3.source.config.SourcePartitionSearcherSettingsKeys import io.lenses.streamreactor.connect.aws.s3.storage.AwsS3StorageInterface import io.lenses.streamreactor.connect.aws.s3.utils.S3ProxyContainerTest import io.lenses.streamreactor.connect.cloud.common.model.UploadableFile +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceSettingsKeys import org.apache.avro import org.apache.avro.SchemaBuilder import org.apache.avro.file.CodecFactory @@ -28,7 +28,7 @@ class S3SourceAvroWithValueAsOptionalArrayEnvelopeTest with AnyFlatSpecLike with Matchers with EitherValues - with SourcePartitionSearcherSettingsKeys { + with CloudSourceSettingsKeys { def DefaultProps: Map[String, String] = defaultProps + ( SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS -> "1000", @@ -155,7 +155,7 @@ class S3SourceAvroWithValueAsOptionalArrayMixValuesEnvelopeTest with AnyFlatSpecLike with Matchers with EitherValues - with SourcePartitionSearcherSettingsKeys { + with CloudSourceSettingsKeys { def DefaultProps: Map[String, String] = defaultProps + ( SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS -> "1000", diff --git a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceJsonEnvelopeTest.scala b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceJsonEnvelopeTest.scala index 5df2a5c36..d158b6951 100644 --- a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceJsonEnvelopeTest.scala +++ b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceJsonEnvelopeTest.scala @@ -1,10 +1,10 @@ package io.lenses.streamreactor.connect.aws.s3.source import cats.implicits.catsSyntaxEitherId -import io.lenses.streamreactor.connect.aws.s3.source.config.SourcePartitionSearcherSettingsKeys import io.lenses.streamreactor.connect.aws.s3.storage.AwsS3StorageInterface import io.lenses.streamreactor.connect.aws.s3.utils.S3ProxyContainerTest import io.lenses.streamreactor.connect.cloud.common.model.UploadableFile +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceSettingsKeys import org.apache.kafka.connect.source.SourceRecord import org.scalatest.EitherValues import org.scalatest.concurrent.Eventually.eventually @@ -22,7 +22,7 @@ class S3SourceJsonEnvelopeTest with AnyFlatSpecLike with Matchers with EitherValues - with SourcePartitionSearcherSettingsKeys { + with CloudSourceSettingsKeys { def DefaultProps: Map[String, String] = defaultProps + ( SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS -> "1000", diff --git a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceParquetEnvelopeTest.scala b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceParquetEnvelopeTest.scala index 21ae2cc1c..05b0d66ed 100644 --- a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceParquetEnvelopeTest.scala +++ b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceParquetEnvelopeTest.scala @@ -1,11 +1,11 @@ package io.lenses.streamreactor.connect.aws.s3.source import cats.implicits.catsSyntaxEitherId -import io.lenses.streamreactor.connect.aws.s3.source.config.SourcePartitionSearcherSettingsKeys import io.lenses.streamreactor.connect.aws.s3.storage.AwsS3StorageInterface import io.lenses.streamreactor.connect.aws.s3.utils.S3ProxyContainerTest import io.lenses.streamreactor.connect.cloud.common.formats.writer.parquet.ParquetOutputFile import io.lenses.streamreactor.connect.cloud.common.model.UploadableFile +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceSettingsKeys import io.lenses.streamreactor.connect.cloud.common.stream.CloudByteArrayOutputStream import org.apache.avro.SchemaBuilder import org.apache.kafka.connect.source.SourceRecord @@ -28,7 +28,7 @@ class S3SourceParquetEnvelopeTest with AnyFlatSpecLike with Matchers with EitherValues - with SourcePartitionSearcherSettingsKeys { + with CloudSourceSettingsKeys { def DefaultProps: Map[String, String] = defaultProps + ( SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS -> "1000", diff --git a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceTaskBucketRootTest.scala b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceTaskBucketRootTest.scala index 02e8c70c2..91c4c828c 100644 --- a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceTaskBucketRootTest.scala +++ b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceTaskBucketRootTest.scala @@ -3,8 +3,8 @@ package io.lenses.streamreactor.connect.aws.s3.source import cats.implicits._ import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings._ import io.lenses.streamreactor.connect.aws.s3.source.S3SourceTaskTest.formats -import io.lenses.streamreactor.connect.aws.s3.source.config.SourcePartitionSearcherSettingsKeys import io.lenses.streamreactor.connect.aws.s3.utils.S3ProxyContainerTest +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceSettingsKeys import org.scalatest.EitherValues import org.scalatest.concurrent.Eventually.eventually import org.scalatest.flatspec.AnyFlatSpecLike @@ -20,7 +20,7 @@ class S3SourceTaskBucketRootTest with AnyFlatSpecLike with Matchers with EitherValues - with SourcePartitionSearcherSettingsKeys { + with CloudSourceSettingsKeys { def DefaultProps: Map[String, String] = defaultProps + ( SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS -> "1000", diff --git a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceTaskTest.scala b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceTaskTest.scala index 9d3b4c62a..26158dba5 100644 --- a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceTaskTest.scala +++ b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceTaskTest.scala @@ -6,7 +6,6 @@ import com.typesafe.scalalogging.LazyLogging import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings._ import io.lenses.streamreactor.connect.aws.s3.model.location.S3LocationValidator import io.lenses.streamreactor.connect.aws.s3.source.S3SourceTaskTest.formats -import io.lenses.streamreactor.connect.aws.s3.source.config.SourcePartitionSearcherSettingsKeys import io.lenses.streamreactor.connect.aws.s3.storage.AwsS3DirectoryLister import io.lenses.streamreactor.connect.aws.s3.storage.AwsS3StorageInterface import io.lenses.streamreactor.connect.aws.s3.utils.S3ProxyContainerTest @@ -16,6 +15,7 @@ import io.lenses.streamreactor.connect.cloud.common.config.Format.Bytes import io.lenses.streamreactor.connect.cloud.common.config.FormatOptions import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocation import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocationValidator +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceSettingsKeys import io.lenses.streamreactor.connect.cloud.common.storage.DirectoryFindCompletionConfig import io.lenses.streamreactor.connect.cloud.common.storage.DirectoryFindResults import org.apache.kafka.connect.source.SourceTaskContext @@ -51,7 +51,7 @@ class S3SourceTaskTest with LazyLogging with BeforeAndAfter with Eventually - with SourcePartitionSearcherSettingsKeys { + with CloudSourceSettingsKeys { override implicit def patienceConfig: PatienceConfig = PatienceConfig(timeout = Span(10, Seconds), interval = Span(500, Milliseconds)) diff --git a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceTaskXmlReaderTest.scala b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceTaskXmlReaderTest.scala index eb084789d..0ca3e8cb6 100644 --- a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceTaskXmlReaderTest.scala +++ b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceTaskXmlReaderTest.scala @@ -1,10 +1,10 @@ package io.lenses.streamreactor.connect.aws.s3.source import cats.implicits._ -import io.lenses.streamreactor.connect.aws.s3.source.config.SourcePartitionSearcherSettingsKeys import io.lenses.streamreactor.connect.aws.s3.storage.AwsS3StorageInterface import io.lenses.streamreactor.connect.aws.s3.utils.S3ProxyContainerTest import io.lenses.streamreactor.connect.cloud.common.model.UploadableFile +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceSettingsKeys import org.apache.kafka.connect.source.SourceRecord import org.scalatest.EitherValues import org.scalatest.concurrent.Eventually.eventually @@ -22,7 +22,7 @@ class S3SourceTaskXmlReaderTest with AnyFlatSpecLike with Matchers with EitherValues - with SourcePartitionSearcherSettingsKeys { + with CloudSourceSettingsKeys { def DefaultProps: Map[String, String] = defaultProps + ( SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS -> "1000", diff --git a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/utils/S3ProxyContainerTest.scala b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/utils/S3ProxyContainerTest.scala index 1e7a22e4e..5ae2a61b4 100644 --- a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/utils/S3ProxyContainerTest.scala +++ b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/aws/s3/utils/S3ProxyContainerTest.scala @@ -2,9 +2,10 @@ package io.lenses.streamreactor.connect.aws.s3.utils import com.typesafe.scalalogging.LazyLogging import io.lenses.streamreactor.connect.aws.s3.auth.AwsS3ClientCreator import io.lenses.streamreactor.connect.aws.s3.config.AuthMode -import io.lenses.streamreactor.connect.aws.s3.config.S3Config +import io.lenses.streamreactor.connect.aws.s3.config.S3ConnectionConfig import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings._ import io.lenses.streamreactor.connect.aws.s3.sink.S3SinkTask +import io.lenses.streamreactor.connect.aws.s3.sink.config.S3SinkConfig import io.lenses.streamreactor.connect.aws.s3.storage.AwsS3StorageInterface import io.lenses.streamreactor.connect.aws.s3.storage.S3FileMetadata import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId @@ -24,7 +25,7 @@ import java.nio.file.Files import scala.util.Try trait S3ProxyContainerTest - extends CloudPlatformEmulatorSuite[S3FileMetadata, AwsS3StorageInterface, S3SinkTask, S3Client] + extends CloudPlatformEmulatorSuite[S3FileMetadata, AwsS3StorageInterface, S3SinkConfig, S3Client, S3SinkTask] with TaskIndexKey with LazyLogging { @@ -40,7 +41,7 @@ trait S3ProxyContainerTest override def createClient(): Either[Throwable, S3Client] = { - val s3Config: S3Config = S3Config( + val s3Config: S3ConnectionConfig = S3ConnectionConfig( region = Some("eu-west-1"), accessKey = Some(s3Container.identity.identity), secretKey = Some(s3Container.identity.credential), diff --git a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/cloud/common/sink/CloudPlatformEmulatorSuite.scala b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/cloud/common/sink/CloudPlatformEmulatorSuite.scala index cccc5cbaf..0ec9e4cc5 100644 --- a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/cloud/common/sink/CloudPlatformEmulatorSuite.scala +++ b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/cloud/common/sink/CloudPlatformEmulatorSuite.scala @@ -3,6 +3,7 @@ package io.lenses.streamreactor.connect.cloud.common.sink import cats.implicits.catsSyntaxEitherId import cats.implicits.catsSyntaxOptionId import cats.implicits.toBifunctorOps +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudSinkConfig import io.lenses.streamreactor.connect.cloud.common.storage.FileMetadata import io.lenses.streamreactor.connect.cloud.common.storage.StorageInterface import io.lenses.streamreactor.connect.cloud.common.utils.RemoteFileHelper @@ -13,8 +14,13 @@ import org.scalatest.flatspec.AnyFlatSpec import scala.util.Try -trait CloudPlatformEmulatorSuite[SM <: FileMetadata, SI <: StorageInterface[SM], T <: CloudSinkTask[SM], C] - extends AnyFlatSpec +trait CloudPlatformEmulatorSuite[ + SM <: FileMetadata, + SI <: StorageInterface[SM], + CSC <: CloudSinkConfig, + C, + T <: CloudSinkTask[SM, CSC, C], +] extends AnyFlatSpec with BeforeAndAfter with BeforeAndAfterAll with RemoteFileHelper[SI] { diff --git a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/cloud/common/sink/CoreSinkTaskTestCases.scala b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/cloud/common/sink/CoreSinkTaskTestCases.scala index 2afdc07d9..dac4832fc 100644 --- a/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/cloud/common/sink/CoreSinkTaskTestCases.scala +++ b/kafka-connect-aws-s3/src/it/scala/io/lenses/streamreactor/connect/cloud/common/sink/CoreSinkTaskTestCases.scala @@ -5,6 +5,7 @@ import io.lenses.streamreactor.common.config.base.const.TraitConfigConst.MAX_RET import io.lenses.streamreactor.common.config.base.const.TraitConfigConst.RETRY_INTERVAL_PROP_SUFFIX import com.opencsv.CSVReader import com.typesafe.scalalogging.LazyLogging +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudSinkConfig import io.lenses.streamreactor.connect.cloud.common.formats.AvroFormatReader import io.lenses.streamreactor.connect.cloud.common.formats.reader.ParquetFormatReader import io.lenses.streamreactor.connect.cloud.common.formats.writer.BytesFormatWriter @@ -43,9 +44,14 @@ import scala.jdk.CollectionConverters.SeqHasAsJava import scala.util.Failure import scala.util.Success import scala.util.Try -abstract class CoreSinkTaskTestCases[SM <: FileMetadata, SI <: StorageInterface[SM], ST <: CloudSinkTask[SM], C]( - unitUnderTest: String, -) extends CloudPlatformEmulatorSuite[SM, SI, ST, C] +abstract class CoreSinkTaskTestCases[ + SM <: FileMetadata, + SI <: StorageInterface[SM], + CSC <: CloudSinkConfig, + C, + T <: CloudSinkTask[SM, CSC, C], +](unitUnderTest: String, +) extends CloudPlatformEmulatorSuite[SM, SI, CSC, C, T] with Matchers with MockitoSugar with LazyLogging { @@ -2059,7 +2065,7 @@ abstract class CoreSinkTaskTestCases[SM <: FileMetadata, SI <: StorageInterface[ struct } - private def createHeaders[T](keyValuePair: (String, T)*): lang.Iterable[Header] = { + private def createHeaders[HX](keyValuePair: (String, HX)*): lang.Iterable[Header] = { val headers = new ConnectHeaders() keyValuePair.foreach { case (key: String, value) => headers.add(key, value, null) diff --git a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/auth/AwsS3ClientCreator.scala b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/auth/AwsS3ClientCreator.scala index cd064f96d..f66741538 100644 --- a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/auth/AwsS3ClientCreator.scala +++ b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/auth/AwsS3ClientCreator.scala @@ -18,7 +18,7 @@ package io.lenses.streamreactor.connect.aws.s3.auth import cats.implicits.catsSyntaxEitherId import cats.implicits.toBifunctorOps import io.lenses.streamreactor.connect.aws.s3.config.AuthMode -import io.lenses.streamreactor.connect.aws.s3.config.S3Config +import io.lenses.streamreactor.connect.aws.s3.config.S3ConnectionConfig import io.lenses.streamreactor.connect.cloud.common.auth.ClientCreator import software.amazon.awssdk.auth.credentials.AwsBasicCredentials import software.amazon.awssdk.auth.credentials.AwsCredentials @@ -36,14 +36,14 @@ import java.net.URI import java.time.Duration import scala.util.Try -object AwsS3ClientCreator extends ClientCreator[S3Config, S3Client] { +object AwsS3ClientCreator extends ClientCreator[S3ConnectionConfig, S3Client] { private val missingCredentialsError = "Configured to use credentials however one or both of `AWS_ACCESS_KEY` or `AWS_SECRET_KEY` are missing." private val defaultCredentialsProvider: AwsCredentialsProvider = DefaultCredentialsProvider.create() - def make(config: S3Config): Either[Throwable, S3Client] = + def make(config: S3ConnectionConfig): Either[Throwable, S3Client] = for { retryPolicy <- Try { RetryPolicy @@ -92,7 +92,7 @@ object AwsS3ClientCreator extends ClientCreator[S3Config, S3Client] { } } yield s3Client - private def credentialsFromConfig(awsConfig: S3Config): Either[String, AwsCredentialsProvider] = + private def credentialsFromConfig(awsConfig: S3ConnectionConfig): Either[String, AwsCredentialsProvider] = awsConfig.accessKey.zip(awsConfig.secretKey) match { case Some((access, secret)) => new AwsCredentialsProvider { @@ -101,7 +101,7 @@ object AwsS3ClientCreator extends ClientCreator[S3Config, S3Client] { case None => missingCredentialsError.asLeft } - private def credentialsProvider(config: S3Config): Either[String, AwsCredentialsProvider] = + private def credentialsProvider(config: S3ConnectionConfig): Either[String, AwsCredentialsProvider] = config.authMode match { case AuthMode.Credentials => credentialsFromConfig(config) case AuthMode.Default => defaultCredentialsProvider.asRight[String] diff --git a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/config/S3ConfigSettings.scala b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/config/S3ConfigSettings.scala index 26871d1be..c51928ea5 100644 --- a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/config/S3ConfigSettings.scala +++ b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/config/S3ConfigSettings.scala @@ -75,13 +75,6 @@ object S3ConfigSettings { s"Maximum index files to allow per topic/partition. Advisable to not raise this: if a large number of files build up this means there is a problem with file deletion." val SEEK_MAX_INDEX_FILES_DEFAULT = 5 - val SOURCE_PARTITION_EXTRACTOR_TYPE = s"$CONNECTOR_PREFIX.source.partition.extractor.type" - val SOURCE_PARTITION_EXTRACTOR_TYPE_DOC = - "If you want to read to specific partitions when running the source. Options are 'hierarchical' (to match the sink's hierarchical file storage pattern) and 'regex' (supply a custom regex). Any other value will ignore original partitions and they should be evenly distributed through available partitions (Kafka dependent)." - - val SOURCE_PARTITION_EXTRACTOR_REGEX = s"$CONNECTOR_PREFIX.source.partition.extractor.regex" - val SOURCE_PARTITION_EXTRACTOR_REGEX_DOC = "If reading filename from regex, supply the regex here." - val POOL_MAX_CONNECTIONS = s"$CONNECTOR_PREFIX.pool.max.connections" val POOL_MAX_CONNECTIONS_DOC = "Max connections in pool. -1: Use default according to underlying client." val POOL_MAX_CONNECTIONS_DEFAULT: Int = -1 diff --git a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/config/S3Config.scala b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/config/S3ConnectionConfig.scala similarity index 92% rename from kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/config/S3Config.scala rename to kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/config/S3ConnectionConfig.scala index 8fc6a4801..c496736e5 100644 --- a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/config/S3Config.scala +++ b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/config/S3ConnectionConfig.scala @@ -21,12 +21,13 @@ import io.lenses.streamreactor.common.errors.ThrowErrorPolicy import enumeratum.Enum import enumeratum.EnumEntry import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings._ -import io.lenses.streamreactor.connect.cloud.common.config.CloudConfig import io.lenses.streamreactor.connect.cloud.common.config.ConfigParse.getBoolean import io.lenses.streamreactor.connect.cloud.common.config.ConfigParse.getInt import io.lenses.streamreactor.connect.cloud.common.config.ConfigParse.getLong import io.lenses.streamreactor.connect.cloud.common.config.ConfigParse.getPassword import io.lenses.streamreactor.connect.cloud.common.config.ConfigParse.getString +import io.lenses.streamreactor.connect.cloud.common.config.RetryConfig +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudConnectionConfig import scala.collection.immutable @@ -42,9 +43,9 @@ object AuthMode extends Enum[AuthMode] { } -object S3Config { +object S3ConnectionConfig { - def apply(props: Map[String, _]): S3Config = S3Config( + def apply(props: Map[String, _]): S3ConnectionConfig = S3ConnectionConfig( getString(props, AWS_REGION), getPassword(props, AWS_ACCESS_KEY), getPassword(props, AWS_SECRET_KEY), @@ -77,8 +78,6 @@ object S3Config { ) } -case class RetryConfig(numberOfRetries: Int, errorRetryInterval: Long) - case class HttpTimeoutConfig(socketTimeout: Option[Int], connectionTimeout: Option[Long]) case class ConnectionPoolConfig(maxConnections: Int) @@ -88,7 +87,7 @@ object ConnectionPoolConfig { maxConns.filterNot(_ == -1).map(ConnectionPoolConfig(_)) } -case class S3Config( +case class S3ConnectionConfig( region: Option[String], accessKey: Option[String], secretKey: Option[String], @@ -100,4 +99,4 @@ case class S3Config( httpRetryConfig: RetryConfig = RetryConfig(HTTP_NBR_OF_RETIRES_DEFAULT, HTTP_ERROR_RETRY_INTERVAL_DEFAULT), timeouts: HttpTimeoutConfig = HttpTimeoutConfig(None, None), connectionPoolConfig: Option[ConnectionPoolConfig] = Option.empty, -) extends CloudConfig +) extends CloudConnectionConfig diff --git a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3SinkTask.scala b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3SinkTask.scala index c34840270..672025fe6 100644 --- a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3SinkTask.scala +++ b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/sink/S3SinkTask.scala @@ -15,52 +15,42 @@ */ package io.lenses.streamreactor.connect.aws.s3.sink -import io.lenses.streamreactor.common.errors.RetryErrorPolicy import io.lenses.streamreactor.common.utils.JarManifest import io.lenses.streamreactor.connect.aws.s3.auth.AwsS3ClientCreator -import io.lenses.streamreactor.connect.aws.s3.config.S3Config import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings import io.lenses.streamreactor.connect.aws.s3.model.location.S3LocationValidator import io.lenses.streamreactor.connect.aws.s3.sink.config.S3SinkConfig import io.lenses.streamreactor.connect.aws.s3.storage.AwsS3StorageInterface import io.lenses.streamreactor.connect.aws.s3.storage.S3FileMetadata +import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId import io.lenses.streamreactor.connect.cloud.common.sink.CloudSinkTask import io.lenses.streamreactor.connect.cloud.common.sink.WriterManagerCreator -import io.lenses.streamreactor.connect.cloud.common.sink.writer.WriterManager - -import scala.util.Try +import software.amazon.awssdk.services.s3.S3Client object S3SinkTask {} class S3SinkTask - extends CloudSinkTask[S3FileMetadata]( + extends CloudSinkTask[S3FileMetadata, S3SinkConfig, S3Client]( S3ConfigSettings.CONNECTOR_PREFIX, "/aws-s3-sink-ascii.txt", JarManifest(S3SinkTask.getClass.getProtectionDomain.getCodeSource.getLocation), - )( - S3LocationValidator, ) { - private val writerManagerCreator = new WriterManagerCreator[S3FileMetadata, S3SinkConfig]() + val writerManagerCreator = new WriterManagerCreator[S3FileMetadata, S3SinkConfig]() + + override def createStorageInterface( + connectorTaskId: ConnectorTaskId, + config: S3SinkConfig, + cloudClient: S3Client, + ): AwsS3StorageInterface = + new AwsS3StorageInterface(connectorTaskId, cloudClient, config.batchDelete) - def createWriterMan(props: Map[String, String]): Either[Throwable, WriterManager[S3FileMetadata]] = - for { - config <- S3SinkConfig.fromProps(props) - s3Client <- AwsS3ClientCreator.make(config.s3Config) - storageInterface = new AwsS3StorageInterface(connectorTaskId, s3Client, config.batchDelete) - _ <- Try(setErrorRetryInterval(config.s3Config)).toEither - writerManager <- Try(writerManagerCreator.from(config)(connectorTaskId, storageInterface)).toEither - _ <- Try(initialize( - config.s3Config.connectorRetryConfig.numberOfRetries, - config.s3Config.errorPolicy, - )).toEither - } yield writerManager + override def createClient(config: S3SinkConfig): Either[Throwable, S3Client] = + AwsS3ClientCreator.make(config.connectionConfig) - private def setErrorRetryInterval(s3Config: S3Config): Unit = - //if error policy is retry set retry interval - s3Config.errorPolicy match { - case RetryErrorPolicy() => context.timeout(s3Config.connectorRetryConfig.errorRetryInterval) - case _ => - } + override def convertPropsToConfig( + connectorTaskId: ConnectorTaskId, + props: Map[String, String], + ): Either[Throwable, S3SinkConfig] = S3SinkConfig.fromProps(connectorTaskId, props)(S3LocationValidator) } diff --git a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3ConsumerGroupsSinkConfig.scala b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3ConsumerGroupsSinkConfig.scala index 10f097f30..46f75baf3 100644 --- a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3ConsumerGroupsSinkConfig.scala +++ b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3ConsumerGroupsSinkConfig.scala @@ -22,7 +22,7 @@ import io.lenses.streamreactor.connect.cloud.common.consumers.CloudObjectKey case class S3ConsumerGroupsSinkConfig( location: CloudObjectKey, - config: S3Config, + config: S3ConnectionConfig, ) object S3ConsumerGroupsSinkConfig extends PropertiesHelper { @@ -46,7 +46,7 @@ object S3ConsumerGroupsSinkConfig extends PropertiesHelper { } yield { S3ConsumerGroupsSinkConfig( bucket, - S3Config(props), + S3ConnectionConfig(props), ) } } diff --git a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3SinkConfig.scala b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3SinkConfig.scala index b03b6181a..0ca138d48 100644 --- a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3SinkConfig.scala +++ b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3SinkConfig.scala @@ -15,41 +15,46 @@ */ package io.lenses.streamreactor.connect.aws.s3.sink.config -import io.lenses.streamreactor.connect.aws.s3.config.S3Config import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings.SEEK_MAX_INDEX_FILES +import io.lenses.streamreactor.connect.aws.s3.config.S3ConnectionConfig import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudSinkConfig +import io.lenses.streamreactor.connect.cloud.common.config.traits.PropsToConfigConverter import io.lenses.streamreactor.connect.cloud.common.model.CompressionCodec import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocationValidator -import io.lenses.streamreactor.connect.cloud.common.sink.config import io.lenses.streamreactor.connect.cloud.common.sink.config.CloudSinkBucketOptions -import io.lenses.streamreactor.connect.cloud.common.sink.config.CloudSinkConfig import io.lenses.streamreactor.connect.cloud.common.sink.config.OffsetSeekerOptions -object S3SinkConfig { +import scala.util.Try - def fromProps( - props: Map[String, String], +object S3SinkConfig extends PropsToConfigConverter[S3SinkConfig] { + + override def fromProps( + connectorTaskId: ConnectorTaskId, + props: Map[String, String], )( implicit - connectorTaskId: ConnectorTaskId, cloudLocationValidator: CloudLocationValidator, ): Either[Throwable, S3SinkConfig] = - S3SinkConfig(S3SinkConfigDefBuilder(props)) + for { + conf <- Try(S3SinkConfigDefBuilder(props)).toEither + s3SinkConfig <- S3SinkConfig.apply(connectorTaskId, conf) + } yield s3SinkConfig - def apply( + private def apply( + connectorTaskId: ConnectorTaskId, s3ConfigDefBuilder: S3SinkConfigDefBuilder, )( implicit - connectorTaskId: ConnectorTaskId, cloudLocationValidator: CloudLocationValidator, ): Either[Throwable, S3SinkConfig] = for { - sinkBucketOptions <- config.CloudSinkBucketOptions(s3ConfigDefBuilder) + sinkBucketOptions <- CloudSinkBucketOptions(connectorTaskId, s3ConfigDefBuilder) offsetSeekerOptions = OffsetSeekerOptions( s3ConfigDefBuilder.getInt(SEEK_MAX_INDEX_FILES), ) } yield S3SinkConfig( - S3Config(s3ConfigDefBuilder.getParsedValues), + S3ConnectionConfig(s3ConfigDefBuilder.getParsedValues), sinkBucketOptions, offsetSeekerOptions, s3ConfigDefBuilder.getCompressionCodec(), @@ -59,7 +64,7 @@ object S3SinkConfig { } case class S3SinkConfig( - s3Config: S3Config, + connectionConfig: S3ConnectionConfig, bucketOptions: Seq[CloudSinkBucketOptions] = Seq.empty, offsetSeekerOptions: OffsetSeekerOptions, compressionCodec: CompressionCodec, diff --git a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3SinkConfigDef.scala b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3SinkConfigDef.scala index 8a775ea7b..ae99b96cb 100644 --- a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3SinkConfigDef.scala +++ b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3SinkConfigDef.scala @@ -15,13 +15,10 @@ */ package io.lenses.streamreactor.connect.aws.s3.sink.config -import cats.implicits.catsSyntaxEitherId -import com.typesafe.scalalogging.LazyLogging import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings._ import io.lenses.streamreactor.connect.aws.s3.config._ import io.lenses.streamreactor.connect.aws.s3.config.processors.kcql.DeprecationConfigDefProcessor -import io.lenses.streamreactor.connect.cloud.common.config.processors.ConfigDefProcessor -import io.lenses.streamreactor.connect.cloud.common.config.processors.LowerCaseKeyConfigDefProcessor +import io.lenses.streamreactor.connect.cloud.common.config.CloudConfigDef import io.lenses.streamreactor.connect.cloud.common.sink.config.FlushConfigKeys import io.lenses.streamreactor.connect.cloud.common.sink.config.LocalStagingAreaConfigKeys import io.lenses.streamreactor.connect.cloud.common.sink.config.padding.PaddingStrategyConfigKeys @@ -29,9 +26,6 @@ import org.apache.kafka.common.config.ConfigDef import org.apache.kafka.common.config.ConfigDef.Importance import org.apache.kafka.common.config.ConfigDef.Type -import java.util -import scala.jdk.CollectionConverters._ - object S3SinkConfigDef extends CommonConfigDef with FlushConfigKeys @@ -66,37 +60,4 @@ object S3SinkConfigDef } -class S3SinkConfigDef() extends ConfigDef with LazyLogging { - - private val processorChain: List[ConfigDefProcessor] = - List(new LowerCaseKeyConfigDefProcessor(CONNECTOR_PREFIX), new DeprecationConfigDefProcessor) - - override def parse(jProps: util.Map[_, _]): util.Map[String, AnyRef] = { - val scalaProps: Map[Any, Any] = jProps.asScala.toMap - processProperties(scalaProps) match { - case Left(exception) => throw exception - case Right(value) => super.parse(value.asJava) - } - } - - private def processProperties(scalaProps: Map[Any, Any]): Either[Throwable, Map[Any, Any]] = { - val stringProps = scalaProps.collect { case (k: String, v: AnyRef) => (k.toLowerCase, v) } - val nonStringProps = scalaProps -- stringProps.keySet - processStringKeyedProperties(stringProps) match { - case Left(exception) => exception.asLeft[Map[Any, Any]] - case Right(stringKeyedProps) => (nonStringProps ++ stringKeyedProps).asRight - } - } - - def processStringKeyedProperties(stringProps: Map[String, Any]): Either[Throwable, Map[String, Any]] = { - var remappedProps: Map[String, Any] = stringProps - for (proc <- processorChain) { - proc.process(remappedProps) match { - case Left(exception) => return exception.asLeft[Map[String, AnyRef]] - case Right(properties) => remappedProps = properties - } - } - remappedProps.asRight - } - -} +class S3SinkConfigDef() extends CloudConfigDef(CONNECTOR_PREFIX, new DeprecationConfigDefProcessor()) {} diff --git a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceTask.scala b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceTask.scala index 53edf7aa9..bf239a7a0 100644 --- a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceTask.scala +++ b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/S3SourceTask.scala @@ -14,92 +14,58 @@ * limitations under the License. */ package io.lenses.streamreactor.connect.aws.s3.source - -import cats.effect.FiberIO -import cats.effect.IO -import cats.effect.Ref -import cats.effect.unsafe.implicits.global -import cats.implicits.catsSyntaxOptionId -import io.lenses.streamreactor.common.utils.AsciiArtPrinter.printAsciiHeader -import io.lenses.streamreactor.common.utils.JarManifest import com.typesafe.scalalogging.LazyLogging +import io.lenses.streamreactor.connect.aws.s3.auth.AwsS3ClientCreator +import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings.CONNECTOR_PREFIX import io.lenses.streamreactor.connect.aws.s3.model.location.S3LocationValidator -import io.lenses.streamreactor.connect.aws.s3.source.state.S3SourceState -import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocation +import io.lenses.streamreactor.connect.aws.s3.source.config.S3SourceConfig +import io.lenses.streamreactor.connect.aws.s3.source.distribution.S3PartitionSearcher +import io.lenses.streamreactor.connect.aws.s3.storage.AwsS3StorageInterface +import io.lenses.streamreactor.connect.aws.s3.storage.S3FileMetadata +import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocationValidator -import io.lenses.streamreactor.connect.cloud.common.source.SourceContextReader -import io.lenses.streamreactor.connect.cloud.common.source.state.CloudSourceTaskState -import io.lenses.streamreactor.connect.cloud.common.utils.MapUtils -import org.apache.kafka.connect.source.SourceRecord -import org.apache.kafka.connect.source.SourceTask - -import java.util -import java.util.Collections -import scala.jdk.CollectionConverters._ -class S3SourceTask extends SourceTask with LazyLogging { - - implicit val cloudLocationValidator: CloudLocationValidator = S3LocationValidator - private val contextOffsetFn: CloudLocation => Option[CloudLocation] = - SourceContextReader.getCurrentOffset(() => context) - - private val manifest = JarManifest(getClass.getProtectionDomain.getCodeSource.getLocation) - - @volatile - private var s3SourceTaskState: Option[CloudSourceTaskState] = None - - @volatile - private var cancelledRef: Option[Ref[IO, Boolean]] = None - - private var partitionDiscoveryLoop: Option[FiberIO[Unit]] = None +import io.lenses.streamreactor.connect.cloud.common.source.CloudSourceTask +import io.lenses.streamreactor.connect.cloud.common.source.state.PartitionSearcher +import software.amazon.awssdk.services.s3.S3Client - override def version(): String = manifest.version() +import scala.jdk.CollectionConverters.IteratorHasAsScala - /** - * Start sets up readers for every configured connection in the properties - */ - override def start(props: util.Map[String, String]): Unit = { +class S3SourceTask + extends CloudSourceTask[ + S3FileMetadata, + S3SourceConfig, + S3Client, + ] + with LazyLogging { - printAsciiHeader(manifest, "/aws-s3-source-ascii.txt") + val validator: CloudLocationValidator = S3LocationValidator - logger.debug(s"Received call to S3SourceTask.start with ${props.size()} properties") + override def createStorageInterface( + connectorTaskId: ConnectorTaskId, + config: S3SourceConfig, + s3Client: S3Client, + ): AwsS3StorageInterface = + new AwsS3StorageInterface(connectorTaskId, s3Client, config.batchDelete) - val contextProperties: Map[String, String] = - Option(context).flatMap(c => Option(c.configs()).map(_.asScala.toMap)).getOrElse(Map.empty) - val mergedProperties: Map[String, String] = MapUtils.mergeProps(contextProperties, props.asScala.toMap) - (for { - result <- S3SourceState.make(mergedProperties, contextOffsetFn) - fiber <- result.partitionDiscoveryLoop.start - } yield { - s3SourceTaskState = result.state.some - cancelledRef = result.cancelledRef.some - partitionDiscoveryLoop = fiber.some - }).unsafeRunSync() - } + override def createClient(config: S3SourceConfig): Either[Throwable, S3Client] = + AwsS3ClientCreator.make(config.connectionConfig) - override def stop(): Unit = { - logger.info(s"Stopping S3 source task") - (s3SourceTaskState, cancelledRef, partitionDiscoveryLoop) match { - case (Some(state), Some(signal), Some(fiber)) => stopInternal(state, signal, fiber) - case _ => logger.info("There is no state to stop.") - } - logger.info(s"Stopped S3 source task") - } + override def convertPropsToConfig( + connectorTaskId: ConnectorTaskId, + props: Map[String, String], + ): Either[Throwable, S3SourceConfig] = S3SourceConfig.fromProps(connectorTaskId, props)(validator) - override def poll(): util.List[SourceRecord] = - s3SourceTaskState.fold(Collections.emptyList[SourceRecord]()) { state => - state.poll().unsafeRunSync().asJava - } + override def createPartitionSearcher( + connectorTaskId: ConnectorTaskId, + config: S3SourceConfig, + client: S3Client, + ): PartitionSearcher = + new S3PartitionSearcher( + config.bucketOptions.map(_.sourceBucketAndPrefix), + config.partitionSearcher, + connectorTaskId, + client.listObjectsV2Paginator(_).iterator().asScala, + ) - private def stopInternal(state: CloudSourceTaskState, signal: Ref[IO, Boolean], fiber: FiberIO[Unit]): Unit = { - (for { - _ <- signal.set(true) - _ <- state.close() - // Don't join the fiber if it's already been cancelled. It will take potentially the interval time to complete - // and this can create issues on Connect. The task will be terminated and the resource cleaned up by the GC. - //_ <- fiber.join.timeout(1.minute).attempt.void - } yield ()).unsafeRunSync() - cancelledRef = None - partitionDiscoveryLoop = None - s3SourceTaskState = None - } + override def connectorPrefix: String = CONNECTOR_PREFIX } diff --git a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfig.scala b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfig.scala index 799269c7a..488ca8f8a 100644 --- a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfig.scala +++ b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfig.scala @@ -15,52 +15,39 @@ */ package io.lenses.streamreactor.connect.aws.s3.source.config -import cats.implicits.toTraverseOps -import io.lenses.kcql.Kcql -import io.lenses.streamreactor.connect.aws.s3.config.S3Config -import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings.SOURCE_ORDERING_TYPE -import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings.SOURCE_PARTITION_EXTRACTOR_REGEX -import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings.SOURCE_PARTITION_EXTRACTOR_TYPE +import io.lenses.streamreactor.connect.aws.s3.config.S3ConnectionConfig import io.lenses.streamreactor.connect.aws.s3.model.location.S3LocationValidator import io.lenses.streamreactor.connect.aws.s3.storage.S3FileMetadata -import io.lenses.streamreactor.connect.cloud.common.config.ConfigParse.getString -import io.lenses.streamreactor.connect.cloud.common.config.FormatSelection -import io.lenses.streamreactor.connect.cloud.common.config.kcqlprops.PropsKeyEntry -import io.lenses.streamreactor.connect.cloud.common.config.kcqlprops.PropsKeyEnum +import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudSourceConfig +import io.lenses.streamreactor.connect.cloud.common.config.traits.PropsToConfigConverter import io.lenses.streamreactor.connect.cloud.common.model.CompressionCodec -import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocation import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocationValidator -import io.lenses.streamreactor.connect.cloud.common.source.config.OrderingType -import io.lenses.streamreactor.connect.cloud.common.source.config.PartitionExtractor +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceBucketOptions import io.lenses.streamreactor.connect.cloud.common.source.config.PartitionSearcherOptions -import io.lenses.streamreactor.connect.cloud.common.source.config.kcqlprops.CloudSourceProps -import io.lenses.streamreactor.connect.cloud.common.source.config.kcqlprops.CloudSourcePropsSchema -import io.lenses.streamreactor.connect.cloud.common.storage.FileListError -import io.lenses.streamreactor.connect.cloud.common.storage.ListOfKeysResponse -import io.lenses.streamreactor.connect.cloud.common.storage.StorageInterface -import io.lenses.streamreactor.connect.config.kcqlprops.KcqlProperties -import scala.util.Try +object S3SourceConfig extends PropsToConfigConverter[S3SourceConfig] { -object S3SourceConfig { + implicit val CloudLocationValidator: CloudLocationValidator = S3LocationValidator - def fromProps( - props: Map[String, String], + override def fromProps( + connectorTaskId: ConnectorTaskId, + props: Map[String, String], + )( + implicit + cloudLocationValidator: CloudLocationValidator, ): Either[Throwable, S3SourceConfig] = - S3SourceConfig(S3SourceConfigDefBuilder(props)) + apply(S3SourceConfigDefBuilder(props)) def apply(s3ConfigDefBuilder: S3SourceConfigDefBuilder): Either[Throwable, S3SourceConfig] = { val parsedValues = s3ConfigDefBuilder.getParsedValues for { - sbo <- SourceBucketOptions( + sbo <- CloudSourceBucketOptions[S3FileMetadata]( s3ConfigDefBuilder, - PartitionExtractor( - getString(parsedValues, SOURCE_PARTITION_EXTRACTOR_TYPE).getOrElse("none"), - getString(parsedValues, SOURCE_PARTITION_EXTRACTOR_REGEX), - ), + s3ConfigDefBuilder.getPartitionExtractor(parsedValues), ) } yield S3SourceConfig( - S3Config(parsedValues), + S3ConnectionConfig(parsedValues), sbo, s3ConfigDefBuilder.getCompressionCodec(), s3ConfigDefBuilder.getPartitionSearcherOptions(parsedValues), @@ -68,79 +55,13 @@ object S3SourceConfig { ) } + } case class S3SourceConfig( - s3Config: S3Config, - bucketOptions: Seq[SourceBucketOptions] = Seq.empty, + connectionConfig: S3ConnectionConfig, + bucketOptions: Seq[CloudSourceBucketOptions[S3FileMetadata]] = Seq.empty, compressionCodec: CompressionCodec, partitionSearcher: PartitionSearcherOptions, batchDelete: Boolean, -) - -case class SourceBucketOptions( - sourceBucketAndPrefix: CloudLocation, - targetTopic: String, - format: FormatSelection, - recordsLimit: Int, - filesLimit: Int, - partitionExtractor: Option[PartitionExtractor], - orderingType: OrderingType, - hasEnvelope: Boolean, -) { - def createBatchListerFn( - storageInterface: StorageInterface[S3FileMetadata], - ): Option[S3FileMetadata] => Either[FileListError, Option[ListOfKeysResponse[S3FileMetadata]]] = - orderingType - .getBatchLister - .listBatch( - storageInterface = storageInterface, - bucket = sourceBucketAndPrefix.bucket, - prefix = sourceBucketAndPrefix.prefix, - numResults = filesLimit, - ) - - def getPartitionExtractorFn: String => Option[Int] = - partitionExtractor.fold((_: String) => Option.empty[Int])(_.extract) - -} - -object SourceBucketOptions { - private val DEFAULT_RECORDS_LIMIT = 10000 - private val DEFAULT_FILES_LIMIT = 1000 - - implicit val cloudLocationValidator: CloudLocationValidator = S3LocationValidator - - def apply( - config: S3SourceConfigDefBuilder, - partitionExtractor: Option[PartitionExtractor], - ): Either[Throwable, Seq[SourceBucketOptions]] = - config.getKCQL.map { - kcql: Kcql => - for { - source <- CloudLocation.splitAndValidate(kcql.getSource) - format <- FormatSelection.fromKcql(kcql, CloudSourcePropsSchema.schema) - sourceProps = CloudSourceProps.fromKcql(kcql) - - //extract the envelope. of not present default to false - hasEnvelope <- extractEnvelope(sourceProps) - - } yield SourceBucketOptions( - source, - kcql.getTarget, - format = format, - recordsLimit = if (kcql.getLimit < 1) DEFAULT_RECORDS_LIMIT else kcql.getLimit, - filesLimit = if (kcql.getBatchSize < 1) DEFAULT_FILES_LIMIT else kcql.getBatchSize, - partitionExtractor = partitionExtractor, - orderingType = Try(config.getString(SOURCE_ORDERING_TYPE)).toOption.flatMap( - OrderingType.withNameInsensitiveOption, - ).getOrElse(OrderingType.AlphaNumeric), - hasEnvelope = hasEnvelope.getOrElse(false), - ) - }.toSeq.traverse(identity) - - private def extractEnvelope( - properties: KcqlProperties[PropsKeyEntry, PropsKeyEnum.type], - ): Either[Throwable, Option[Boolean]] = - properties.getOptionalBoolean(PropsKeyEnum.StoreEnvelope) -} +) extends CloudSourceConfig[S3FileMetadata] diff --git a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfigDef.scala b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfigDef.scala index f3294f05a..d0eedcec6 100644 --- a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfigDef.scala +++ b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfigDef.scala @@ -15,99 +15,24 @@ */ package io.lenses.streamreactor.connect.aws.s3.source.config -import cats.implicits.catsSyntaxEitherId -import com.typesafe.scalalogging.LazyLogging import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings._ import io.lenses.streamreactor.connect.aws.s3.config._ import io.lenses.streamreactor.connect.aws.s3.config.processors.kcql.DeprecationConfigDefProcessor -import io.lenses.streamreactor.connect.cloud.common.config.processors.ConfigDefProcessor -import io.lenses.streamreactor.connect.cloud.common.config.processors.LowerCaseKeyConfigDefProcessor +import io.lenses.streamreactor.connect.cloud.common.config.CloudConfigDef +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceSettingsKeys import org.apache.kafka.common.config.ConfigDef -import org.apache.kafka.common.config.ConfigDef.Importance -import org.apache.kafka.common.config.ConfigDef.Type -import java.util -import scala.collection.immutable.ListMap -import scala.jdk.CollectionConverters._ - -object S3SourceConfigDef extends CommonConfigDef with SourcePartitionSearcherSettingsKeys { +object S3SourceConfigDef extends CommonConfigDef with CloudSourceSettingsKeys { override def connectorPrefix: String = CONNECTOR_PREFIX override val config: ConfigDef = { - val settings = super.config - .define( - SOURCE_PARTITION_EXTRACTOR_TYPE, - Type.STRING, - null, - Importance.LOW, - SOURCE_PARTITION_EXTRACTOR_TYPE_DOC, - "Source", - 1, - ConfigDef.Width.MEDIUM, - SOURCE_PARTITION_EXTRACTOR_TYPE, - ) - .define( - SOURCE_PARTITION_EXTRACTOR_REGEX, - Type.STRING, - null, - Importance.LOW, - SOURCE_PARTITION_EXTRACTOR_REGEX_DOC, - "Source", - 2, - ConfigDef.Width.MEDIUM, - SOURCE_PARTITION_EXTRACTOR_REGEX, - ) - .define( - SOURCE_ORDERING_TYPE, - Type.STRING, - SOURCE_ORDERING_TYPE_DEFAULT, - Importance.LOW, - SOURCE_ORDERING_TYPE_DOC, - "Source", - 6, - ConfigDef.Width.MEDIUM, - SOURCE_ORDERING_TYPE, - ) + val settings = super.config + addSourceOrderingSettings(settings) addSourcePartitionSearcherSettings(settings) + addSourcePartitionExtractorSettings(settings) } } -class S3SourceConfigDef() extends ConfigDef with LazyLogging { - - private val processorChain: List[ConfigDefProcessor] = - List(new LowerCaseKeyConfigDefProcessor(CONNECTOR_PREFIX), new DeprecationConfigDefProcessor) - - override def parse(jProps: util.Map[_, _]): util.Map[String, AnyRef] = { - val scalaProps: Map[Any, Any] = jProps.asScala.toMap - processProperties(scalaProps) match { - case Left(exception) => throw exception - case Right(value) => super.parse(value.asJava) - } - } - - private def processProperties(scalaProps: Map[Any, Any]): Either[Throwable, Map[Any, Any]] = { - val stringProps = scalaProps.collect { case (k: String, v: AnyRef) => (k.toLowerCase, v) } - val nonStringProps = scalaProps -- stringProps.keySet - processStringKeyedProperties(stringProps) match { - case Left(exception) => exception.asLeft[Map[Any, Any]] - case Right(stringKeyedProps) => (nonStringProps ++ stringKeyedProps).asRight - } - } - - def writeInOrder(remappedProps: Map[String, Any]): ListMap[String, Any] = - ListMap(remappedProps.toSeq.sortBy(_._1): _*) - - def processStringKeyedProperties(stringProps: Map[String, Any]): Either[Throwable, Map[String, Any]] = { - var remappedProps: Map[String, Any] = stringProps - for (proc <- processorChain) { - proc.process(remappedProps) match { - case Left(exception) => return exception.asLeft[Map[String, AnyRef]] - case Right(properties) => remappedProps = properties - } - } - remappedProps.asRight - } - -} +class S3SourceConfigDef() extends CloudConfigDef(CONNECTOR_PREFIX, new DeprecationConfigDefProcessor()) {} diff --git a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfigDefBuilder.scala b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfigDefBuilder.scala index 1f7924e12..f5588cbb8 100644 --- a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfigDefBuilder.scala +++ b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfigDefBuilder.scala @@ -15,22 +15,14 @@ */ package io.lenses.streamreactor.connect.aws.s3.source.config -import io.lenses.streamreactor.common.config.base.traits._ import io.lenses.streamreactor.connect.aws.s3.config.DeleteModeSettings import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings -import io.lenses.streamreactor.connect.cloud.common.config.CompressionCodecSettings +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceConfigDefBuilder import scala.jdk.CollectionConverters.MapHasAsScala case class S3SourceConfigDefBuilder(props: Map[String, String]) - extends BaseConfig(S3ConfigSettings.CONNECTOR_PREFIX, S3SourceConfigDef.config, props) - with KcqlSettings - with ErrorPolicySettings - with NumberRetriesSettings - with UserSettings - with ConnectionSettings - with CompressionCodecSettings - with SourcePartitionSearcherSettings + extends CloudSourceConfigDefBuilder(S3ConfigSettings.CONNECTOR_PREFIX, S3SourceConfigDef.config, props) with DeleteModeSettings { def getParsedValues: Map[String, _] = values().asScala.toMap diff --git a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/config/SourcePartitionSearcherSettings.scala b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/config/SourcePartitionSearcherSettings.scala deleted file mode 100644 index 106ce6c74..000000000 --- a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/config/SourcePartitionSearcherSettings.scala +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Copyright 2017-2024 Lenses.io Ltd - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.lenses.streamreactor.connect.aws.s3.source.config - -import io.lenses.streamreactor.common.config.base.traits.BaseSettings -import io.lenses.streamreactor.common.config.base.traits.WithConnectorPrefix -import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings._ -import io.lenses.streamreactor.connect.cloud.common.config.ConfigParse.getLong -import io.lenses.streamreactor.connect.cloud.common.source.config.PartitionSearcherOptions -import io.lenses.streamreactor.connect.cloud.common.source.config.PartitionSearcherOptions.ExcludeIndexes -import org.apache.kafka.common.config.ConfigDef -import org.apache.kafka.common.config.ConfigDef.Importance -import org.apache.kafka.common.config.ConfigDef.Type - -import scala.concurrent.duration.DurationLong - -trait SourcePartitionSearcherSettingsKeys extends WithConnectorPrefix { - - val SOURCE_PARTITION_SEARCH_RECURSE_LEVELS: String = s"$CONNECTOR_PREFIX.partition.search.recurse.levels" - val SOURCE_PARTITION_SEARCH_RECURSE_LEVELS_DOC: String = - "When searching for new partitions on the S3 filesystem, how many levels deep to recurse." - val SOURCE_PARTITION_SEARCH_RECURSE_LEVELS_DEFAULT: Int = 0 - - val SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS: String = s"$CONNECTOR_PREFIX.partition.search.interval" - val SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS_DOC: String = - "The interval in milliseconds between searching for new partitions. Defaults to 5 minutes." - val SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS_DEFAULT: Long = 300000L - - val SOURCE_PARTITION_SEARCH_MODE: String = s"$CONNECTOR_PREFIX.partition.search.continuous" - val SOURCE_PARTITION_SEARCH_MODE_DOC: String = - "If set to true, it will be continuously search for new partitions. Otherwise it is a one-off operation. Defaults to true." - - def addSourcePartitionSearcherSettings(configDef: ConfigDef): ConfigDef = - configDef.define( - SOURCE_PARTITION_SEARCH_RECURSE_LEVELS, - Type.INT, - SOURCE_PARTITION_SEARCH_RECURSE_LEVELS_DEFAULT, - Importance.LOW, - SOURCE_PARTITION_SEARCH_RECURSE_LEVELS_DOC, - "Source", - 3, - ConfigDef.Width.MEDIUM, - SOURCE_PARTITION_SEARCH_RECURSE_LEVELS, - ) - .define( - SOURCE_PARTITION_SEARCH_MODE, - Type.BOOLEAN, - true, - Importance.LOW, - SOURCE_PARTITION_SEARCH_MODE_DOC, - "Source", - 4, - ConfigDef.Width.MEDIUM, - SOURCE_PARTITION_SEARCH_MODE, - ) - .define( - SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS, - Type.LONG, - SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS_DEFAULT, - Importance.LOW, - SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS_DOC, - "Source", - 5, - ConfigDef.Width.MEDIUM, - SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS, - ) -} -trait SourcePartitionSearcherSettings extends BaseSettings with SourcePartitionSearcherSettingsKeys { - - def getPartitionSearcherOptions(props: Map[String, _]): PartitionSearcherOptions = - PartitionSearcherOptions( - recurseLevels = getInt(SOURCE_PARTITION_SEARCH_RECURSE_LEVELS), - continuous = getBoolean(SOURCE_PARTITION_SEARCH_MODE), - interval = getLong(props, SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS).getOrElse( - SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS_DEFAULT, - ).millis, - wildcardExcludes = ExcludeIndexes, - ) -} diff --git a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/distribution/PartitionSearcher.scala b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/distribution/S3PartitionSearcher.scala similarity index 78% rename from kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/distribution/PartitionSearcher.scala rename to kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/distribution/S3PartitionSearcher.scala index a97a7cae1..12759fd54 100644 --- a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/distribution/PartitionSearcher.scala +++ b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/distribution/S3PartitionSearcher.scala @@ -23,17 +23,34 @@ import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocation import io.lenses.streamreactor.connect.cloud.common.source.config.PartitionSearcherOptions import io.lenses.streamreactor.connect.cloud.common.source.distribution.PartitionSearcherResponse +import io.lenses.streamreactor.connect.cloud.common.source.state.PartitionSearcher import io.lenses.streamreactor.connect.cloud.common.storage.DirectoryFindCompletionConfig import software.amazon.awssdk.services.s3.model.ListObjectsV2Request import software.amazon.awssdk.services.s3.model.ListObjectsV2Response -class PartitionSearcher( +/** + * Class implementing a partition searcher for S3 cloud storage. + * This class searches for new partitions within specified roots in S3. + * + * @param roots The list of root locations in which to search for partitions. + * @param settings The configuration options for partition searching. + * @param connectorTaskId The identifier for the connector task. + * @param listS3ObjF A function to list objects in S3 buckets. + */ +class S3PartitionSearcher( roots: Seq[CloudLocation], settings: PartitionSearcherOptions, connectorTaskId: ConnectorTaskId, listS3ObjF: ListObjectsV2Request => Iterator[ListObjectsV2Response], -) extends LazyLogging { +) extends PartitionSearcher + with LazyLogging { + /** + * Finds new partitions based on the provided last found partition responses. + * + * @param lastFound The previously found partition responses. + * @return A sequence of new partition responses. + */ def find( lastFound: Seq[PartitionSearcherResponse], ): IO[Seq[PartitionSearcherResponse]] = diff --git a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/state/S3SourceBuilder.scala b/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/state/S3SourceBuilder.scala deleted file mode 100644 index cf1b6ac5c..000000000 --- a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/state/S3SourceBuilder.scala +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Copyright 2017-2024 Lenses.io Ltd - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.lenses.streamreactor.connect.aws.s3.source.state - -import cats.effect.IO -import cats.effect.kernel.Ref -import com.typesafe.scalalogging.StrictLogging -import io.lenses.streamreactor.connect.aws.s3.auth.AwsS3ClientCreator -import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings.CONNECTOR_PREFIX -import io.lenses.streamreactor.connect.aws.s3.source.config.S3SourceConfig -import io.lenses.streamreactor.connect.aws.s3.source.distribution.PartitionSearcher -import io.lenses.streamreactor.connect.aws.s3.storage.AwsS3StorageInterface -import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskIdCreator -import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocation -import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocationValidator -import io.lenses.streamreactor.connect.cloud.common.source.reader.PartitionDiscovery -import io.lenses.streamreactor.connect.cloud.common.source.reader.ReaderManager -import io.lenses.streamreactor.connect.cloud.common.source.reader.ReaderManagerState -import io.lenses.streamreactor.connect.cloud.common.source.state.CloudSourceTaskState - -import scala.jdk.CollectionConverters.IteratorHasAsScala - -object S3SourceState extends StrictLogging { - def make( - props: Map[String, String], - contextOffsetFn: CloudLocation => Option[CloudLocation], - )( - implicit - cloudLocationValidator: CloudLocationValidator, - ): IO[BuilderResult] = - for { - connectorTaskId <- IO.fromEither(new ConnectorTaskIdCreator(CONNECTOR_PREFIX).fromProps(props)) - config <- IO.fromEither(S3SourceConfig.fromProps(props)) - s3Client <- IO.fromEither(AwsS3ClientCreator.make(config.s3Config)) - storageInterface <- IO.delay(new AwsS3StorageInterface(connectorTaskId, s3Client, config.batchDelete)) - partitionSearcher <- IO.delay( - new PartitionSearcher( - config.bucketOptions.map(_.sourceBucketAndPrefix), - config.partitionSearcher, - connectorTaskId, - s3Client.listObjectsV2Paginator(_).iterator().asScala, - ), - ) - - readerManagerState <- Ref[IO].of(ReaderManagerState(Seq.empty, Seq.empty)) - cancelledRef <- Ref[IO].of(false) - } yield { - val readerManagerCreateFn: (CloudLocation, String) => IO[ReaderManager] = (root, path) => { - ReaderManagerBuilder( - root, - path, - storageInterface, - connectorTaskId, - contextOffsetFn, - location => config.bucketOptions.find(sb => sb.sourceBucketAndPrefix == location), - ) - } - val partitionDiscoveryLoop = PartitionDiscovery.run(connectorTaskId, - config.partitionSearcher, - partitionSearcher.find, - readerManagerCreateFn, - readerManagerState, - cancelledRef, - ) - BuilderResult(new CloudSourceTaskState(readerManagerState.get.map(_.readerManagers)), - cancelledRef, - partitionDiscoveryLoop, - ) - } -} - -case class BuilderResult( - state: CloudSourceTaskState, - cancelledRef: Ref[IO, Boolean], - partitionDiscoveryLoop: IO[Unit], -) diff --git a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/config/S3ConfigTest.scala b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/config/S3ConfigTest.scala index 88c84eedb..259826dc7 100644 --- a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/config/S3ConfigTest.scala +++ b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/config/S3ConfigTest.scala @@ -19,6 +19,7 @@ import io.lenses.streamreactor.common.errors.NoopErrorPolicy import io.lenses.streamreactor.common.errors.RetryErrorPolicy import io.lenses.streamreactor.common.errors.ThrowErrorPolicy import com.typesafe.scalalogging.LazyLogging +import io.lenses.streamreactor.connect.cloud.common.config.RetryConfig import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers import org.scalatest.prop.TableDrivenPropertyChecks._ @@ -41,7 +42,7 @@ class S3ConfigTest extends AnyFlatSpec with Matchers with LazyLogging { forAll(errorPolicyValuesMap) { (name, value, clazz) => logger.debug("Executing {}", name) - S3Config(Map("connect.s3.error.policy" -> value)).errorPolicy should be(clazz) + S3ConnectionConfig(Map("connect.s3.error.policy" -> value)).errorPolicy should be(clazz) } } @@ -57,7 +58,7 @@ class S3ConfigTest extends AnyFlatSpec with Matchers with LazyLogging { forAll(retryValuesMap) { (name: String, ret: Any, interval: Any, result: RetryConfig) => logger.debug("Executing {}", name) - S3Config(Map( + S3ConnectionConfig(Map( "connect.s3.max.retries" -> ret, "connect.s3.retry.interval" -> interval, )).connectorRetryConfig should be(result) @@ -68,7 +69,7 @@ class S3ConfigTest extends AnyFlatSpec with Matchers with LazyLogging { forAll(retryValuesMap) { (name: String, ret: Any, interval: Any, result: RetryConfig) => logger.debug("Executing {}", name) - S3Config(Map( + S3ConnectionConfig(Map( "connect.s3.http.max.retries" -> ret, "connect.s3.http.retry.interval" -> interval, )).httpRetryConfig should be(result) diff --git a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/config/S3ConsumerGroupsSinkConfigTest.scala b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/config/S3ConsumerGroupsSinkConfigTest.scala index 0bec19587..dcc86d83b 100644 --- a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/config/S3ConsumerGroupsSinkConfigTest.scala +++ b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/config/S3ConsumerGroupsSinkConfigTest.scala @@ -19,6 +19,7 @@ import cats.implicits.catsSyntaxOptionId import io.lenses.streamreactor.common.errors.ThrowErrorPolicy import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings._ import io.lenses.streamreactor.connect.aws.s3.sink.config.S3ConsumerGroupsSinkConfig +import io.lenses.streamreactor.connect.cloud.common.config.RetryConfig import io.lenses.streamreactor.connect.cloud.common.consumers.CloudObjectKey import org.scalatest.funsuite.AnyFunSuite import org.scalatest.matchers.should.Matchers @@ -40,7 +41,7 @@ class S3ConsumerGroupsSinkConfigTest extends AnyFunSuite with Matchers { value should be( S3ConsumerGroupsSinkConfig( CloudObjectKey("bucket", "a/b/c".some), - S3Config( + S3ConnectionConfig( Some("eu-west-1"), Some("access"), Some("secret"), @@ -74,7 +75,7 @@ class S3ConsumerGroupsSinkConfigTest extends AnyFunSuite with Matchers { value should be( S3ConsumerGroupsSinkConfig( CloudObjectKey("bucket", "a/b/c".some), - S3Config( + S3ConnectionConfig( Some("eu-west-1"), Some("access"), Some("secret"), diff --git a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3ConsumerGroupsSinkConfigTest.scala b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3ConsumerGroupsSinkConfigTest.scala index f16fe3934..cafeb1049 100644 --- a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3ConsumerGroupsSinkConfigTest.scala +++ b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3ConsumerGroupsSinkConfigTest.scala @@ -19,9 +19,9 @@ import cats.implicits.catsSyntaxOptionId import io.lenses.streamreactor.common.errors.ThrowErrorPolicy import io.lenses.streamreactor.connect.aws.s3.config.AuthMode import io.lenses.streamreactor.connect.aws.s3.config.HttpTimeoutConfig -import io.lenses.streamreactor.connect.aws.s3.config.RetryConfig -import io.lenses.streamreactor.connect.aws.s3.config.S3Config +import io.lenses.streamreactor.connect.aws.s3.config.S3ConnectionConfig import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings._ +import io.lenses.streamreactor.connect.cloud.common.config.RetryConfig import io.lenses.streamreactor.connect.cloud.common.consumers.CloudObjectKey import org.scalatest.funsuite.AnyFunSuite import org.scalatest.matchers.should.Matchers @@ -43,7 +43,7 @@ class S3ConsumerGroupsSinkConfigTest extends AnyFunSuite with Matchers { value should be( S3ConsumerGroupsSinkConfig( CloudObjectKey("bucket", "a/b/c".some), - S3Config( + S3ConnectionConfig( Some("eu-west-1"), Some("access"), Some("secret"), @@ -77,7 +77,7 @@ class S3ConsumerGroupsSinkConfigTest extends AnyFunSuite with Matchers { value should be( S3ConsumerGroupsSinkConfig( CloudObjectKey("bucket", "a/b/c".some), - S3Config( + S3ConnectionConfig( Some("eu-west-1"), Some("access"), Some("secret"), diff --git a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3SinkConfigDefBuilderTest.scala b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3SinkConfigDefBuilderTest.scala index 06281829f..2cbb40508 100644 --- a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3SinkConfigDefBuilderTest.scala +++ b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3SinkConfigDefBuilderTest.scala @@ -64,7 +64,7 @@ class S3SinkConfigDefBuilderTest extends AnyFlatSpec with MockitoSugar with Matc "connect.s3.kcql" -> s"insert into mybucket:myprefix select * from $TopicName PARTITIONBY _key STOREAS CSV WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1", ) - CloudSinkBucketOptions(S3SinkConfigDefBuilder(props)) match { + CloudSinkBucketOptions(connectorTaskId, S3SinkConfigDefBuilder(props)) match { case Left(value) => fail(value.toString) case Right(value) => value.map(_.dataStorage) should be(List(DataStorageSettings.Default)) } @@ -75,7 +75,7 @@ class S3SinkConfigDefBuilderTest extends AnyFlatSpec with MockitoSugar with Matc "connect.s3.kcql" -> s"insert into mybucket:myprefix select * from $TopicName PARTITIONBY _key STOREAS `JSON` WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true)", ) - config.CloudSinkBucketOptions(S3SinkConfigDefBuilder(props)) match { + config.CloudSinkBucketOptions(connectorTaskId, S3SinkConfigDefBuilder(props)) match { case Left(value) => fail(value.toString) case Right(value) => value.map(_.dataStorage) should be(List(DataStorageSettings.enabled)) } @@ -86,7 +86,7 @@ class S3SinkConfigDefBuilderTest extends AnyFlatSpec with MockitoSugar with Matc "connect.s3.kcql" -> s"insert into mybucket:myprefix select * from $TopicName PARTITIONBY _key STOREAS `PARQUET` WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true, '${DataStorageSettings.StoreKeyKey}'=true, '${DataStorageSettings.StoreValueKey}'=true, '${DataStorageSettings.StoreMetadataKey}'=false, '${DataStorageSettings.StoreHeadersKey}'=false)", ) - config.CloudSinkBucketOptions(S3SinkConfigDefBuilder(props)) match { + config.CloudSinkBucketOptions(connectorTaskId, S3SinkConfigDefBuilder(props)) match { case Left(value) => fail(value.toString) case Right(value) => value.map(_.dataStorage) should be(List(DataStorageSettings(true, true, true, false, false))) @@ -115,7 +115,7 @@ class S3SinkConfigDefBuilderTest extends AnyFlatSpec with MockitoSugar with Matc |""".stripMargin, ) - config.CloudSinkBucketOptions(S3SinkConfigDefBuilder(props)) match { + config.CloudSinkBucketOptions(connectorTaskId, S3SinkConfigDefBuilder(props)) match { case Left(value) => fail(value.toString) case Right(value) => value.map(_.dataStorage) should be(List(DataStorageSettings(true, true, true, false, false), @@ -191,7 +191,7 @@ class S3SinkConfigDefBuilderTest extends AnyFlatSpec with MockitoSugar with Matc "connect.s3.kcql" -> s"insert into $BucketName:$PrefixName select * from $TopicName STOREAS `JSON` WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true, '${DataStorageSettings.StoreKeyKey}'=true, '${DataStorageSettings.StoreValueKey}'=true, '${DataStorageSettings.StoreMetadataKey}'=false, '${DataStorageSettings.StoreHeadersKey}'=false)", ) - config.CloudSinkBucketOptions(S3SinkConfigDefBuilder(props)) match { + config.CloudSinkBucketOptions(connectorTaskId, S3SinkConfigDefBuilder(props)) match { case Left(value) => fail(value.toString) case Right(value) => value.map(_.dataStorage) should be(List(DataStorageSettings(envelope = true, @@ -208,7 +208,7 @@ class S3SinkConfigDefBuilderTest extends AnyFlatSpec with MockitoSugar with Matc "connect.s3.kcql" -> s"insert into $BucketName:$PrefixName select * from $TopicName STOREAS `JSON` WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true, '${DataStorageSettings.StoreKeyKey}'=true, '${DataStorageSettings.StoreValueKey}'=true, '${DataStorageSettings.StoreMetadataKey}'=false, '${DataStorageSettings.StoreHeadersKey}'=false)", ) - config.CloudSinkBucketOptions(S3SinkConfigDefBuilder(props)) match { + config.CloudSinkBucketOptions(connectorTaskId, S3SinkConfigDefBuilder(props)) match { case Left(value) => fail(value.toString) case Right(value) => value.map(_.dataStorage) should be(List(DataStorageSettings(envelope = true, @@ -225,7 +225,9 @@ class S3SinkConfigDefBuilderTest extends AnyFlatSpec with MockitoSugar with Matc "connect.s3.kcql" -> s"insert into $BucketName:$PrefixName select * from $TopicName STOREAS `BYTES_VALUEONLY` WITH_FLUSH_COUNT = 1", ) - config.CloudSinkBucketOptions(S3SinkConfigDefBuilder(props)).left.value.getMessage should startWith( + config.CloudSinkBucketOptions(connectorTaskId, + S3SinkConfigDefBuilder(props), + ).left.value.getMessage should startWith( "Unsupported format - BYTES_VALUEONLY. Please note", ) } @@ -235,7 +237,9 @@ class S3SinkConfigDefBuilderTest extends AnyFlatSpec with MockitoSugar with Matc "connect.s3.kcql" -> s"insert into $BucketName:$PrefixName select * from $TopicName STOREAS `BYTES` WITH_FLUSH_COUNT = 3", ) - config.CloudSinkBucketOptions(S3SinkConfigDefBuilder(props)).left.value.getMessage should startWith( + config.CloudSinkBucketOptions(connectorTaskId, + S3SinkConfigDefBuilder(props), + ).left.value.getMessage should startWith( "FLUSH_COUNT > 1 is not allowed for BYTES", ) } diff --git a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3SinkConfigTest.scala b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3SinkConfigTest.scala index b17c04d33..f3d01e622 100644 --- a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3SinkConfigTest.scala +++ b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/sink/config/S3SinkConfigTest.scala @@ -32,7 +32,7 @@ class S3SinkConfigTest extends AnyFunSuite with Matchers { "connect.s3.kcql" -> s"insert into mybucket:myprefix select * from TopicName PARTITIONBY _key STOREAS `CSV` WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true)", ) - CloudSinkBucketOptions(S3SinkConfigDefBuilder(props)) match { + CloudSinkBucketOptions(connectorTaskId, S3SinkConfigDefBuilder(props)) match { case Left(value) => value.getMessage shouldBe "Envelope is not supported for format CSV." case Right(_) => fail("Should fail since envelope and CSV storage is not allowed") } @@ -43,7 +43,7 @@ class S3SinkConfigTest extends AnyFunSuite with Matchers { "connect.s3.kcql" -> s"insert into mybucket:myprefix select * from TopicName PARTITIONBY _key STOREAS `Parquet` WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true)", ) - config.CloudSinkBucketOptions(S3SinkConfigDefBuilder(props)) match { + config.CloudSinkBucketOptions(connectorTaskId, S3SinkConfigDefBuilder(props)) match { case Left(error) => fail("Should not fail since envelope and Parquet storage is allowed", error) case Right(_) => succeed } @@ -53,7 +53,7 @@ class S3SinkConfigTest extends AnyFunSuite with Matchers { "connect.s3.kcql" -> s"insert into mybucket:myprefix select * from TopicName PARTITIONBY _key STOREAS `Avro` WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true)", ) - config.CloudSinkBucketOptions(S3SinkConfigDefBuilder(props)) match { + config.CloudSinkBucketOptions(connectorTaskId, S3SinkConfigDefBuilder(props)) match { case Left(error) => fail("Should not fail since envelope and Avro storage is allowed", error) case Right(_) => succeed } @@ -63,7 +63,7 @@ class S3SinkConfigTest extends AnyFunSuite with Matchers { "connect.s3.kcql" -> s"insert into mybucket:myprefix select * from TopicName PARTITIONBY _key STOREAS `Json` WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true)", ) - config.CloudSinkBucketOptions(S3SinkConfigDefBuilder(props)) match { + config.CloudSinkBucketOptions(connectorTaskId, S3SinkConfigDefBuilder(props)) match { case Left(error) => fail("Should not fail since envelope and Json storage is allowed", error) case Right(_) => succeed } @@ -73,7 +73,7 @@ class S3SinkConfigTest extends AnyFunSuite with Matchers { "connect.s3.kcql" -> s"insert into mybucket:myprefix select * from TopicName PARTITIONBY _key STOREAS `Text` WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true)", ) - config.CloudSinkBucketOptions(S3SinkConfigDefBuilder(props)) match { + config.CloudSinkBucketOptions(connectorTaskId, S3SinkConfigDefBuilder(props)) match { case Left(value) => value.getMessage shouldBe "Envelope is not supported for format TEXT." case Right(_) => fail("Should fail since text and envelope storage is not allowed") } @@ -83,7 +83,7 @@ class S3SinkConfigTest extends AnyFunSuite with Matchers { "connect.s3.kcql" -> s"insert into mybucket:myprefix select * from TopicName PARTITIONBY _key STOREAS `Bytes` WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true)", ) - config.CloudSinkBucketOptions(S3SinkConfigDefBuilder(props)) match { + config.CloudSinkBucketOptions(connectorTaskId, S3SinkConfigDefBuilder(props)) match { case Left(value) => value.getMessage shouldBe "Envelope is not supported for format BYTES." case Right(_) => fail("Should fail since envelope and bytes storage is not allowed") } diff --git a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfigTest.scala b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfigTest.scala index aace14384..77684388e 100644 --- a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfigTest.scala +++ b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfigTest.scala @@ -18,10 +18,11 @@ package io.lenses.streamreactor.connect.aws.s3.source.config import io.lenses.streamreactor.connect.aws.s3.config.AuthMode import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings._ import io.lenses.streamreactor.connect.cloud.common.config.TaskIndexKey +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceSettingsKeys import org.scalatest.funsuite.AnyFunSuite import org.scalatest.matchers.should.Matchers -class S3SourceConfigTest extends AnyFunSuite with Matchers with TaskIndexKey with SourcePartitionSearcherSettingsKeys { +class S3SourceConfigTest extends AnyFunSuite with Matchers with TaskIndexKey with CloudSourceSettingsKeys { private val Identity: String = "identity" private val Credential: String = "credential" private val BucketName: String = "mybucket" diff --git a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfigTests.scala b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfigTests.scala index 7e8633a85..cb87a7ffa 100644 --- a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfigTests.scala +++ b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/source/config/S3SourceConfigTests.scala @@ -16,17 +16,25 @@ package io.lenses.streamreactor.connect.aws.s3.source.config import io.lenses.streamreactor.connect.aws.s3.config.S3ConfigSettings._ +import io.lenses.streamreactor.connect.aws.s3.model.location.S3LocationValidator +import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId import io.lenses.streamreactor.connect.cloud.common.config.TaskIndexKey import io.lenses.streamreactor.connect.cloud.common.source.config.PartitionSearcherOptions +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceSettingsKeys import io.lenses.streamreactor.connect.cloud.common.source.config.PartitionSearcherOptions.ExcludeIndexes import org.scalatest.funsuite.AnyFunSuite import org.scalatest.matchers.should.Matchers import scala.concurrent.duration._ -class S3SourceConfigTests extends AnyFunSuite with Matchers with TaskIndexKey with SourcePartitionSearcherSettingsKeys { +class S3SourceConfigTests extends AnyFunSuite with Matchers with TaskIndexKey with CloudSourceSettingsKeys { + + implicit val taskId = ConnectorTaskId("test", 1, 1) + implicit val validator = S3LocationValidator + test("default recursive levels is 0") { S3SourceConfig.fromProps( + taskId, Map( SOURCE_PARTITION_SEARCH_MODE -> "false", SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS -> "1000", @@ -41,6 +49,7 @@ class S3SourceConfigTests extends AnyFunSuite with Matchers with TaskIndexKey wi } test("partition search options disables the continuous search") { S3SourceConfig.fromProps( + taskId, Map( SOURCE_PARTITION_SEARCH_RECURSE_LEVELS -> "1", SOURCE_PARTITION_SEARCH_MODE -> "false", @@ -56,6 +65,7 @@ class S3SourceConfigTests extends AnyFunSuite with Matchers with TaskIndexKey wi } test("enable continuous partitions polling") { S3SourceConfig.fromProps( + taskId, Map( SOURCE_PARTITION_SEARCH_RECURSE_LEVELS -> "1", SOURCE_PARTITION_SEARCH_MODE -> "true", @@ -71,6 +81,7 @@ class S3SourceConfigTests extends AnyFunSuite with Matchers with TaskIndexKey wi } test("not specifying the SOURCE_PARTITION_SEARCH_MODE defaults to true") { S3SourceConfig.fromProps( + taskId, Map( SOURCE_PARTITION_SEARCH_RECURSE_LEVELS -> "1", SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS -> "1000", diff --git a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/source/reader/S3PartitionDiscoveryTest.scala b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/source/reader/S3PartitionDiscoveryTest.scala index b142ae083..c19149794 100644 --- a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/source/reader/S3PartitionDiscoveryTest.scala +++ b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/source/reader/S3PartitionDiscoveryTest.scala @@ -20,7 +20,7 @@ import cats.effect.kernel.Ref import cats.effect.unsafe.implicits.global import cats.implicits.catsSyntaxOptionId import io.lenses.streamreactor.connect.aws.s3.model.location.S3LocationValidator -import io.lenses.streamreactor.connect.aws.s3.source.distribution.PartitionSearcher +import io.lenses.streamreactor.connect.aws.s3.source.distribution.S3PartitionSearcher import io.lenses.streamreactor.connect.aws.s3.storage.MockS3Client import io.lenses.streamreactor.connect.aws.s3.storage.S3Page import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId @@ -64,12 +64,12 @@ class S3PartitionDiscoveryTest extends AnyFlatSpecLike with Matchers with Mockit fiber <- PartitionDiscovery.run( connectorTaskId, options, - new PartitionSearcher(List( - CloudLocation("bucket", None), - ), - options, - connectorTaskId, - s3Client.listObjectsV2Paginator(_).iterator().asScala, + new S3PartitionSearcher(List( + CloudLocation("bucket", None), + ), + options, + connectorTaskId, + s3Client.listObjectsV2Paginator(_).iterator().asScala, ).find, (_, _) => IO(new ReaderManager(limit, @@ -129,12 +129,12 @@ class S3PartitionDiscoveryTest extends AnyFlatSpecLike with Matchers with Mockit fiber <- PartitionDiscovery.run( connectorTaskId, options, - new PartitionSearcher(List( - CloudLocation("bucket", None), - ), - options, - connectorTaskId, - s3Client.listObjectsV2Paginator(_).iterator().asScala, + new S3PartitionSearcher(List( + CloudLocation("bucket", None), + ), + options, + connectorTaskId, + s3Client.listObjectsV2Paginator(_).iterator().asScala, ).find, (_, _) => IO(new ReaderManager(limit, @@ -186,12 +186,12 @@ class S3PartitionDiscoveryTest extends AnyFlatSpecLike with Matchers with Mockit fiber <- PartitionDiscovery.run( connectorTaskId, options, - new PartitionSearcher(List( - CloudLocation("bucket", "prefix1/".some), - ), - options, - connectorTaskId, - s3Client.listObjectsV2Paginator(_).iterator().asScala, + new S3PartitionSearcher(List( + CloudLocation("bucket", "prefix1/".some), + ), + options, + connectorTaskId, + s3Client.listObjectsV2Paginator(_).iterator().asScala, ).find, (_, _) => IO(new ReaderManager(limit, @@ -244,12 +244,12 @@ class S3PartitionDiscoveryTest extends AnyFlatSpecLike with Matchers with Mockit fiber <- PartitionDiscovery.run( taskId, options, - new PartitionSearcher(List( - CloudLocation("bucket", "prefix1/".some), - ), - options, - taskId, - s3Client.listObjectsV2Paginator(_).iterator().asScala, + new S3PartitionSearcher(List( + CloudLocation("bucket", "prefix1/".some), + ), + options, + taskId, + s3Client.listObjectsV2Paginator(_).iterator().asScala, ).find, ( _, diff --git a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/source/state/ReaderManagerBuilderTest.scala b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/source/state/ReaderManagerBuilderTest.scala index 68af3448c..4ecb5e8ab 100644 --- a/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/source/state/ReaderManagerBuilderTest.scala +++ b/kafka-connect-aws-s3/src/test/scala/io/lenses/streamreactor/connect/aws/s3/source/state/ReaderManagerBuilderTest.scala @@ -17,13 +17,14 @@ package io.lenses.streamreactor.connect.aws.s3.source.state import cats.effect.testing.scalatest.AsyncIOSpec import io.lenses.streamreactor.connect.aws.s3.model.location.S3LocationValidator -import io.lenses.streamreactor.connect.aws.s3.source.config.SourceBucketOptions import io.lenses.streamreactor.connect.aws.s3.storage.S3FileMetadata import io.lenses.streamreactor.connect.cloud.common.config.AvroFormatSelection import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocation import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocationValidator +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceBucketOptions import io.lenses.streamreactor.connect.cloud.common.source.config.OrderingType +import io.lenses.streamreactor.connect.cloud.common.source.state.ReaderManagerBuilder import io.lenses.streamreactor.connect.cloud.common.storage.StorageInterface import org.mockito.MockitoSugar.mock import org.scalatest.flatspec.AsyncFlatSpec @@ -46,7 +47,15 @@ class ReaderManagerBuilderTest extends AsyncFlatSpec with AsyncIOSpec with Match rootValue = Some(in) rootValue } - val sbo = SourceBucketOptions(root, "topic", AvroFormatSelection, 100, 100, None, OrderingType.LastModified, false) + val sbo = CloudSourceBucketOptions[S3FileMetadata](root, + "topic", + AvroFormatSelection, + 100, + 100, + None, + OrderingType.LastModified, + false, + ) val taskId = ConnectorTaskId("test", 3, 1) ReaderManagerBuilder(root, path, si, taskId, contextF, _ => Some(sbo)) .asserting(_ => rootValue shouldBe Some(root.copy(prefix = Some(path)))) diff --git a/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/auth/DatalakeClientCreator.scala b/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/auth/DatalakeClientCreator.scala index 2feda0805..d81f67512 100644 --- a/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/auth/DatalakeClientCreator.scala +++ b/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/auth/DatalakeClientCreator.scala @@ -27,15 +27,15 @@ import io.lenses.streamreactor.connect.cloud.common.auth.ClientCreator import io.lenses.streamreactor.connect.datalake.config.AuthMode import io.lenses.streamreactor.connect.datalake.config.AuthMode.ConnectionString import io.lenses.streamreactor.connect.datalake.config.AuthMode.Credentials -import io.lenses.streamreactor.connect.datalake.config.AzureConfig +import io.lenses.streamreactor.connect.datalake.config.AzureConnectionConfig import io.lenses.streamreactor.connect.datalake.config.ConnectionPoolConfig import java.time.Duration import scala.util.Try -object DatalakeClientCreator extends ClientCreator[AzureConfig, DataLakeServiceClient] { +object DatalakeClientCreator extends ClientCreator[AzureConnectionConfig, DataLakeServiceClient] { - def make(config: AzureConfig): Either[Throwable, DataLakeServiceClient] = { + def make(config: AzureConnectionConfig): Either[Throwable, DataLakeServiceClient] = { require(config != null, "AzureDataLakeConfig cannot be null") config.authMode match { @@ -50,7 +50,7 @@ object DatalakeClientCreator extends ClientCreator[AzureConfig, DataLakeServiceC } } - private def createHttpClient(config: AzureConfig): HttpClient = { + private def createHttpClient(config: AzureConnectionConfig): HttpClient = { val httpClientOptions = new HttpClientOptions() config.timeouts.socketTimeout.foreach(millis => httpClientOptions.setReadTimeout(Duration.ofMillis(millis))) @@ -77,7 +77,7 @@ object DatalakeClientCreator extends ClientCreator[AzureConfig, DataLakeServiceC }.toEither private def createDataLakeClientWithSharedKey( - config: AzureConfig, + config: AzureConnectionConfig, authMode: Credentials, ): Either[Throwable, DataLakeServiceClient] = Try { @@ -93,7 +93,9 @@ object DatalakeClientCreator extends ClientCreator[AzureConfig, DataLakeServiceC }.toEither - private def createDataLakeClientWithDefaultCredential(config: AzureConfig): Either[Throwable, DataLakeServiceClient] = + private def createDataLakeClientWithDefaultCredential( + config: AzureConnectionConfig, + ): Either[Throwable, DataLakeServiceClient] = Try { val builder = new DataLakeServiceClientBuilder() .credential(new DefaultAzureCredentialBuilder().build()) diff --git a/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/config/AzureConfig.scala b/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/config/AzureConnectionConfig.scala similarity index 91% rename from kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/config/AzureConfig.scala rename to kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/config/AzureConnectionConfig.scala index 2c4affade..6e89e608d 100644 --- a/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/config/AzureConfig.scala +++ b/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/config/AzureConnectionConfig.scala @@ -18,15 +18,16 @@ package io.lenses.streamreactor.connect.datalake.config import io.lenses.streamreactor.common.errors.ErrorPolicy import io.lenses.streamreactor.common.errors.ErrorPolicyEnum import io.lenses.streamreactor.common.errors.ThrowErrorPolicy -import io.lenses.streamreactor.connect.cloud.common.config.CloudConfig import io.lenses.streamreactor.connect.cloud.common.config.ConfigParse.getInt import io.lenses.streamreactor.connect.cloud.common.config.ConfigParse.getLong import io.lenses.streamreactor.connect.cloud.common.config.ConfigParse.getString +import io.lenses.streamreactor.connect.cloud.common.config.RetryConfig +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudConnectionConfig import io.lenses.streamreactor.connect.datalake.config.AzureConfigSettings._ -object AzureConfig { +object AzureConnectionConfig { - def apply(props: Map[String, _], authMode: AuthMode): AzureConfig = AzureConfig( + def apply(props: Map[String, _], authMode: AuthMode): AzureConnectionConfig = AzureConnectionConfig( authMode, getString(props, ENDPOINT), getErrorPolicy(props), @@ -53,8 +54,6 @@ object AzureConfig { ) } -case class RetryConfig(numberOfRetries: Int, errorRetryInterval: Long) - case class HttpTimeoutConfig(socketTimeout: Option[Long], connectionTimeout: Option[Long]) case class ConnectionPoolConfig(maxConnections: Int) @@ -64,7 +63,7 @@ object ConnectionPoolConfig { maxConns.filterNot(_ == -1).map(ConnectionPoolConfig(_)) } -case class AzureConfig( +case class AzureConnectionConfig( authMode: AuthMode, endpoint: Option[String] = None, errorPolicy: ErrorPolicy = ThrowErrorPolicy(), @@ -72,4 +71,4 @@ case class AzureConfig( httpRetryConfig: RetryConfig = RetryConfig(HTTP_NBR_OF_RETIRES_DEFAULT, HTTP_ERROR_RETRY_INTERVAL_DEFAULT), timeouts: HttpTimeoutConfig = HttpTimeoutConfig(None, None), connectionPoolConfig: Option[ConnectionPoolConfig] = Option.empty, -) extends CloudConfig +) extends CloudConnectionConfig diff --git a/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/sink/DatalakeSinkTask.scala b/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/sink/DatalakeSinkTask.scala index 42f2c6ac4..27bdfcbdc 100644 --- a/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/sink/DatalakeSinkTask.scala +++ b/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/sink/DatalakeSinkTask.scala @@ -15,50 +15,38 @@ */ package io.lenses.streamreactor.connect.datalake.sink -import io.lenses.streamreactor.common.errors.RetryErrorPolicy +import com.azure.storage.file.datalake.DataLakeServiceClient import io.lenses.streamreactor.common.utils.JarManifest +import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId import io.lenses.streamreactor.connect.cloud.common.sink.CloudSinkTask -import io.lenses.streamreactor.connect.cloud.common.sink.WriterManagerCreator -import io.lenses.streamreactor.connect.cloud.common.sink.writer.WriterManager +import io.lenses.streamreactor.connect.cloud.common.storage.StorageInterface import io.lenses.streamreactor.connect.datalake.auth.DatalakeClientCreator -import io.lenses.streamreactor.connect.datalake.config.AzureConfig import io.lenses.streamreactor.connect.datalake.config.AzureConfigSettings import io.lenses.streamreactor.connect.datalake.model.location.DatalakeLocationValidator import io.lenses.streamreactor.connect.datalake.sink.config.DatalakeSinkConfig import io.lenses.streamreactor.connect.datalake.storage.DatalakeFileMetadata import io.lenses.streamreactor.connect.datalake.storage.DatalakeStorageInterface - -import scala.util.Try object DatalakeSinkTask {} class DatalakeSinkTask - extends CloudSinkTask[DatalakeFileMetadata]( + extends CloudSinkTask[DatalakeFileMetadata, DatalakeSinkConfig, DataLakeServiceClient]( AzureConfigSettings.CONNECTOR_PREFIX, "/datalake-sink-ascii.txt", JarManifest(DatalakeSinkTask.getClass.getProtectionDomain.getCodeSource.getLocation), - )( - DatalakeLocationValidator, ) { - private val writerManagerCreator = new WriterManagerCreator[DatalakeFileMetadata, DatalakeSinkConfig]() + override def createClient(config: DatalakeSinkConfig): Either[Throwable, DataLakeServiceClient] = + DatalakeClientCreator.make(config.connectionConfig) - def createWriterMan(props: Map[String, String]): Either[Throwable, WriterManager[DatalakeFileMetadata]] = - for { - config <- DatalakeSinkConfig.fromProps(props) - s3Client <- DatalakeClientCreator.make(config.s3Config) - storageInterface = new DatalakeStorageInterface(connectorTaskId, s3Client) - _ <- Try(setErrorRetryInterval(config.s3Config)).toEither - writerManager <- Try(writerManagerCreator.from(config)(connectorTaskId, storageInterface)).toEither - _ <- Try(initialize( - config.s3Config.connectorRetryConfig.numberOfRetries, - config.s3Config.errorPolicy, - )).toEither - } yield writerManager + override def createStorageInterface( + connectorTaskId: ConnectorTaskId, + config: DatalakeSinkConfig, + cloudClient: DataLakeServiceClient, + ): StorageInterface[DatalakeFileMetadata] = new DatalakeStorageInterface(connectorTaskId, cloudClient) - private def setErrorRetryInterval(s3Config: AzureConfig): Unit = - //if error policy is retry set retry interval - s3Config.errorPolicy match { - case RetryErrorPolicy() => context.timeout(s3Config.connectorRetryConfig.errorRetryInterval) - case _ => - } + override def convertPropsToConfig( + connectorTaskId: ConnectorTaskId, + props: Map[String, String], + ): Either[Throwable, DatalakeSinkConfig] = + DatalakeSinkConfig.fromProps(connectorTaskId, props)(DatalakeLocationValidator) } diff --git a/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/sink/config/DatalakeSinkConfig.scala b/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/sink/config/DatalakeSinkConfig.scala index 8c295de46..24ed7a027 100644 --- a/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/sink/config/DatalakeSinkConfig.scala +++ b/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/sink/config/DatalakeSinkConfig.scala @@ -16,40 +16,41 @@ package io.lenses.streamreactor.connect.datalake.sink.config import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudSinkConfig +import io.lenses.streamreactor.connect.cloud.common.config.traits.PropsToConfigConverter import io.lenses.streamreactor.connect.cloud.common.model.CompressionCodec import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocationValidator import io.lenses.streamreactor.connect.cloud.common.sink.config.CloudSinkBucketOptions -import io.lenses.streamreactor.connect.cloud.common.sink.config.CloudSinkConfig import io.lenses.streamreactor.connect.cloud.common.sink.config.OffsetSeekerOptions -import io.lenses.streamreactor.connect.datalake.config.AzureConfig +import io.lenses.streamreactor.connect.datalake.config.AzureConnectionConfig import io.lenses.streamreactor.connect.datalake.config.AzureConfigSettings.SEEK_MAX_INDEX_FILES -object DatalakeSinkConfig { +object DatalakeSinkConfig extends PropsToConfigConverter[DatalakeSinkConfig] { def fromProps( - props: Map[String, String], + connectorTaskId: ConnectorTaskId, + props: Map[String, String], )( implicit - connectorTaskId: ConnectorTaskId, cloudLocationValidator: CloudLocationValidator, ): Either[Throwable, DatalakeSinkConfig] = - DatalakeSinkConfig(DatalakeSinkConfigDefBuilder(props)) + DatalakeSinkConfig(connectorTaskId, DatalakeSinkConfigDefBuilder(props)) def apply( + connectorTaskId: ConnectorTaskId, s3ConfigDefBuilder: DatalakeSinkConfigDefBuilder, )( implicit - connectorTaskId: ConnectorTaskId, cloudLocationValidator: CloudLocationValidator, ): Either[Throwable, DatalakeSinkConfig] = for { authMode <- s3ConfigDefBuilder.getAuthMode - sinkBucketOptions <- CloudSinkBucketOptions(s3ConfigDefBuilder) + sinkBucketOptions <- CloudSinkBucketOptions(connectorTaskId, s3ConfigDefBuilder) offsetSeekerOptions = OffsetSeekerOptions( s3ConfigDefBuilder.getInt(SEEK_MAX_INDEX_FILES), ) } yield DatalakeSinkConfig( - AzureConfig(s3ConfigDefBuilder.getParsedValues, authMode), + AzureConnectionConfig(s3ConfigDefBuilder.getParsedValues, authMode), sinkBucketOptions, offsetSeekerOptions, s3ConfigDefBuilder.getCompressionCodec(), @@ -58,7 +59,7 @@ object DatalakeSinkConfig { } case class DatalakeSinkConfig( - s3Config: AzureConfig, + connectionConfig: AzureConnectionConfig, bucketOptions: Seq[CloudSinkBucketOptions] = Seq.empty, offsetSeekerOptions: OffsetSeekerOptions, compressionCodec: CompressionCodec, diff --git a/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/sink/config/DatalakeSinkConfigDef.scala b/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/sink/config/DatalakeSinkConfigDef.scala index 5cadda60f..eeb957621 100644 --- a/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/sink/config/DatalakeSinkConfigDef.scala +++ b/kafka-connect-azure-datalake/src/main/scala/io/lenses/streamreactor/connect/datalake/sink/config/DatalakeSinkConfigDef.scala @@ -15,10 +15,7 @@ */ package io.lenses.streamreactor.connect.datalake.sink.config -import cats.implicits.catsSyntaxEitherId -import com.typesafe.scalalogging.LazyLogging -import io.lenses.streamreactor.connect.cloud.common.config.processors.ConfigDefProcessor -import io.lenses.streamreactor.connect.cloud.common.config.processors.LowerCaseKeyConfigDefProcessor +import io.lenses.streamreactor.connect.cloud.common.config.CloudConfigDef import io.lenses.streamreactor.connect.cloud.common.sink.config.FlushConfigKeys import io.lenses.streamreactor.connect.cloud.common.sink.config.LocalStagingAreaConfigKeys import io.lenses.streamreactor.connect.cloud.common.sink.config.padding.PaddingStrategyConfigKeys @@ -28,9 +25,6 @@ import org.apache.kafka.common.config.ConfigDef import org.apache.kafka.common.config.ConfigDef.Importance import org.apache.kafka.common.config.ConfigDef.Type -import java.util -import scala.jdk.CollectionConverters._ - object DatalakeSinkConfigDef extends CommonConfigDef with FlushConfigKeys @@ -65,37 +59,4 @@ object DatalakeSinkConfigDef } -class DatalakeSinkConfigDef() extends ConfigDef with LazyLogging { - - private val processorChain: List[ConfigDefProcessor] = - List(new LowerCaseKeyConfigDefProcessor(CONNECTOR_PREFIX)) - - override def parse(jProps: util.Map[_, _]): util.Map[String, AnyRef] = { - val scalaProps: Map[Any, Any] = jProps.asScala.toMap - processProperties(scalaProps) match { - case Left(exception) => throw exception - case Right(value) => super.parse(value.asJava) - } - } - - private def processProperties(scalaProps: Map[Any, Any]): Either[Throwable, Map[Any, Any]] = { - val stringProps = scalaProps.collect { case (k: String, v: AnyRef) => (k.toLowerCase, v) } - val nonStringProps = scalaProps -- stringProps.keySet - processStringKeyedProperties(stringProps) match { - case Left(exception) => exception.asLeft[Map[Any, Any]] - case Right(stringKeyedProps) => (nonStringProps ++ stringKeyedProps).asRight - } - } - - private def processStringKeyedProperties(stringProps: Map[String, Any]): Either[Throwable, Map[String, Any]] = { - var remappedProps: Map[String, Any] = stringProps - for (proc <- processorChain) { - proc.process(remappedProps) match { - case Left(exception) => return exception.asLeft[Map[String, AnyRef]] - case Right(properties) => remappedProps = properties - } - } - remappedProps.asRight - } - -} +class DatalakeSinkConfigDef() extends CloudConfigDef(CONNECTOR_PREFIX) {} diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/auth/ClientCreator.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/auth/ClientCreator.scala index 9684a4340..72c47ea90 100644 --- a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/auth/ClientCreator.scala +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/auth/ClientCreator.scala @@ -15,8 +15,8 @@ */ package io.lenses.streamreactor.connect.cloud.common.auth -import io.lenses.streamreactor.connect.cloud.common.config.CloudConfig +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudConnectionConfig -trait ClientCreator[CT <: CloudConfig, X] { +trait ClientCreator[CT <: CloudConnectionConfig, X] { def make(config: CT): Either[Throwable, X] } diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/CloudConfigDef.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/CloudConfigDef.scala new file mode 100644 index 000000000..52405267c --- /dev/null +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/CloudConfigDef.scala @@ -0,0 +1,62 @@ +/* + * Copyright 2017-2024 Lenses.io Ltd + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.lenses.streamreactor.connect.cloud.common.config + +import cats.implicits.catsSyntaxEitherId +import com.typesafe.scalalogging.LazyLogging +import io.lenses.streamreactor.connect.cloud.common.config.processors.ConfigDefProcessor +import io.lenses.streamreactor.connect.cloud.common.config.processors.LowerCaseKeyConfigDefProcessor +import org.apache.kafka.common.config.ConfigDef + +import java.util +import scala.jdk.CollectionConverters.MapHasAsJava +import scala.jdk.CollectionConverters.MapHasAsScala + +class CloudConfigDef(connectorPrefix: String, processors: ConfigDefProcessor*) extends ConfigDef with LazyLogging { + + private val processorChain: List[ConfigDefProcessor] = { + List(new LowerCaseKeyConfigDefProcessor(connectorPrefix)) ++ processors + } + + override def parse(jProps: util.Map[_, _]): util.Map[String, AnyRef] = { + val scalaProps: Map[Any, Any] = jProps.asScala.toMap + processProperties(scalaProps) match { + case Left(exception) => throw exception + case Right(value) => super.parse(value.asJava) + } + } + + private def processProperties(scalaProps: Map[Any, Any]): Either[Throwable, Map[Any, Any]] = { + val stringProps = scalaProps.collect { case (k: String, v: AnyRef) => (k.toLowerCase, v) } + val nonStringProps = scalaProps -- stringProps.keySet + processStringKeyedProperties(stringProps) match { + case Left(exception) => exception.asLeft[Map[Any, Any]] + case Right(stringKeyedProps) => (nonStringProps ++ stringKeyedProps).asRight + } + } + + private def processStringKeyedProperties(stringProps: Map[String, Any]): Either[Throwable, Map[String, Any]] = { + var remappedProps: Map[String, Any] = stringProps + for (proc <- processorChain) { + proc.process(remappedProps) match { + case Left(exception) => return exception.asLeft[Map[String, AnyRef]] + case Right(properties) => remappedProps = properties + } + } + remappedProps.asRight + } + +} diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/CloudConfig.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/RetryConfig.scala similarity index 90% rename from kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/CloudConfig.scala rename to kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/RetryConfig.scala index 7e7503352..fd2314905 100644 --- a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/CloudConfig.scala +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/RetryConfig.scala @@ -14,4 +14,4 @@ * limitations under the License. */ package io.lenses.streamreactor.connect.cloud.common.config -trait CloudConfig +case class RetryConfig(numberOfRetries: Int, errorRetryInterval: Long) diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/traits/CloudConfig.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/traits/CloudConfig.scala new file mode 100644 index 000000000..57c205961 --- /dev/null +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/traits/CloudConfig.scala @@ -0,0 +1,101 @@ +/* + * Copyright 2017-2024 Lenses.io Ltd + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.lenses.streamreactor.connect.cloud.common.config.traits + +import io.lenses.streamreactor.connect.cloud.common.model.CompressionCodec +import io.lenses.streamreactor.connect.cloud.common.sink.config.CloudSinkBucketOptions +import io.lenses.streamreactor.connect.cloud.common.sink.config.OffsetSeekerOptions +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceBucketOptions +import io.lenses.streamreactor.connect.cloud.common.source.config.PartitionSearcherOptions +import io.lenses.streamreactor.connect.cloud.common.storage.FileMetadata + +/** + * Trait representing a generic cloud configuration. + * This trait serves as a marker trait for cloud-specific configuration implementations. + */ +trait CloudConfig + +/** + * Trait representing configuration for a cloud sink. + * Extends [[CloudConfig]]. + */ +trait CloudSinkConfig extends CloudConfig { + + /** + * Retrieves the connection configuration for the cloud sink. + * + * @return The connection configuration for the cloud sink. + */ + def connectionConfig: CloudConnectionConfig + + /** + * Retrieves the bucket options for the cloud sink. + * + * @return The bucket options for the cloud sink. + */ + def bucketOptions: Seq[CloudSinkBucketOptions] + + /** + * Retrieves the offset seeker options for the cloud sink. + * + * @return The offset seeker options for the cloud sink. + */ + def offsetSeekerOptions: OffsetSeekerOptions + + /** + * Retrieves the compression codec for the cloud sink. + * + * @return The compression codec for the cloud sink. + */ + def compressionCodec: CompressionCodec +} + +/** + * Trait representing configuration for a cloud source. + * Extends [[CloudConfig]]. + * + * @tparam MD The type of file metadata associated with the cloud source. + */ +trait CloudSourceConfig[MD <: FileMetadata] extends CloudConfig { + + /** + * Retrieves the connection configuration for the cloud source. + * + * @return The connection configuration for the cloud source. + */ + def connectionConfig: CloudConnectionConfig + + /** + * Retrieves the bucket options for the cloud source. + * + * @return The bucket options for the cloud source. + */ + def bucketOptions: Seq[CloudSourceBucketOptions[MD]] + + /** + * Retrieves the compression codec for the cloud source. + * + * @return The compression codec for the cloud source. + */ + def compressionCodec: CompressionCodec + + /** + * Retrieves the partition searcher options for the cloud source. + * + * @return The partition searcher options for the cloud source. + */ + def partitionSearcher: PartitionSearcherOptions +} diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/traits/CloudConnectionConfig.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/traits/CloudConnectionConfig.scala new file mode 100644 index 000000000..b6c49e562 --- /dev/null +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/traits/CloudConnectionConfig.scala @@ -0,0 +1,40 @@ +/* + * Copyright 2017-2024 Lenses.io Ltd + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.lenses.streamreactor.connect.cloud.common.config.traits + +import io.lenses.streamreactor.common.errors.ErrorPolicy +import io.lenses.streamreactor.connect.cloud.common.config.RetryConfig + +/** + * Trait representing configuration for a cloud connection. + * This trait defines methods for retrieving error policy and connector retry configuration. + */ +trait CloudConnectionConfig { + + /** + * Retrieves the error policy for the cloud connection. + * + * @return The error policy for the cloud connection. + */ + def errorPolicy: ErrorPolicy + + /** + * Retrieves the retry configuration for the cloud connection. + * + * @return The retry configuration for the cloud connection. + */ + def connectorRetryConfig: RetryConfig +} diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/traits/PropsToConfigConverter.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/traits/PropsToConfigConverter.scala new file mode 100644 index 000000000..b4dd9b6db --- /dev/null +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/config/traits/PropsToConfigConverter.scala @@ -0,0 +1,51 @@ +/* + * Copyright 2017-2024 Lenses.io Ltd + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.lenses.streamreactor.connect.cloud.common.config.traits + +import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId +import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocationValidator + +/** + * Trait for converting properties to a specific cloud configuration. + * Implementations of this trait are responsible for parsing properties + * and constructing a cloud configuration object of type C. + */ +trait PropsToConfigConverter[C <: CloudConfig] { + + /** + * Converts a map of properties to a cloud configuration object of type C. + * + * @param connectorTaskId The identifier for the connector task. + * This is used to associate the configuration with a specific task. + * @param props The properties to be converted. + * These properties contain the configuration settings. + * @param cloudLocationValidator An implicit validator for cloud locations. + * This validator is used to ensure that cloud location information + * specified in the properties is valid. + * @return Either a Throwable if an error occurs during conversion, + * or a cloud configuration object of type C. + * The left side of the Either contains an error if the conversion fails, + * otherwise, the right side contains the successfully converted configuration. + */ + def fromProps( + connectorTaskId: ConnectorTaskId, + props: Map[String, String], + )( + implicit + cloudLocationValidator: CloudLocationValidator, + ): Either[Throwable, C] + +} diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/sink/CloudSinkTask.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/sink/CloudSinkTask.scala index 4c9d25104..f0d13f1d2 100644 --- a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/sink/CloudSinkTask.scala +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/sink/CloudSinkTask.scala @@ -18,20 +18,22 @@ package io.lenses.streamreactor.connect.cloud.common.sink import cats.implicits.toBifunctorOps import cats.implicits.toShow import io.lenses.streamreactor.common.errors.ErrorHandler +import io.lenses.streamreactor.common.errors.RetryErrorPolicy import io.lenses.streamreactor.common.utils.AsciiArtPrinter.printAsciiHeader import io.lenses.streamreactor.common.utils.JarManifest import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskIdCreator +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudSinkConfig import io.lenses.streamreactor.connect.cloud.common.formats.writer.MessageDetail import io.lenses.streamreactor.connect.cloud.common.formats.writer.NullSinkData import io.lenses.streamreactor.connect.cloud.common.model.Offset import io.lenses.streamreactor.connect.cloud.common.model.Topic import io.lenses.streamreactor.connect.cloud.common.model.TopicPartition -import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocationValidator import io.lenses.streamreactor.connect.cloud.common.sink.conversion.HeaderToStringConverter import io.lenses.streamreactor.connect.cloud.common.sink.conversion.ValueToSinkDataConverter import io.lenses.streamreactor.connect.cloud.common.sink.writer.WriterManager import io.lenses.streamreactor.connect.cloud.common.storage.FileMetadata +import io.lenses.streamreactor.connect.cloud.common.storage.StorageInterface import io.lenses.streamreactor.connect.cloud.common.utils.MapUtils import io.lenses.streamreactor.connect.cloud.common.utils.TimestampUtils import org.apache.kafka.clients.consumer.OffsetAndMetadata @@ -45,18 +47,19 @@ import scala.jdk.CollectionConverters.MapHasAsJava import scala.jdk.CollectionConverters.MapHasAsScala import scala.util.Try -abstract class CloudSinkTask[SM <: FileMetadata]( +abstract class CloudSinkTask[MD <: FileMetadata, C <: CloudSinkConfig, CT]( connectorPrefix: String, sinkAsciiArtResource: String, manifest: JarManifest, -)( - implicit - val cloudLocationValidator: CloudLocationValidator, ) extends SinkTask with ErrorHandler { - private var writerManager: WriterManager[SM] = _ + + private val writerManagerCreator = new WriterManagerCreator[MD, C]() + + private var writerManager: WriterManager[MD] = _ implicit var connectorTaskId: ConnectorTaskId = _ - override def version(): String = manifest.version() + + override def version(): String = manifest.version() override def start(fallbackProps: util.Map[String, String]): Unit = { @@ -76,8 +79,6 @@ abstract class CloudSinkTask[SM <: FileMetadata]( errOrWriterMan.leftMap(throw _).foreach(writerManager = _) } - def createWriterMan(props: Map[String, String]): Either[Throwable, WriterManager[SM]] - private def rollback(topicPartitions: Set[TopicPartition]): Unit = topicPartitions.foreach(writerManager.cleanUp) @@ -244,4 +245,36 @@ abstract class CloudSinkTask[SM <: FileMetadata]( Option(writerManager).foreach(_.close()) writerManager = null } + + def createClient(config: C): Either[Throwable, CT] + + def createStorageInterface(connectorTaskId: ConnectorTaskId, config: C, cloudClient: CT): StorageInterface[MD] + + def convertPropsToConfig(connectorTaskId: ConnectorTaskId, props: Map[String, String]): Either[Throwable, C] + + def createWriterMan(props: Map[String, String]): Either[Throwable, WriterManager[MD]] = + for { + config <- convertPropsToConfig(connectorTaskId, props) + s3Client <- createClient(config) + storageInterface = createStorageInterface(connectorTaskId, config, s3Client) + _ <- setRetryInterval(config) + writerManager <- Try(writerManagerCreator.from(config)(connectorTaskId, storageInterface)).toEither + _ <- initializeFromConfig(config) + } yield writerManager + + private def initializeFromConfig(config: C): Either[Throwable, Unit] = + Try(initialize( + config.connectionConfig.connectorRetryConfig.numberOfRetries, + config.connectionConfig.errorPolicy, + )).toEither + + private def setRetryInterval(config: C): Either[Throwable, Unit] = + Try { + //if error policy is retry set retry interval + config.connectionConfig.errorPolicy match { + case RetryErrorPolicy() => context.timeout(config.connectionConfig.connectorRetryConfig.errorRetryInterval) + case _ => + } + }.toEither + } diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/sink/WriterManagerCreator.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/sink/WriterManagerCreator.scala index 99bcfd10c..12b20703f 100644 --- a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/sink/WriterManagerCreator.scala +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/sink/WriterManagerCreator.scala @@ -18,6 +18,7 @@ package io.lenses.streamreactor.connect.cloud.common.sink import cats.implicits.catsSyntaxEitherId import com.typesafe.scalalogging.LazyLogging import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudSinkConfig import io.lenses.streamreactor.connect.cloud.common.formats import io.lenses.streamreactor.connect.cloud.common.formats.writer.FormatWriter import io.lenses.streamreactor.connect.cloud.common.model.Offset @@ -26,7 +27,6 @@ import io.lenses.streamreactor.connect.cloud.common.model.TopicPartition import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocation import io.lenses.streamreactor.connect.cloud.common.sink.commit.CommitPolicy import io.lenses.streamreactor.connect.cloud.common.sink.config.CloudSinkBucketOptions -import io.lenses.streamreactor.connect.cloud.common.sink.config.CloudSinkConfig import io.lenses.streamreactor.connect.cloud.common.sink.config.PartitionField import io.lenses.streamreactor.connect.cloud.common.sink.naming.KeyNamer import io.lenses.streamreactor.connect.cloud.common.sink.seek.IndexManager diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/sink/config/CloudSinkBucketOptions.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/sink/config/CloudSinkBucketOptions.scala index b2d9e8504..cd317a2af 100644 --- a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/sink/config/CloudSinkBucketOptions.scala +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/sink/config/CloudSinkBucketOptions.scala @@ -39,10 +39,10 @@ import io.lenses.streamreactor.connect.cloud.common.sink.naming.TopicPartitionOf object CloudSinkBucketOptions extends LazyLogging { def apply( - config: CloudSinkConfigDefBuilder, + connectorTaskId: ConnectorTaskId, + config: CloudSinkConfigDefBuilder, )( implicit - connectorTaskId: ConnectorTaskId, cloudLocationValidator: CloudLocationValidator, ): Either[Throwable, Seq[CloudSinkBucketOptions]] = config.getKCQL.map { kcql: Kcql => @@ -66,7 +66,7 @@ object CloudSinkBucketOptions extends LazyLogging { ) } keyNamer = CloudKeyNamer(formatSelection, partitionSelection, fileNamer, paddingService) - stagingArea <- config.getLocalStagingArea() + stagingArea <- config.getLocalStagingArea()(connectorTaskId) target <- CloudLocation.splitAndValidate(kcql.getTarget) storageSettings <- DataStorageSettings.from(sinkProps) _ <- validateEnvelopeAndFormat(formatSelection, storageSettings) diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/CloudSourceTask.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/CloudSourceTask.scala new file mode 100644 index 000000000..888cf1092 --- /dev/null +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/CloudSourceTask.scala @@ -0,0 +1,163 @@ +/* + * Copyright 2017-2024 Lenses.io Ltd + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.lenses.streamreactor.connect.cloud.common.source + +import cats.effect.unsafe.implicits.global +import cats.effect.FiberIO +import cats.effect.IO +import cats.effect.Ref +import cats.implicits.catsSyntaxOptionId +import com.typesafe.scalalogging.LazyLogging +import io.lenses.streamreactor.common.config.base.traits.WithConnectorPrefix +import io.lenses.streamreactor.common.utils.AsciiArtPrinter.printAsciiHeader +import io.lenses.streamreactor.common.utils.JarManifest +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudSourceConfig +import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId +import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskIdCreator +import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocation +import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocationValidator +import io.lenses.streamreactor.connect.cloud.common.source.reader.PartitionDiscovery +import io.lenses.streamreactor.connect.cloud.common.source.reader.ReaderManager +import io.lenses.streamreactor.connect.cloud.common.source.reader.ReaderManagerState +import io.lenses.streamreactor.connect.cloud.common.source.state.CloudSourceTaskState +import io.lenses.streamreactor.connect.cloud.common.source.state.PartitionSearcher +import io.lenses.streamreactor.connect.cloud.common.source.state.ReaderManagerBuilder +import io.lenses.streamreactor.connect.cloud.common.storage.FileMetadata +import io.lenses.streamreactor.connect.cloud.common.storage.StorageInterface +import io.lenses.streamreactor.connect.cloud.common.utils.MapUtils +import org.apache.kafka.connect.source.SourceRecord +import org.apache.kafka.connect.source.SourceTask + +import java.util +import java.util.Collections +import scala.jdk.CollectionConverters.MapHasAsScala +import scala.jdk.CollectionConverters._ +abstract class CloudSourceTask[MD <: FileMetadata, C <: CloudSourceConfig[MD], CT] + extends SourceTask + with LazyLogging + with WithConnectorPrefix { + + def validator: CloudLocationValidator + + private val contextOffsetFn: CloudLocation => Option[CloudLocation] = + SourceContextReader.getCurrentOffset(() => context) + + private val manifest = JarManifest(getClass.getProtectionDomain.getCodeSource.getLocation) + + @volatile + private var s3SourceTaskState: Option[CloudSourceTaskState] = None + + @volatile + private var cancelledRef: Option[Ref[IO, Boolean]] = None + + private var partitionDiscoveryLoop: Option[FiberIO[Unit]] = None + + implicit var connectorTaskId: ConnectorTaskId = _ + + override def version(): String = manifest.version() + + /** + * Start sets up readers for every configured connection in the properties + */ + override def start(props: util.Map[String, String]): Unit = { + + printAsciiHeader(manifest, "/aws-s3-source-ascii.txt") + + logger.debug(s"Received call to S3SourceTask.start with ${props.size()} properties") + + val contextProperties: Map[String, String] = + Option(context).flatMap(c => Option(c.configs()).map(_.asScala.toMap)).getOrElse(Map.empty) + val mergedProperties: Map[String, String] = MapUtils.mergeProps(contextProperties, props.asScala.toMap) + (for { + result <- make(validator, connectorPrefix, mergedProperties, contextOffsetFn) + fiber <- result.partitionDiscoveryLoop.start + } yield { + s3SourceTaskState = result.some + cancelledRef = result.cancelledRef.some + partitionDiscoveryLoop = fiber.some + }).unsafeRunSync() + } + + override def stop(): Unit = { + logger.info(s"Stopping S3 source task") + (s3SourceTaskState, cancelledRef, partitionDiscoveryLoop) match { + case (Some(state), Some(signal), Some(fiber)) => stopInternal(state, signal, fiber) + case _ => logger.info("There is no state to stop.") + } + logger.info(s"Stopped S3 source task") + } + + override def poll(): util.List[SourceRecord] = + s3SourceTaskState.fold(Collections.emptyList[SourceRecord]()) { state => + state.poll().unsafeRunSync().asJava + } + + private def stopInternal(state: CloudSourceTaskState, signal: Ref[IO, Boolean], fiber: FiberIO[Unit]): Unit = { + (for { + _ <- signal.set(true) + _ <- state.close() + // Don't join the fiber if it's already been cancelled. It will take potentially the interval time to complete + // and this can create issues on Connect. The task will be terminated and the resource cleaned up by the GC. + //_ <- fiber.join.timeout(1.minute).attempt.void + } yield ()).unsafeRunSync() + cancelledRef = None + partitionDiscoveryLoop = None + s3SourceTaskState = None + } + + def createClient(config: C): Either[Throwable, CT] + + def make( + validator: CloudLocationValidator, + connectorPrefix: String, + props: Map[String, String], + contextOffsetFn: CloudLocation => Option[CloudLocation], + ): IO[CloudSourceTaskState] = + for { + connectorTaskId <- IO.fromEither(new ConnectorTaskIdCreator(connectorPrefix).fromProps(props)) + config <- IO.fromEither(convertPropsToConfig(connectorTaskId, props)) + s3Client <- IO.fromEither(createClient(config)) + storageInterface: StorageInterface[MD] <- IO.delay(createStorageInterface(connectorTaskId, config, s3Client)) + partitionSearcher <- IO.delay(createPartitionSearcher(connectorTaskId, config, s3Client)) + readerManagerState <- Ref[IO].of(ReaderManagerState(Seq.empty, Seq.empty)) + cancelledRef <- Ref[IO].of(false) + } yield { + val readerManagerCreateFn: (CloudLocation, String) => IO[ReaderManager] = (root, path) => { + ReaderManagerBuilder( + root, + path, + storageInterface, + connectorTaskId, + contextOffsetFn, + location => config.bucketOptions.find(sb => sb.sourceBucketAndPrefix == location), + )(validator) + } + val partitionDiscoveryLoop = PartitionDiscovery.run(connectorTaskId, + config.partitionSearcher, + partitionSearcher.find, + readerManagerCreateFn, + readerManagerState, + cancelledRef, + ) + CloudSourceTaskState(readerManagerState.get.map(_.readerManagers), cancelledRef, partitionDiscoveryLoop) + } + + def createStorageInterface(connectorTaskId: ConnectorTaskId, config: C, s3Client: CT): StorageInterface[MD] + + def convertPropsToConfig(connectorTaskId: ConnectorTaskId, props: Map[String, String]): Either[Throwable, C] + + def createPartitionSearcher(connectorTaskId: ConnectorTaskId, config: C, client: CT): PartitionSearcher +} diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/CloudSourceBucketOptions.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/CloudSourceBucketOptions.scala new file mode 100644 index 000000000..f547a31b6 --- /dev/null +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/CloudSourceBucketOptions.scala @@ -0,0 +1,90 @@ +/* + * Copyright 2017-2024 Lenses.io Ltd + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.lenses.streamreactor.connect.cloud.common.source.config + +import cats.implicits.toTraverseOps +import io.lenses.kcql.Kcql +import io.lenses.streamreactor.connect.cloud.common.config.FormatSelection +import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocation +import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocationValidator +import io.lenses.streamreactor.connect.cloud.common.source.config.kcqlprops.CloudSourceProps +import io.lenses.streamreactor.connect.cloud.common.source.config.kcqlprops.CloudSourcePropsSchema +import io.lenses.streamreactor.connect.cloud.common.storage.FileListError +import io.lenses.streamreactor.connect.cloud.common.storage.FileMetadata +import io.lenses.streamreactor.connect.cloud.common.storage.ListOfKeysResponse +import io.lenses.streamreactor.connect.cloud.common.storage.StorageInterface + +object CloudSourceBucketOptions { + private val DEFAULT_RECORDS_LIMIT = 10000 + private val DEFAULT_FILES_LIMIT = 1000 + + def apply[M <: FileMetadata]( + config: CloudSourceConfigDefBuilder, + partitionExtractor: Option[PartitionExtractor], + )( + implicit + cloudLocationValidator: CloudLocationValidator, + ): Either[Throwable, Seq[CloudSourceBucketOptions[M]]] = + config.getKCQL.map { + kcql: Kcql => + for { + source <- CloudLocation.splitAndValidate(kcql.getSource) + format <- FormatSelection.fromKcql(kcql, CloudSourcePropsSchema.schema) + sourceProps = CloudSourceProps.fromKcql(kcql) + + //extract the envelope. of not present default to false + hasEnvelope <- config.extractEnvelope(sourceProps) + + } yield CloudSourceBucketOptions[M]( + source, + kcql.getTarget, + format = format, + recordsLimit = if (kcql.getLimit < 1) DEFAULT_RECORDS_LIMIT else kcql.getLimit, + filesLimit = if (kcql.getBatchSize < 1) DEFAULT_FILES_LIMIT else kcql.getBatchSize, + partitionExtractor = partitionExtractor, + orderingType = config.extractOrderingType, + hasEnvelope = hasEnvelope.getOrElse(false), + ) + }.toSeq.traverse(identity) + +} + +case class CloudSourceBucketOptions[M <: FileMetadata]( + sourceBucketAndPrefix: CloudLocation, + targetTopic: String, + format: FormatSelection, + recordsLimit: Int, + filesLimit: Int, + partitionExtractor: Option[PartitionExtractor], + orderingType: OrderingType, + hasEnvelope: Boolean, +) { + def createBatchListerFn( + storageInterface: StorageInterface[M], + ): Option[M] => Either[FileListError, Option[ListOfKeysResponse[M]]] = + orderingType + .getBatchLister + .listBatch( + storageInterface = storageInterface, + bucket = sourceBucketAndPrefix.bucket, + prefix = sourceBucketAndPrefix.prefix, + numResults = filesLimit, + ) + + def getPartitionExtractorFn: String => Option[Int] = + partitionExtractor.fold((_: String) => Option.empty[Int])(_.extract) + +} diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/CloudSourceConfigDefBuilder.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/CloudSourceConfigDefBuilder.scala new file mode 100644 index 000000000..53a55731a --- /dev/null +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/CloudSourceConfigDefBuilder.scala @@ -0,0 +1,38 @@ +/* + * Copyright 2017-2024 Lenses.io Ltd + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.lenses.streamreactor.connect.cloud.common.source.config + +import io.lenses.streamreactor.common.config.base.traits.BaseConfig +import io.lenses.streamreactor.common.config.base.traits.ConnectionSettings +import io.lenses.streamreactor.common.config.base.traits.ErrorPolicySettings +import io.lenses.streamreactor.common.config.base.traits.KcqlSettings +import io.lenses.streamreactor.common.config.base.traits.NumberRetriesSettings +import io.lenses.streamreactor.common.config.base.traits.UserSettings +import io.lenses.streamreactor.connect.cloud.common.config.CompressionCodecSettings +import org.apache.kafka.common.config.ConfigDef + +abstract class CloudSourceConfigDefBuilder( + connectorPrefix: String, + configDef: ConfigDef, + props: Map[String, String], +) extends BaseConfig(connectorPrefix, configDef, props) + with CloudSourceSettings + with KcqlSettings + with ErrorPolicySettings + with NumberRetriesSettings + with UserSettings + with ConnectionSettings + with CompressionCodecSettings {} diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/CloudSourceSettings.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/CloudSourceSettings.scala new file mode 100644 index 000000000..556f35154 --- /dev/null +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/CloudSourceSettings.scala @@ -0,0 +1,57 @@ +/* + * Copyright 2017-2024 Lenses.io Ltd + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.lenses.streamreactor.connect.cloud.common.source.config + +import io.lenses.streamreactor.common.config.base.traits.BaseSettings +import io.lenses.streamreactor.connect.cloud.common.config.ConfigParse +import io.lenses.streamreactor.connect.cloud.common.config.ConfigParse.getLong +import io.lenses.streamreactor.connect.cloud.common.config.kcqlprops.PropsKeyEntry +import io.lenses.streamreactor.connect.cloud.common.config.kcqlprops.PropsKeyEnum +import io.lenses.streamreactor.connect.cloud.common.source.config.PartitionSearcherOptions.ExcludeIndexes +import io.lenses.streamreactor.connect.cloud.common.storage.FileMetadata +import io.lenses.streamreactor.connect.config.kcqlprops.KcqlProperties + +import scala.concurrent.duration.DurationLong +import scala.util.Try + +trait CloudSourceSettings extends BaseSettings with CloudSourceSettingsKeys { + + def extractOrderingType[M <: FileMetadata] = + Try(getString(SOURCE_ORDERING_TYPE)).toOption.flatMap( + OrderingType.withNameInsensitiveOption, + ).getOrElse(OrderingType.AlphaNumeric) + + def getPartitionExtractor(parsedValues: Map[String, _]): Option[PartitionExtractor] = PartitionExtractor( + ConfigParse.getString(parsedValues, SOURCE_PARTITION_EXTRACTOR_TYPE).getOrElse("none"), + ConfigParse.getString(parsedValues, SOURCE_PARTITION_EXTRACTOR_REGEX), + ) + + def extractEnvelope( + properties: KcqlProperties[PropsKeyEntry, PropsKeyEnum.type], + ): Either[Throwable, Option[Boolean]] = + properties.getOptionalBoolean(PropsKeyEnum.StoreEnvelope) + + def getPartitionSearcherOptions(props: Map[String, _]): PartitionSearcherOptions = + PartitionSearcherOptions( + recurseLevels = getInt(SOURCE_PARTITION_SEARCH_RECURSE_LEVELS), + continuous = getBoolean(SOURCE_PARTITION_SEARCH_MODE), + interval = getLong(props, SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS).getOrElse( + SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS_DEFAULT, + ).millis, + wildcardExcludes = ExcludeIndexes, + ) + +} diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/CloudSourceSettingsKeys.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/CloudSourceSettingsKeys.scala new file mode 100644 index 000000000..f39f4ba02 --- /dev/null +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/CloudSourceSettingsKeys.scala @@ -0,0 +1,120 @@ +/* + * Copyright 2017-2024 Lenses.io Ltd + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.lenses.streamreactor.connect.cloud.common.source.config +import io.lenses.streamreactor.common.config.base.traits.WithConnectorPrefix +import org.apache.kafka.common.config.ConfigDef +import org.apache.kafka.common.config.ConfigDef.Importance +import org.apache.kafka.common.config.ConfigDef.Type + +trait CloudSourceSettingsKeys extends WithConnectorPrefix { + val SOURCE_PARTITION_SEARCH_RECURSE_LEVELS: String = s"$connectorPrefix.partition.search.recurse.levels" + private val SOURCE_PARTITION_SEARCH_RECURSE_LEVELS_DOC: String = + "When searching for new partitions on the S3 filesystem, how many levels deep to recurse." + private val SOURCE_PARTITION_SEARCH_RECURSE_LEVELS_DEFAULT: Int = 0 + + val SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS: String = s"$connectorPrefix.partition.search.interval" + private val SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS_DOC: String = + "The interval in milliseconds between searching for new partitions. Defaults to 5 minutes." + val SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS_DEFAULT: Long = 300000L + + val SOURCE_PARTITION_SEARCH_MODE: String = s"$connectorPrefix.partition.search.continuous" + private val SOURCE_PARTITION_SEARCH_MODE_DOC: String = + "If set to true, it will be continuously search for new partitions. Otherwise it is a one-off operation. Defaults to true." + + def addSourceOrderingSettings(configDef: ConfigDef): ConfigDef = + configDef + .define( + SOURCE_ORDERING_TYPE, + Type.STRING, + SOURCE_ORDERING_TYPE_DEFAULT, + Importance.LOW, + SOURCE_ORDERING_TYPE_DOC, + "Source", + 6, + ConfigDef.Width.MEDIUM, + SOURCE_ORDERING_TYPE, + ) + + def addSourcePartitionSearcherSettings(configDef: ConfigDef): ConfigDef = + configDef.define( + SOURCE_PARTITION_SEARCH_RECURSE_LEVELS, + Type.INT, + SOURCE_PARTITION_SEARCH_RECURSE_LEVELS_DEFAULT, + Importance.LOW, + SOURCE_PARTITION_SEARCH_RECURSE_LEVELS_DOC, + "Source", + 3, + ConfigDef.Width.MEDIUM, + SOURCE_PARTITION_SEARCH_RECURSE_LEVELS, + ) + .define( + SOURCE_PARTITION_SEARCH_MODE, + Type.BOOLEAN, + true, + Importance.LOW, + SOURCE_PARTITION_SEARCH_MODE_DOC, + "Source", + 4, + ConfigDef.Width.MEDIUM, + SOURCE_PARTITION_SEARCH_MODE, + ) + .define( + SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS, + Type.LONG, + SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS_DEFAULT, + Importance.LOW, + SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS_DOC, + "Source", + 5, + ConfigDef.Width.MEDIUM, + SOURCE_PARTITION_SEARCH_INTERVAL_MILLIS, + ) + + val SOURCE_PARTITION_EXTRACTOR_TYPE = s"$connectorPrefix.source.partition.extractor.type" + val SOURCE_PARTITION_EXTRACTOR_TYPE_DOC = + "If you want to read to specific partitions when running the source. Options are 'hierarchical' (to match the sink's hierarchical file storage pattern) and 'regex' (supply a custom regex). Any other value will ignore original partitions and they should be evenly distributed through available partitions (Kafka dependent)." + + val SOURCE_PARTITION_EXTRACTOR_REGEX = s"$connectorPrefix.source.partition.extractor.regex" + val SOURCE_PARTITION_EXTRACTOR_REGEX_DOC = "If reading filename from regex, supply the regex here." + + val SOURCE_ORDERING_TYPE: String = s"$connectorPrefix.ordering.type" + val SOURCE_ORDERING_TYPE_DOC: String = "AlphaNumeric (the default)" + val SOURCE_ORDERING_TYPE_DEFAULT: String = "AlphaNumeric" + + def addSourcePartitionExtractorSettings(configDef: ConfigDef): ConfigDef = configDef.define( + SOURCE_PARTITION_EXTRACTOR_TYPE, + Type.STRING, + null, + Importance.LOW, + SOURCE_PARTITION_EXTRACTOR_TYPE_DOC, + "Source", + 1, + ConfigDef.Width.MEDIUM, + SOURCE_PARTITION_EXTRACTOR_TYPE, + ) + .define( + SOURCE_PARTITION_EXTRACTOR_REGEX, + Type.STRING, + null, + Importance.LOW, + SOURCE_PARTITION_EXTRACTOR_REGEX_DOC, + "Source", + 2, + ConfigDef.Width.MEDIUM, + SOURCE_PARTITION_EXTRACTOR_REGEX, + ) + +} diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/PartitionExtractor.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/PartitionExtractor.scala index c86ef9f0e..a3217a66a 100644 --- a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/PartitionExtractor.scala +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/PartitionExtractor.scala @@ -27,6 +27,7 @@ sealed trait PartitionExtractor { } object PartitionExtractor extends LazyLogging { + def apply(extractorType: String, extractorRegex: Option[String]): Option[PartitionExtractor] = extractorType.toLowerCase match { case "regex" => diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/S3SourceBucketOptions.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/S3SourceBucketOptions.scala new file mode 100644 index 000000000..861a5e2ce --- /dev/null +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/config/S3SourceBucketOptions.scala @@ -0,0 +1,15 @@ +/* + * Copyright 2017-2024 Lenses.io Ltd + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/state/CloudSourceTaskState.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/state/CloudSourceTaskState.scala index cf51d76ce..9d1f1ffc5 100644 --- a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/state/CloudSourceTaskState.scala +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/state/CloudSourceTaskState.scala @@ -16,12 +16,15 @@ package io.lenses.streamreactor.connect.cloud.common.source.state import cats.effect.IO -import cats.implicits._ +import cats.effect.kernel.Ref +import cats.implicits.toTraverseOps import io.lenses.streamreactor.connect.cloud.common.source.reader.ReaderManager import org.apache.kafka.connect.source.SourceRecord -class CloudSourceTaskState( - latestReaderManagers: IO[Seq[ReaderManager]], +case class CloudSourceTaskState( + latestReaderManagers: IO[Seq[ReaderManager]], + cancelledRef: Ref[IO, Boolean], + partitionDiscoveryLoop: IO[Unit], ) { def close(): IO[Unit] = diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/state/PartitionSearcher.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/state/PartitionSearcher.scala new file mode 100644 index 000000000..292346b4f --- /dev/null +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/state/PartitionSearcher.scala @@ -0,0 +1,36 @@ +/* + * Copyright 2017-2024 Lenses.io Ltd + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.lenses.streamreactor.connect.cloud.common.source.state + +import cats.effect.IO +import io.lenses.streamreactor.connect.cloud.common.source.distribution.PartitionSearcherResponse + +/** + * Trait defining a partition searcher. + * Implementations of this trait are responsible for finding partitions based on previous search results. + */ +trait PartitionSearcher { + + /** + * Finds partitions based on the provided last found partition responses. + * + * @param lastFound The previously found partition responses. + * @return An IO monad containing a sequence of new partition responses. + */ + def find( + lastFound: Seq[PartitionSearcherResponse], + ): IO[Seq[PartitionSearcherResponse]] +} diff --git a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/state/ReaderManagerBuilder.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/state/ReaderManagerBuilder.scala similarity index 82% rename from kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/state/ReaderManagerBuilder.scala rename to kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/state/ReaderManagerBuilder.scala index b2b085abb..b31a4c3ee 100644 --- a/kafka-connect-aws-s3/src/main/scala/io/lenses/streamreactor/connect/aws/s3/source/state/ReaderManagerBuilder.scala +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/source/state/ReaderManagerBuilder.scala @@ -13,18 +13,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.lenses.streamreactor.connect.aws.s3.source.state +package io.lenses.streamreactor.connect.cloud.common.source.state import cats.effect.IO import cats.effect.Ref -import io.lenses.streamreactor.connect.aws.s3.source.config.SourceBucketOptions -import io.lenses.streamreactor.connect.aws.s3.storage.S3FileMetadata import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocation import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocationValidator +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceBucketOptions import io.lenses.streamreactor.connect.cloud.common.source.files.CloudSourceFileQueue import io.lenses.streamreactor.connect.cloud.common.source.reader.ReaderManager import io.lenses.streamreactor.connect.cloud.common.source.reader.ResultReader +import io.lenses.streamreactor.connect.cloud.common.storage.FileMetadata import io.lenses.streamreactor.connect.cloud.common.storage.StorageInterface import org.apache.kafka.connect.errors.ConnectException @@ -32,13 +32,13 @@ import org.apache.kafka.connect.errors.ConnectException * Responsible for creating an instance of {{{ReaderManager}}} for a given path. */ object ReaderManagerBuilder { - def apply( + def apply[M <: FileMetadata]( root: CloudLocation, path: String, - storageInterface: StorageInterface[S3FileMetadata], + storageInterface: StorageInterface[M], connectorTaskId: ConnectorTaskId, contextOffsetFn: CloudLocation => Option[CloudLocation], - findSboF: CloudLocation => Option[SourceBucketOptions], + findSboF: CloudLocation => Option[CloudSourceBucketOptions[M]], )( implicit cloudLocationValidator: CloudLocationValidator, @@ -51,12 +51,12 @@ object ReaderManagerBuilder { ) ref <- Ref[IO].of(Option.empty[ResultReader]) adaptedRoot = root.copy(prefix = Some(path)) - adaptedSbo = sbo.copy(sourceBucketAndPrefix = adaptedRoot) + adaptedSbo = sbo.copy[M](sourceBucketAndPrefix = adaptedRoot) listingFn = adaptedSbo.createBatchListerFn(storageInterface) source = contextOffsetFn(adaptedRoot).fold { - new CloudSourceFileQueue[S3FileMetadata](connectorTaskId, listingFn) + new CloudSourceFileQueue[M](connectorTaskId, listingFn) } { location => - CloudSourceFileQueue.from[S3FileMetadata]( + CloudSourceFileQueue.from[M]( listingFn, storageInterface, location, diff --git a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/sink/config/CloudSinkConfig.scala b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/traits/PropsParser.scala similarity index 66% rename from kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/sink/config/CloudSinkConfig.scala rename to kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/traits/PropsParser.scala index ec8107058..52556d8df 100644 --- a/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/sink/config/CloudSinkConfig.scala +++ b/kafka-connect-cloud-common/src/main/scala/io/lenses/streamreactor/connect/cloud/common/traits/PropsParser.scala @@ -13,13 +13,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.lenses.streamreactor.connect.cloud.common.sink.config +package io.lenses.streamreactor.connect.cloud.common.traits -import io.lenses.streamreactor.connect.cloud.common.model.CompressionCodec +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudConnectionConfig -trait CloudSinkConfig { +trait PropsParser { + + def parse[CConfig <: CloudConnectionConfig](props: Map[String, String]): Either[Throwable, CConfig] - def bucketOptions: Seq[CloudSinkBucketOptions] - def offsetSeekerOptions: OffsetSeekerOptions - def compressionCodec: CompressionCodec } diff --git a/kafka-connect-cloud-common/src/test/scala/io/lenses/streamreactor/connect/cloud/common/sink/WriterManagerCreatorTest.scala b/kafka-connect-cloud-common/src/test/scala/io/lenses/streamreactor/connect/cloud/common/sink/WriterManagerCreatorTest.scala index 814741d99..38d92077b 100644 --- a/kafka-connect-cloud-common/src/test/scala/io/lenses/streamreactor/connect/cloud/common/sink/WriterManagerCreatorTest.scala +++ b/kafka-connect-cloud-common/src/test/scala/io/lenses/streamreactor/connect/cloud/common/sink/WriterManagerCreatorTest.scala @@ -15,11 +15,15 @@ */ package io.lenses.streamreactor.connect.cloud.common.sink +import io.lenses.streamreactor.common.errors.ErrorPolicy +import io.lenses.streamreactor.common.errors.NoopErrorPolicy +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudConnectionConfig +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudSinkConfig import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId +import io.lenses.streamreactor.connect.cloud.common.config.RetryConfig import io.lenses.streamreactor.connect.cloud.common.model.CompressionCodec import io.lenses.streamreactor.connect.cloud.common.model.CompressionCodecName import io.lenses.streamreactor.connect.cloud.common.sink.config.CloudSinkBucketOptions -import io.lenses.streamreactor.connect.cloud.common.sink.config.CloudSinkConfig import io.lenses.streamreactor.connect.cloud.common.sink.config.OffsetSeekerOptions import io.lenses.streamreactor.connect.cloud.common.sink.writer.WriterManager import io.lenses.streamreactor.connect.cloud.common.storage.FileMetadata @@ -32,7 +36,13 @@ import java.time.Instant class WriterManagerCreatorTest extends AnyFunSuite with Matchers with MockitoSugar { + case class FakeConnectionConfig( + errorPolicy: ErrorPolicy, + connectorRetryConfig: RetryConfig, + ) extends CloudConnectionConfig + case class FakeCloudSinkConfig( + connectionConfig: FakeConnectionConfig, bucketOptions: Seq[CloudSinkBucketOptions], offsetSeekerOptions: OffsetSeekerOptions, compressionCodec: CompressionCodec, @@ -47,6 +57,7 @@ class WriterManagerCreatorTest extends AnyFunSuite with Matchers with MockitoSug test("create WriterManager from GCPStorageSinkConfig") { val config = FakeCloudSinkConfig( + connectionConfig = FakeConnectionConfig(NoopErrorPolicy(), RetryConfig(1, 1L)), bucketOptions = Seq.empty, offsetSeekerOptions = OffsetSeekerOptions(maxIndexFiles = 10), compressionCodec = CompressionCodecName.ZSTD.toCodec(), diff --git a/kafka-connect-gcp-storage/src/it/scala/io/lenses/streamreactor/connect/cloud/common/sink/CloudPlatformEmulatorSuite.scala b/kafka-connect-gcp-storage/src/it/scala/io/lenses/streamreactor/connect/cloud/common/sink/CloudPlatformEmulatorSuite.scala index cccc5cbaf..0ec9e4cc5 100644 --- a/kafka-connect-gcp-storage/src/it/scala/io/lenses/streamreactor/connect/cloud/common/sink/CloudPlatformEmulatorSuite.scala +++ b/kafka-connect-gcp-storage/src/it/scala/io/lenses/streamreactor/connect/cloud/common/sink/CloudPlatformEmulatorSuite.scala @@ -3,6 +3,7 @@ package io.lenses.streamreactor.connect.cloud.common.sink import cats.implicits.catsSyntaxEitherId import cats.implicits.catsSyntaxOptionId import cats.implicits.toBifunctorOps +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudSinkConfig import io.lenses.streamreactor.connect.cloud.common.storage.FileMetadata import io.lenses.streamreactor.connect.cloud.common.storage.StorageInterface import io.lenses.streamreactor.connect.cloud.common.utils.RemoteFileHelper @@ -13,8 +14,13 @@ import org.scalatest.flatspec.AnyFlatSpec import scala.util.Try -trait CloudPlatformEmulatorSuite[SM <: FileMetadata, SI <: StorageInterface[SM], T <: CloudSinkTask[SM], C] - extends AnyFlatSpec +trait CloudPlatformEmulatorSuite[ + SM <: FileMetadata, + SI <: StorageInterface[SM], + CSC <: CloudSinkConfig, + C, + T <: CloudSinkTask[SM, CSC, C], +] extends AnyFlatSpec with BeforeAndAfter with BeforeAndAfterAll with RemoteFileHelper[SI] { diff --git a/kafka-connect-gcp-storage/src/it/scala/io/lenses/streamreactor/connect/cloud/common/sink/CoreSinkTaskTestCases.scala b/kafka-connect-gcp-storage/src/it/scala/io/lenses/streamreactor/connect/cloud/common/sink/CoreSinkTaskTestCases.scala index 390a5a6cf..250a9f707 100644 --- a/kafka-connect-gcp-storage/src/it/scala/io/lenses/streamreactor/connect/cloud/common/sink/CoreSinkTaskTestCases.scala +++ b/kafka-connect-gcp-storage/src/it/scala/io/lenses/streamreactor/connect/cloud/common/sink/CoreSinkTaskTestCases.scala @@ -5,6 +5,7 @@ import io.lenses.streamreactor.common.config.base.const.TraitConfigConst.MAX_RET import io.lenses.streamreactor.common.config.base.const.TraitConfigConst.RETRY_INTERVAL_PROP_SUFFIX import com.opencsv.CSVReader import com.typesafe.scalalogging.LazyLogging +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudSinkConfig import io.lenses.streamreactor.connect.cloud.common.formats.AvroFormatReader import io.lenses.streamreactor.connect.cloud.common.formats.reader.ParquetFormatReader import io.lenses.streamreactor.connect.cloud.common.formats.writer.BytesFormatWriter @@ -43,9 +44,14 @@ import scala.jdk.CollectionConverters.SeqHasAsJava import scala.util.Failure import scala.util.Success import scala.util.Try -abstract class CoreSinkTaskTestCases[SM <: FileMetadata, SI <: StorageInterface[SM], ST <: CloudSinkTask[SM], C]( - unitUnderTest: String, -) extends CloudPlatformEmulatorSuite[SM, SI, ST, C] +abstract class CoreSinkTaskTestCases[ + SM <: FileMetadata, + SI <: StorageInterface[SM], + CSC <: CloudSinkConfig, + C, + T <: CloudSinkTask[SM, CSC, C], +](unitUnderTest: String, +) extends CloudPlatformEmulatorSuite[SM, SI, CSC, C, T] with Matchers with MockitoSugar with LazyLogging { @@ -2059,7 +2065,7 @@ abstract class CoreSinkTaskTestCases[SM <: FileMetadata, SI <: StorageInterface[ struct } - private def createHeaders[T](keyValuePair: (String, T)*): lang.Iterable[Header] = { + private def createHeaders[HX](keyValuePair: (String, HX)*): lang.Iterable[Header] = { val headers = new ConnectHeaders() keyValuePair.foreach { case (key: String, value) => headers.add(key, value, null) diff --git a/kafka-connect-gcp-storage/src/it/scala/io/lenses/streamreactor/connect/gcp/storage/sink/GCPStorageSinkTaskTest.scala b/kafka-connect-gcp-storage/src/it/scala/io/lenses/streamreactor/connect/gcp/storage/sink/GCPStorageSinkTaskTest.scala index 3431b5dd2..b7da4700b 100644 --- a/kafka-connect-gcp-storage/src/it/scala/io/lenses/streamreactor/connect/gcp/storage/sink/GCPStorageSinkTaskTest.scala +++ b/kafka-connect-gcp-storage/src/it/scala/io/lenses/streamreactor/connect/gcp/storage/sink/GCPStorageSinkTaskTest.scala @@ -18,12 +18,19 @@ package io.lenses.streamreactor.connect.gcp.storage.sink import com.google.cloud.storage.Storage import io.lenses.streamreactor.connect.cloud.common.sink.CoreSinkTaskTestCases +import io.lenses.streamreactor.connect.gcp.storage.sink.config.GCPStorageSinkConfig import io.lenses.streamreactor.connect.gcp.storage.storage.GCPStorageFileMetadata import io.lenses.streamreactor.connect.gcp.storage.storage.GCPStorageStorageInterface import io.lenses.streamreactor.connect.gcp.storage.utils.GCPProxyContainerTest class GCPStorageSinkTaskTest - extends CoreSinkTaskTestCases[GCPStorageFileMetadata, GCPStorageStorageInterface, GCPStorageSinkTask, Storage]( + extends CoreSinkTaskTestCases[ + GCPStorageFileMetadata, + GCPStorageStorageInterface, + GCPStorageSinkConfig, + Storage, + GCPStorageSinkTask, + ]( "GCPStorageSinkTask", ) with GCPProxyContainerTest {} diff --git a/kafka-connect-gcp-storage/src/it/scala/io/lenses/streamreactor/connect/gcp/storage/utils/GCPProxyContainerTest.scala b/kafka-connect-gcp-storage/src/it/scala/io/lenses/streamreactor/connect/gcp/storage/utils/GCPProxyContainerTest.scala index 6580710c0..80daa010d 100644 --- a/kafka-connect-gcp-storage/src/it/scala/io/lenses/streamreactor/connect/gcp/storage/utils/GCPProxyContainerTest.scala +++ b/kafka-connect-gcp-storage/src/it/scala/io/lenses/streamreactor/connect/gcp/storage/utils/GCPProxyContainerTest.scala @@ -11,9 +11,10 @@ import io.lenses.streamreactor.connect.gcp.storage.auth.GCPStorageClientCreator import io.lenses.streamreactor.connect.gcp.storage.config.GCPConfigSettings._ import io.lenses.streamreactor.connect.gcp.storage.config.AuthMode import io.lenses.streamreactor.connect.gcp.storage.config.AuthModeSettingsConfigKeys -import io.lenses.streamreactor.connect.gcp.storage.config.GCPConfig +import io.lenses.streamreactor.connect.gcp.storage.config.GCPConnectionConfig import io.lenses.streamreactor.connect.gcp.storage.config.UploadConfigKeys import io.lenses.streamreactor.connect.gcp.storage.sink.GCPStorageSinkTask +import io.lenses.streamreactor.connect.gcp.storage.sink.config.GCPStorageSinkConfig import io.lenses.streamreactor.connect.gcp.storage.storage.GCPStorageFileMetadata import io.lenses.streamreactor.connect.gcp.storage.storage.GCPStorageStorageInterface import io.lenses.streamreactor.connect.testcontainers.GCPStorageContainer @@ -25,7 +26,13 @@ import java.nio.file.Files import scala.util.Try trait GCPProxyContainerTest - extends CloudPlatformEmulatorSuite[GCPStorageFileMetadata, GCPStorageStorageInterface, GCPStorageSinkTask, Storage] + extends CloudPlatformEmulatorSuite[ + GCPStorageFileMetadata, + GCPStorageStorageInterface, + GCPStorageSinkConfig, + Storage, + GCPStorageSinkTask, + ] with TaskIndexKey with AuthModeSettingsConfigKeys with UploadConfigKeys @@ -42,7 +49,7 @@ trait GCPProxyContainerTest override def createClient(): Either[Throwable, Storage] = { - val gcpConfig: GCPConfig = GCPConfig( + val gcpConfig: GCPConnectionConfig = GCPConnectionConfig( projectId = Some("test"), quotaProjectId = Option.empty, authMode = AuthMode.None, diff --git a/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/auth/GCPStorageClientCreator.scala b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/auth/GCPStorageClientCreator.scala index 19dcdfad4..efbfad096 100644 --- a/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/auth/GCPStorageClientCreator.scala +++ b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/auth/GCPStorageClientCreator.scala @@ -23,20 +23,20 @@ import com.google.cloud.storage.StorageOptions import com.google.cloud.NoCredentials import com.google.cloud.TransportOptions import io.lenses.streamreactor.connect.cloud.common.auth.ClientCreator +import io.lenses.streamreactor.connect.cloud.common.config.RetryConfig import io.lenses.streamreactor.connect.gcp.storage.config.AuthMode.None import io.lenses.streamreactor.connect.gcp.storage.config.AuthMode -import io.lenses.streamreactor.connect.gcp.storage.config.GCPConfig +import io.lenses.streamreactor.connect.gcp.storage.config.GCPConnectionConfig import io.lenses.streamreactor.connect.gcp.storage.config.HttpTimeoutConfig -import io.lenses.streamreactor.connect.gcp.storage.config.RetryConfig import org.threeten.bp.Duration import java.io.ByteArrayInputStream import java.io.FileInputStream import scala.util.Try -object GCPStorageClientCreator extends ClientCreator[GCPConfig, Storage] { +object GCPStorageClientCreator extends ClientCreator[GCPConnectionConfig, Storage] { - def make(config: GCPConfig): Either[Throwable, Storage] = + def make(config: GCPConnectionConfig): Either[Throwable, Storage] = Try { val builder = StorageOptions .newBuilder() diff --git a/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/config/GCPConfigSettings.scala b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/config/GCPConfigSettings.scala index b35459600..78d43d769 100644 --- a/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/config/GCPConfigSettings.scala +++ b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/config/GCPConfigSettings.scala @@ -72,4 +72,8 @@ object GCPConfigSettings { s"Maximum index files to allow per topic/partition. Advisable to not raise this: if a large number of files build up this means there is a problem with file deletion." val SEEK_MAX_INDEX_FILES_DEFAULT = 5 + val SOURCE_ORDERING_TYPE: String = s"$CONNECTOR_PREFIX.ordering.type" + val SOURCE_ORDERING_TYPE_DOC: String = "AlphaNumeric (the default)" + val SOURCE_ORDERING_TYPE_DEFAULT: String = "AlphaNumeric" + } diff --git a/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/config/GCPConfig.scala b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/config/GCPConnectionConfig.scala similarity index 90% rename from kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/config/GCPConfig.scala rename to kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/config/GCPConnectionConfig.scala index 207ead8ed..75a4024d0 100644 --- a/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/config/GCPConfig.scala +++ b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/config/GCPConnectionConfig.scala @@ -18,7 +18,6 @@ package io.lenses.streamreactor.connect.gcp.storage.config import io.lenses.streamreactor.common.errors.ErrorPolicy import io.lenses.streamreactor.common.errors.ErrorPolicyEnum import io.lenses.streamreactor.common.errors.ThrowErrorPolicy -import io.lenses.streamreactor.connect.cloud.common.config.CloudConfig import io.lenses.streamreactor.connect.cloud.common.config.ConfigParse._ import io.lenses.streamreactor.connect.cloud.common.config.ConfigParse.getString import GCPConfigSettings.ERROR_POLICY @@ -36,10 +35,12 @@ import GCPConfigSettings.HTTP_NBR_OF_RETRIES import GCPConfigSettings.HTTP_SOCKET_TIMEOUT import GCPConfigSettings.NBR_OF_RETIRES_DEFAULT import GCPConfigSettings.NBR_OF_RETRIES +import io.lenses.streamreactor.connect.cloud.common.config.RetryConfig +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudConnectionConfig -object GCPConfig { +object GCPConnectionConfig { - def apply(props: Map[String, _], authMode: AuthMode): GCPConfig = GCPConfig( + def apply(props: Map[String, _], authMode: AuthMode): GCPConnectionConfig = GCPConnectionConfig( getString(props, GCP_PROJECT_ID), getString(props, GCP_QUOTA_PROJECT_ID), authMode, @@ -65,11 +66,9 @@ object GCPConfig { ) } -case class RetryConfig(numberOfRetries: Int, errorRetryInterval: Long) - case class HttpTimeoutConfig(socketTimeout: Option[Long], connectionTimeout: Option[Long]) -case class GCPConfig( +case class GCPConnectionConfig( projectId: Option[String], quotaProjectId: Option[String], authMode: AuthMode, @@ -78,4 +77,4 @@ case class GCPConfig( connectorRetryConfig: RetryConfig = RetryConfig(NBR_OF_RETIRES_DEFAULT, ERROR_RETRY_INTERVAL_DEFAULT), httpRetryConfig: RetryConfig = RetryConfig(HTTP_NBR_OF_RETIRES_DEFAULT, HTTP_ERROR_RETRY_INTERVAL_DEFAULT), timeouts: HttpTimeoutConfig = HttpTimeoutConfig(None, None), -) extends CloudConfig +) extends CloudConnectionConfig diff --git a/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/sink/GCPStorageSinkTask.scala b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/sink/GCPStorageSinkTask.scala index 37f3cb83f..40c72cf47 100644 --- a/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/sink/GCPStorageSinkTask.scala +++ b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/sink/GCPStorageSinkTask.scala @@ -15,51 +15,39 @@ */ package io.lenses.streamreactor.connect.gcp.storage.sink -import io.lenses.streamreactor.common.errors.RetryErrorPolicy +import com.google.cloud.storage.Storage import io.lenses.streamreactor.common.utils.JarManifest +import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId import io.lenses.streamreactor.connect.cloud.common.sink.CloudSinkTask -import io.lenses.streamreactor.connect.cloud.common.sink.WriterManagerCreator -import io.lenses.streamreactor.connect.cloud.common.sink.writer.WriterManager +import io.lenses.streamreactor.connect.cloud.common.storage.StorageInterface import io.lenses.streamreactor.connect.gcp.storage.auth.GCPStorageClientCreator -import io.lenses.streamreactor.connect.gcp.storage.config.GCPConfig import io.lenses.streamreactor.connect.gcp.storage.config.GCPConfigSettings import io.lenses.streamreactor.connect.gcp.storage.model.location.GCPStorageLocationValidator import io.lenses.streamreactor.connect.gcp.storage.sink.config.GCPStorageSinkConfig import io.lenses.streamreactor.connect.gcp.storage.storage.GCPStorageFileMetadata import io.lenses.streamreactor.connect.gcp.storage.storage.GCPStorageStorageInterface -import scala.util.Try - object GCPStorageSinkTask {} class GCPStorageSinkTask - extends CloudSinkTask[GCPStorageFileMetadata]( + extends CloudSinkTask[GCPStorageFileMetadata, GCPStorageSinkConfig, Storage]( GCPConfigSettings.CONNECTOR_PREFIX, "/gcpstorage-sink-ascii.txt", JarManifest(GCPStorageSinkTask.getClass.getProtectionDomain.getCodeSource.getLocation), - )( - GCPStorageLocationValidator, ) { - private val writerManagerCreator = new WriterManagerCreator[GCPStorageFileMetadata, GCPStorageSinkConfig]() - - def createWriterMan(props: Map[String, String]): Either[Throwable, WriterManager[GCPStorageFileMetadata]] = - for { - config <- GCPStorageSinkConfig.fromProps(props) - gcpClient <- GCPStorageClientCreator.make(config.gcpConfig) - storageInterface = new GCPStorageStorageInterface(connectorTaskId, gcpClient, config.avoidResumableUpload) - _ <- Try(setErrorRetryInterval(config.gcpConfig)).toEither - writerManager <- Try(writerManagerCreator.from(config)(connectorTaskId, storageInterface)).toEither - _ <- Try(initialize( - config.gcpConfig.connectorRetryConfig.numberOfRetries, - config.gcpConfig.errorPolicy, - )).toEither - } yield writerManager + override def createClient(config: GCPStorageSinkConfig): Either[Throwable, Storage] = + GCPStorageClientCreator.make(config.connectionConfig) - private def setErrorRetryInterval(gcpConfig: GCPConfig): Unit = - //if error policy is retry set retry interval - gcpConfig.errorPolicy match { - case RetryErrorPolicy() => context.timeout(gcpConfig.connectorRetryConfig.errorRetryInterval) - case _ => - } + override def createStorageInterface( + connectorTaskId: ConnectorTaskId, + config: GCPStorageSinkConfig, + cloudClient: Storage, + ): StorageInterface[GCPStorageFileMetadata] = + new GCPStorageStorageInterface(connectorTaskId, cloudClient, avoidReumableUpload = config.avoidResumableUpload) + override def convertPropsToConfig( + connectorTaskId: ConnectorTaskId, + props: Map[String, String], + ): Either[Throwable, GCPStorageSinkConfig] = + GCPStorageSinkConfig.fromProps(connectorTaskId, props)(GCPStorageLocationValidator) } diff --git a/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/sink/config/GCPStorageSinkConfig.scala b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/sink/config/GCPStorageSinkConfig.scala index db25da91e..1dc2ecad4 100644 --- a/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/sink/config/GCPStorageSinkConfig.scala +++ b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/sink/config/GCPStorageSinkConfig.scala @@ -16,40 +16,41 @@ package io.lenses.streamreactor.connect.gcp.storage.sink.config import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudSinkConfig +import io.lenses.streamreactor.connect.cloud.common.config.traits.PropsToConfigConverter import io.lenses.streamreactor.connect.cloud.common.model.CompressionCodec import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocationValidator import io.lenses.streamreactor.connect.cloud.common.sink.config.CloudSinkBucketOptions -import io.lenses.streamreactor.connect.cloud.common.sink.config.CloudSinkConfig import io.lenses.streamreactor.connect.cloud.common.sink.config.OffsetSeekerOptions -import io.lenses.streamreactor.connect.gcp.storage.config.GCPConfig +import io.lenses.streamreactor.connect.gcp.storage.config.GCPConnectionConfig import io.lenses.streamreactor.connect.gcp.storage.config.GCPConfigSettings.SEEK_MAX_INDEX_FILES -object GCPStorageSinkConfig { +object GCPStorageSinkConfig extends PropsToConfigConverter[GCPStorageSinkConfig] { def fromProps( - props: Map[String, String], + connectorTaskId: ConnectorTaskId, + props: Map[String, String], )( implicit - connectorTaskId: ConnectorTaskId, cloudLocationValidator: CloudLocationValidator, ): Either[Throwable, GCPStorageSinkConfig] = - GCPStorageSinkConfig(GCPStorageSinkConfigDefBuilder(props)) + GCPStorageSinkConfig(connectorTaskId, GCPStorageSinkConfigDefBuilder(props)) def apply( + connectorTaskId: ConnectorTaskId, gcpConfigDefBuilder: GCPStorageSinkConfigDefBuilder, )( implicit - connectorTaskId: ConnectorTaskId, cloudLocationValidator: CloudLocationValidator, ): Either[Throwable, GCPStorageSinkConfig] = for { authMode <- gcpConfigDefBuilder.getAuthMode - sinkBucketOptions <- CloudSinkBucketOptions(gcpConfigDefBuilder) + sinkBucketOptions <- CloudSinkBucketOptions(connectorTaskId, gcpConfigDefBuilder) offsetSeekerOptions = OffsetSeekerOptions( gcpConfigDefBuilder.getInt(SEEK_MAX_INDEX_FILES), ) } yield GCPStorageSinkConfig( - GCPConfig(gcpConfigDefBuilder.getParsedValues, authMode), + GCPConnectionConfig(gcpConfigDefBuilder.getParsedValues, authMode), sinkBucketOptions, offsetSeekerOptions, gcpConfigDefBuilder.getCompressionCodec(), @@ -59,7 +60,7 @@ object GCPStorageSinkConfig { } case class GCPStorageSinkConfig( - gcpConfig: GCPConfig, + connectionConfig: GCPConnectionConfig, bucketOptions: Seq[CloudSinkBucketOptions] = Seq.empty, offsetSeekerOptions: OffsetSeekerOptions, compressionCodec: CompressionCodec, diff --git a/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/sink/config/GCPStorageSinkConfigDef.scala b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/sink/config/GCPStorageSinkConfigDef.scala index 8c208e63c..2070d3a71 100644 --- a/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/sink/config/GCPStorageSinkConfigDef.scala +++ b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/sink/config/GCPStorageSinkConfigDef.scala @@ -15,10 +15,7 @@ */ package io.lenses.streamreactor.connect.gcp.storage.sink.config -import cats.implicits.catsSyntaxEitherId -import com.typesafe.scalalogging.LazyLogging -import io.lenses.streamreactor.connect.cloud.common.config.processors.ConfigDefProcessor -import io.lenses.streamreactor.connect.cloud.common.config.processors.LowerCaseKeyConfigDefProcessor +import io.lenses.streamreactor.connect.cloud.common.config.CloudConfigDef import io.lenses.streamreactor.connect.cloud.common.sink.config.FlushConfigKeys import io.lenses.streamreactor.connect.cloud.common.sink.config.LocalStagingAreaConfigKeys import io.lenses.streamreactor.connect.cloud.common.sink.config.padding.PaddingStrategyConfigKeys @@ -31,9 +28,6 @@ import org.apache.kafka.common.config.ConfigDef import org.apache.kafka.common.config.ConfigDef.Importance import org.apache.kafka.common.config.ConfigDef.Type -import java.util -import scala.jdk.CollectionConverters._ - object GCPStorageSinkConfigDef extends CommonConfigDef with FlushConfigKeys @@ -70,37 +64,4 @@ object GCPStorageSinkConfigDef } -class GCPStorageSinkConfigDef() extends ConfigDef with LazyLogging { - - private val processorChain: List[ConfigDefProcessor] = - List(new LowerCaseKeyConfigDefProcessor(CONNECTOR_PREFIX)) - - override def parse(jProps: util.Map[_, _]): util.Map[String, AnyRef] = { - val scalaProps: Map[Any, Any] = jProps.asScala.toMap - processProperties(scalaProps) match { - case Left(exception) => throw exception - case Right(value) => super.parse(value.asJava) - } - } - - private def processProperties(scalaProps: Map[Any, Any]): Either[Throwable, Map[Any, Any]] = { - val stringProps = scalaProps.collect { case (k: String, v: AnyRef) => (k.toLowerCase, v) } - val nonStringProps = scalaProps -- stringProps.keySet - processStringKeyedProperties(stringProps) match { - case Left(exception) => exception.asLeft[Map[Any, Any]] - case Right(stringKeyedProps) => (nonStringProps ++ stringKeyedProps).asRight - } - } - - private def processStringKeyedProperties(stringProps: Map[String, Any]): Either[Throwable, Map[String, Any]] = { - var remappedProps: Map[String, Any] = stringProps - for (proc <- processorChain) { - proc.process(remappedProps) match { - case Left(exception) => return exception.asLeft[Map[String, AnyRef]] - case Right(properties) => remappedProps = properties - } - } - remappedProps.asRight - } - -} +class GCPStorageSinkConfigDef() extends CloudConfigDef(CONNECTOR_PREFIX) {} diff --git a/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/source/config/GCPStorageSourceConfig.scala b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/source/config/GCPStorageSourceConfig.scala new file mode 100644 index 000000000..3bee76121 --- /dev/null +++ b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/source/config/GCPStorageSourceConfig.scala @@ -0,0 +1,68 @@ +/* + * Copyright 2017-2024 Lenses.io Ltd + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.lenses.streamreactor.connect.gcp.storage.source.config + +import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId +import io.lenses.streamreactor.connect.cloud.common.config.traits.CloudSourceConfig +import io.lenses.streamreactor.connect.cloud.common.config.traits.PropsToConfigConverter +import io.lenses.streamreactor.connect.cloud.common.model.CompressionCodec +import io.lenses.streamreactor.connect.cloud.common.model.location.CloudLocationValidator +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceBucketOptions +import io.lenses.streamreactor.connect.cloud.common.source.config.PartitionSearcherOptions +import io.lenses.streamreactor.connect.gcp.storage.config.GCPConnectionConfig +import io.lenses.streamreactor.connect.gcp.storage.model.location.GCPStorageLocationValidator +import io.lenses.streamreactor.connect.gcp.storage.storage.GCPStorageFileMetadata + +import scala.util.Try + +object GCPStorageSourceConfig extends PropsToConfigConverter[GCPStorageSourceConfig] { + + implicit val CloudLocationValidator: CloudLocationValidator = GCPStorageLocationValidator + + override def fromProps( + connectorTaskId: ConnectorTaskId, + props: Map[String, String], + )( + implicit + cloudLocationValidator: CloudLocationValidator, + ): Either[Throwable, GCPStorageSourceConfig] = + Try(GCPStorageSourceConfig(GCPStorageSourceConfigDefBuilder(props))).toEither.flatten + + def apply(gcpConfigDefBuilder: GCPStorageSourceConfigDefBuilder): Either[Throwable, GCPStorageSourceConfig] = { + val parsedValues = gcpConfigDefBuilder.getParsedValues + for { + authMode <- gcpConfigDefBuilder.getAuthMode + sbo <- CloudSourceBucketOptions[GCPStorageFileMetadata]( + gcpConfigDefBuilder, + gcpConfigDefBuilder.getPartitionExtractor(parsedValues), + ) + } yield GCPStorageSourceConfig( + GCPConnectionConfig(parsedValues, authMode), + sbo, + gcpConfigDefBuilder.getCompressionCodec(), + gcpConfigDefBuilder.getPartitionSearcherOptions(parsedValues), + ) + + } + +} + +case class GCPStorageSourceConfig( + connectionConfig: GCPConnectionConfig, + bucketOptions: Seq[CloudSourceBucketOptions[GCPStorageFileMetadata]] = Seq.empty, + compressionCodec: CompressionCodec, + partitionSearcher: PartitionSearcherOptions, +) extends CloudSourceConfig[GCPStorageFileMetadata] diff --git a/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/source/config/GCPStorageSourceConfigDef.scala b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/source/config/GCPStorageSourceConfigDef.scala new file mode 100644 index 000000000..24a4f4697 --- /dev/null +++ b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/source/config/GCPStorageSourceConfigDef.scala @@ -0,0 +1,37 @@ +/* + * Copyright 2017-2024 Lenses.io Ltd + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.lenses.streamreactor.connect.gcp.storage.source.config + +import io.lenses.streamreactor.connect.cloud.common.config.CloudConfigDef +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceSettingsKeys +import io.lenses.streamreactor.connect.gcp.storage.config.CommonConfigDef +import io.lenses.streamreactor.connect.gcp.storage.config.GCPConfigSettings.CONNECTOR_PREFIX +import org.apache.kafka.common.config.ConfigDef + +object GCPStorageSourceConfigDef extends CommonConfigDef with CloudSourceSettingsKeys { + + override def connectorPrefix: String = CONNECTOR_PREFIX + + override val config: ConfigDef = { + val settings = super.config + + addSourceOrderingSettings(settings) + addSourcePartitionSearcherSettings(settings) + addSourcePartitionExtractorSettings(settings) + } +} + +class GCPStorageSourceConfigDef() extends CloudConfigDef(CONNECTOR_PREFIX) {} diff --git a/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/source/config/GCPStorageSourceConfigDefBuilder.scala b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/source/config/GCPStorageSourceConfigDefBuilder.scala new file mode 100644 index 000000000..905053ded --- /dev/null +++ b/kafka-connect-gcp-storage/src/main/scala/io/lenses/streamreactor/connect/gcp/storage/source/config/GCPStorageSourceConfigDefBuilder.scala @@ -0,0 +1,30 @@ +/* + * Copyright 2017-2024 Lenses.io Ltd + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.lenses.streamreactor.connect.gcp.storage.source.config + +import io.lenses.streamreactor.connect.cloud.common.source.config.CloudSourceConfigDefBuilder +import io.lenses.streamreactor.connect.gcp.storage.config.AuthModeSettings +import io.lenses.streamreactor.connect.gcp.storage.config.GCPConfigSettings + +import scala.jdk.CollectionConverters.MapHasAsScala + +case class GCPStorageSourceConfigDefBuilder(props: Map[String, String]) + extends CloudSourceConfigDefBuilder(GCPConfigSettings.CONNECTOR_PREFIX, GCPStorageSourceConfigDef.config, props) + with AuthModeSettings { + + def getParsedValues: Map[String, _] = values().asScala.toMap + +} diff --git a/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/auth/GCPStorageClientCreatorTest.scala b/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/auth/GCPStorageClientCreatorTest.scala index 485bdddb0..e6388ece1 100644 --- a/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/auth/GCPStorageClientCreatorTest.scala +++ b/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/auth/GCPStorageClientCreatorTest.scala @@ -18,10 +18,10 @@ package io.lenses.streamreactor.connect.gcp.storage.auth import cats.implicits.catsSyntaxOptionId import com.google.cloud.TransportOptions import com.google.cloud.http.HttpTransportOptions +import io.lenses.streamreactor.connect.cloud.common.config.RetryConfig import io.lenses.streamreactor.connect.gcp.storage.config.AuthMode -import io.lenses.streamreactor.connect.gcp.storage.config.GCPConfig +import io.lenses.streamreactor.connect.gcp.storage.config.GCPConnectionConfig import io.lenses.streamreactor.connect.gcp.storage.config.HttpTimeoutConfig -import io.lenses.streamreactor.connect.gcp.storage.config.RetryConfig import org.apache.commons.io.IOUtils import org.apache.kafka.common.config.types.Password import org.scalatest.EitherValues @@ -36,7 +36,7 @@ class GCPStorageClientCreatorTest extends AnyFunSuite with Matchers with EitherV private val jsonCredsUrl: URL = getClass.getResource("/test-gcp-credentials.json") - private val defaultConfig = GCPConfig( + private val defaultConfig = GCPConnectionConfig( host = Some("custom-host"), projectId = Some("project-id"), quotaProjectId = Some("quota-project-id"), diff --git a/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/config/GCPConfigTest.scala b/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/config/GCPConfigTest.scala index 765e84e24..8477c6fd9 100644 --- a/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/config/GCPConfigTest.scala +++ b/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/config/GCPConfigTest.scala @@ -34,6 +34,7 @@ import io.lenses.streamreactor.common.errors.NoopErrorPolicy import io.lenses.streamreactor.common.errors.RetryErrorPolicy import io.lenses.streamreactor.common.errors.ThrowErrorPolicy import com.typesafe.scalalogging.LazyLogging +import io.lenses.streamreactor.connect.cloud.common.config.RetryConfig import org.mockito.MockitoSugar import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers @@ -59,7 +60,7 @@ class GCPConfigTest extends AnyFlatSpec with Matchers with LazyLogging with Mock forAll(errorPolicyValuesMap) { (name, value, clazz) => logger.debug("Executing {}", name) - GCPConfig(Map("connect.gcpstorage.error.policy" -> value), authMode).errorPolicy should be(clazz) + GCPConnectionConfig(Map("connect.gcpstorage.error.policy" -> value), authMode).errorPolicy should be(clazz) } } @@ -75,11 +76,11 @@ class GCPConfigTest extends AnyFlatSpec with Matchers with LazyLogging with Mock forAll(retryValuesMap) { (name: String, ret: Any, interval: Any, result: RetryConfig) => logger.debug("Executing {}", name) - GCPConfig(Map( - "connect.gcpstorage.max.retries" -> ret, - "connect.gcpstorage.retry.interval" -> interval, - ), - authMode, + GCPConnectionConfig(Map( + "connect.gcpstorage.max.retries" -> ret, + "connect.gcpstorage.retry.interval" -> interval, + ), + authMode, ).connectorRetryConfig should be(result) } } @@ -88,11 +89,11 @@ class GCPConfigTest extends AnyFlatSpec with Matchers with LazyLogging with Mock forAll(retryValuesMap) { (name: String, ret: Any, interval: Any, result: RetryConfig) => logger.debug("Executing {}", name) - GCPConfig(Map( - "connect.gcpstorage.http.max.retries" -> ret, - "connect.gcpstorage.http.retry.interval" -> interval, - ), - authMode, + GCPConnectionConfig(Map( + "connect.gcpstorage.http.max.retries" -> ret, + "connect.gcpstorage.http.retry.interval" -> interval, + ), + authMode, ).httpRetryConfig should be(result) } } diff --git a/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/sink/config/GCPStorageGCPStorageSinkConfigDefBuilderTest.scala b/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/sink/config/GCPStorageGCPStorageSinkConfigDefBuilderTest.scala index 20740e23a..e23483f16 100644 --- a/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/sink/config/GCPStorageGCPStorageSinkConfigDefBuilderTest.scala +++ b/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/sink/config/GCPStorageGCPStorageSinkConfigDefBuilderTest.scala @@ -67,7 +67,7 @@ class GCPStorageGCPStorageSinkConfigDefBuilderTest "connect.gcpstorage.kcql" -> s"insert into mybucket:myprefix select * from $TopicName PARTITIONBY _key STOREAS CSV WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1", ) - CloudSinkBucketOptions(GCPStorageSinkConfigDefBuilder(props)) match { + CloudSinkBucketOptions(connectorTaskId, GCPStorageSinkConfigDefBuilder(props)) match { case Left(value) => fail(value.toString) case Right(value) => value.map(_.dataStorage) should be(List(DataStorageSettings.Default)) } @@ -78,7 +78,7 @@ class GCPStorageGCPStorageSinkConfigDefBuilderTest "connect.gcpstorage.kcql" -> s"insert into mybucket:myprefix select * from $TopicName PARTITIONBY _key STOREAS `JSON` WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true)", ) - CloudSinkBucketOptions(GCPStorageSinkConfigDefBuilder(props)) match { + CloudSinkBucketOptions(connectorTaskId, GCPStorageSinkConfigDefBuilder(props)) match { case Left(value) => fail(value.toString) case Right(value) => value.map(_.dataStorage) should be(List(DataStorageSettings.enabled)) } @@ -89,7 +89,7 @@ class GCPStorageGCPStorageSinkConfigDefBuilderTest "connect.gcpstorage.kcql" -> s"insert into mybucket:myprefix select * from $TopicName PARTITIONBY _key STOREAS `PARQUET` WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true, '${DataStorageSettings.StoreKeyKey}'=true, '${DataStorageSettings.StoreValueKey}'=true, '${DataStorageSettings.StoreMetadataKey}'=false, '${DataStorageSettings.StoreHeadersKey}'=false)", ) - CloudSinkBucketOptions(GCPStorageSinkConfigDefBuilder(props)) match { + CloudSinkBucketOptions(connectorTaskId, GCPStorageSinkConfigDefBuilder(props)) match { case Left(value) => fail(value.toString) case Right(value) => value.map(_.dataStorage) should be(List(DataStorageSettings(true, true, true, false, false))) @@ -118,7 +118,7 @@ class GCPStorageGCPStorageSinkConfigDefBuilderTest |""".stripMargin, ) - CloudSinkBucketOptions(GCPStorageSinkConfigDefBuilder(props)) match { + CloudSinkBucketOptions(connectorTaskId, GCPStorageSinkConfigDefBuilder(props)) match { case Left(value) => fail(value.toString) case Right(value) => value.map(_.dataStorage) should be( @@ -203,7 +203,7 @@ class GCPStorageGCPStorageSinkConfigDefBuilderTest "connect.gcpstorage.kcql" -> s"insert into $BucketName:$PrefixName select * from $TopicName STOREAS `JSON` WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true, '${DataStorageSettings.StoreKeyKey}'=true, '${DataStorageSettings.StoreValueKey}'=true, '${DataStorageSettings.StoreMetadataKey}'=false, '${DataStorageSettings.StoreHeadersKey}'=false)", ) - CloudSinkBucketOptions(GCPStorageSinkConfigDefBuilder(props)) match { + CloudSinkBucketOptions(connectorTaskId, GCPStorageSinkConfigDefBuilder(props)) match { case Left(value) => fail(value.toString) case Right(value) => value.map(_.dataStorage) should be(List(DataStorageSettings(envelope = true, @@ -220,7 +220,7 @@ class GCPStorageGCPStorageSinkConfigDefBuilderTest "connect.gcpstorage.kcql" -> s"insert into $BucketName:$PrefixName select * from $TopicName STOREAS `JSON` WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true, '${DataStorageSettings.StoreKeyKey}'=true, '${DataStorageSettings.StoreValueKey}'=true, '${DataStorageSettings.StoreMetadataKey}'=false, '${DataStorageSettings.StoreHeadersKey}'=false)", ) - CloudSinkBucketOptions(GCPStorageSinkConfigDefBuilder(props)) match { + CloudSinkBucketOptions(connectorTaskId, GCPStorageSinkConfigDefBuilder(props)) match { case Left(value) => fail(value.toString) case Right(value) => value.map(_.dataStorage) should be(List(DataStorageSettings(envelope = true, @@ -237,7 +237,9 @@ class GCPStorageGCPStorageSinkConfigDefBuilderTest "connect.gcpstorage.kcql" -> s"insert into $BucketName:$PrefixName select * from $TopicName STOREAS `BYTES_VALUEONLY` WITH_FLUSH_COUNT = 1", ) - CloudSinkBucketOptions(GCPStorageSinkConfigDefBuilder(props)).left.value.getMessage should startWith( + CloudSinkBucketOptions(connectorTaskId, + GCPStorageSinkConfigDefBuilder(props), + ).left.value.getMessage should startWith( "Unsupported format - BYTES_VALUEONLY. Please note", ) } @@ -247,7 +249,9 @@ class GCPStorageGCPStorageSinkConfigDefBuilderTest "connect.gcpstorage.kcql" -> s"insert into $BucketName:$PrefixName select * from $TopicName STOREAS `BYTES` WITH_FLUSH_COUNT = 3", ) - CloudSinkBucketOptions(GCPStorageSinkConfigDefBuilder(props)).left.value.getMessage should startWith( + CloudSinkBucketOptions(connectorTaskId, + GCPStorageSinkConfigDefBuilder(props), + ).left.value.getMessage should startWith( "FLUSH_COUNT > 1 is not allowed for BYTES", ) } diff --git a/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/sink/config/GCPStorageSinkConfigTest.scala b/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/sink/config/GCPStorageSinkConfigTest.scala index 2ca073343..8ad0fba25 100644 --- a/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/sink/config/GCPStorageSinkConfigTest.scala +++ b/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/sink/config/GCPStorageSinkConfigTest.scala @@ -31,7 +31,7 @@ class GCPStorageSinkConfigTest extends AnyFunSuite with Matchers { "connect.gcpstorage.kcql" -> s"insert into mybucket:myprefix select * from TopicName PARTITIONBY _key STOREAS `CSV` WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true)", ) - CloudSinkBucketOptions(GCPStorageSinkConfigDefBuilder(props)) match { + CloudSinkBucketOptions(connectorTaskId, GCPStorageSinkConfigDefBuilder(props)) match { case Left(value) => value.getMessage shouldBe "Envelope is not supported for format CSV." case Right(_) => fail("Should fail since envelope and CSV storage is not allowed") } @@ -42,7 +42,7 @@ class GCPStorageSinkConfigTest extends AnyFunSuite with Matchers { "connect.gcpstorage.kcql" -> s"insert into mybucket:myprefix select * from TopicName PARTITIONBY _key STOREAS `Parquet` WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true)", ) - CloudSinkBucketOptions(GCPStorageSinkConfigDefBuilder(props)) match { + CloudSinkBucketOptions(connectorTaskId, GCPStorageSinkConfigDefBuilder(props)) match { case Left(error) => fail("Should not fail since envelope and Parquet storage is allowed", error) case Right(_) => succeed } @@ -52,7 +52,7 @@ class GCPStorageSinkConfigTest extends AnyFunSuite with Matchers { "connect.gcpstorage.kcql" -> s"insert into mybucket:myprefix select * from TopicName PARTITIONBY _key STOREAS `Avro` WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true)", ) - CloudSinkBucketOptions(GCPStorageSinkConfigDefBuilder(props)) match { + CloudSinkBucketOptions(connectorTaskId, GCPStorageSinkConfigDefBuilder(props)) match { case Left(error) => fail("Should not fail since envelope and Avro storage is allowed", error) case Right(_) => succeed } @@ -62,7 +62,7 @@ class GCPStorageSinkConfigTest extends AnyFunSuite with Matchers { "connect.gcpstorage.kcql" -> s"insert into mybucket:myprefix select * from TopicName PARTITIONBY _key STOREAS `Json` WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true)", ) - CloudSinkBucketOptions(GCPStorageSinkConfigDefBuilder(props)) match { + CloudSinkBucketOptions(connectorTaskId, GCPStorageSinkConfigDefBuilder(props)) match { case Left(error) => fail("Should not fail since envelope and Json storage is allowed", error) case Right(_) => succeed } @@ -72,7 +72,7 @@ class GCPStorageSinkConfigTest extends AnyFunSuite with Matchers { "connect.gcpstorage.kcql" -> s"insert into mybucket:myprefix select * from TopicName PARTITIONBY _key STOREAS `Text` WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true)", ) - CloudSinkBucketOptions(GCPStorageSinkConfigDefBuilder(props)) match { + CloudSinkBucketOptions(connectorTaskId, GCPStorageSinkConfigDefBuilder(props)) match { case Left(value) => value.getMessage shouldBe "Envelope is not supported for format TEXT." case Right(_) => fail("Should fail since text and envelope storage is not allowed") } @@ -82,7 +82,7 @@ class GCPStorageSinkConfigTest extends AnyFunSuite with Matchers { "connect.gcpstorage.kcql" -> s"insert into mybucket:myprefix select * from TopicName PARTITIONBY _key STOREAS `Bytes` WITHPARTITIONER=Values WITH_FLUSH_COUNT = 1 PROPERTIES('${DataStorageSettings.StoreEnvelopeKey}'=true)", ) - CloudSinkBucketOptions(GCPStorageSinkConfigDefBuilder(props)) match { + CloudSinkBucketOptions(connectorTaskId, GCPStorageSinkConfigDefBuilder(props)) match { case Left(value) => value.getMessage shouldBe "Envelope is not supported for format BYTES." case Right(_) => fail("Should fail since envelope and bytes storage is not allowed") } diff --git a/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/source/config/GCPStorageSourceConfigTest.scala b/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/source/config/GCPStorageSourceConfigTest.scala new file mode 100644 index 000000000..376f568f4 --- /dev/null +++ b/kafka-connect-gcp-storage/src/test/scala/io/lenses/streamreactor/connect/gcp/storage/source/config/GCPStorageSourceConfigTest.scala @@ -0,0 +1,91 @@ +/* + * Copyright 2017-2024 Lenses.io Ltd + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.lenses.streamreactor.connect.gcp.storage.source.config + +import io.lenses.streamreactor.connect.cloud.common.config.ConnectorTaskId +import io.lenses.streamreactor.connect.gcp.storage.model.location.GCPStorageLocationValidator +import org.apache.kafka.common.config.ConfigException +import org.scalatest.EitherValues +import org.scalatest.funsuite.AnyFunSuite +import org.scalatest.matchers.should.Matchers._ + +class GCPStorageSourceConfigTest extends AnyFunSuite with EitherValues { + + val taskId = ConnectorTaskId("name", 1, 1) + implicit val validator = GCPStorageLocationValidator + test("fromProps should reject configuration when no kcql string is provided") { + val props = Map[String, String]() + val result = GCPStorageSourceConfig.fromProps(taskId, props) + + assertEitherException( + result, + classOf[ConfigException].getName, + "Missing required configuration \"connect.gcpstorage.kcql\" which has no default value.", + ) + } + + test("fromProps should reject configuration when kcql doesn't parse") { + val props = Map[String, String]( + "connect.gcpstorage.kcql" -> "flibble dibble dop", + ) + val result = GCPStorageSourceConfig.fromProps(taskId, props) + assertEitherException( + result, + classOf[IllegalArgumentException].getName, + "Invalid syntax.failed to parse at line 1 due to mismatched input 'flibble' expecting {INSERT, UPSERT, UPDATE, SELECT}", + ) + } + + test("fromProps should reject configuration when invalid bucket name is provided") { + val props = Map[String, String]( + "connect.gcpstorage.kcql" -> "select * from myBucket insert into myTopic", + ) + val result = GCPStorageSourceConfig.fromProps(taskId, props) + assertEitherException(result, + classOf[IllegalArgumentException].getName, + "Invalid bucket name (Rule: Bucket name should match regex", + ) + } + + test("fromProps should reject configuration when invalid auth mode is provided") { + val props = Map[String, String]( + "connect.gcpstorage.kcql" -> "select * from myBucket.azure insert into myTopic", + "connect.gcpstorage.gcp.auth.mode" -> "plain-and-unencrypted", + ) + val result = GCPStorageSourceConfig.fromProps(taskId, props) + assertEitherException(result, classOf[ConfigException].getName, "Unsupported auth mode `plain-and-unencrypted`") + } + + test("apply should return Right with GCPStorageSourceConfig when valid properties are provided") { + val props = Map[String, String]( + "connect.gcpstorage.kcql" -> "select * from myBucket.azure insert into myTopic", + "connect.gcpstorage.gcp.auth.mode" -> "credentials", + ) + val result = GCPStorageSourceConfig.fromProps(taskId, props) + result.isRight shouldBe true + } + + private def assertEitherException( + result: Either[Throwable, GCPStorageSourceConfig], + expectedExceptionClass: String, + expectedMessage: String, + ): Any = + result.left.value match { + case ex if expectedExceptionClass == ex.getClass.getName => + ex.getMessage should be(expectedMessage) + case ex => fail(s"Unexpected exception, was a ${ex.getClass.getName}") + } +}