From 249319ebe5f20f2ccef7d1dde75fe5a1337b4014 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Mon, 21 Sep 2020 20:23:26 -0700 Subject: [PATCH 001/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index d21416f1a..b8f0b8eb3 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.0.0-SNAPSHOT + 10.1.0-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 12b4de4c5..b2545cbe7 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.0.0-SNAPSHOT + 10.1.0-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.0.x + HEAD From afa8ba5a7428edb2761f4599086990beab52577a Mon Sep 17 00:00:00 2001 From: Daniel Date: Thu, 22 Oct 2020 11:06:15 -0700 Subject: [PATCH 002/169] CC-11945: Fix empty header check for storing headers (#365) --- .../KeyValueHeaderRecordWriterProvider.java | 22 +- .../connect/s3/TopicPartitionWriterTest.java | 228 +++++++++++++++++- 2 files changed, 239 insertions(+), 11 deletions(-) diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/KeyValueHeaderRecordWriterProvider.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/KeyValueHeaderRecordWriterProvider.java index 031fbedc0..814303d75 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/KeyValueHeaderRecordWriterProvider.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/KeyValueHeaderRecordWriterProvider.java @@ -86,20 +86,22 @@ public RecordWriter getRecordWriter(S3SinkConnectorConfig conf, String filename) public void write(SinkRecord sinkRecord) { valueWriter.write(sinkRecord); // null check happens in sink task // keyWriter != null means writing keys is turned on - if (keyWriter != null && sinkRecord.key() == null) { - throw new DataException( - String.format("Key cannot be null for SinkRecord: %s", sinkRecord) - ); - } else if (keyWriter != null) { + if (keyWriter != null) { + if (sinkRecord.key() == null) { + throw new DataException( + String.format("Key cannot be null for SinkRecord: %s", sinkRecord) + ); + } keyWriter.write(sinkRecord); } // headerWriter != null means writing headers is turned on - if (headerWriter != null && sinkRecord.headers() == null) { - throw new DataException( - String.format("Headers cannot be null for SinkRecord: %s", sinkRecord) - ); - } else if (headerWriter != null) { + if (headerWriter != null) { + if (sinkRecord.headers() == null || sinkRecord.headers().isEmpty()) { + throw new DataException( + String.format("Headers cannot be null for SinkRecord: %s", sinkRecord) + ); + } headerWriter.write(sinkRecord); } } diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java index 7b89c05da..192ecee31 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java @@ -19,12 +19,21 @@ import com.amazonaws.services.s3.model.S3ObjectSummary; import com.amazonaws.services.s3.model.Tag; import io.confluent.common.utils.SystemTime; +import io.confluent.connect.s3.format.KeyValueHeaderRecordWriterProvider; +import io.confluent.connect.s3.format.RecordViewSetter; +import io.confluent.connect.s3.format.RecordViews.HeaderRecordView; +import io.confluent.connect.s3.format.RecordViews.KeyRecordView; +import io.confluent.kafka.serializers.NonRecordContainer; +import org.apache.avro.util.Utf8; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.connector.ConnectRecord; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.errors.RetriableException; +import org.apache.kafka.connect.header.ConnectHeaders; +import org.apache.kafka.connect.header.Header; import org.apache.kafka.connect.sink.SinkRecord; import org.easymock.EasyMock; import org.joda.time.DateTime; @@ -44,7 +53,6 @@ import java.util.concurrent.atomic.AtomicInteger; import io.confluent.common.utils.MockTime; -import io.confluent.common.utils.SystemTime; import io.confluent.common.utils.Time; import io.confluent.connect.s3.format.avro.AvroFormat; import io.confluent.connect.s3.storage.S3OutputStream; @@ -67,11 +75,17 @@ import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; public class TopicPartitionWriterTest extends TestWithMockedS3 { // The default private static final String ZERO_PAD_FMT = "%010d"; + private enum RecordElement { + KEYS, + HEADERS, + VALUES + } private RecordWriterProvider writerProvider; private S3Storage storage; @@ -900,6 +914,108 @@ public void testAddingS3ObjectTags() throws Exception{ verifyTags(expectedTaggedFiles); } + @Test + public void testExceptionOnNullKeys() { + String recordValue = "1"; + int kafkaOffset = 1; + SinkRecord faultyRecord = new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, null, + Schema.STRING_SCHEMA, recordValue, kafkaOffset, 0L, TimestampType.NO_TIMESTAMP_TYPE, sampleHeaders()); + + Exception thrownException = assertThrows(DataException.class, () -> writeRecordWithKeysAndHeaders(faultyRecord)); + String expectedMessage = String.format("Key cannot be null for SinkRecord: %s", faultyRecord); + assertEquals(expectedMessage, thrownException.getMessage()); + } + + @Test + public void testExceptionOnEmptyHeaders() { + String recordValue = "1"; + int kafkaOffset = 1; + SinkRecord faultyRecord = new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, "key", + Schema.STRING_SCHEMA, recordValue, kafkaOffset, 0L, TimestampType.NO_TIMESTAMP_TYPE, Collections.emptyList()); + + Exception thrownException = assertThrows(DataException.class, () -> writeRecordWithKeysAndHeaders(faultyRecord)); + String expectedMessage = String.format("Headers cannot be null for SinkRecord: %s", faultyRecord); + assertEquals(expectedMessage, thrownException.getMessage()); + } + + @Test + public void testExceptionOnNullHeaders() { + String recordValue = "1"; + int kafkaOffset = 1; + SinkRecord faultyRecord = new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, "key", + Schema.STRING_SCHEMA, recordValue, kafkaOffset, 0L, TimestampType.NO_TIMESTAMP_TYPE, null); + + Exception thrownException = assertThrows(DataException.class, () -> writeRecordWithKeysAndHeaders(faultyRecord)); + String expectedMessage = String.format("Headers cannot be null for SinkRecord: %s", faultyRecord); + assertEquals(expectedMessage, thrownException.getMessage()); + } + + @Test + public void testRecordKeysAnsHeadersWritten() throws Exception { + writeRecordWithKeysAndHeaders(null); + } + + private void writeRecordWithKeysAndHeaders(SinkRecord faultyRecord) throws Exception { + setUp(); + // Define the partitioner + Partitioner partitioner = new DefaultPartitioner<>(); + partitioner.configure(parsedConfig); + + // setup key record provider for writing record key files. + RecordWriterProvider keyWriterProvider = + new AvroFormat(storage).getRecordWriterProvider(); + ((RecordViewSetter) keyWriterProvider).setRecordView(new KeyRecordView()); + // setup header record provider for writing record header files. + RecordWriterProvider headerWriterProvider = + new AvroFormat(storage).getRecordWriterProvider(); + ((RecordViewSetter) headerWriterProvider).setRecordView(new HeaderRecordView()); + // initialize the KVHWriterProvider with header and key writers turned on. + RecordWriterProvider writerProvider = new KeyValueHeaderRecordWriterProvider( + new AvroFormat(storage).getRecordWriterProvider(), + keyWriterProvider, + headerWriterProvider + ); + + TopicPartitionWriter topicPartitionWriter = new TopicPartitionWriter( + TOPIC_PARTITION, storage, writerProvider, partitioner, connectorConfig, context); + + Schema schema = createSchema(); + List records = createRecordBatches(schema, 3, 3); + List sinkRecords = createSinkRecordsWithHeaders(records, "key", schema); + + if (faultyRecord != null) { + topicPartitionWriter.buffer(faultyRecord); + } + + for (SinkRecord record : sinkRecords) { + topicPartitionWriter.buffer(record); + } + + // Test actual write + topicPartitionWriter.write(); + topicPartitionWriter.close(); + + String dirPrefix = partitioner.generatePartitionedPath(TOPIC, "partition=" + PARTITION); + + List expectedValueFiles = new ArrayList<>(); + expectedValueFiles.add(FileUtils.fileKeyToCommit(topicsDir, dirPrefix, TOPIC_PARTITION, 0, extension, ZERO_PAD_FMT)); + expectedValueFiles.add(FileUtils.fileKeyToCommit(topicsDir, dirPrefix, TOPIC_PARTITION, 3, extension, ZERO_PAD_FMT)); + expectedValueFiles.add(FileUtils.fileKeyToCommit(topicsDir, dirPrefix, TOPIC_PARTITION, 6, extension, ZERO_PAD_FMT)); + verifyRecordElement(expectedValueFiles, 3, sinkRecords, RecordElement.VALUES); + + List expectedHeaderFiles = new ArrayList<>(); + expectedHeaderFiles.add(FileUtils.fileKeyToCommit(topicsDir, dirPrefix, TOPIC_PARTITION, 0, ".headers.avro", ZERO_PAD_FMT)); + expectedHeaderFiles.add(FileUtils.fileKeyToCommit(topicsDir, dirPrefix, TOPIC_PARTITION, 3, ".headers.avro", ZERO_PAD_FMT)); + expectedHeaderFiles.add(FileUtils.fileKeyToCommit(topicsDir, dirPrefix, TOPIC_PARTITION, 6, ".headers.avro", ZERO_PAD_FMT)); + verifyRecordElement(expectedHeaderFiles, 3, sinkRecords, RecordElement.HEADERS); + + List expectedKeyFiles = new ArrayList<>(); + expectedKeyFiles.add(FileUtils.fileKeyToCommit(topicsDir, dirPrefix, TOPIC_PARTITION, 0, ".keys.avro", ZERO_PAD_FMT)); + expectedKeyFiles.add(FileUtils.fileKeyToCommit(topicsDir, dirPrefix, TOPIC_PARTITION, 3, ".keys.avro", ZERO_PAD_FMT)); + expectedKeyFiles.add(FileUtils.fileKeyToCommit(topicsDir, dirPrefix, TOPIC_PARTITION, 6, ".keys.avro", ZERO_PAD_FMT)); + verifyRecordElement(expectedKeyFiles, 3, sinkRecords, RecordElement.KEYS); + } + private Struct createRecord(Schema schema, int ibase, float fbase) { return new Struct(schema) .put("boolean", true) @@ -935,6 +1051,11 @@ private List createSinkRecords(List records, String key, Sch return createSinkRecords(records, key, schema, 0); } + // Given a list of records, create a list of sink records with contiguous offsets. + private List createSinkRecordsWithHeaders(List records, String key, Schema schema) { + return createSinkRecordsWithHeaders(records, key, schema, 0); + } + // Given a list of records, create a list of sink records with contiguous offsets. private List createSinkRecords(List records, String key, Schema schema, int startOffset) { ArrayList sinkRecords = new ArrayList<>(); @@ -945,6 +1066,23 @@ private List createSinkRecords(List records, String key, Sch return sinkRecords; } + // Given a list of records, create a list of sink records with contiguous offsets. + private List createSinkRecordsWithHeaders(List records, String key, Schema schema, int startOffset) { + ArrayList sinkRecords = new ArrayList<>(); + for (int i = 0; i < records.size(); ++i) { + sinkRecords.add(new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, records.get(i), + i + startOffset, 0L, TimestampType.NO_TIMESTAMP_TYPE, sampleHeaders())); + } + return sinkRecords; + } + + private Iterable
sampleHeaders() { + return new ConnectHeaders() + .addString("first-header-key", "first-header-value") + .addLong("second-header-key", 8L) + .addFloat("third-header-key", 6.5f); + } + // Given a list of records, create a list of sink records with contiguous offsets. private List createSinkRecordsWithTimestamp(List records, String key, Schema schema, int startOffset, long startTime, long timeStep) { @@ -980,6 +1118,94 @@ private void verify(List expectedFileKeys, int expectedSize, Schema sche } } + // based on verify() + private void verifyRecordElement(List expectedFileKeys, int expectedSize, List records, RecordElement fileType) + throws IOException { + + List summaries = listObjects(S3_TEST_BUCKET_NAME, null, s3); + List actualFiles; + switch (fileType) { + case KEYS: + actualFiles = getS3FileListKeys(summaries); + break; + case HEADERS: + actualFiles = getS3FileListHeaders(summaries); + break; + default: + actualFiles = getS3FileListValues(summaries); + break; + } + + Collections.sort(actualFiles); + Collections.sort(expectedFileKeys); + assertEquals(actualFiles, expectedFileKeys); + + int index = 0; + for (String fileKey : actualFiles) { + Collection actualRecords = readRecordsAvro(S3_TEST_BUCKET_NAME, fileKey, s3); + assertEquals(expectedSize, actualRecords.size()); + for (Object avroRecord : actualRecords) { + + SinkRecord currentRecord = records.get(index++); + Object expectedRecord; + if (fileKey.endsWith(".headers.avro")) { + Schema headerSchema = new HeaderRecordView().getViewSchema(currentRecord, false); + Object value = new HeaderRecordView().getView(currentRecord, false); + expectedRecord = ((NonRecordContainer) format.getAvroData().fromConnectData(headerSchema, value)).getValue(); + } else if (fileKey.endsWith(".keys.avro")) { + expectedRecord = ((NonRecordContainer) format.getAvroData().fromConnectData(currentRecord.keySchema(), currentRecord.key())).getValue(); + expectedRecord = new Utf8((String) expectedRecord); // fix assert conflicts due to java string and avro utf8 + } else { + expectedRecord = format.getAvroData().fromConnectData(currentRecord.valueSchema(), currentRecord.value()); + } + assertEquals(expectedRecord, avroRecord); + } + } + } + + // whether a filename contains any of the extensions + private boolean filenameContainsExtensions(String filename, List extensions) { + for (String extension : extensions){ + if (filename.contains(extension)) { + return true; + } + } + return false; + } + + // filter for values only. + private List getS3FileListValues(List summaries) { + List excludeExtensions = Arrays.asList(".headers.avro", ".keys.avro"); + List filteredFiles = new ArrayList<>(); + for (S3ObjectSummary summary : summaries) { + String fileKey = summary.getKey(); + if (!filenameContainsExtensions(fileKey, excludeExtensions)) { + filteredFiles.add(fileKey); + } + } + return filteredFiles; + } + + private List getS3FileListHeaders(List summaries) { + return getS3FileListFilter(summaries, ".headers.avro"); + } + + private List getS3FileListKeys(List summaries) { + return getS3FileListFilter(summaries, ".keys.avro"); + } + + // filter for keys or headers + private List getS3FileListFilter(List summaries, String extension) { + List filteredFiles = new ArrayList<>(); + for (S3ObjectSummary summary : summaries) { + String fileKey = summary.getKey(); + if (fileKey.contains(extension)) { + filteredFiles.add(fileKey); + } + } + return filteredFiles; + } + private void verifyTags(Map> expectedTaggedFiles) throws IOException { List summaries = listObjects(S3_TEST_BUCKET_NAME, null, s3); From 05a7bd7042d755d1f6e43298450945887ee65803 Mon Sep 17 00:00:00 2001 From: Yang Lei Date: Mon, 5 Apr 2021 13:41:15 -0400 Subject: [PATCH 003/169] CCMSG-493: enable IT (#398) * enable IT * enable IT * revise aws credentials * revise comment * revise jenkins * remove comments * add node type * fmt * revise env.WORKSPACE * revise how to retrieve aws credential * revise readme * revise secret * revise pom * revise credential file name * revise credential file name * revise doc * revise readme * revise readme * revise where IT plugins is * revise aws credential * fix typo * revise upstream * revise upstream project * move plugin definition for IT to parent * remove env variable from Jenkinsfile * revise file name in Jenkinsfile * revise upstreamProjects * revise upstreamProjects * revise upstreamProjects * revise upstreamProjects * revise pom * revise pom * revise pom * format * fmt * revise aws credential * revise from comment * revise from comment * revise from comment * add log Co-authored-by: Yang Lei --- Jenkinsfile | 9 +- README.md | 27 ++++++ kafka-connect-s3/pom.xml | 40 +++++++++ .../connect/s3/integration/README.md | 12 --- .../s3/integration/S3SinkConnectorIT.java | 88 ++++++++++--------- 5 files changed, 123 insertions(+), 53 deletions(-) delete mode 100644 kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/README.md diff --git a/Jenkinsfile b/Jenkinsfile index c391c96fa..ed77610ad 100755 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -1,8 +1,15 @@ #!/usr/bin/env groovy + +def getVaultSecretsList() { + return [["connect/s3sink_it", "creds", "/tmp/s3_sink_aws_credentials.json", "AWS_CREDENTIALS_PATH"]] +} + common { slackChannel = '#connect-warn' - upstreamProjects = 'confluentinc/kafka-connect-storage-common' + upstreamProjects = 'confluentinc/kafka-connect-storage-common-parent' + nodeLabel = 'docker-oraclejdk8' pintMerge = true twistlockCveScan = true + secret_file_list = getVaultSecretsList() downStreamValidate = false } diff --git a/README.md b/README.md index 2f1a89de8..2a31c6c70 100644 --- a/README.md +++ b/README.md @@ -20,6 +20,33 @@ for guidance on this process. You can build *kafka-connect-storage-cloud* with Maven using the standard lifecycle phases. +# Integration Tests + +## Test Setup + +The following resources need to be setup to run integration tests with a real S3 instance. + +### AWS Credentials + +The integration tests follow the `DefaultAWSCredentialsProviderChain` for S3 authentication. +One of the simplest ways to test locally is to configure the `.aws/credentials` file. +You can also define environment variable as described [here](https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html) + +Besides above, we support using `AWS_CREDENTIALS_PATH` to point a json format credential file. This is used for Jenkins. +e.g. the following script will create the json file using the known environment variables. + +``` +cat << EOF > s3_credentials.json +{ +"aws_access_key_id": "$AWS_ACCESS_KEY_ID", +"aws_secret_access_key": "$AWS_SECRET_ACCESS_KEY" +} +EOF +``` + +## Running the Tests +Tests can be run manually using the `mvn integration-test` command +or through IntelliJ by running the class. # Contribute diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 193c45212..4001d27c8 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -40,6 +40,9 @@ 0.2.2 1.0.1 1.15.0 + 2.22.1 + false + 2.22.1 @@ -234,6 +237,36 @@ v@{project.version} + + org.apache.maven.plugins + maven-surefire-plugin + + + org.apache.kafka.test.IntegrationTest + + + + maven-failsafe-plugin + ${maven.failsafe.plugin.version} + + + org.apache.maven.surefire + surefire-junit47 + ${surefire-junit47.version} + + + + ${skipIntegrationTests} + org.apache.kafka.test.IntegrationTest + + + + + integration-test + + + + @@ -392,5 +425,12 @@ + + jenkins + + false + + + diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/README.md b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/README.md deleted file mode 100644 index dd7a0ff8c..000000000 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/README.md +++ /dev/null @@ -1,12 +0,0 @@ -## Integration Test Setup - -The following resources need to be setup to run integration tests with a real S3 instance. - -### AWS Credentials -The integration tests follow the `DefaultAWSCredentialsProviderChain` for S3 authentication. -One of the simplest ways to ensure access to S3 for these tests is by configuring -the `.aws/credentials` file. - -## Running the Tests -Tests can be run manually using the `mvn integration-test` command -or through IntelliJ by running the class. \ No newline at end of file diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java index c89d0dab3..239a668bf 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java @@ -19,6 +19,8 @@ import static io.confluent.connect.s3.S3SinkConnectorConfig.S3_BUCKET_CONFIG; import static io.confluent.connect.s3.S3SinkConnectorConfig.STORE_KAFKA_HEADERS_CONFIG; import static io.confluent.connect.s3.S3SinkConnectorConfig.STORE_KAFKA_KEYS_CONFIG; +import static io.confluent.connect.s3.S3SinkConnectorConfig.AWS_ACCESS_KEY_ID_CONFIG; +import static io.confluent.connect.s3.S3SinkConnectorConfig.AWS_SECRET_ACCESS_KEY_CONFIG; import static io.confluent.connect.storage.StorageSinkConnectorConfig.FLUSH_SIZE_CONFIG; import static io.confluent.connect.storage.StorageSinkConnectorConfig.FORMAT_CLASS_CONFIG; import static io.confluent.connect.storage.common.StorageCommonConfig.STORE_URL_CONFIG; @@ -32,6 +34,7 @@ import com.amazonaws.auth.AWSStaticCredentialsProvider; import com.amazonaws.auth.AnonymousAWSCredentials; +import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.client.builder.AwsClientBuilder.EndpointConfiguration; import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.AmazonS3ClientBuilder; @@ -47,7 +50,6 @@ import io.confluent.connect.s3.format.json.JsonFormat; import io.confluent.connect.s3.format.parquet.ParquetFormat; import io.confluent.connect.s3.storage.S3Storage; -import io.findify.s3mock.S3Mock; import java.io.BufferedReader; import java.io.File; import java.io.FileReader; @@ -108,12 +110,9 @@ public class S3SinkConnectorIT extends BaseConnectorIT { private static final Logger log = LoggerFactory.getLogger(S3SinkConnectorIT.class); private static final ObjectMapper jsonMapper = new ObjectMapper(); - private static final String JENKINS_HOME = "JENKINS_HOME"; // AWS configs - private static final String AWS_CRED_PATH = System.getProperty("user.home") + "/.aws/credentials"; private static final String AWS_REGION = "us-west-2"; - private static final String MOCK_S3_URL = "http://localhost:8001"; - private static final int MOCK_S3_PORT = 8001; + private static final String AWS_CREDENTIALS_PATH = "AWS_CREDENTIALS_PATH"; // local dir configs private static final String TEST_RESOURCES_PATH = "src/test/resources/"; private static final String TEST_DOWNLOAD_PATH = TEST_RESOURCES_PATH + "downloaded-files/"; @@ -146,13 +145,9 @@ public class S3SinkConnectorIT extends BaseConnectorIT { // custom producer to enable sending records with headers private Producer producer; - protected static boolean useMockClient() { - File creds = new File(AWS_CRED_PATH); - return System.getenv(JENKINS_HOME) != null || !creds.exists(); - } - @BeforeClass public static void setupClient() { + log.info("Starting ITs..."); S3Client = getS3Client(); if (S3Client.doesBucketExistV2(TEST_BUCKET_NAME)) { clearBucket(TEST_BUCKET_NAME); @@ -164,6 +159,7 @@ public static void setupClient() { @AfterClass public static void deleteBucket() { S3Client.deleteBucket(TEST_BUCKET_NAME); + log.info("Finished ITs, removed S3 bucket"); } @Before @@ -177,9 +173,6 @@ public void before() { props.put(FORMAT_CLASS_CONFIG, AvroFormat.class.getName()); props.put(STORAGE_CLASS_CONFIG, S3Storage.class.getName()); props.put(S3_BUCKET_CONFIG, TEST_BUCKET_NAME); - if (useMockClient()) { - props.put(STORE_URL_CONFIG, MOCK_S3_URL); - } // create topics in Kafka KAFKA_TOPICS.forEach(topic -> connect.kafka().createTopic(topic, 1)); } @@ -406,37 +399,24 @@ private Struct getSampleStructVal(Schema structSchema) { } /** - * Get an S3 client based on existing credentials, or a mock client if running on jenkins. + * Get an S3 client based on existing credentials * * @return an authenticated S3 client */ private static AmazonS3 getS3Client() { - if (useMockClient()) { - S3Mock api = new S3Mock.Builder().withPort(MOCK_S3_PORT).withInMemoryBackend().build(); - api.start(); - /* - * AWS S3 client setup. - * withPathStyleAccessEnabled(true) trick is required to overcome S3 default - * DNS-based bucket access scheme - * resulting in attempts to connect to addresses like "bucketname.localhost" - * which requires specific DNS setup. - */ - EndpointConfiguration endpoint = new EndpointConfiguration(MOCK_S3_URL, AWS_REGION); - AmazonS3 mockClient = AmazonS3ClientBuilder - .standard() - .withPathStyleAccessEnabled(true) - .withEndpointConfiguration(endpoint) - .withCredentials(new AWSStaticCredentialsProvider(new AnonymousAWSCredentials())) - .build(); - - log.info("No credentials found, using mock S3 client."); - return mockClient; - } else { - log.info("Credentials found, using real S3 client."); - // DefaultAWSCredentialsProviderChain, - // assumes .aws/credentials is setup and test bucket exists - return AmazonS3ClientBuilder.standard().withRegion(AWS_REGION).build(); - } + Map creds = getAWSCredentialFromPath(); + // If AWS credentials found on AWS_CREDENTIALS_PATH, use them (Jenkins) + if (creds.size() == 2) { + BasicAWSCredentials awsCreds = new BasicAWSCredentials( + creds.get(AWS_ACCESS_KEY_ID_CONFIG), + creds.get(AWS_SECRET_ACCESS_KEY_CONFIG)); + return AmazonS3ClientBuilder.standard() + .withCredentials(new AWSStaticCredentialsProvider(awsCreds)) + .build(); + } + // DefaultAWSCredentialsProviderChain, + // For local testing, ~/.aws/credentials needs to be defined or other environment variables + return AmazonS3ClientBuilder.standard().withRegion(AWS_REGION).build(); } /** @@ -658,5 +638,33 @@ private void setupProperties() { // converters props.put(KEY_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); props.put(VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + // aws credential if exists + props.putAll(getAWSCredentialFromPath()); + } + + private static Map getAWSCredentialFromPath() { + Map map = new HashMap(); + if (!System.getenv().containsKey(AWS_CREDENTIALS_PATH)) { + return map; + } + String path = System.getenv().get(AWS_CREDENTIALS_PATH); + try { + Map creds = new ObjectMapper() + .readValue(new FileReader(path), Map.class); + String value = creds.get("aws_access_key_id"); + if (value != null && !value.isEmpty()) { + map.put(AWS_ACCESS_KEY_ID_CONFIG, value); + } + value = creds.get("aws_secret_access_key"); + if (value != null && !value.isEmpty()) { + map.put(AWS_SECRET_ACCESS_KEY_CONFIG,value); + } + } catch (Exception e) { + e.printStackTrace(); + throw new IllegalArgumentException( + "AWS credentials file not found." + AWS_CREDENTIALS_PATH + ); + } + return map; } } From 092aca9b271880fe21c16a1a6ef9c25813379504 Mon Sep 17 00:00:00 2001 From: Yang Lei Date: Mon, 5 Apr 2021 13:55:42 -0400 Subject: [PATCH 004/169] Revert "CCMSG-493: enable IT (#398)" (#401) This reverts commit 05a7bd7042d755d1f6e43298450945887ee65803. --- Jenkinsfile | 9 +- README.md | 27 ------ kafka-connect-s3/pom.xml | 40 --------- .../connect/s3/integration/README.md | 12 +++ .../s3/integration/S3SinkConnectorIT.java | 88 +++++++++---------- 5 files changed, 53 insertions(+), 123 deletions(-) create mode 100644 kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/README.md diff --git a/Jenkinsfile b/Jenkinsfile index ed77610ad..c391c96fa 100755 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -1,15 +1,8 @@ #!/usr/bin/env groovy - -def getVaultSecretsList() { - return [["connect/s3sink_it", "creds", "/tmp/s3_sink_aws_credentials.json", "AWS_CREDENTIALS_PATH"]] -} - common { slackChannel = '#connect-warn' - upstreamProjects = 'confluentinc/kafka-connect-storage-common-parent' - nodeLabel = 'docker-oraclejdk8' + upstreamProjects = 'confluentinc/kafka-connect-storage-common' pintMerge = true twistlockCveScan = true - secret_file_list = getVaultSecretsList() downStreamValidate = false } diff --git a/README.md b/README.md index 2a31c6c70..2f1a89de8 100644 --- a/README.md +++ b/README.md @@ -20,33 +20,6 @@ for guidance on this process. You can build *kafka-connect-storage-cloud* with Maven using the standard lifecycle phases. -# Integration Tests - -## Test Setup - -The following resources need to be setup to run integration tests with a real S3 instance. - -### AWS Credentials - -The integration tests follow the `DefaultAWSCredentialsProviderChain` for S3 authentication. -One of the simplest ways to test locally is to configure the `.aws/credentials` file. -You can also define environment variable as described [here](https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html) - -Besides above, we support using `AWS_CREDENTIALS_PATH` to point a json format credential file. This is used for Jenkins. -e.g. the following script will create the json file using the known environment variables. - -``` -cat << EOF > s3_credentials.json -{ -"aws_access_key_id": "$AWS_ACCESS_KEY_ID", -"aws_secret_access_key": "$AWS_SECRET_ACCESS_KEY" -} -EOF -``` - -## Running the Tests -Tests can be run manually using the `mvn integration-test` command -or through IntelliJ by running the class. # Contribute diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 4001d27c8..193c45212 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -40,9 +40,6 @@ 0.2.2 1.0.1 1.15.0 - 2.22.1 - false - 2.22.1 @@ -237,36 +234,6 @@ v@{project.version} - - org.apache.maven.plugins - maven-surefire-plugin - - - org.apache.kafka.test.IntegrationTest - - - - maven-failsafe-plugin - ${maven.failsafe.plugin.version} - - - org.apache.maven.surefire - surefire-junit47 - ${surefire-junit47.version} - - - - ${skipIntegrationTests} - org.apache.kafka.test.IntegrationTest - - - - - integration-test - - - - @@ -425,12 +392,5 @@ - - jenkins - - false - - - diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/README.md b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/README.md new file mode 100644 index 000000000..dd7a0ff8c --- /dev/null +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/README.md @@ -0,0 +1,12 @@ +## Integration Test Setup + +The following resources need to be setup to run integration tests with a real S3 instance. + +### AWS Credentials +The integration tests follow the `DefaultAWSCredentialsProviderChain` for S3 authentication. +One of the simplest ways to ensure access to S3 for these tests is by configuring +the `.aws/credentials` file. + +## Running the Tests +Tests can be run manually using the `mvn integration-test` command +or through IntelliJ by running the class. \ No newline at end of file diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java index 239a668bf..c89d0dab3 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java @@ -19,8 +19,6 @@ import static io.confluent.connect.s3.S3SinkConnectorConfig.S3_BUCKET_CONFIG; import static io.confluent.connect.s3.S3SinkConnectorConfig.STORE_KAFKA_HEADERS_CONFIG; import static io.confluent.connect.s3.S3SinkConnectorConfig.STORE_KAFKA_KEYS_CONFIG; -import static io.confluent.connect.s3.S3SinkConnectorConfig.AWS_ACCESS_KEY_ID_CONFIG; -import static io.confluent.connect.s3.S3SinkConnectorConfig.AWS_SECRET_ACCESS_KEY_CONFIG; import static io.confluent.connect.storage.StorageSinkConnectorConfig.FLUSH_SIZE_CONFIG; import static io.confluent.connect.storage.StorageSinkConnectorConfig.FORMAT_CLASS_CONFIG; import static io.confluent.connect.storage.common.StorageCommonConfig.STORE_URL_CONFIG; @@ -34,7 +32,6 @@ import com.amazonaws.auth.AWSStaticCredentialsProvider; import com.amazonaws.auth.AnonymousAWSCredentials; -import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.client.builder.AwsClientBuilder.EndpointConfiguration; import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.AmazonS3ClientBuilder; @@ -50,6 +47,7 @@ import io.confluent.connect.s3.format.json.JsonFormat; import io.confluent.connect.s3.format.parquet.ParquetFormat; import io.confluent.connect.s3.storage.S3Storage; +import io.findify.s3mock.S3Mock; import java.io.BufferedReader; import java.io.File; import java.io.FileReader; @@ -110,9 +108,12 @@ public class S3SinkConnectorIT extends BaseConnectorIT { private static final Logger log = LoggerFactory.getLogger(S3SinkConnectorIT.class); private static final ObjectMapper jsonMapper = new ObjectMapper(); + private static final String JENKINS_HOME = "JENKINS_HOME"; // AWS configs + private static final String AWS_CRED_PATH = System.getProperty("user.home") + "/.aws/credentials"; private static final String AWS_REGION = "us-west-2"; - private static final String AWS_CREDENTIALS_PATH = "AWS_CREDENTIALS_PATH"; + private static final String MOCK_S3_URL = "http://localhost:8001"; + private static final int MOCK_S3_PORT = 8001; // local dir configs private static final String TEST_RESOURCES_PATH = "src/test/resources/"; private static final String TEST_DOWNLOAD_PATH = TEST_RESOURCES_PATH + "downloaded-files/"; @@ -145,9 +146,13 @@ public class S3SinkConnectorIT extends BaseConnectorIT { // custom producer to enable sending records with headers private Producer producer; + protected static boolean useMockClient() { + File creds = new File(AWS_CRED_PATH); + return System.getenv(JENKINS_HOME) != null || !creds.exists(); + } + @BeforeClass public static void setupClient() { - log.info("Starting ITs..."); S3Client = getS3Client(); if (S3Client.doesBucketExistV2(TEST_BUCKET_NAME)) { clearBucket(TEST_BUCKET_NAME); @@ -159,7 +164,6 @@ public static void setupClient() { @AfterClass public static void deleteBucket() { S3Client.deleteBucket(TEST_BUCKET_NAME); - log.info("Finished ITs, removed S3 bucket"); } @Before @@ -173,6 +177,9 @@ public void before() { props.put(FORMAT_CLASS_CONFIG, AvroFormat.class.getName()); props.put(STORAGE_CLASS_CONFIG, S3Storage.class.getName()); props.put(S3_BUCKET_CONFIG, TEST_BUCKET_NAME); + if (useMockClient()) { + props.put(STORE_URL_CONFIG, MOCK_S3_URL); + } // create topics in Kafka KAFKA_TOPICS.forEach(topic -> connect.kafka().createTopic(topic, 1)); } @@ -399,24 +406,37 @@ private Struct getSampleStructVal(Schema structSchema) { } /** - * Get an S3 client based on existing credentials + * Get an S3 client based on existing credentials, or a mock client if running on jenkins. * * @return an authenticated S3 client */ private static AmazonS3 getS3Client() { - Map creds = getAWSCredentialFromPath(); - // If AWS credentials found on AWS_CREDENTIALS_PATH, use them (Jenkins) - if (creds.size() == 2) { - BasicAWSCredentials awsCreds = new BasicAWSCredentials( - creds.get(AWS_ACCESS_KEY_ID_CONFIG), - creds.get(AWS_SECRET_ACCESS_KEY_CONFIG)); - return AmazonS3ClientBuilder.standard() - .withCredentials(new AWSStaticCredentialsProvider(awsCreds)) - .build(); - } - // DefaultAWSCredentialsProviderChain, - // For local testing, ~/.aws/credentials needs to be defined or other environment variables - return AmazonS3ClientBuilder.standard().withRegion(AWS_REGION).build(); + if (useMockClient()) { + S3Mock api = new S3Mock.Builder().withPort(MOCK_S3_PORT).withInMemoryBackend().build(); + api.start(); + /* + * AWS S3 client setup. + * withPathStyleAccessEnabled(true) trick is required to overcome S3 default + * DNS-based bucket access scheme + * resulting in attempts to connect to addresses like "bucketname.localhost" + * which requires specific DNS setup. + */ + EndpointConfiguration endpoint = new EndpointConfiguration(MOCK_S3_URL, AWS_REGION); + AmazonS3 mockClient = AmazonS3ClientBuilder + .standard() + .withPathStyleAccessEnabled(true) + .withEndpointConfiguration(endpoint) + .withCredentials(new AWSStaticCredentialsProvider(new AnonymousAWSCredentials())) + .build(); + + log.info("No credentials found, using mock S3 client."); + return mockClient; + } else { + log.info("Credentials found, using real S3 client."); + // DefaultAWSCredentialsProviderChain, + // assumes .aws/credentials is setup and test bucket exists + return AmazonS3ClientBuilder.standard().withRegion(AWS_REGION).build(); + } } /** @@ -638,33 +658,5 @@ private void setupProperties() { // converters props.put(KEY_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); props.put(VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); - // aws credential if exists - props.putAll(getAWSCredentialFromPath()); - } - - private static Map getAWSCredentialFromPath() { - Map map = new HashMap(); - if (!System.getenv().containsKey(AWS_CREDENTIALS_PATH)) { - return map; - } - String path = System.getenv().get(AWS_CREDENTIALS_PATH); - try { - Map creds = new ObjectMapper() - .readValue(new FileReader(path), Map.class); - String value = creds.get("aws_access_key_id"); - if (value != null && !value.isEmpty()) { - map.put(AWS_ACCESS_KEY_ID_CONFIG, value); - } - value = creds.get("aws_secret_access_key"); - if (value != null && !value.isEmpty()) { - map.put(AWS_SECRET_ACCESS_KEY_CONFIG,value); - } - } catch (Exception e) { - e.printStackTrace(); - throw new IllegalArgumentException( - "AWS credentials file not found." + AWS_CREDENTIALS_PATH - ); - } - return map; } } From 96c8c98d61d7a4c767c9497cb5f3c5f256a9c890 Mon Sep 17 00:00:00 2001 From: kpatelatwork Date: Mon, 27 Sep 2021 12:45:55 -0500 Subject: [PATCH 005/169] =?UTF-8?q?use=20lastIndexOf=20for=20deriving=20ad?= =?UTF-8?q?justed=20filename=20and=20also=20added=20tests=20f=E2=80=A6=20(?= =?UTF-8?q?#453)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * use lastIndexOf for deriving adjusted filename and also added tests for case where extension is not part of filename * added one more corner case when filename is avro * added more corner cases where filename has avro in middle and filename is empty and filename is avro --- .../io/confluent/connect/s3/util/Utils.java | 2 +- .../confluent/connect/s3/util/UtilsTest.java | 29 +++++++++++++++---- 2 files changed, 25 insertions(+), 6 deletions(-) diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/Utils.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/Utils.java index 2c564503b..7d1e36816 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/Utils.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/Utils.java @@ -32,7 +32,7 @@ public class Utils { public static String getAdjustedFilename(RecordView recordView, String filename, String initialExtension) { if (filename.endsWith(initialExtension)) { - int index = filename.indexOf(initialExtension); + int index = filename.lastIndexOf(initialExtension); return filename.substring(0, index) + recordView.getExtension() + initialExtension; } else { // filename is already stripped diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/util/UtilsTest.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/util/UtilsTest.java index 06d94f5c6..3ad7af6ec 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/util/UtilsTest.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/util/UtilsTest.java @@ -22,7 +22,12 @@ public class UtilsTest { "header.avro", "sample-filename.avro", "sample.filename.avro", - "sample.file.name.avro" + "sample.file.name.avro", + "sample.file.avro.RawEvent.avro", + "x", + "avro", + ".avro", + "fooavrobar.txt" )); @Test @@ -36,7 +41,12 @@ public void getAdjustedFilenameForValues() { "header.avro", "sample-filename.avro", "sample.filename.avro", - "sample.file.name.avro" + "sample.file.name.avro", + "sample.file.avro.RawEvent.avro", + "x.avro", + "avro.avro", + ".avro", + "fooavrobar.txt.avro" )); for (int i = 0; i < expectedFilenames.size(); i++) { @@ -56,7 +66,12 @@ public void getAdjustedFilenameForKeys() { "header.keys.avro", "sample-filename.keys.avro", "sample.filename.keys.avro", - "sample.file.name.keys.avro" + "sample.file.name.keys.avro", + "sample.file.avro.RawEvent.keys.avro", + "x.keys.avro", + "avro.keys.avro", + ".keys.avro", + "fooavrobar.txt.keys.avro" )); for (int i = 0; i < expectedFilenames.size(); i++) { @@ -76,7 +91,12 @@ public void getAdjustedFilenameForHeaders() { "header.headers.avro", "sample-filename.headers.avro", "sample.filename.headers.avro", - "sample.file.name.headers.avro" + "sample.file.name.headers.avro", + "sample.file.avro.RawEvent.headers.avro", + "x.headers.avro", + "avro.headers.avro", + ".headers.avro", + "fooavrobar.txt.headers.avro" )); for (int i = 0; i < expectedFilenames.size(); i++) { @@ -106,5 +126,4 @@ public void testParquetEncodingExtensions() { assertEquals(expectedFilenames.get(i), adjustedFilename); } } - } From c8593f5952971b8e8fcfecaab91ac71d5fa06dc0 Mon Sep 17 00:00:00 2001 From: Chris Olivier Date: Wed, 29 Dec 2021 12:23:46 -0800 Subject: [PATCH 006/169] MINOR: Remove guava usage and superfluous template arguments. (#470) * Remove guava usage and some unused/unthrown code * cleanup Co-authored-by: Chris Olivier --- .../main/java/io/confluent/connect/s3/S3SinkTask.java | 9 +-------- .../connect/s3/integration/S3SinkConnectorIT.java | 2 +- 2 files changed, 2 insertions(+), 9 deletions(-) diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java index a831042b9..7e9532b60 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java @@ -16,7 +16,6 @@ package io.confluent.connect.s3; import com.amazonaws.AmazonClientException; -import com.google.common.annotations.VisibleForTesting; import io.confluent.connect.s3.S3SinkConnectorConfig.BehaviorOnNullValues; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; @@ -161,8 +160,7 @@ private Format newFormat(String formatClassConfig return formatClass.getConstructor(S3Storage.class).newInstance(storage); } - @VisibleForTesting - public RecordWriterProvider newRecordWriterProvider( + RecordWriterProvider newRecordWriterProvider( S3SinkConnectorConfig config) throws ClassNotFoundException, InvocationTargetException, InstantiationException, NoSuchMethodException, IllegalAccessException { @@ -327,9 +325,4 @@ private TopicPartitionWriter newTopicPartitionWriter(TopicPartition tp) { reporter ); } - - // Visible for testing - Format getFormat() { - return format; - } } diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java index 7c98be8a8..967c3a695 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java @@ -725,7 +725,7 @@ private void setupProperties() { } private static Map getAWSCredentialFromPath() { - Map map = new HashMap(); + Map map = new HashMap<>(); if (!System.getenv().containsKey(AWS_CREDENTIALS_PATH)) { return map; } From d212b64be1d5d72e5aee158c966fa490a04142c7 Mon Sep 17 00:00:00 2001 From: kpatelatwork Date: Mon, 10 Jan 2022 14:37:36 -0600 Subject: [PATCH 007/169] CCMSG-1582 fixed a build error in testcase happened due to merge of fixes from 5.5.3 https://github.com/confluentinc/kafka-connect-storage-cloud/pull/473 (#475) --- .../io/confluent/connect/s3/TopicPartitionWriterTest.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java index 0ff89c39b..db31e38cf 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java @@ -627,7 +627,8 @@ public void testWriteRecordsAfterScheduleRotationExpiryButNoResetShouldGoToSameF time.sleep(SYSTEM.milliseconds()); TopicPartitionWriter topicPartitionWriter = new TopicPartitionWriter( - TOPIC_PARTITION, storage, writerProvider, partitioner, connectorConfig, context, time); + TOPIC_PARTITION, storage, writerProvider, partitioner, connectorConfig, context, time, + null); // sleep for 11 minutes after startup time.sleep(TimeUnit.MINUTES.toMillis(11)); @@ -690,7 +691,8 @@ public void testWriteRecordsAfterCurrentScheduleRotationExpiryShouldGoToSameFile time.sleep(SYSTEM.milliseconds()); TopicPartitionWriter topicPartitionWriter = new TopicPartitionWriter( - TOPIC_PARTITION, storage, writerProvider, partitioner, connectorConfig, context, time); + TOPIC_PARTITION, storage, writerProvider, partitioner, connectorConfig, context, time, + null); // sleep for 11 minutes after startup time.sleep(TimeUnit.MINUTES.toMillis(11)); From c31709b06cf226af77b6349cdceecf02241a2414 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Wed, 19 Jan 2022 12:32:48 -0800 Subject: [PATCH 008/169] MINOR: Contribution guidelines for developers (#481) * MINOR: Contribution guidelines for developers * MINOR: title formatting * Apply suggestions from code review * Update CONTRIBUTING.md --- CONTRIBUTING.md | 181 ++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 181 insertions(+) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 7ed9f32cb..50721ea09 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -1,3 +1,5 @@ +# Contributing + In order for us to consider merging a contribution, you will need to sign our **C**ontributor **L**icense **A**greement. @@ -5,3 +7,182 @@ In order for us to consider merging a contribution, you will need to sign our > [Wikipedia](http://en.wikipedia.org/wiki/Contributor_License_Agreement) You can read and sign our full Contributor License Agreement [here](http://clabot.confluent.io/cla). + +## Reporting Bugs and Issues + +Report bugs and issues by creating a new GitHub issue. Prior to creating an issue, please search +through existing issues so that you are not creating duplicate ones. If a pull request exists that +corresponds to the issue, mention this pull request on the GitHub issue. + +## Guidelines for Contributing Code, Examples, Documentation + +Code changes are submitted via a pull request (PR). When submitting a PR use the following +guidelines: + +* Follow the style guide below +* Add/update documentation appropriately for the change you are making. +* Non-trivial changes should include unit tests covering the new functionality and potentially integration tests. +* Bug fixes should include unit tests and/or integration tests proving the issue is fixed. +* Try to keep pull requests short and submit separate ones for unrelated features. +* Keep formatting changes in separate commits to make code reviews easier and distinguish them from actual code changes. + +### Code Style +This connector is using a coding style that generally follows the [Google Java coding standard guide](https://google.github.io/styleguide/javaguide.html). + +Some conventions worth mentioning are: + +* Indentation (single tab) is 2 spaces. +* All import statements are listed explicitly. The wildcard (*) is not used in imports. +* Imports are groups as follows: +``` + import all packages not listed below (all other imports) + + import all javax.* packages + import all java.* packages + + import all io.confluent.* packages + + import static packages +``` +* Javadoc is highly recommended and often required during reviews in interfaces and public or protected classes and methods. + +### Titles and changelogs + +The title of a pull request is used as an entry on the release notes (aka changelogs) of the +connector in every release. + +For this reason, please use a brief but descriptive title for your pull request. If GitHub shortens +your pull request title when you issue the pull request adding the excessive part to the pull +request description, make sure that you correct the title before or after you issue the pull +request. + +If the fix is a minor fix you are encouraged to use the tag `MINOR:` followed by your pull request +title. You may link the corresponding issue to the description of your pull request but adding it to +the title will not be useful during changelog generation. + +When reverting a previous commit, use the prefix `Revert ` on the pull request title (automatically +added by GitHub when a pull request is created to revert an existing commit). + +### Tests +Every pull request should contain a sufficient amount of tests that assess your suggested code +changes. It’s highly recommended that you also check the code coverage of the production code you +are adding to make sure that your changes are covered sufficiently by the test code. + +### Description +Including a good description when you issue your pull requests helps significantly with reviews. +Feel free to follow the template that is when issuing a pull request and mention how your changes +are tested. + +### Backporting Commits +If your code changes are essentially bug fixes that make sense to backport to existing releases make sure to target the earliest release branch (e.g. 2.0.x) that should contain your changes. When selecting the release branch you should also consider how easy it will be to resolve any conflicts in newer release branches, including the `master` branch. + +## Github Workflow + +1. Fork the connector repository into your GitHub account: https://github.com/confluentinc/kafka-connect-storage-cloud/fork + +2. Clone your fork of the GitHub repository, replacing `` with your GitHub username. + + Use ssh (recommended): + + ```bash + git clone git@github.com:/kafka-connect-storage-cloud.git + ``` + + Or https: + + ```bash + git clone https://github.com//kafka-connect-storage-cloud.git + ``` + +3. Add a remote to keep up with upstream changes. + + ```bash + git remote add upstream https://github.com/confluentinc/kafka-connect-storage-cloud.git + ``` + + If you already have a copy, fetch upstream changes. + + ```bash + git fetch upstream + ``` + + or + + ```bash + git remote update + ``` + +4. Create a feature branch to work in. + + ```bash + git checkout -b feature-xyz upstream/master + ``` + +5. Work in your feature branch. + + ```bash + git commit -a --verbose + ``` + +6. Periodically rebase your changes + + ```bash + git pull --rebase + ``` + +7. When done, combine ("squash") related commits into a single one + + ```bash + git rebase -i upstream/master + ``` + + This will open your editor and allow you to re-order commits and merge them: + - Re-order the lines to change commit order (to the extent possible without creating conflicts) + - Prefix commits using `s` (squash) or `f` (fixup) to merge extraneous commits. + +8. Submit a pull-request + + ```bash + git push origin feature-xyz + ``` + + Go to your fork main page + + ```bash + https://github.com//kafka-connect-storage-cloud.git + ``` + + If you recently pushed your changes GitHub will automatically pop up a `Compare & pull request` + button for any branches you recently pushed to. If you click that button it will automatically + offer you to submit your pull-request to the `confluentinc` connector repository. + + - Give your pull-request a meaningful title as described [above](#titles-and-changelogs). + - In the description, explain your changes and the problem they are solving. + +9. Addressing code review comments + + Repeat steps 5. through 7. to address any code review comments and rebase your changes if necessary. + + Push your updated changes to update the pull request + + ```bash + git push origin [--force] feature-xyz + ``` + + `--force` may be necessary to overwrite your existing pull request in case your + commit history was changed when performing the rebase. + + Note: Be careful when using `--force` since you may lose data if you are not careful. + + ```bash + git push origin --force feature-xyz + ``` + +## Useful Resources for Developers + +1. Connector Developer Guide: https://docs.confluent.io/platform/current/connect/devguide.html +2. A Guide to the Confluent Verified Integrations Program: https://www.confluent.io/blog/guide-to-confluent-verified-integrations-program/ +3. Verification Guide for Confluent Platform Integrations: https://cdn.confluent.io/wp-content/uploads/Verification-Guide-Confluent-Platform-Connectors-Integrations.pdf +4. From Zero to Hero with Kafka Connect: https://www.confluent.io/kafka-summit-lon19/from-zero-to-hero-with-kafka-connect/ +5. 4 Steps to Creating Apache Kafka Connectors with the Kafka Connect API: https://www.confluent.io/blog/create-dynamic-kafka-connect-source-connectors/ +6. How to Write a Connector for Kafka Connect – Deep Dive into Configuration Handling: https://www.confluent.io/blog/write-a-kafka-connect-connector-with-configuration-handling/ From cde8564d648ec2c88e66472b3c3b2536d99f7951 Mon Sep 17 00:00:00 2001 From: Konstantine Karantasis Date: Wed, 19 Jan 2022 12:47:53 -0800 Subject: [PATCH 009/169] Update README.md to include link to contribution guidelines (#483) --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 2f1a89de8..21582cad8 100644 --- a/README.md +++ b/README.md @@ -25,6 +25,7 @@ You can build *kafka-connect-storage-cloud* with Maven using the standard lifecy - Source Code: https://github.com/confluentinc/kafka-connect-storage-cloud - Issue Tracker: https://github.com/confluentinc/kafka-connect-storage-cloud/issues +- Learn how to work with the connector's source code by reading our [Development and Contribution guidelines](CONTRIBUTING.md). # License From ac9b823ec02f681c07b8197d250cfd16d75ce18a Mon Sep 17 00:00:00 2001 From: Chris Olivier Date: Fri, 28 Jan 2022 16:14:02 -0800 Subject: [PATCH 010/169] Merge 5.5.x into 10.0.x (#486) * CCMSG-1531 Support null items within arrays with Parquet writer (#485) * CCMSG-1531 Support null items within arrays with Parquet writer * remove errant blank test * PR comments, naming convention * test * PR comments * the code had too many comments Co-authored-by: Chris Olivier * Refreshed branch colivier/10.0.x * Fix tests that expect deterministically-sorted partition set Co-authored-by: Chris Olivier --- .../parquet/ParquetRecordWriterProvider.java | 56 +++- .../connect/s3/DataWriterAvroTest.java | 23 +- .../connect/s3/DataWriterParquetTest.java | 240 +++++++++++++++++- .../connect/s3/DataWriterTestBase.java | 27 +- .../connect/s3/TestWithMockedS3.java | 4 +- .../connect/s3/TopicPartitionWriterTest.java | 2 +- 6 files changed, 337 insertions(+), 15 deletions(-) diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/parquet/ParquetRecordWriterProvider.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/parquet/ParquetRecordWriterProvider.java index b4e2b5e08..24de63b4a 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/parquet/ParquetRecordWriterProvider.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/parquet/ParquetRecordWriterProvider.java @@ -27,10 +27,12 @@ import io.confluent.connect.storage.format.RecordWriter; import io.confluent.connect.storage.format.RecordWriterProvider; import org.apache.avro.generic.GenericRecord; +import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.parquet.avro.AvroParquetWriter; +import org.apache.parquet.avro.AvroWriteSupport; import org.apache.parquet.hadoop.ParquetFileWriter; import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.io.OutputFile; @@ -39,6 +41,8 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.HashSet; +import java.util.Set; public class ParquetRecordWriterProvider extends RecordViewSetter implements RecordWriterProvider { @@ -75,14 +79,26 @@ public void write(SinkRecord record) { org.apache.avro.Schema avroSchema = avroData.fromConnectSchema(schema); s3ParquetOutputFile = new S3ParquetOutputFile(storage, adjustedFilename); - writer = AvroParquetWriter - .builder(s3ParquetOutputFile) + AvroParquetWriter.Builder builder = + AvroParquetWriter.builder(s3ParquetOutputFile) .withSchema(avroSchema) .withWriteMode(ParquetFileWriter.Mode.OVERWRITE) .withDictionaryEncoding(true) .withCompressionCodec(storage.conf().parquetCompressionCodecName()) - .withPageSize(PAGE_SIZE) - .build(); + .withPageSize(PAGE_SIZE); + if (schemaHasArrayOfOptionalItems(schema, /*seenSchemas=*/null)) { + // If the schema contains an array of optional items, then + // it is possible that the array may have null items during the + // writing process. In this case, we set a flag so as not to + // incur a NullPointerException + log.debug( + "Setting \"" + AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE + + "\" to false because the schema contains an array " + + "with optional items" + ); + builder.config(AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE, "false"); + } + writer = builder.build(); } catch (IOException e) { throw new ConnectException(e); } @@ -121,6 +137,38 @@ public void commit() { }; } + /** + * Check if any schema (or nested schema) is an array of optional items + * @param schema The shema to check + * @return 'true' if the schema contains an array with optional items. + */ + /* VisibleForTesting */ + public static boolean schemaHasArrayOfOptionalItems(Schema schema, Set seenSchemas) { + // First, check for infinitely recursing schemas + if (seenSchemas == null) { + seenSchemas = new HashSet<>(); + } else if (seenSchemas.contains(schema)) { + return false; + } + seenSchemas.add(schema); + switch (schema.type()) { + case STRUCT: + for (Field field : schema.fields()) { + if (schemaHasArrayOfOptionalItems(field.schema(), seenSchemas)) { + return true; + } + } + return false; + case MAP: + return schemaHasArrayOfOptionalItems(schema.valueSchema(), seenSchemas); + case ARRAY: + return schema.valueSchema().isOptional() + || schemaHasArrayOfOptionalItems(schema.valueSchema(), seenSchemas); + default: + return false; + } + } + private static class S3ParquetOutputFile implements OutputFile { private static final int DEFAULT_BLOCK_SIZE = 0; private S3Storage storage; diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/DataWriterAvroTest.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/DataWriterAvroTest.java index 11faf7b0e..75b917a9c 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/DataWriterAvroTest.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/DataWriterAvroTest.java @@ -46,6 +46,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TreeMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -891,11 +892,17 @@ protected List createRecordsWithAlteringSchemas(int size, long start return sinkRecords; } - protected List createRecordsInterleaved(int size, long startOffset, Set partitions) { + protected List createRecordsInterleaved( + int size, + long startOffset, + Set partitionSet + ) { String key = "key"; Schema schema = createSchema(); Struct record = createRecord(schema); + Collection partitions = sortedPartitions(partitionSet); + List sinkRecords = new ArrayList<>(); for (long offset = startOffset, total = 0; total < size; ++offset) { for (TopicPartition tp : partitions) { @@ -957,7 +964,8 @@ protected void verify(List sinkRecords, long[] validOffsets, Set sinkRecords, long[] validOffsets, Set partitions, + protected void verify(List sinkRecords, long[] validOffsets, + Set partitions, boolean skipFileListing) throws IOException { if (!skipFileListing) { @@ -979,8 +987,12 @@ protected void verify(List sinkRecords, long[] validOffsets, Set actualOffsets, long[] validOffsets, - Set partitions) { + protected void verifyOffsets( + Map actualOffsets, + long[] validOffsets, + Set partitionSet + ) { + Collection partitions = sortedPartitions(partitionSet); int i = 0; Map expectedOffsets = new HashMap<>(); for (TopicPartition tp : partitions) { @@ -995,8 +1007,9 @@ protected void verifyOffsets(Map actualOffset protected void verifyRawOffsets( Map actualOffsets, long[] validOffsets, - Set partitions + Set partitionSet ) { + Collection partitions = sortedPartitions(partitionSet); int i = 0; Map expectedOffsets = new HashMap<>(); for (TopicPartition tp : partitions) { diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/DataWriterParquetTest.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/DataWriterParquetTest.java index f0bb0f655..cce668980 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/DataWriterParquetTest.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/DataWriterParquetTest.java @@ -16,6 +16,8 @@ package io.confluent.connect.s3; +import com.amazonaws.services.s3.model.S3ObjectSummary; +import io.confluent.connect.s3.format.parquet.ParquetRecordWriterProvider; import org.apache.avro.util.Utf8; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; @@ -27,6 +29,7 @@ import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.codehaus.plexus.util.StringUtils; import org.junit.After; import org.junit.Test; @@ -37,6 +40,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; @@ -51,6 +55,7 @@ import io.confluent.connect.storage.partitioner.PartitionerConfig; import io.confluent.connect.storage.partitioner.TimeBasedPartitioner; import io.confluent.kafka.serializers.NonRecordContainer; +import org.testcontainers.shaded.com.google.common.collect.ImmutableList; import static org.apache.kafka.common.utils.Time.SYSTEM; import static org.junit.Assert.assertEquals; @@ -181,6 +186,30 @@ public void testWriteInterleavedRecordsInMultiplePartitions() throws Exception { verify(sinkRecords, validOffsets, context.assignment()); } + /** + * Test for parquet writer with null array item(s) arrays + * @link https://github.com/confluentinc/kafka-connect-storage-cloud/issues/339 + * @throws Exception + */ + @Test + public void testWriteRecordsInMultiplePartitionsWithArrayOfOptionalString() throws Exception { + setUp(); + task = new S3SinkTask(connectorConfig, context, storage, partitioner, format, SYSTEM_TIME); + + List sinkRecords = createRecordsWithArrayOfOptionalString( + 7, + context.assignment() + ); + // Perform write + task.put(sinkRecords); + task.close(context.assignment()); + task.stop(); + + // Offsets where each file should start (embedded in the file name) + long[] validOffsets = {0, 3, 6}; + verify(sinkRecords, validOffsets, context.assignment()); + } + @Test public void testWriteInterleavedRecordsInMultiplePartitionsNonZeroInitialOffset() throws Exception { setUp(); @@ -513,6 +542,206 @@ public void testCorrectRecordWriterPartialThisB() throws Exception { testCorrectRecordWriterHelper("this.is.parquet.dir"); } + class SchemaConfig { + public String name; + public boolean optionalItems = false; + public boolean regularItems = false; + public boolean mapRegular = false; + public boolean mapOptional = false; + public SchemaConfig nested = null; + public SchemaConfig nestedArray = null; + + public SchemaConfig() {} + + public SchemaConfig( + String name, + boolean regularItems, + boolean optionalItems, + boolean mapRegular, + boolean mapOptional, + SchemaConfig nested, + SchemaConfig nestedArray + ) { + this.name = name; + this.optionalItems = optionalItems; + this.regularItems = regularItems; + this.mapRegular = mapRegular; + this.mapOptional = mapOptional; + this.nested = nested; + this.nestedArray = nestedArray; + } + + public boolean hasOptionalItems() { + if (optionalItems || mapOptional) { + return true; + } + if (nested != null && nested.hasOptionalItems()) { + return true; + } + return nestedArray != null && nestedArray.hasOptionalItems(); + } + + private Schema create() { + SchemaBuilder builder = SchemaBuilder.struct(); + if (StringUtils.isNotBlank(name)) { + builder.name(name); + } + if (regularItems) { + builder.field("regular_items", SchemaBuilder.array(Schema.STRING_SCHEMA).build()); + } + if (optionalItems) { + builder.field("optional_items", SchemaBuilder.array(Schema.OPTIONAL_STRING_SCHEMA).build()); + } + if (mapRegular) { + builder.field("regular_map", SchemaBuilder.map( + Schema.STRING_SCHEMA, + SchemaBuilder.array(Schema.STRING_SCHEMA).build() + ).build()); + } + if (mapOptional) { + builder.field("optional_map", SchemaBuilder.map( + Schema.STRING_SCHEMA, + SchemaBuilder.array(Schema.OPTIONAL_STRING_SCHEMA).build() + ).build()); + } + if (this.nested != null) { + builder.field("nested", nested.create()); + } + if (this.nestedArray != null) { + builder.field("nested_array", SchemaBuilder.array(nestedArray.create())); + } + return builder.build(); + } + + } + + /** + * Tests ParquetRecordWriterProvider::schemaHasArrayOfOptionalItems() + * + * Test permutations of schemas and schema nesting with and without optional array items + * somewhere within the schema. + */ + @Test + public void testSchemaHasArrayOfOptionalItems() { + for (int regularItems = 0; regularItems < 2; ++regularItems) { + for (int optionalItems = 0; optionalItems < 2; ++optionalItems) { + for (int mapRegular = 0; mapRegular < 2; ++mapRegular) { + for (int mapOptional = 0; mapOptional < 2; ++mapOptional) { + for (int hasNested = 0; hasNested < 2; ++hasNested) { + for (int hasNestedArray = 0; hasNestedArray < 2; ++hasNestedArray) { + SchemaConfig conf = new SchemaConfig(); + SchemaConfig nested = null, nestedArray = null; + if (hasNested != 0) { + // Invert tests so as to hit in isolation + nested = new SchemaConfig( + "nested_schema", + regularItems == 0, + optionalItems == 0, + mapRegular == 0, + mapOptional == 0, + /*nested=*/ null, + /*nestedArray=*/null + ); + } + if (hasNestedArray != 0) { + // Invert tests so as to hit in isolation + nestedArray = new SchemaConfig( + "array_of_schema", + regularItems == 0, + optionalItems == 0, + mapRegular == 0, + mapOptional == 0, + /*nested=*/ null, + /*nestedArray=*/null + ); + } + conf.nested = new SchemaConfig( + "nested_schema", + regularItems != 0, + optionalItems != 0, + mapRegular != 0, + mapOptional != 0, + nested, + nestedArray + ); + Schema schema = conf.create(); + if (schema.fields().size() == 0) { + // Base case of everything is false + continue; + } + final boolean hasArrayOptional = + ParquetRecordWriterProvider.schemaHasArrayOfOptionalItems( + schema, + /*seenSchemas=*/null + ); + final boolean shouldHaveArrayOptional = conf.hasOptionalItems(); + assertEquals(hasArrayOptional, shouldHaveArrayOptional); + } + } + } + } + } + } + } + + private List createRecordsWithArrayOfOptionalString( + int size, + Set partitions + ) { + SchemaConfig conf = new SchemaConfig(); + conf.regularItems = true; + conf.optionalItems = true; + conf.nested = new SchemaConfig( + "nested_schema", + true, + true, + false, + false, + null, + null + ); + + Schema schema = conf.create(); + + List sinkRecords = new ArrayList<>(); + for (TopicPartition tp : partitions) { + // We're going to alternate internal and external array elements as null + boolean hasString = true; + for (long offset = 0; offset < size; ++offset) { + LinkedList optionalList = new LinkedList<>(); + // Alternate edge and internal as null items + optionalList.add(hasString ? "item-1" : null); + optionalList.add(hasString ? null : "item-2"); + optionalList.add(hasString ? "item-3" : null); + Struct struct = new Struct(schema) + .put("optional_items", optionalList) + .put("regular_items", ImmutableList.of("reg-1", "reg-2")) + .put( + // Nested struct + "nested", + new Struct(schema.field("nested").schema()) + // Nested option string array + .put("optional_items", optionalList.clone()) + // Nested regular string array + .put("regular_items", ImmutableList.of("reg-1", "reg-2")) + ); + sinkRecords.add( + new SinkRecord( + TOPIC, + tp.partition(), + Schema.STRING_SCHEMA, + "key", + schema, + struct, + offset + ) + ); + hasString = !hasString; + } + } + return sinkRecords; + } + /** * Return a list of new records starting at zero offset. * @@ -548,11 +777,17 @@ protected List createRecords(int size, long startOffset, Set createRecordsInterleaved(int size, long startOffset, Set partitions) { + protected List createRecordsInterleaved( + int size, + long startOffset, + Set partitionSet + ) { String key = "key"; Schema schema = createSchema(); Struct record = createRecord(schema); + Collection partitions = sortedPartitions(partitionSet); + List sinkRecords = new ArrayList<>(); for (long offset = startOffset; offset < startOffset + size; ++offset) { for (TopicPartition tp : partitions) { @@ -707,7 +942,8 @@ protected String getDirectory(String topic, int partition) { } protected void verifyOffsets(Map actualOffsets, Long[] validOffsets, - Set partitions) { + Set partitionSet) { + Collection partitions = sortedPartitions(partitionSet); int i = 0; Map expectedOffsets = new HashMap<>(); for (TopicPartition tp : partitions) { diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/DataWriterTestBase.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/DataWriterTestBase.java index 1a0d23c18..74b6fe4e6 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/DataWriterTestBase.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/DataWriterTestBase.java @@ -39,6 +39,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TreeMap; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; @@ -147,7 +148,8 @@ protected List getExpectedFiles(long[] validOffsets, TopicPartition tp, return expectedFiles; } - protected List getExpectedFiles(long[] validOffsets, Collection partitions, + protected List getExpectedFiles(long[] validOffsets, + Collection partitions, String extension) { List expectedFiles = new ArrayList<>(); for (TopicPartition tp : partitions) { @@ -167,7 +169,7 @@ protected void verifyFileListing(List expectedFiles) throws IOException assertThat(actualFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); } - protected void verifyFileListing(long[] validOffsets, Set partitions, + protected void verifyFileListing(long[] validOffsets, Collection partitions, String extension) throws IOException { List expectedFiles = getExpectedFiles(validOffsets, partitions, extension); verifyFileListing(expectedFiles); @@ -239,4 +241,25 @@ protected void testCorrectRecordWriterHelper( verify(records, validOffsets, partitions); } + /** + * Sort a collection of TopicPartition objects by the partition number. + * Expectation is that there are not duplicate partition numbers in the set + * (i.e. only one topic). + * @param partitions Collection of TopicPartition objects + * @return Coll3ection of TopicPartition sorted by TopicPartition::partition() + */ + protected static Collection sortedPartitions( + Collection partitions + ) { + // Sort by partition # + TreeMap map = new TreeMap<>(); + for (TopicPartition partition : partitions) { + if (map.containsKey(partition.partition())) { + throw new RuntimeException("A duplicate partition number not expected."); + } + map.put(partition.partition(), partition); + } + return map.values(); + } + } diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TestWithMockedS3.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TestWithMockedS3.java index 9b5a3e2cf..35f457db0 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TestWithMockedS3.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TestWithMockedS3.java @@ -87,7 +87,9 @@ public void setUp() throws Exception { @Override public void tearDown() throws Exception { super.tearDown(); - s3mock.shutdown(); // shutdown the Akka and HTTP frameworks to close all connections + if (s3mock != null) { + s3mock.shutdown(); // shutdown the Akka and HTTP frameworks to close all connections + } } public static List listObjects(String bucket, String prefix, AmazonS3 s3) { diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java index db31e38cf..1d634299a 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java @@ -132,7 +132,7 @@ public void setUp() throws Exception { format = new AvroFormat(storage); s3.createBucket(S3_TEST_BUCKET_NAME); - assertTrue(s3.doesBucketExist(S3_TEST_BUCKET_NAME)); + assertTrue(s3.doesBucketExistV2(S3_TEST_BUCKET_NAME)); Format format = new AvroFormat(storage); writerProvider = format.getRecordWriterProvider(); From cd2dc18fa2166f3528bc9bd1592762194190075a Mon Sep 17 00:00:00 2001 From: Chris Olivier Date: Mon, 7 Feb 2022 13:32:39 -0800 Subject: [PATCH 011/169] MINOR: Suppress warning which causes IntelliJ to error after Maven import (#489) Co-authored-by: Chris Olivier --- .../io/confluent/connect/s3/integration/S3SinkConnectorIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java index 967c3a695..2c6a5a73f 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java @@ -104,6 +104,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +@SuppressWarnings({"unchecked", "deprecation"}) @Category(IntegrationTest.class) public class S3SinkConnectorIT extends BaseConnectorIT { From fb68c7ed841757eafdda4baa4e15dfc2b109f69d Mon Sep 17 00:00:00 2001 From: Spiro Date: Tue, 2 Aug 2022 11:47:03 +0800 Subject: [PATCH 012/169] Elastic byte buffer for staging s3-part to save memory in S3OutputStream (#538) --- .../connect/s3/S3SinkConnectorConfig.java | 44 +++ .../confluent/connect/s3/storage/ByteBuf.java | 36 ++ .../connect/s3/storage/ElasticByteBuffer.java | 157 +++++++++ .../connect/s3/storage/S3OutputStream.java | 13 +- .../connect/s3/storage/SimpleByteBuffer.java | 58 +++ .../s3/storage/ElasticByteBufferTest.java | 332 ++++++++++++++++++ 6 files changed, 637 insertions(+), 3 deletions(-) create mode 100644 kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/ByteBuf.java create mode 100644 kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/ElasticByteBuffer.java create mode 100644 kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/SimpleByteBuffer.java create mode 100644 kafka-connect-s3/src/test/java/io/confluent/connect/s3/storage/ElasticByteBufferTest.java diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java index 6f80d560a..e7e06b291 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java @@ -172,6 +172,16 @@ public class S3SinkConnectorConfig extends StorageSinkConnectorConfig { public static final String HEADERS_FORMAT_CLASS_CONFIG = "headers.format.class"; public static final Class HEADERS_FORMAT_CLASS_DEFAULT = AvroFormat.class; + /** + * Elastic buffer to save memory. {@link io.confluent.connect.s3.storage.S3OutputStream#buffer} + */ + + public static final String ELASTIC_BUFFER_ENABLE = "s3.elastic.buffer.enable"; + public static final boolean ELASTIC_BUFFER_ENABLE_DEFAULT = false; + + public static final String ELASTIC_BUFFER_INIT_CAPACITY = "s3.elastic.buffer.init.capacity"; + public static final int ELASTIC_BUFFER_INIT_CAPACITY_DEFAULT = 128 * 1024; // 128KB + private final String name; private final Map propertyToConfig = new HashMap<>(); @@ -660,6 +670,32 @@ public static ConfigDef newConfigDef() { Width.SHORT, "Enable Path Style Access to S3" ); + configDef.define( + ELASTIC_BUFFER_ENABLE, + Type.BOOLEAN, + ELASTIC_BUFFER_ENABLE_DEFAULT, + Importance.LOW, + "Specifies whether or not to allocate elastic buffer for staging s3-part to save memory." + + " Note that this may cause decreased performance or increased CPU usage", + group, + ++orderInGroup, + Width.LONG, + "Enable elastic buffer to staging s3-part" + ); + + configDef.define( + ELASTIC_BUFFER_INIT_CAPACITY, + Type.INT, + ELASTIC_BUFFER_INIT_CAPACITY_DEFAULT, + atLeast(4096), + Importance.LOW, + "Elastic buffer initial capacity.", + group, + ++orderInGroup, + Width.LONG, + "Elastic buffer initial capacity" + ); + } return configDef; } @@ -811,6 +847,14 @@ public String getFormatByteArrayLineSeparator() { return FORMAT_BYTEARRAY_LINE_SEPARATOR_DEFAULT; } + public boolean getElasticBufferEnable() { + return getBoolean(ELASTIC_BUFFER_ENABLE); + } + + public int getElasticBufferInitCap() { + return getInt(ELASTIC_BUFFER_INIT_CAPACITY); + } + protected static String parseName(Map props) { String nameProp = props.get("name"); return nameProp != null ? nameProp : "S3-sink"; diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/ByteBuf.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/ByteBuf.java new file mode 100644 index 000000000..9dede235b --- /dev/null +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/ByteBuf.java @@ -0,0 +1,36 @@ +/* + * Copyright 2018 Confluent Inc. + * + * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.connect.s3.storage; + +/** + * A interface for S3OutputStream to write s3-part. + */ +public interface ByteBuf { + + void put(byte b); + + void put(byte[] src, int offset, int length); + + boolean hasRemaining(); + + int remaining(); + + int position(); + + void clear(); + + byte[] array(); +} diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/ElasticByteBuffer.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/ElasticByteBuffer.java new file mode 100644 index 000000000..505988ed4 --- /dev/null +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/ElasticByteBuffer.java @@ -0,0 +1,157 @@ +/* + * Copyright 2018 Confluent Inc. + * + * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.connect.s3.storage; + +import java.nio.BufferOverflowException; + +/** + * A elastic byte buffer with a logic size as max size. + * The formula to expand: initCapacity * 2 ^ (incrementFactor * N) + */ +public class ElasticByteBuffer implements ByteBuf { + + public static final int INCREMENT_FACTOR = 1; + + /* logical capacity */ + private int capacity; + + /* initial physical capacity */ + private int initPhysicalCap; + + /* the next position to write */ + private int position; + + /* physical buf */ + private byte[] buf; + + public ElasticByteBuffer(int capacity, int initPhysicalCap) { + if (capacity <= 0) { + throw new IllegalArgumentException("capacity must greater than zero"); + } + + if (initPhysicalCap <= 0) { + throw new IllegalArgumentException("initial physical capacity must greater than zero"); + } + + this.capacity = capacity; + this.initPhysicalCap = initPhysicalCap; + + initialize(); + } + + private void initialize() { + this.position = 0; + int initCapacity = Math.min(this.capacity, this.initPhysicalCap); + this.buf = new byte[initCapacity]; + } + + private void expand() { + int currSize = this.buf.length; + int calNewSize = currSize << INCREMENT_FACTOR; + + int newSize = 0; + if (calNewSize < currSize) { + // down overflow + newSize = this.capacity; + } else { + newSize = Math.min(this.capacity, calNewSize); + } + + byte[] currBuf = this.buf; + this.buf = new byte[newSize]; + System.arraycopy(currBuf, 0, this.buf, 0, currSize); + } + + public void put(byte b) { + if (!hasRemaining()) { + throw new BufferOverflowException(); + } + + if (physicalRemaining() <= 0) { + // expand physical buf + expand(); + } + + this.buf[this.position] = b; + this.position++; + } + + public void put(byte[] src, int offset, int length) { + + checkBounds(offset, length, src.length); + + if (remaining() < length) { + throw new BufferOverflowException(); + } + + int remainingOffset = offset; + int remainingLen = length; + while (true) { + if (physicalRemaining() <= 0) { + // expand physical buf + expand(); + } + + if (physicalRemaining() >= remainingLen) { + System.arraycopy(src, remainingOffset, this.buf, this.position, remainingLen); + this.position += remainingLen; + break; + } else { + int physicalRemaining = physicalRemaining(); + System.arraycopy(src, remainingOffset, this.buf, this.position, physicalRemaining); + this.position += physicalRemaining; + remainingOffset += physicalRemaining; + remainingLen -= physicalRemaining; + } + } + } + + static void checkBounds(int off, int len, int size) { // package-private + if ((off | len | (off + len) | (size - (off + len))) < 0) { + throw new IndexOutOfBoundsException(); + } + } + + public int physicalRemaining() { + return this.buf.length - this.position; + } + + public boolean hasRemaining() { + return capacity > position; + } + + public int remaining() { + return capacity - position; + } + + public int position() { + return this.position; + } + + public void clear() { + if (this.buf.length <= this.initPhysicalCap) { + // has not ever expanded, just reset position + this.position = 0; + } else { + initialize(); + } + } + + public final byte[] array() { + return this.buf; + } + +} diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3OutputStream.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3OutputStream.java index 50e0d9817..a8afac74f 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3OutputStream.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3OutputStream.java @@ -38,7 +38,6 @@ import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.OutputStream; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import java.util.function.Supplier; @@ -60,7 +59,7 @@ public class S3OutputStream extends PositionOutputStream { private final int partSize; private final CannedAccessControlList cannedAcl; private boolean closed; - private final ByteBuffer buffer; + private final ByteBuf buffer; private MultipartUpload multiPartUpload; private final CompressionType compressionType; private final int compressionLevel; @@ -80,7 +79,15 @@ public S3OutputStream(String key, S3SinkConnectorConfig conf, AmazonS3 s3) { this.partSize = conf.getPartSize(); this.cannedAcl = conf.getCannedAcl(); this.closed = false; - this.buffer = ByteBuffer.allocate(this.partSize); + + final boolean elasticBufEnable = conf.getElasticBufferEnable(); + if (elasticBufEnable) { + final int elasticBufInitialCap = conf.getElasticBufferInitCap(); + this.buffer = new ElasticByteBuffer(this.partSize, elasticBufInitialCap); + } else { + this.buffer = new SimpleByteBuffer(this.partSize); + } + this.progressListener = new ConnectProgressListener(); this.multiPartUpload = null; this.compressionType = conf.getCompressionType(); diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/SimpleByteBuffer.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/SimpleByteBuffer.java new file mode 100644 index 000000000..e997d1baf --- /dev/null +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/SimpleByteBuffer.java @@ -0,0 +1,58 @@ +/* + * Copyright 2018 Confluent Inc. + * + * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.connect.s3.storage; + +import java.nio.ByteBuffer; + +/** + * A simple byte buf + */ +public class SimpleByteBuffer implements ByteBuf { + + private ByteBuffer buffer; + + public SimpleByteBuffer(int capacity) { + this.buffer = ByteBuffer.allocate(capacity); + } + + public void put(byte b) { + this.buffer.put(b); + } + + public void put(byte[] src, int offset, int length) { + this.buffer.put(src, offset, length); + } + + public boolean hasRemaining() { + return this.buffer.hasRemaining(); + } + + public int remaining() { + return this.buffer.remaining(); + } + + public int position() { + return this.buffer.position(); + } + + public void clear() { + this.buffer.clear(); + } + + public byte[] array() { + return this.buffer.array(); + } +} diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/storage/ElasticByteBufferTest.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/storage/ElasticByteBufferTest.java new file mode 100644 index 000000000..5bcd9422f --- /dev/null +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/storage/ElasticByteBufferTest.java @@ -0,0 +1,332 @@ +package io.confluent.connect.s3.storage; + +import org.apache.commons.lang.RandomStringUtils; +import org.apache.commons.lang.math.RandomUtils; +import org.junit.Test; +import java.nio.BufferOverflowException; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +public class ElasticByteBufferTest { + + public static final int INIT_CAP = 128 * 1024; + + @Test(expected = IllegalArgumentException.class) + public void testIllegalCapacity1() { + ElasticByteBuffer buf = new ElasticByteBuffer(-1, INIT_CAP); + } + + @Test(expected = IllegalArgumentException.class) + public void testIllegalCapacity2() { + ElasticByteBuffer buf = new ElasticByteBuffer(0, INIT_CAP); + } + + @Test + public void testLessThanInitCapacityPut1() { + ElasticByteBuffer buf = new ElasticByteBuffer(1024, INIT_CAP); + + assertEquals(1024, buf.physicalRemaining()); + assertEquals(1024, buf.remaining()); + assertEquals(0, buf.position()); + assertTrue(buf.hasRemaining()); + assertEquals(1024, buf.array().length); + + buf.put((byte) 0x01); + assertEquals(1023, buf.physicalRemaining()); + assertEquals(1023, buf.remaining()); + assertEquals(1, buf.position()); + assertTrue(buf.hasRemaining()); + assertEquals(1024, buf.array().length); + + byte[] randomBytes = RandomStringUtils.randomAlphanumeric(1023).getBytes(); + for (byte randomByte : randomBytes) { + buf.put(randomByte); + } + + assertEquals(0, buf.physicalRemaining()); + assertEquals(0, buf.remaining()); + assertEquals(1024, buf.position()); + assertFalse(buf.hasRemaining()); + assertEquals(1024, buf.array().length); + + assertThrows(BufferOverflowException.class, () -> { + buf.put((byte) 0x01); + }); + + } + + @Test + public void testLessThanInitCapacityPut2() { + + int cap = 1024; + ElasticByteBuffer buf = new ElasticByteBuffer(cap, INIT_CAP); + + byte[] randomBytes1 = RandomStringUtils.randomAlphanumeric(4).getBytes(); + buf.put(randomBytes1, 0, randomBytes1.length); + + assertEquals(1020, buf.physicalRemaining()); + assertEquals(1020, buf.remaining()); + assertEquals(4, buf.position()); + assertTrue(buf.hasRemaining()); + assertEquals(cap, buf.array().length); + + byte[] randomBytes2 = RandomStringUtils.randomAlphanumeric(1019).getBytes(); + buf.put(randomBytes2, 0, randomBytes2.length); + + assertEquals(1, buf.physicalRemaining()); + assertEquals(1, buf.remaining()); + assertEquals(1023, buf.position()); + assertTrue(buf.hasRemaining()); + assertEquals(cap, buf.array().length); + + byte[] randomBytes3 = RandomStringUtils.randomAlphanumeric(2).getBytes(); + assertThrows(BufferOverflowException.class, () -> { + buf.put(randomBytes3, 0, randomBytes3.length); + }); + + buf.put(new byte[] {0x01}, 0, 1); + assertEquals(0, buf.physicalRemaining()); + assertEquals(0, buf.remaining()); + assertEquals(cap, buf.position()); + assertFalse(buf.hasRemaining()); + assertEquals(cap, buf.array().length); + } + + @Test + public void testLessThanInitCapacityClear() { + ElasticByteBuffer buf = new ElasticByteBuffer(1024, INIT_CAP); + + byte[] randomBytes1 = RandomStringUtils.randomAlphanumeric(4).getBytes(); + buf.put(randomBytes1, 0, randomBytes1.length); + + byte[] arrayBeforeClear = buf.array(); + buf.clear(); + byte[] arrayAfterClear = buf.array(); + assertEquals(arrayAfterClear.length, arrayBeforeClear.length); + assertSame(arrayAfterClear, arrayBeforeClear); + } + + + @Test + public void testGreaterThanInitCapacityPut1() { + + int cap = 10 * 1024 * 1024; + ElasticByteBuffer buf = new ElasticByteBuffer(cap, INIT_CAP); + + assertEquals(INIT_CAP, buf.physicalRemaining()); + assertEquals(cap, buf.remaining()); + assertEquals(0, buf.position()); + assertTrue(buf.hasRemaining()); + assertEquals(INIT_CAP, buf.array().length); + + byte[] randomBytes1 = RandomStringUtils.randomAlphanumeric(INIT_CAP).getBytes(); + for (byte randomByte : randomBytes1) { + buf.put(randomByte); + } + + assertEquals(0, buf.physicalRemaining()); + assertEquals(cap - INIT_CAP, buf.remaining()); + assertEquals(INIT_CAP, buf.position()); + assertTrue(buf.hasRemaining()); + assertEquals(INIT_CAP, buf.array().length); + + int testBytesLen1 = 5; + byte[] randomBytes2 = RandomStringUtils.randomAlphanumeric(testBytesLen1).getBytes(); + for (byte randomByte : randomBytes2) { + buf.put(randomByte); + } + + int exceptNewPhysicalSize = INIT_CAP * 2; + + assertEquals(exceptNewPhysicalSize - (INIT_CAP + testBytesLen1), buf.physicalRemaining()); + assertEquals(cap - (INIT_CAP + testBytesLen1), buf.remaining()); + assertEquals(INIT_CAP + testBytesLen1, buf.position()); + assertTrue(buf.hasRemaining()); + assertEquals(exceptNewPhysicalSize, buf.array().length); + + int remaining = cap - (INIT_CAP + testBytesLen1); + byte[] randomBytes3 = RandomStringUtils.randomAlphanumeric(remaining).getBytes(); + for (byte randomByte : randomBytes3) { + buf.put(randomByte); + } + + assertEquals(0, buf.physicalRemaining()); + assertEquals(0, buf.remaining()); + assertEquals(cap, buf.position()); + assertFalse(buf.hasRemaining()); + assertEquals(cap, buf.array().length); + + assertThrows(BufferOverflowException.class, ()->{ + buf.put((byte) 0x01); + }); + } + + @Test + public void testGreaterThanInitCapacityPut2() { + int cap = 10 * 1024 * 1024; + ElasticByteBuffer buf = new ElasticByteBuffer(cap, INIT_CAP); + + assertEquals(INIT_CAP, buf.physicalRemaining()); + assertEquals(cap, buf.remaining()); + assertEquals(0, buf.position()); + assertTrue(buf.hasRemaining()); + assertEquals(INIT_CAP, buf.array().length); + + byte[] randomBytes1 = RandomStringUtils.randomAlphanumeric(INIT_CAP).getBytes(); + buf.put(randomBytes1, 0, randomBytes1.length); + + assertEquals(0, buf.physicalRemaining()); + assertEquals(cap - INIT_CAP, buf.remaining()); + assertEquals(INIT_CAP, buf.position()); + assertTrue(buf.hasRemaining()); + assertEquals(INIT_CAP, buf.array().length); + + int testBytesLen1 = 5; + byte[] randomBytes2 = RandomStringUtils.randomAlphanumeric(testBytesLen1).getBytes(); + buf.put(randomBytes2, 0, randomBytes2.length); + + int expectNewPhysicalSize = INIT_CAP * 2; + + assertEquals(expectNewPhysicalSize - (INIT_CAP + testBytesLen1), buf.physicalRemaining()); + assertEquals(cap - (INIT_CAP + testBytesLen1), buf.remaining()); + assertEquals(INIT_CAP + testBytesLen1, buf.position()); + assertTrue(buf.hasRemaining()); + assertEquals(expectNewPhysicalSize, buf.array().length); + + int remainingLessOne = cap - (INIT_CAP + testBytesLen1) - 1; + byte[] randomBytes3 = RandomStringUtils.randomAlphanumeric(remainingLessOne).getBytes(); + buf.put(randomBytes3, 0, randomBytes3.length); + + assertEquals(1, buf.physicalRemaining()); + assertEquals(1, buf.remaining()); + assertEquals(cap - 1, buf.position()); + assertTrue(buf.hasRemaining()); + assertEquals(cap, buf.array().length); + + assertThrows(BufferOverflowException.class, ()->{ + buf.put(new byte[] {0x01, 0x02}, 0, 2); + }); + + buf.put(new byte[] {0x01}, 0, 1); + assertEquals(0, buf.physicalRemaining()); + assertEquals(0, buf.remaining()); + assertEquals(cap, buf.position()); + assertFalse(buf.hasRemaining()); + assertEquals(cap, buf.array().length); + + } + + @Test + public void testGreaterThanInitCapacityClear() { + int cap = 10 * 1024 * 1024; + ElasticByteBuffer buf = new ElasticByteBuffer(cap, INIT_CAP); + + byte[] randomBytes1 = RandomStringUtils.randomAlphanumeric(5 * 1024 * 1024).getBytes(); + buf.put(randomBytes1, 0, randomBytes1.length); + + byte[] arrayBeforeClear = buf.array(); + buf.clear(); + byte[] arrayAfterClear = buf.array(); + + assertEquals(0, buf.position()); + assertTrue(buf.hasRemaining()); + assertEquals(INIT_CAP, buf.physicalRemaining()); + assertEquals(cap, buf.remaining()); + + assertEquals(INIT_CAP, arrayAfterClear.length); + assertTrue(arrayAfterClear.length < arrayBeforeClear.length); + assertNotSame(arrayAfterClear, arrayBeforeClear); + } + + @Test + public void testLessThanInitSizeDataPut1() { + int cap = 1024; + ElasticByteBuffer buf = new ElasticByteBuffer(cap, INIT_CAP); + + int testBytesLen1 = 4; + String data1 = RandomStringUtils.randomAlphanumeric(testBytesLen1); + byte[] randomBytes1 = data1.getBytes(); + for (byte randomByte : randomBytes1) { + buf.put(randomByte); + } + + assertEquals(data1, new String(buf.array(), 0, buf.position())); + + int testBytesLen2 = 1020; + String data2 = RandomStringUtils.randomAlphanumeric(testBytesLen2); + byte[] randomBytes2 = data2.getBytes(); + for (byte randomByte : randomBytes2) { + buf.put(randomByte); + } + + assertEquals(data1 + data2, new String(buf.array(), 0, buf.position())); + } + + @Test + public void testLessThanInitSizeDataPut2() { + int cap = 1024; + ElasticByteBuffer buf = new ElasticByteBuffer(cap, INIT_CAP); + + int testBytesLen1 = 4; + String data1 = RandomStringUtils.randomAlphanumeric(testBytesLen1); + byte[] randomBytes1 = data1.getBytes(); + buf.put(randomBytes1, 0, randomBytes1.length); + + assertEquals(data1, new String(buf.array(), 0, buf.position())); + + int testBytesLen2 = 1020; + String data2 = RandomStringUtils.randomAlphanumeric(testBytesLen2); + byte[] randomBytes2 = data2.getBytes(); + buf.put(randomBytes2, 0, randomBytes2.length); + + assertEquals(data1 + data2, new String(buf.array(), 0, buf.position())); + } + + @Test + public void testGreaterThanInitSizeDataPut1() { + int cap = 5 * 1024 * 1024; + ElasticByteBuffer buf = new ElasticByteBuffer(cap, INIT_CAP); + + int testBytesLen1 = RandomUtils.nextInt(cap); + String data1 = RandomStringUtils.randomAlphanumeric(testBytesLen1); + byte[] randomBytes1 = data1.getBytes(); + for (byte randomByte : randomBytes1) { + buf.put(randomByte); + } + + assertEquals(data1, new String(buf.array(), 0, buf.position())); + + int testBytesLen2 = cap - testBytesLen1; + String data2 = RandomStringUtils.randomAlphanumeric(testBytesLen2); + byte[] randomBytes2 = data2.getBytes(); + for (byte randomByte : randomBytes2) { + buf.put(randomByte); + } + + assertEquals(data1 + data2, new String(buf.array(), 0, buf.position())); + } + + @Test + public void testGreaterThanInitSizeDataPut2() { + int cap = 5 * 1024 * 1024; + ElasticByteBuffer buf = new ElasticByteBuffer(cap, INIT_CAP); + + int testBytesLen1 = RandomUtils.nextInt(cap); + String data1 = RandomStringUtils.randomAlphanumeric(testBytesLen1); + byte[] randomBytes1 = data1.getBytes(); + buf.put(randomBytes1, 0, randomBytes1.length); + + assertEquals(data1, new String(buf.array(), 0, buf.position())); + + int testBytesLen2 = cap - testBytesLen1; + String data2 = RandomStringUtils.randomAlphanumeric(testBytesLen2); + byte[] randomBytes2 = data2.getBytes(); + buf.put(randomBytes2, 0, randomBytes2.length); + + assertEquals(data1 + data2, new String(buf.array(), 0, buf.position())); + } +} \ No newline at end of file From d83f942cacf540c2cc833e1a1976666095ee6eb6 Mon Sep 17 00:00:00 2001 From: sudeshwasnik Date: Tue, 2 Aug 2022 16:14:32 +0530 Subject: [PATCH 013/169] [maven-release-plugin] prepare branch 10.1.x --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 857d465c9..a1e6683b5 100644 --- a/pom.xml +++ b/pom.xml @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - HEAD + 10.1.x From af769885933104a8fc2ba030f1615069d47aafef Mon Sep 17 00:00:00 2001 From: Sudesh Wasnik <67974618+sudeshwasnik@users.noreply.github.com> Date: Tue, 2 Aug 2022 17:08:19 +0530 Subject: [PATCH 014/169] Update master to reflect new branch cut chnages (#540) --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index ce9101e3c..20e4b83c3 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.0-SNAPSHOT + 10.2.0-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index a1e6683b5..77f07a047 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.0-SNAPSHOT + 10.2.0-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + HEAD From 15ee7a221848fe77cf4d4496823491e27e7203eb Mon Sep 17 00:00:00 2001 From: Sudesh Wasnik Date: Wed, 3 Aug 2022 03:07:52 +0000 Subject: [PATCH 015/169] [maven-release-plugin] prepare release v10.1.0 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index ce9101e3c..2e6dc450f 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.0-SNAPSHOT + 10.1.0 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index a1e6683b5..2f7f759bf 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.0-SNAPSHOT + 10.1.0 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + v10.1.0 From 0fcca2e855b1020b18663744289426b1b3d2a3ec Mon Sep 17 00:00:00 2001 From: Sudesh Wasnik Date: Wed, 3 Aug 2022 03:07:58 +0000 Subject: [PATCH 016/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 2e6dc450f..51471bb54 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.0 + 10.1.1-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 2f7f759bf..0a7659294 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.0 + 10.1.1-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.1.0 + 10.1.x From eb919b908853cbef70b89274ecac9f0ced024087 Mon Sep 17 00:00:00 2001 From: Siddhartha Agarwal Date: Wed, 17 Aug 2022 05:58:11 +0000 Subject: [PATCH 017/169] [maven-release-plugin] prepare release v10.1.1 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 51471bb54..f1a7c0f29 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.1-SNAPSHOT + 10.1.1 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 3ac2ba90a..a3c89ba2a 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.1-SNAPSHOT + 10.1.1 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + v10.1.1 From d298fa4930fa8f6dd7b5aef3585bd5ae2e300ba2 Mon Sep 17 00:00:00 2001 From: Siddhartha Agarwal Date: Wed, 17 Aug 2022 05:58:17 +0000 Subject: [PATCH 018/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index f1a7c0f29..53fcd1bc0 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.1 + 10.1.2-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index a3c89ba2a..37bb114b9 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.1 + 10.1.2-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.1.1 + 10.1.x From 14cb8bc93ef045a8e6c9e9030f10875a0f0eeba9 Mon Sep 17 00:00:00 2001 From: pkuntal Date: Mon, 22 Aug 2022 22:25:38 +0530 Subject: [PATCH 019/169] Using Access key and secret access to create Assume Role Provider. --- .../connect/s3/S3SinkConnectorConfig.java | 15 ++++++- .../AwsAssumeRoleCredentialsProvider.java | 39 ++++++++++++++++--- .../connect/s3/storage/S3Storage.java | 19 ++------- .../connect/s3/S3SinkConnectorConfigTest.java | 8 ++-- 4 files changed, 54 insertions(+), 27 deletions(-) diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java index e7e06b291..a521f41c8 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java @@ -17,11 +17,14 @@ import com.amazonaws.ClientConfiguration; import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.AWSStaticCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; import com.amazonaws.regions.RegionUtils; import com.amazonaws.regions.Regions; import com.amazonaws.services.s3.model.CannedAccessControlList; import com.amazonaws.services.s3.model.SSEAlgorithm; +import io.confluent.connect.storage.common.util.StringUtils; import org.apache.kafka.common.Configurable; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; @@ -774,7 +777,7 @@ public int getPartSize() { } @SuppressWarnings("unchecked") - public AWSCredentialsProvider getCredentialsProvider() { + public AWSCredentialsProvider getCredentialsProvider(S3SinkConnectorConfig config) { try { AWSCredentialsProvider provider = ((Class) getClass(S3SinkConnectorConfig.CREDENTIALS_PROVIDER_CLASS_CONFIG)).newInstance(); @@ -785,6 +788,16 @@ public AWSCredentialsProvider getCredentialsProvider() { CREDENTIALS_PROVIDER_CONFIG_PREFIX.length() )); ((Configurable) provider).configure(configs); + } else { + final String accessKeyId = config.getString(AWS_ACCESS_KEY_ID_CONFIG); + final String secretKey = config.getPassword(AWS_SECRET_ACCESS_KEY_CONFIG).value(); + if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKey)) { + /*log.info("Returning new credentials provider using the access key id and " + + "the secret access key that were directly supplied through the connector's " + + "configuration"); */ + BasicAWSCredentials basicCredentials = new BasicAWSCredentials(accessKeyId, secretKey); + provider = new AWSStaticCredentialsProvider(basicCredentials); + } } return provider; diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/auth/AwsAssumeRoleCredentialsProvider.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/auth/AwsAssumeRoleCredentialsProvider.java index 9547098c7..a14eaf6da 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/auth/AwsAssumeRoleCredentialsProvider.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/auth/AwsAssumeRoleCredentialsProvider.java @@ -17,15 +17,21 @@ package io.confluent.connect.s3.auth; import com.amazonaws.auth.AWSCredentials; -import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.AWSStaticCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider; +import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder; +import io.confluent.connect.storage.common.util.StringUtils; import org.apache.kafka.common.Configurable; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import java.util.Map; +import static io.confluent.connect.s3.S3SinkConnectorConfig.AWS_ACCESS_KEY_ID_CONFIG; +import static io.confluent.connect.s3.S3SinkConnectorConfig.AWS_SECRET_ACCESS_KEY_CONFIG; + /** * AWS credentials provider that uses the AWS Security Token Service to assume a Role and create a * temporary, short-lived session to use for authentication. This credentials provider does not @@ -59,21 +65,42 @@ public class AwsAssumeRoleCredentialsProvider implements AWSCredentialsProvider, private String roleExternalId; private String roleSessionName; + private BasicAWSCredentials basicCredentials; + private String accessKeyId; + private String secretKey; + @Override public void configure(Map configs) { AbstractConfig config = new AbstractConfig(STS_CONFIG_DEF, configs); roleArn = config.getString(ROLE_ARN_CONFIG); roleExternalId = config.getString(ROLE_EXTERNAL_ID_CONFIG); roleSessionName = config.getString(ROLE_SESSION_NAME_CONFIG); + final String accessKeyId = config.getString(AWS_ACCESS_KEY_ID_CONFIG); + final String secretKey = config.getPassword(AWS_SECRET_ACCESS_KEY_CONFIG).value(); + if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKey)) { + basicCredentials = new BasicAWSCredentials(accessKeyId, secretKey); + } else { + basicCredentials = null; + } } @Override public AWSCredentials getCredentials() { - return new STSAssumeRoleSessionCredentialsProvider.Builder(roleArn, roleSessionName) - .withStsClient(AWSSecurityTokenServiceClientBuilder.defaultClient()) - .withExternalId(roleExternalId) - .build() - .getCredentials(); + if (basicCredentials != null) { + return new STSAssumeRoleSessionCredentialsProvider.Builder(roleArn, roleSessionName) + .withStsClient(AWSSecurityTokenServiceClientBuilder.standard() + .withCredentials(new AWSStaticCredentialsProvider( + new BasicAWSCredentials(accessKeyId, secretKey))).build()) + .withExternalId(roleExternalId) + .build() + .getCredentials(); + } else { + return new STSAssumeRoleSessionCredentialsProvider.Builder(roleArn, roleSessionName) + .withStsClient(AWSSecurityTokenServiceClientBuilder.defaultClient()) + .withExternalId(roleExternalId) + .build() + .getCredentials(); + } } @Override diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3Storage.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3Storage.java index 206741324..c4a52599c 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3Storage.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3Storage.java @@ -19,8 +19,6 @@ import com.amazonaws.PredefinedClientConfigurations; import com.amazonaws.SdkClientException; import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.auth.AWSStaticCredentialsProvider; -import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.client.builder.AwsClientBuilder; import com.amazonaws.regions.Regions; import com.amazonaws.retry.PredefinedBackoffStrategies; @@ -47,8 +45,6 @@ import io.confluent.connect.storage.Storage; import io.confluent.connect.storage.common.util.StringUtils; -import static io.confluent.connect.s3.S3SinkConnectorConfig.AWS_ACCESS_KEY_ID_CONFIG; -import static io.confluent.connect.s3.S3SinkConnectorConfig.AWS_SECRET_ACCESS_KEY_CONFIG; import static io.confluent.connect.s3.S3SinkConnectorConfig.REGION_CONFIG; import static io.confluent.connect.s3.S3SinkConnectorConfig.S3_PATH_STYLE_ACCESS_ENABLED_CONFIG; import static io.confluent.connect.s3.S3SinkConnectorConfig.S3_PROXY_URL_CONFIG; @@ -174,18 +170,9 @@ protected RetryPolicy newFullJitterRetryPolicy(S3SinkConnectorConfig config) { } protected AWSCredentialsProvider newCredentialsProvider(S3SinkConnectorConfig config) { - final String accessKeyId = config.getString(AWS_ACCESS_KEY_ID_CONFIG); - final String secretKey = config.getPassword(AWS_SECRET_ACCESS_KEY_CONFIG).value(); - if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKey)) { - log.info("Returning new credentials provider using the access key id and " - + "the secret access key that were directly supplied through the connector's " - + "configuration"); - BasicAWSCredentials basicCredentials = new BasicAWSCredentials(accessKeyId, secretKey); - return new AWSStaticCredentialsProvider(basicCredentials); - } - log.info( - "Returning new credentials provider based on the configured credentials provider class"); - return config.getCredentialsProvider(); + log.info("Returning new credentials provider based on the configured " + + "credentials provider class"); + return config.getCredentialsProvider(config); } @Override diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorConfigTest.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorConfigTest.java index 93b86d6b2..6aac5e465 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorConfigTest.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorConfigTest.java @@ -218,7 +218,7 @@ public void testConfigurableCredentialProvider() { ); connectorConfig = new S3SinkConnectorConfig(properties); - AWSCredentialsProvider credentialsProvider = connectorConfig.getCredentialsProvider(); + AWSCredentialsProvider credentialsProvider = connectorConfig.getCredentialsProvider(connectorConfig); assertEquals(ACCESS_KEY_VALUE, credentialsProvider.getCredentials().getAWSAccessKeyId()); assertEquals(SECRET_KEY_VALUE, credentialsProvider.getCredentials().getAWSSecretKey()); @@ -246,7 +246,7 @@ public void testConfigurableAwsAssumeRoleCredentialsProvider() { connectorConfig = new S3SinkConnectorConfig(properties); AwsAssumeRoleCredentialsProvider credentialsProvider = - (AwsAssumeRoleCredentialsProvider) connectorConfig.getCredentialsProvider(); + (AwsAssumeRoleCredentialsProvider) connectorConfig.getCredentialsProvider(connectorConfig); } @Test @@ -279,7 +279,7 @@ public void testConfigurableCredentialProviderMissingConfigs() { ); connectorConfig = new S3SinkConnectorConfig(properties); - assertThrows("are mandatory configuration properties", ConfigException.class, () -> connectorConfig.getCredentialsProvider()); + assertThrows("are mandatory configuration properties", ConfigException.class, () -> connectorConfig.getCredentialsProvider(connectorConfig)); } @Test @@ -304,7 +304,7 @@ public void testConfigurableAwsAssumeRoleCredentialsProviderMissingConfigs() { connectorConfig = new S3SinkConnectorConfig(properties); AwsAssumeRoleCredentialsProvider credentialsProvider = - (AwsAssumeRoleCredentialsProvider) connectorConfig.getCredentialsProvider(); + (AwsAssumeRoleCredentialsProvider) connectorConfig.getCredentialsProvider(connectorConfig); assertThrows("Missing required configuration", ConfigException.class, () -> credentialsProvider.configure(properties)); } From 5009c06c44184bdb2215b6c35cf96a02214c2732 Mon Sep 17 00:00:00 2001 From: pkuntal Date: Mon, 22 Aug 2022 22:27:27 +0530 Subject: [PATCH 020/169] Using Access key and secret access to create Assume Role Provider. --- .../connect/s3/auth/AwsAssumeRoleCredentialsProvider.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/auth/AwsAssumeRoleCredentialsProvider.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/auth/AwsAssumeRoleCredentialsProvider.java index a14eaf6da..690fc2e9a 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/auth/AwsAssumeRoleCredentialsProvider.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/auth/AwsAssumeRoleCredentialsProvider.java @@ -66,8 +66,6 @@ public class AwsAssumeRoleCredentialsProvider implements AWSCredentialsProvider, private String roleSessionName; private BasicAWSCredentials basicCredentials; - private String accessKeyId; - private String secretKey; @Override public void configure(Map configs) { @@ -89,8 +87,7 @@ public AWSCredentials getCredentials() { if (basicCredentials != null) { return new STSAssumeRoleSessionCredentialsProvider.Builder(roleArn, roleSessionName) .withStsClient(AWSSecurityTokenServiceClientBuilder.standard() - .withCredentials(new AWSStaticCredentialsProvider( - new BasicAWSCredentials(accessKeyId, secretKey))).build()) + .withCredentials(new AWSStaticCredentialsProvider(basicCredentials)).build()) .withExternalId(roleExternalId) .build() .getCredentials(); From 488dc530b782a4d828fa7d04995e7a88ac8107fa Mon Sep 17 00:00:00 2001 From: pkuntal Date: Tue, 23 Aug 2022 00:20:43 +0530 Subject: [PATCH 021/169] Using Access key and secret access to create Assume Role Provider. --- .../io/confluent/connect/s3/S3SinkConnectorConfig.java | 9 ++++++--- .../s3/auth/AwsAssumeRoleCredentialsProvider.java | 4 ++-- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java index a521f41c8..825c77194 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java @@ -787,14 +787,17 @@ public AWSCredentialsProvider getCredentialsProvider(S3SinkConnectorConfig confi configs.remove(CREDENTIALS_PROVIDER_CLASS_CONFIG.substring( CREDENTIALS_PROVIDER_CONFIG_PREFIX.length() )); + + configs.put(AWS_ACCESS_KEY_ID_CONFIG, + config.getString(AWS_ACCESS_KEY_ID_CONFIG)); + configs.put(AWS_SECRET_ACCESS_KEY_CONFIG, + config.getPassword(AWS_SECRET_ACCESS_KEY_CONFIG).value() + ); ((Configurable) provider).configure(configs); } else { final String accessKeyId = config.getString(AWS_ACCESS_KEY_ID_CONFIG); final String secretKey = config.getPassword(AWS_SECRET_ACCESS_KEY_CONFIG).value(); if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKey)) { - /*log.info("Returning new credentials provider using the access key id and " - + "the secret access key that were directly supplied through the connector's " - + "configuration"); */ BasicAWSCredentials basicCredentials = new BasicAWSCredentials(accessKeyId, secretKey); provider = new AWSStaticCredentialsProvider(basicCredentials); } diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/auth/AwsAssumeRoleCredentialsProvider.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/auth/AwsAssumeRoleCredentialsProvider.java index 690fc2e9a..2dc6b23da 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/auth/AwsAssumeRoleCredentialsProvider.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/auth/AwsAssumeRoleCredentialsProvider.java @@ -73,8 +73,8 @@ public void configure(Map configs) { roleArn = config.getString(ROLE_ARN_CONFIG); roleExternalId = config.getString(ROLE_EXTERNAL_ID_CONFIG); roleSessionName = config.getString(ROLE_SESSION_NAME_CONFIG); - final String accessKeyId = config.getString(AWS_ACCESS_KEY_ID_CONFIG); - final String secretKey = config.getPassword(AWS_SECRET_ACCESS_KEY_CONFIG).value(); + final String accessKeyId = (String) configs.get(AWS_ACCESS_KEY_ID_CONFIG); + final String secretKey = (String) configs.get(AWS_SECRET_ACCESS_KEY_CONFIG); if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKey)) { basicCredentials = new BasicAWSCredentials(accessKeyId, secretKey); } else { From 905831b66a53b1c830c599864bfb35dafb791e21 Mon Sep 17 00:00:00 2001 From: pkuntal Date: Wed, 24 Aug 2022 18:26:23 +0530 Subject: [PATCH 022/169] Using Access key and secret access to create Assume Role Provider. --- .../connect/s3/S3SinkConnectorConfig.java | 22 ++++++++++++------- .../connect/s3/storage/S3Storage.java | 2 +- .../connect/s3/S3SinkConnectorConfigTest.java | 8 +++---- 3 files changed, 19 insertions(+), 13 deletions(-) diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java index 825c77194..bd7a9eeeb 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java @@ -772,12 +772,20 @@ public CannedAccessControlList getCannedAcl() { return CannedAclValidator.ACLS_BY_HEADER_VALUE.get(getString(ACL_CANNED_CONFIG)); } + public String awsAccessKeyId() { + return getString(AWS_ACCESS_KEY_ID_CONFIG); + } + + public Password awsSecretKeyId() { + return getPassword(AWS_SECRET_ACCESS_KEY_CONFIG); + } + public int getPartSize() { return getInt(PART_SIZE_CONFIG); } @SuppressWarnings("unchecked") - public AWSCredentialsProvider getCredentialsProvider(S3SinkConnectorConfig config) { + public AWSCredentialsProvider getCredentialsProvider() { try { AWSCredentialsProvider provider = ((Class) getClass(S3SinkConnectorConfig.CREDENTIALS_PROVIDER_CLASS_CONFIG)).newInstance(); @@ -788,15 +796,13 @@ public AWSCredentialsProvider getCredentialsProvider(S3SinkConnectorConfig confi CREDENTIALS_PROVIDER_CONFIG_PREFIX.length() )); - configs.put(AWS_ACCESS_KEY_ID_CONFIG, - config.getString(AWS_ACCESS_KEY_ID_CONFIG)); - configs.put(AWS_SECRET_ACCESS_KEY_CONFIG, - config.getPassword(AWS_SECRET_ACCESS_KEY_CONFIG).value() - ); + configs.put(AWS_ACCESS_KEY_ID_CONFIG, awsAccessKeyId()); + configs.put(AWS_SECRET_ACCESS_KEY_CONFIG, awsSecretKeyId().value()); + ((Configurable) provider).configure(configs); } else { - final String accessKeyId = config.getString(AWS_ACCESS_KEY_ID_CONFIG); - final String secretKey = config.getPassword(AWS_SECRET_ACCESS_KEY_CONFIG).value(); + final String accessKeyId = awsAccessKeyId(); + final String secretKey = awsSecretKeyId().value(); if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKey)) { BasicAWSCredentials basicCredentials = new BasicAWSCredentials(accessKeyId, secretKey); provider = new AWSStaticCredentialsProvider(basicCredentials); diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3Storage.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3Storage.java index c4a52599c..6e01c4559 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3Storage.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3Storage.java @@ -172,7 +172,7 @@ protected RetryPolicy newFullJitterRetryPolicy(S3SinkConnectorConfig config) { protected AWSCredentialsProvider newCredentialsProvider(S3SinkConnectorConfig config) { log.info("Returning new credentials provider based on the configured " + "credentials provider class"); - return config.getCredentialsProvider(config); + return config.getCredentialsProvider(); } @Override diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorConfigTest.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorConfigTest.java index 6aac5e465..93b86d6b2 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorConfigTest.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorConfigTest.java @@ -218,7 +218,7 @@ public void testConfigurableCredentialProvider() { ); connectorConfig = new S3SinkConnectorConfig(properties); - AWSCredentialsProvider credentialsProvider = connectorConfig.getCredentialsProvider(connectorConfig); + AWSCredentialsProvider credentialsProvider = connectorConfig.getCredentialsProvider(); assertEquals(ACCESS_KEY_VALUE, credentialsProvider.getCredentials().getAWSAccessKeyId()); assertEquals(SECRET_KEY_VALUE, credentialsProvider.getCredentials().getAWSSecretKey()); @@ -246,7 +246,7 @@ public void testConfigurableAwsAssumeRoleCredentialsProvider() { connectorConfig = new S3SinkConnectorConfig(properties); AwsAssumeRoleCredentialsProvider credentialsProvider = - (AwsAssumeRoleCredentialsProvider) connectorConfig.getCredentialsProvider(connectorConfig); + (AwsAssumeRoleCredentialsProvider) connectorConfig.getCredentialsProvider(); } @Test @@ -279,7 +279,7 @@ public void testConfigurableCredentialProviderMissingConfigs() { ); connectorConfig = new S3SinkConnectorConfig(properties); - assertThrows("are mandatory configuration properties", ConfigException.class, () -> connectorConfig.getCredentialsProvider(connectorConfig)); + assertThrows("are mandatory configuration properties", ConfigException.class, () -> connectorConfig.getCredentialsProvider()); } @Test @@ -304,7 +304,7 @@ public void testConfigurableAwsAssumeRoleCredentialsProviderMissingConfigs() { connectorConfig = new S3SinkConnectorConfig(properties); AwsAssumeRoleCredentialsProvider credentialsProvider = - (AwsAssumeRoleCredentialsProvider) connectorConfig.getCredentialsProvider(connectorConfig); + (AwsAssumeRoleCredentialsProvider) connectorConfig.getCredentialsProvider(); assertThrows("Missing required configuration", ConfigException.class, () -> credentialsProvider.configure(properties)); } From 7484111cbffd893bc7ddb1e4c7e0374abfbd6fa4 Mon Sep 17 00:00:00 2001 From: Manasjyoti Sharma Date: Wed, 7 Sep 2022 18:38:30 +0530 Subject: [PATCH 023/169] CCDB-4880: Adding a S3 Sink Connector config to modify behavior w.r.t tagging errors. (#558) * CCDB-4880: Adding a S3 Sink Connector config to decide whether to ignore object tagging errors or not. * CCDB-4880: Adding s3.object.behavior.on.tagging.error & UTs for same. * CCDB-4880: Adding retries for TopicPartitionWriter#tagFile method. * CCDB-4880: Incorporating review comments to rename IgnoreFailBehavior to IgnoreOrFailBehavior. --- README.md | 8 ++ .../connect/s3/S3SinkConnectorConfig.java | 26 +++++- .../io/confluent/connect/s3/S3SinkTask.java | 4 +- .../connect/s3/TopicPartitionWriter.java | 28 +++++- .../confluent/connect/s3/util/RetryUtil.java | 78 +++++++++++++++++ .../connect/s3/S3SinkConnectorConfigTest.java | 10 +++ .../confluent/connect/s3/S3SinkTaskTest.java | 2 +- .../connect/s3/TopicPartitionWriterTest.java | 85 ++++++++++++++++++- .../s3/integration/S3SinkConnectorIT.java | 4 +- 9 files changed, 230 insertions(+), 15 deletions(-) create mode 100644 kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/RetryUtil.java diff --git a/README.md b/README.md index 21582cad8..ae6c2a413 100644 --- a/README.md +++ b/README.md @@ -20,6 +20,14 @@ for guidance on this process. You can build *kafka-connect-storage-cloud* with Maven using the standard lifecycle phases. +# Running Integration Tests +Integration tests are run as part of `mvn install`; however one needs to first configure the environment variable`AWS_CREDENTIALS_PATH` to point to a json file path with following structure: +``` +{ + "aws_access_key_id": "", + "aws_secret_access_key": "" +} +``` # Contribute diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java index e7e06b291..880bcf374 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java @@ -80,6 +80,11 @@ public class S3SinkConnectorConfig extends StorageSinkConnectorConfig { public static final String S3_OBJECT_TAGGING_CONFIG = "s3.object.tagging"; public static final boolean S3_OBJECT_TAGGING_DEFAULT = false; + public static final String S3_OBJECT_BEHAVIOR_ON_TAGGING_ERROR_CONFIG = + "s3.object.behavior.on.tagging.error"; + public static final String S3_OBJECT_BEHAVIOR_ON_TAGGING_ERROR_DEFAULT = + IgnoreOrFailBehavior.IGNORE.toString(); + public static final String SSEA_CONFIG = "s3.ssea.name"; public static final String SSEA_DEFAULT = ""; @@ -152,7 +157,7 @@ public class S3SinkConnectorConfig extends StorageSinkConnectorConfig { ClientConfiguration.DEFAULT_USE_EXPECT_CONTINUE; public static final String BEHAVIOR_ON_NULL_VALUES_CONFIG = "behavior.on.null.values"; - public static final String BEHAVIOR_ON_NULL_VALUES_DEFAULT = BehaviorOnNullValues.FAIL.toString(); + public static final String BEHAVIOR_ON_NULL_VALUES_DEFAULT = IgnoreOrFailBehavior.FAIL.toString(); /** * Maximum back-off time when retrying failed requests. @@ -281,6 +286,19 @@ public static ConfigDef newConfigDef() { "S3 Object Tagging" ); + configDef.define( + S3_OBJECT_BEHAVIOR_ON_TAGGING_ERROR_CONFIG, + Type.STRING, + S3_OBJECT_BEHAVIOR_ON_TAGGING_ERROR_DEFAULT, + IgnoreOrFailBehavior.VALIDATOR, + Importance.LOW, + "How to handle S3 object tagging error. Valid options are 'ignore' and 'fail'.", + group, + ++orderInGroup, + Width.SHORT, + "Behavior for S3 object tagging error" + ); + configDef.define( REGION_CONFIG, Type.STRING, @@ -591,7 +609,7 @@ public static ConfigDef newConfigDef() { BEHAVIOR_ON_NULL_VALUES_CONFIG, Type.STRING, BEHAVIOR_ON_NULL_VALUES_DEFAULT, - BehaviorOnNullValues.VALIDATOR, + IgnoreOrFailBehavior.VALIDATOR, Importance.LOW, "How to handle records with a null value (i.e. Kafka tombstone records)." + " Valid options are 'ignore' and 'fail'.", @@ -1140,7 +1158,7 @@ public String nullValueBehavior() { return getString(BEHAVIOR_ON_NULL_VALUES_CONFIG); } - public enum BehaviorOnNullValues { + public enum IgnoreOrFailBehavior { IGNORE, FAIL; @@ -1164,7 +1182,7 @@ public String toString() { }; public static String[] names() { - BehaviorOnNullValues[] behaviors = values(); + IgnoreOrFailBehavior[] behaviors = values(); String[] result = new String[behaviors.length]; for (int i = 0; i < behaviors.length; i++) { diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java index 7e9532b60..e5ad7fb32 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java @@ -16,7 +16,7 @@ package io.confluent.connect.s3; import com.amazonaws.AmazonClientException; -import io.confluent.connect.s3.S3SinkConnectorConfig.BehaviorOnNullValues; +import io.confluent.connect.s3.S3SinkConnectorConfig.IgnoreOrFailBehavior; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.errors.ConnectException; @@ -249,7 +249,7 @@ public void put(Collection records) throws ConnectException { private boolean maybeSkipOnNullValue(SinkRecord record) { if (record.value() == null) { if (connectorConfig.nullValueBehavior() - .equalsIgnoreCase(BehaviorOnNullValues.IGNORE.toString())) { + .equalsIgnoreCase(IgnoreOrFailBehavior.IGNORE.toString())) { log.debug( "Null valued record from topic '{}', partition {} and offset {} was skipped.", record.topic(), diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/TopicPartitionWriter.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/TopicPartitionWriter.java index f47e222d3..b22e8a1f8 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/TopicPartitionWriter.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/TopicPartitionWriter.java @@ -17,6 +17,7 @@ import com.amazonaws.SdkClientException; import io.confluent.connect.s3.storage.S3Storage; +import io.confluent.connect.s3.util.RetryUtil; import io.confluent.connect.storage.errors.PartitionException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.data.Schema; @@ -53,6 +54,9 @@ import io.confluent.connect.storage.schema.StorageSchemaCompatibility; import io.confluent.connect.storage.util.DateTimeUtils; +import static io.confluent.connect.s3.S3SinkConnectorConfig.S3_PART_RETRIES_CONFIG; +import static io.confluent.connect.s3.S3SinkConnectorConfig.S3_RETRY_BACKOFF_CONFIG; + public class TopicPartitionWriter { private static final Logger log = LoggerFactory.getLogger(TopicPartitionWriter.class); @@ -68,6 +72,7 @@ public class TopicPartitionWriter { private final Queue buffer; private final SinkTaskContext context; private final boolean isTaggingEnabled; + private final boolean ignoreTaggingErrors; private int recordCount; private final int flushSize; private final long rotateIntervalMs; @@ -127,6 +132,9 @@ public TopicPartitionWriter(TopicPartition tp, ? ((TimeBasedPartitioner) partitioner).getTimestampExtractor() : null; isTaggingEnabled = connectorConfig.getBoolean(S3SinkConnectorConfig.S3_OBJECT_TAGGING_CONFIG); + ignoreTaggingErrors = connectorConfig.getString( + S3SinkConnectorConfig.S3_OBJECT_BEHAVIOR_ON_TAGGING_ERROR_CONFIG) + .equalsIgnoreCase(S3SinkConnectorConfig.IgnoreOrFailBehavior.IGNORE.toString()); flushSize = connectorConfig.getInt(S3SinkConnectorConfig.FLUSH_SIZE_CONFIG); topicsDir = connectorConfig.getString(StorageCommonConfig.TOPICS_DIR_CONFIG); rotateIntervalMs = connectorConfig.getLong(S3SinkConnectorConfig.ROTATE_INTERVAL_MS_CONFIG); @@ -599,7 +607,11 @@ private void commitFiles() { String encodedPartition = entry.getKey(); commitFile(encodedPartition); if (isTaggingEnabled) { - tagFile(encodedPartition, entry.getValue()); + RetryUtil.exponentialBackoffRetry(() -> tagFile(encodedPartition, entry.getValue()), + ConnectException.class, + connectorConfig.getInt(S3_PART_RETRIES_CONFIG), + connectorConfig.getLong(S3_RETRY_BACKOFF_CONFIG) + ); } startOffsets.remove(encodedPartition); endOffsets.remove(encodedPartition); @@ -659,10 +671,18 @@ private void tagFile(String encodedPartition, String s3ObjectPath) { log.info("Tagged S3 object {} with starting offset {}, ending offset {}, record count {}", s3ObjectPath, startOffset, endOffset, recordCount); } catch (SdkClientException e) { - log.warn("Unable to tag S3 object {}. Ignoring.", s3ObjectPath, e); + if (ignoreTaggingErrors) { + log.warn("Unable to tag S3 object {}. Ignoring.", s3ObjectPath, e); + } else { + throw new ConnectException(String.format("Unable to tag S3 object %s", s3ObjectPath), e); + } } catch (Exception e) { - log.warn("Unrecoverable exception while attempting to tag S3 object {}. Ignoring.", - s3ObjectPath, e); + if (ignoreTaggingErrors) { + log.warn("Unrecoverable exception while attempting to tag S3 object {}. Ignoring.", + s3ObjectPath, e); + } else { + throw new ConnectException(String.format("Unable to tag S3 object %s", s3ObjectPath), e); + } } } } diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/RetryUtil.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/RetryUtil.java new file mode 100644 index 000000000..fc2a32afd --- /dev/null +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/RetryUtil.java @@ -0,0 +1,78 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.connect.s3.util; + +import org.apache.kafka.connect.errors.ConnectException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.TimeUnit; + +public class RetryUtil { + private static final Logger log = LoggerFactory.getLogger(RetryUtil.class); + + /** + * @param runnable + * An executable piece of code which will be retried in case an expected exception occurred. + * @param exceptionClass + * The function will be retried only if it throws an instance of exceptionClass. + * @param totalNumIterations + * This is the overall execution count e.g. if 3 then it'd be retried a max of twice. + * @param delayInMillis + * Delay between 2 retries, each time it'd be doubled. + */ + public static void exponentialBackoffRetry(final Runnable runnable, + final Class exceptionClass, + final int totalNumIterations, + final long delayInMillis) throws ConnectException { + long expDelayInMillis = delayInMillis; + for (int i = 1; i <= totalNumIterations; i++) { + try { + runnable.run(); + break; + } catch (Exception e) { + if (e.getClass().equals(exceptionClass)) { + log.warn("Attempt {} of {} failed.", i, totalNumIterations, e); + if (i == totalNumIterations) { + wrapAndThrowAsConnectException(e); + } else { + log.warn("Awaiting {} milliseconds before retrying.", expDelayInMillis); + await(expDelayInMillis); + expDelayInMillis <<= 1; + } + } else { + wrapAndThrowAsConnectException(e); + } + } + } + } + + private static void wrapAndThrowAsConnectException(Exception e) throws ConnectException { + if (e instanceof ConnectException) { + throw (ConnectException) e; + } + throw new ConnectException(e); + } + + private static void await(long millis) { + try { + TimeUnit.MILLISECONDS.sleep(millis); + } catch (InterruptedException e) { + // Restore the interrupted status + Thread.currentThread().interrupt(); + } + } +} diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorConfigTest.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorConfigTest.java index 93b86d6b2..4509a2a81 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorConfigTest.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorConfigTest.java @@ -321,6 +321,16 @@ public void testConfigurableS3ObjectTaggingConfigs() { properties.put(S3SinkConnectorConfig.S3_OBJECT_TAGGING_CONFIG, "false"); connectorConfig = new S3SinkConnectorConfig(properties); assertEquals(false, connectorConfig.get(S3SinkConnectorConfig.S3_OBJECT_TAGGING_CONFIG)); + + properties.put(S3SinkConnectorConfig.S3_OBJECT_BEHAVIOR_ON_TAGGING_ERROR_CONFIG, "ignore"); + connectorConfig = new S3SinkConnectorConfig(properties); + assertEquals("ignore", + connectorConfig.get(S3SinkConnectorConfig.S3_OBJECT_BEHAVIOR_ON_TAGGING_ERROR_CONFIG)); + + properties.put(S3SinkConnectorConfig.S3_OBJECT_BEHAVIOR_ON_TAGGING_ERROR_CONFIG, "fail"); + connectorConfig = new S3SinkConnectorConfig(properties); + assertEquals("fail", + connectorConfig.get(S3SinkConnectorConfig.S3_OBJECT_BEHAVIOR_ON_TAGGING_ERROR_CONFIG)); } private void assertDefaultPartitionerVisibility(List values) { diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkTaskTest.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkTaskTest.java index 38b5589cd..2270cc3f0 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkTaskTest.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkTaskTest.java @@ -118,7 +118,7 @@ public void testWriteNullRecords() throws Exception { task.initialize(mockContext); properties.put(S3SinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, - S3SinkConnectorConfig.BehaviorOnNullValues.IGNORE.toString()); + S3SinkConnectorConfig.IgnoreOrFailBehavior.IGNORE.toString()); task.start(properties); verifyAll(); diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java index 048f72a69..826ecbf10 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java @@ -15,6 +15,7 @@ package io.confluent.connect.s3; +import com.amazonaws.SdkClientException; import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.model.S3ObjectSummary; import com.amazonaws.services.s3.model.Tag; @@ -91,8 +92,7 @@ import static org.apache.kafka.common.utils.Time.SYSTEM; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -160,6 +160,30 @@ public S3OutputStream create(String path, boolean overwrite, Class formatClas extension = writerProvider.getExtension(); } + public void setUpWithTaggingException(boolean mockSdkClientException) throws Exception { + super.setUp(); + + s3 = newS3Client(connectorConfig); + storage = new S3Storage(connectorConfig, url, S3_TEST_BUCKET_NAME, s3) { + @Override + public void addTags(String fileName, Map tags) throws SdkClientException { + if (mockSdkClientException) { + throw new SdkClientException("Mock SdkClientException while tagging"); + } + throw new RuntimeException("Mock RuntimeException while tagging"); + } + }; + + format = new AvroFormat(storage); + + s3.createBucket(S3_TEST_BUCKET_NAME); + assertTrue(s3.doesBucketExistV2(S3_TEST_BUCKET_NAME)); + + Format format = new AvroFormat(storage); + writerProvider = format.getRecordWriterProvider(); + extension = writerProvider.getExtension(); + } + @After @Override public void tearDown() throws Exception { @@ -1098,6 +1122,34 @@ public void testAddingS3ObjectTags() throws Exception{ verifyTags(expectedTaggedFiles); } + @Test + public void testIgnoreS3ObjectTaggingSdkClientException() throws Exception { + // Tagging error occurred (SdkClientException) but getting ignored. + testS3ObjectTaggingErrorHelper(true, true); + } + + @Test + public void testIgnoreS3ObjectTaggingRuntimeException() throws Exception { + // Tagging error occurred (RuntimeException) but getting ignored. + testS3ObjectTaggingErrorHelper(false, true); + } + + @Test + public void testFailS3ObjectTaggingSdkClientException() throws Exception { + ConnectException exception = assertThrows(ConnectException.class, + () -> testS3ObjectTaggingErrorHelper(true, false)); + assertEquals("Unable to tag S3 object topics_test-topic_partition=12_test-topic#12#0000000000.avro", exception.getMessage()); + assertEquals("Mock SdkClientException while tagging", exception.getCause().getMessage()); + } + + @Test + public void testFailS3ObjectTaggingRuntimeException() throws Exception { + ConnectException exception = assertThrows(ConnectException.class, () -> + testS3ObjectTaggingErrorHelper(false, false)); + assertEquals("Unable to tag S3 object topics_test-topic_partition=12_test-topic#12#0000000000.avro", exception.getMessage()); + assertEquals("Mock RuntimeException while tagging", exception.getCause().getMessage()); + } + @Test public void testExceptionOnNullKeysReported() throws Exception { String recordValue = "1"; @@ -1636,6 +1688,35 @@ private void verifyTags(Map> expectedTaggedFiles) } } + private void testS3ObjectTaggingErrorHelper(boolean mockSdkClientException, boolean ignoreTaggingError) throws Exception { + // Enabling tagging and setting behavior for tagging error. + localProps.put(S3SinkConnectorConfig.S3_OBJECT_TAGGING_CONFIG, "true"); + localProps.put(S3SinkConnectorConfig.S3_OBJECT_BEHAVIOR_ON_TAGGING_ERROR_CONFIG, ignoreTaggingError ? "ignore" : "fail"); + + // Setup mock exception while tagging + setUpWithTaggingException(mockSdkClientException); + + // Define the partitioner + Partitioner partitioner = new DefaultPartitioner<>(); + partitioner.configure(parsedConfig); + TopicPartitionWriter topicPartitionWriter = new TopicPartitionWriter( + TOPIC_PARTITION, storage, writerProvider, partitioner, connectorConfig, context, null); + + String key = "key"; + Schema schema = createSchema(); + List records = createRecordBatches(schema, 3, 3); + + Collection sinkRecords = createSinkRecords(records, key, schema); + + for (SinkRecord record : sinkRecords) { + topicPartitionWriter.buffer(record); + } + + // Invoke write so as to simulate tagging error. + topicPartitionWriter.write(); + topicPartitionWriter.close(); + } + public static class MockedWallclockTimestampExtractor implements TimestampExtractor { public final MockTime time; diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java index a5487d0ec..a4bb0ca7c 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java @@ -42,7 +42,7 @@ import com.google.common.collect.ImmutableMap; import io.confluent.connect.s3.S3SinkConnector; -import io.confluent.connect.s3.S3SinkConnectorConfig.BehaviorOnNullValues; +import io.confluent.connect.s3.S3SinkConnectorConfig.IgnoreOrFailBehavior; import io.confluent.connect.s3.format.avro.AvroFormat; import io.confluent.connect.s3.format.json.JsonFormat; import io.confluent.connect.s3.format.parquet.ParquetFormat; @@ -302,7 +302,7 @@ private void testBasicRecordsWritten( @Test public void testFaultyRecordsReportedToDLQ() throws Throwable { props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); - props.put(BEHAVIOR_ON_NULL_VALUES_CONFIG, BehaviorOnNullValues.IGNORE.toString()); + props.put(BEHAVIOR_ON_NULL_VALUES_CONFIG, IgnoreOrFailBehavior.IGNORE.toString()); props.put(STORE_KAFKA_KEYS_CONFIG, "true"); props.put(STORE_KAFKA_HEADERS_CONFIG, "true"); props.put(DLQ_TOPIC_CONFIG, DLQ_TOPIC_NAME); From 86edf289e6098e5186d7366edae61f8df60d8a43 Mon Sep 17 00:00:00 2001 From: Confluent Jenkins Bot Date: Wed, 7 Sep 2022 14:35:17 +0000 Subject: [PATCH 024/169] [maven-release-plugin] prepare branch 10.2.x --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d622eeab0..688a5ab77 100644 --- a/pom.xml +++ b/pom.xml @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - HEAD + 10.2.x From b9baa8afe4542104c534fff7ffcb9c4652d2e5e9 Mon Sep 17 00:00:00 2001 From: Confluent Jenkins Bot Date: Wed, 7 Sep 2022 14:35:23 +0000 Subject: [PATCH 025/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index e0bd68b00..0f1dcabd5 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.0-SNAPSHOT + 10.3.0-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 688a5ab77..54a8a25fb 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.0-SNAPSHOT + 10.3.0-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.2.x + HEAD From 07d5452c361026d2e75444b33abd5eb2873dd3a9 Mon Sep 17 00:00:00 2001 From: Manasjyoti Sharma Date: Wed, 7 Sep 2022 14:50:07 +0000 Subject: [PATCH 026/169] [maven-release-plugin] prepare release v10.2.0 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index e0bd68b00..64a5b9c09 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.0-SNAPSHOT + 10.2.0 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 688a5ab77..0ad505a49 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.0-SNAPSHOT + 10.2.0 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.2.x + v10.2.0 From c29b24235619c03a185f5d4ade390631986e5ba8 Mon Sep 17 00:00:00 2001 From: Manasjyoti Sharma Date: Wed, 7 Sep 2022 14:50:13 +0000 Subject: [PATCH 027/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 64a5b9c09..430dbab30 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.0 + 10.2.1-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 0ad505a49..3d845f370 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.0 + 10.2.1-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.2.0 + 10.2.x From ea26e21a4081000e08c4f1e3e00945c6015db793 Mon Sep 17 00:00:00 2001 From: Siddhartha Agarwal Date: Mon, 3 Oct 2022 05:01:32 +0000 Subject: [PATCH 028/169] [maven-release-plugin] prepare release v10.1.2 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index d21bd2744..cfa94741a 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.2-SNAPSHOT + 10.1.2 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index ecbe3a3e2..8e202724d 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.2-SNAPSHOT + 10.1.2 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + v10.1.2 From 3698e151ebfc0d5e1851626d7b3936163ba7b769 Mon Sep 17 00:00:00 2001 From: Siddhartha Agarwal Date: Mon, 3 Oct 2022 05:01:38 +0000 Subject: [PATCH 029/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index cfa94741a..b32dbd4bd 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.2 + 10.1.3-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 8e202724d..ba5dad3bf 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.2 + 10.1.3-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.1.2 + 10.1.x From 38dd335da9544bcd344918dcaa4a180d492833f2 Mon Sep 17 00:00:00 2001 From: Siddhartha Agarwal Date: Mon, 3 Oct 2022 05:04:51 +0000 Subject: [PATCH 030/169] [maven-release-plugin] prepare release v10.2.1 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 430dbab30..0f31ebc07 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.1-SNAPSHOT + 10.2.1 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index c4d9bf96d..fd1fc1766 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.1-SNAPSHOT + 10.2.1 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.2.x + v10.2.1 From b813810c0cfc8d795cbcea69856daba11c84b44f Mon Sep 17 00:00:00 2001 From: Siddhartha Agarwal Date: Mon, 3 Oct 2022 05:04:56 +0000 Subject: [PATCH 031/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 0f31ebc07..b9635ebca 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.1 + 10.2.2-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index fd1fc1766..f48d5520e 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.1 + 10.2.2-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.2.1 + 10.2.x From 938226034bdd553fb8e6322216585c4673aff87c Mon Sep 17 00:00:00 2001 From: Arihant Jain Date: Fri, 7 Oct 2022 12:47:15 +0000 Subject: [PATCH 032/169] [maven-release-plugin] prepare release v10.1.3 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index b32dbd4bd..ace1ad8f7 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.3-SNAPSHOT + 10.1.3 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index ba5dad3bf..3d3973561 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.3-SNAPSHOT + 10.1.3 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + v10.1.3 From 84f59d10622971f3a08bd4642c1ea7e39870d3b2 Mon Sep 17 00:00:00 2001 From: Arihant Jain Date: Fri, 7 Oct 2022 12:47:21 +0000 Subject: [PATCH 033/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index ace1ad8f7..b24e9d884 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.3 + 10.1.4-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 3d3973561..4716ba7d2 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.3 + 10.1.4-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.1.3 + 10.1.x From 8a40c7a747b8e1a90ce3f0dae301d7c730d5bbca Mon Sep 17 00:00:00 2001 From: Arihant Jain Date: Fri, 7 Oct 2022 16:18:23 +0000 Subject: [PATCH 034/169] [maven-release-plugin] prepare release v10.2.2 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index b9635ebca..37263adad 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.2-SNAPSHOT + 10.2.2 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index f48d5520e..0799667ec 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.2-SNAPSHOT + 10.2.2 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.2.x + v10.2.2 From 87f5095185433b4661ad0579694df5e9f749632a Mon Sep 17 00:00:00 2001 From: Arihant Jain Date: Fri, 7 Oct 2022 16:18:29 +0000 Subject: [PATCH 035/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 37263adad..7d8e3bd5e 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.2 + 10.2.3-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 0799667ec..17b8f07d4 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.2 + 10.2.3-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.2.2 + 10.2.x From 86c3547a7823d25eeca4d3ff2d9459c27efca7f4 Mon Sep 17 00:00:00 2001 From: pkuntal Date: Mon, 7 Nov 2022 12:59:01 +0530 Subject: [PATCH 036/169] bumping hadoop and jettison version --- pom.xml | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4716ba7d2..082b42704 100644 --- a/pom.xml +++ b/pom.xml @@ -64,6 +64,8 @@ 6.1.6 + 3.2.3 + 1.5.1 @@ -121,7 +123,7 @@ io.confluent kafka-connect-storage-common-htrace-core4-shaded - ${kafka.connect.storage.common.version} + ${project.version} com.fasterxml.jackson.core @@ -133,6 +135,11 @@ jackson-core ${jackson.version} + + org.codehaus.jettison + jettison + ${jettison.version} + org.apache.hadoop hadoop-common From 6f81ce10f9a86d86292555d28708ac8abfc8d53b Mon Sep 17 00:00:00 2001 From: pkuntal Date: Mon, 7 Nov 2022 12:59:01 +0530 Subject: [PATCH 037/169] bumping hadoop and jettison version --- pom.xml | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 17b8f07d4..16a4acd25 100644 --- a/pom.xml +++ b/pom.xml @@ -64,6 +64,8 @@ 6.1.6 + 3.2.3 + 1.5.1 @@ -121,7 +123,7 @@ io.confluent kafka-connect-storage-common-htrace-core4-shaded - ${kafka.connect.storage.common.version} + ${project.version} com.fasterxml.jackson.core @@ -133,6 +135,11 @@ jackson-core ${jackson.version} + + org.codehaus.jettison + jettison + ${jettison.version} + org.apache.hadoop hadoop-common From 7dbd090e94a168f19d6422da82228323bfe87481 Mon Sep 17 00:00:00 2001 From: pkuntal Date: Tue, 8 Nov 2022 09:21:50 +0530 Subject: [PATCH 038/169] updating pom.xml to add missing common-lang dependency --- pom.xml | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 082b42704..5cbc44f78 100644 --- a/pom.xml +++ b/pom.xml @@ -123,7 +123,7 @@ io.confluent kafka-connect-storage-common-htrace-core4-shaded - ${project.version} + ${kafka.connect.storage.common.version} com.fasterxml.jackson.core @@ -140,6 +140,12 @@ jettison ${jettison.version} + + + commons-lang + commons-lang + 2.6 + org.apache.hadoop hadoop-common @@ -293,4 +299,4 @@ - + \ No newline at end of file From 806e2fb6bd15e6935a0ace199bb86469958c233e Mon Sep 17 00:00:00 2001 From: pkuntal Date: Tue, 8 Nov 2022 10:36:09 +0530 Subject: [PATCH 039/169] Explicitly add woodstox with 5.4.0 as hadoop-common extracts 5.3.0 version --- pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pom.xml b/pom.xml index 16a4acd25..2f3fd2809 100644 --- a/pom.xml +++ b/pom.xml @@ -66,6 +66,7 @@ 6.1.6 3.2.3 1.5.1 + 5.4.0 @@ -140,6 +141,11 @@ jettison ${jettison.version} + + com.fasterxml.woodstox + woodstox-core + ${woodstox.version} + org.apache.hadoop hadoop-common From 3aebcd11e0d222f5f8911c61de7d683ca7e56411 Mon Sep 17 00:00:00 2001 From: pkuntal Date: Tue, 8 Nov 2022 10:36:34 +0530 Subject: [PATCH 040/169] Revert "Explicitly add woodstox with 5.4.0 as hadoop-common extracts 5.3.0 version" This reverts commit 806e2fb6bd15e6935a0ace199bb86469958c233e. --- pom.xml | 6 ------ 1 file changed, 6 deletions(-) diff --git a/pom.xml b/pom.xml index 2f3fd2809..16a4acd25 100644 --- a/pom.xml +++ b/pom.xml @@ -66,7 +66,6 @@ 6.1.6 3.2.3 1.5.1 - 5.4.0 @@ -141,11 +140,6 @@ jettison ${jettison.version} - - com.fasterxml.woodstox - woodstox-core - ${woodstox.version} - org.apache.hadoop hadoop-common From de9e1f3a512f9856e2e1e2f59714eb3968db6b61 Mon Sep 17 00:00:00 2001 From: pkuntal Date: Tue, 8 Nov 2022 11:24:04 +0530 Subject: [PATCH 041/169] Explicitly add woodstox with 5.4.0 as hadoop-common extracts 5.3.0 version --- pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pom.xml b/pom.xml index 16a4acd25..ee52696cf 100644 --- a/pom.xml +++ b/pom.xml @@ -66,6 +66,7 @@ 6.1.6 3.2.3 1.5.1 + <5.4.0> @@ -140,6 +141,11 @@ jettison ${jettison.version} + + com.fasterxml.woodstox + woodstox-core + ${woodstox.version} + org.apache.hadoop hadoop-common From 971c42238ed0a844ad020ca34610a98364bfb675 Mon Sep 17 00:00:00 2001 From: pkuntal Date: Tue, 8 Nov 2022 11:27:06 +0530 Subject: [PATCH 042/169] Explicitly add woodstox with 5.4.0 as hadoop-common extracts 5.3.0 version --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index ee52696cf..a3f15d759 100644 --- a/pom.xml +++ b/pom.xml @@ -66,7 +66,7 @@ 6.1.6 3.2.3 1.5.1 - <5.4.0> + 5.4.0 From 709f1b5225a0645dab5f48cac17bb895da72f6a4 Mon Sep 17 00:00:00 2001 From: Pooja Kuntal Date: Fri, 11 Nov 2022 12:00:18 +0000 Subject: [PATCH 043/169] [maven-release-plugin] prepare release v10.1.4 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 8 +++----- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index b24e9d884..d91abd111 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.4-SNAPSHOT + 10.1.4 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 4388a8915..73b3afb18 100644 --- a/pom.xml +++ b/pom.xml @@ -14,9 +14,7 @@ ~ WARRANTIES OF ANY KIND, either express or implied. See the License for the ~ specific language governing permissions and limitations under the License. --> - + 4.0.0 @@ -29,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.4-SNAPSHOT + 10.1.4 kafka-connect-storage-cloud Confluent, Inc. @@ -52,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + v10.1.4 From 6cc0f4da0656109ad3a475470d711b111a9052b5 Mon Sep 17 00:00:00 2001 From: Pooja Kuntal Date: Fri, 11 Nov 2022 12:00:23 +0000 Subject: [PATCH 044/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index d91abd111..aa054464c 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.4 + 10.1.5-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 73b3afb18..c591d0eba 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.4 + 10.1.5-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.1.4 + 10.1.x From f349ec3246966d81b4b3cf269e6f0ed311d83e1f Mon Sep 17 00:00:00 2001 From: Pooja Kuntal Date: Fri, 11 Nov 2022 12:03:54 +0000 Subject: [PATCH 045/169] [maven-release-plugin] prepare release v10.2.3 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 8 +++----- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 7d8e3bd5e..dd9c65200 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.3-SNAPSHOT + 10.2.3 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 09a7ec726..5de71e862 100644 --- a/pom.xml +++ b/pom.xml @@ -14,9 +14,7 @@ ~ WARRANTIES OF ANY KIND, either express or implied. See the License for the ~ specific language governing permissions and limitations under the License. --> - + 4.0.0 @@ -29,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.3-SNAPSHOT + 10.2.3 kafka-connect-storage-cloud Confluent, Inc. @@ -52,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.2.x + v10.2.3 From 0957f97cd69a0c5dee347660f99a3617bc31a859 Mon Sep 17 00:00:00 2001 From: Pooja Kuntal Date: Fri, 11 Nov 2022 12:03:59 +0000 Subject: [PATCH 046/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index dd9c65200..cd51e2d92 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.3 + 10.2.4-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 5de71e862..35c56cb9c 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.3 + 10.2.4-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.2.3 + 10.2.x From 22d0312a2266e731e81ae564c26b65249b4220da Mon Sep 17 00:00:00 2001 From: Pooja Kuntal Date: Thu, 17 Nov 2022 07:20:26 +0000 Subject: [PATCH 047/169] [maven-release-plugin] prepare release v10.1.5 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index aa054464c..f0f5d1d5a 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.5-SNAPSHOT + 10.1.5 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 02768aade..9b76ba697 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.5-SNAPSHOT + 10.1.5 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + v10.1.5 From f85ed9535275df9c0ab07b1f990c71ea5651b31b Mon Sep 17 00:00:00 2001 From: Pooja Kuntal Date: Thu, 17 Nov 2022 07:20:33 +0000 Subject: [PATCH 048/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index f0f5d1d5a..b7cdcb0cd 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.5 + 10.1.6-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 9b76ba697..f730ba946 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.5 + 10.1.6-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.1.5 + 10.1.x From abde7de14bd085e8826996dfd46f3b0dba8c8cbb Mon Sep 17 00:00:00 2001 From: Pooja Kuntal Date: Thu, 17 Nov 2022 07:24:39 +0000 Subject: [PATCH 049/169] [maven-release-plugin] prepare release v10.2.4 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index cd51e2d92..828e45773 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.4-SNAPSHOT + 10.2.4 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 9fd037388..825a519f7 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.4-SNAPSHOT + 10.2.4 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.2.x + v10.2.4 From 38ea1f7f876272f669561f46d200d09039ddd1bf Mon Sep 17 00:00:00 2001 From: Pooja Kuntal Date: Thu, 17 Nov 2022 07:24:45 +0000 Subject: [PATCH 050/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 828e45773..ec19d56eb 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.4 + 10.2.5-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 825a519f7..4c1aee587 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.4 + 10.2.5-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.2.4 + 10.2.x From 2d29cb02e07026ac0b035c93e472171fadc01f25 Mon Sep 17 00:00:00 2001 From: pkuntal Date: Thu, 17 Nov 2022 18:05:27 +0530 Subject: [PATCH 051/169] [maven-release-plugin] prepare branch 10.3.x --- pom.xml | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index 80cb5b51f..5dfffbbf1 100644 --- a/pom.xml +++ b/pom.xml @@ -14,9 +14,7 @@ ~ WARRANTIES OF ANY KIND, either express or implied. See the License for the ~ specific language governing permissions and limitations under the License. --> - + 4.0.0 @@ -52,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - HEAD + 10.3.x From 7366de0914d5bb33cc6debb867006f49f103793f Mon Sep 17 00:00:00 2001 From: pkuntal Date: Thu, 17 Nov 2022 18:05:47 +0530 Subject: [PATCH 052/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 0f1dcabd5..e64fb7d4d 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.0-SNAPSHOT + 10.4.0-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 5dfffbbf1..f4a5e3716 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.0-SNAPSHOT + 10.4.0-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.3.x + HEAD From 2d0dd1f5f2902db92a57dcf0b4b5c195bf1aa14a Mon Sep 17 00:00:00 2001 From: Pooja Kuntal Date: Fri, 18 Nov 2022 05:51:22 +0000 Subject: [PATCH 053/169] [maven-release-plugin] prepare release v10.3.0 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 0f1dcabd5..6bdf61973 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.0-SNAPSHOT + 10.3.0 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 5dfffbbf1..6e5ce124c 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.0-SNAPSHOT + 10.3.0 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.3.x + v10.3.0 From 80505b8ce3a0c50ac9247b9ce85922662d217ac6 Mon Sep 17 00:00:00 2001 From: Pooja Kuntal Date: Fri, 18 Nov 2022 05:51:28 +0000 Subject: [PATCH 054/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 6bdf61973..ba4f5997f 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.0 + 10.3.1-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 6e5ce124c..79bc36301 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.0 + 10.3.1-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.3.0 + 10.3.x From 75f56c8db3b49d097dd09cd2849b9861506628cd Mon Sep 17 00:00:00 2001 From: Arihant Jain <105902781+arihant-confluent@users.noreply.github.com> Date: Thu, 1 Dec 2022 16:55:44 +0530 Subject: [PATCH 055/169] CCMSG-2232: Add support for topic record name strategy (#598) * add topic record name strategy * add unit tests * refactor schema partitioner constructor * handle null valueschema case and added more tests * added junit vintage to run unit tests * Update kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java Co-authored-by: Sudesh Wasnik <67974618+sudeshwasnik@users.noreply.github.com> * Update kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/SchemaPartitioner.java Co-authored-by: Sudesh Wasnik <67974618+sudeshwasnik@users.noreply.github.com> * Update kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java Co-authored-by: Sudesh Wasnik <67974618+sudeshwasnik@users.noreply.github.com> * Update kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java Co-authored-by: Sudesh Wasnik <67974618+sudeshwasnik@users.noreply.github.com> * removed system out log Co-authored-by: Sudesh Wasnik <67974618+sudeshwasnik@users.noreply.github.com> --- .../connect/s3/S3SinkConnectorConfig.java | 45 +++ .../io/confluent/connect/s3/S3SinkTask.java | 5 +- .../connect/s3/util/SchemaPartitioner.java | 80 +++++ .../connect/s3/S3SinkConnectorConfigTest.java | 60 +++- .../connect/s3/TopicPartitionWriterTest.java | 316 +++++++++++++++++- .../connect/s3/storage/S3StorageTest.java | 2 +- pom.xml | 7 +- 7 files changed, 495 insertions(+), 20 deletions(-) create mode 100644 kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/SchemaPartitioner.java diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java index 97a9541e7..007cda838 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java @@ -190,6 +190,20 @@ public class S3SinkConnectorConfig extends StorageSinkConnectorConfig { public static final String ELASTIC_BUFFER_INIT_CAPACITY = "s3.elastic.buffer.init.capacity"; public static final int ELASTIC_BUFFER_INIT_CAPACITY_DEFAULT = 128 * 1024; // 128KB + /** + * Append schema name in s3-path + */ + + public static final String SCHEMA_PARTITION_AFFIX_TYPE_CONFIG = + "s3.schema.partition.affix.type"; + public static final String SCHEMA_PARTITION_AFFIX_TYPE_DEFAULT = AffixType.NONE.name(); + public static final String SCHEMA_PARTITION_AFFIX_TYPE_DOC = "Append the record schema name " + + "to prefix or suffix in the s3 path after the topic name." + + " None will not append the schema name in the s3 path."; + + private static final GenericRecommender SCHEMA_PARTITION_AFFIX_TYPE_RECOMMENDER = + new GenericRecommender(); + private final String name; private final Map propertyToConfig = new HashMap<>(); @@ -241,6 +255,9 @@ public class S3SinkConnectorConfig extends StorageSinkConnectorConfig { FieldPartitioner.class ) ); + + SCHEMA_PARTITION_AFFIX_TYPE_RECOMMENDER.addValidValues( + Arrays.stream(AffixType.names()).collect(Collectors.toList())); } public static ConfigDef newConfigDef() { @@ -621,6 +638,20 @@ public static ConfigDef newConfigDef() { Width.SHORT, "Behavior for null-valued records" ); + + configDef.define( + SCHEMA_PARTITION_AFFIX_TYPE_CONFIG, + Type.STRING, + SCHEMA_PARTITION_AFFIX_TYPE_DEFAULT, + ConfigDef.ValidString.in(AffixType.names()), + Importance.LOW, + SCHEMA_PARTITION_AFFIX_TYPE_DOC, + group, + ++orderInGroup, + Width.LONG, + "Schema Partition Affix Type", + SCHEMA_PARTITION_AFFIX_TYPE_RECOMMENDER + ); } { @@ -921,6 +952,10 @@ public Object get(String key) { return map; } + public AffixType getSchemaPartitionAffixType() { + return AffixType.valueOf(getString(SCHEMA_PARTITION_AFFIX_TYPE_CONFIG)); + } + private static class PartRange implements ConfigDef.Validator { // S3 specific limit final int min = 5 * 1024 * 1024; @@ -1220,6 +1255,16 @@ public String toString() { } } + public enum AffixType { + SUFFIX, + PREFIX, + NONE; + + public static String[] names() { + return Arrays.stream(values()).map(AffixType::name).toArray(String[]::new); + } + } + public static void main(String[] args) { System.out.println(getConfig().toEnrichedRst()); } diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java index e5ad7fb32..6d05d4755 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java @@ -17,6 +17,7 @@ import com.amazonaws.AmazonClientException; import io.confluent.connect.s3.S3SinkConnectorConfig.IgnoreOrFailBehavior; +import io.confluent.connect.s3.util.SchemaPartitioner; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.errors.ConnectException; @@ -204,8 +205,10 @@ private Partitioner newPartitioner(S3SinkConnectorConfig config) plainValues.put(originalKey, originals.get(originalKey)); } } + if (config.getSchemaPartitionAffixType() != S3SinkConnectorConfig.AffixType.NONE) { + partitioner = new SchemaPartitioner<>(partitioner); + } partitioner.configure(plainValues); - return partitioner; } diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/SchemaPartitioner.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/SchemaPartitioner.java new file mode 100644 index 000000000..0139121d1 --- /dev/null +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/SchemaPartitioner.java @@ -0,0 +1,80 @@ +/* +* Copyright 2018 Confluent Inc. +* +* Licensed under the Confluent Community License (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.confluent.io/confluent-community-license +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT +* WARRANTIES OF ANY KIND, either express or implied. See the License for the +* specific language governing permissions and limitations under the License. +*/ + +package io.confluent.connect.s3.util; + +import io.confluent.connect.s3.S3SinkConnectorConfig; +import io.confluent.connect.storage.common.StorageCommonConfig; +import io.confluent.connect.storage.partitioner.Partitioner; +import java.util.List; +import java.util.Map; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.sink.SinkRecord; + +import static io.confluent.connect.s3.S3SinkConnectorConfig.SCHEMA_PARTITION_AFFIX_TYPE_CONFIG; + +public class SchemaPartitioner implements Partitioner { + + private final Partitioner delegatePartitioner; + private S3SinkConnectorConfig.AffixType schemaAffixType; + private String delim; + + public SchemaPartitioner(Partitioner delegatePartitioner) { + this.delegatePartitioner = delegatePartitioner; + } + + @Override + public void configure(Map config) { + this.delim = (String) config.get(StorageCommonConfig.DIRECTORY_DELIM_CONFIG); + this.schemaAffixType = S3SinkConnectorConfig.AffixType.valueOf( + (String) config.get(SCHEMA_PARTITION_AFFIX_TYPE_CONFIG)); + delegatePartitioner.configure(config); + } + + @Override + public String encodePartition(SinkRecord sinkRecord) { + String encodePartition = this.delegatePartitioner.encodePartition(sinkRecord); + Schema valueSchema = sinkRecord.valueSchema(); + String valueSchemaName = valueSchema != null ? valueSchema.name() : null; + return generateSchemaBasedPath(encodePartition, valueSchemaName); + } + + @Override + public String encodePartition(SinkRecord sinkRecord, long nowInMillis) { + String encodePartition = this.delegatePartitioner.encodePartition(sinkRecord, nowInMillis); + Schema valueSchema = sinkRecord.valueSchema(); + String valueSchemaName = valueSchema != null ? valueSchema.name() : null; + return generateSchemaBasedPath(encodePartition, valueSchemaName); + } + + private String generateSchemaBasedPath(String encodedPartition, String schemaName) { + if (schemaAffixType == S3SinkConnectorConfig.AffixType.PREFIX) { + return "schema_name=" + schemaName + this.delim + encodedPartition; + } else { + return encodedPartition + this.delim + "schema_name=" + schemaName; + } + } + + @Override + public String generatePartitionedPath(String topic, String encodedPartition) { + return delegatePartitioner.generatePartitionedPath(topic, encodedPartition); + } + + @Override + public List partitionFields() { + return delegatePartitioner.partitionFields(); + } +} \ No newline at end of file diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorConfigTest.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorConfigTest.java index 4509a2a81..cf9dd8bb7 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorConfigTest.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorConfigTest.java @@ -31,6 +31,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import java.util.stream.IntStream; import io.confluent.connect.s3.auth.AwsAssumeRoleCredentialsProvider; @@ -47,8 +48,10 @@ import io.confluent.connect.storage.partitioner.TimeBasedPartitioner; import io.confluent.connect.avro.AvroDataConfig; +import static io.confluent.connect.s3.S3SinkConnectorConfig.AffixType; import static io.confluent.connect.s3.S3SinkConnectorConfig.HEADERS_FORMAT_CLASS_CONFIG; import static io.confluent.connect.s3.S3SinkConnectorConfig.KEYS_FORMAT_CLASS_CONFIG; +import static io.confluent.connect.s3.S3SinkConnectorConfig.SCHEMA_PARTITION_AFFIX_TYPE_CONFIG; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThrows; @@ -112,21 +115,24 @@ public void testRecommendedValues() { TimeBasedPartitioner.class, FieldPartitioner.class ); + List expectedSchemaPartitionerAffixTypes = Arrays.stream( + S3SinkConnectorConfig.AffixType.names()).collect(Collectors.toList()); List values = S3SinkConnectorConfig.getConfig().validate(properties); for (ConfigValue val : values) { - if (val.value() instanceof Class) { - switch (val.name()) { - case StorageCommonConfig.STORAGE_CLASS_CONFIG: - assertEquals(expectedStorageClasses, val.recommendedValues()); - break; - case S3SinkConnectorConfig.FORMAT_CLASS_CONFIG: - assertEquals(expectedFormatClasses, val.recommendedValues()); - break; - case PartitionerConfig.PARTITIONER_CLASS_CONFIG: - assertEquals(expectedPartitionerClasses, val.recommendedValues()); - break; - } + switch (val.name()) { + case StorageCommonConfig.STORAGE_CLASS_CONFIG: + assertEquals(expectedStorageClasses, val.recommendedValues()); + break; + case S3SinkConnectorConfig.FORMAT_CLASS_CONFIG: + assertEquals(expectedFormatClasses, val.recommendedValues()); + break; + case PartitionerConfig.PARTITIONER_CLASS_CONFIG: + assertEquals(expectedPartitionerClasses, val.recommendedValues()); + break; + case SCHEMA_PARTITION_AFFIX_TYPE_CONFIG: + assertEquals(expectedSchemaPartitionerAffixTypes, val.recommendedValues()); + break; } } } @@ -214,7 +220,7 @@ public void testConfigurableCredentialProvider() { ); properties.put( configPrefix.concat(DummyAssertiveCredentialsProvider.CONFIGS_NUM_KEY_NAME), - "3" + "5" ); connectorConfig = new S3SinkConnectorConfig(properties); @@ -571,5 +577,33 @@ public void testHeaderFormatClassSupported() { connectorConfig = new S3SinkConnectorConfig(properties); assertEquals(ParquetFormat.class, connectorConfig.getClass(HEADERS_FORMAT_CLASS_CONFIG)); } + + @Test + public void testSchemaPartitionerAffixTypDefault() { + properties.remove(SCHEMA_PARTITION_AFFIX_TYPE_CONFIG); + connectorConfig = new S3SinkConnectorConfig(properties); + assertEquals(AffixType.NONE, connectorConfig.getSchemaPartitionAffixType()); + } + + @Test + public void testSchemaPartitionerAffixType() { + properties.put(SCHEMA_PARTITION_AFFIX_TYPE_CONFIG, AffixType.NONE.name()); + connectorConfig = new S3SinkConnectorConfig(properties); + assertEquals(AffixType.NONE, connectorConfig.getSchemaPartitionAffixType()); + + properties.put(SCHEMA_PARTITION_AFFIX_TYPE_CONFIG, AffixType.PREFIX.name()); + connectorConfig = new S3SinkConnectorConfig(properties); + assertEquals(AffixType.PREFIX, connectorConfig.getSchemaPartitionAffixType()); + + properties.put(SCHEMA_PARTITION_AFFIX_TYPE_CONFIG, AffixType.SUFFIX.name()); + connectorConfig = new S3SinkConnectorConfig(properties); + assertEquals(S3SinkConnectorConfig.AffixType.SUFFIX, connectorConfig.getSchemaPartitionAffixType()); + } + + @Test(expected = ConfigException.class) + public void testSchemaPartitionerAffixTypeExceptionOnWrongValue() { + properties.put(SCHEMA_PARTITION_AFFIX_TYPE_CONFIG, "Random"); + new S3SinkConnectorConfig(properties); + } } diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java index 826ecbf10..c95ad0284 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/TopicPartitionWriterTest.java @@ -28,10 +28,10 @@ import io.confluent.connect.s3.format.RecordViews.KeyRecordView; import io.confluent.connect.s3.format.json.JsonFormat; import io.confluent.connect.s3.storage.CompressionType; +import io.confluent.connect.s3.util.SchemaPartitioner; import io.confluent.connect.storage.errors.PartitionException; import io.confluent.kafka.serializers.NonRecordContainer; -import java.util.HashSet; -import java.util.Set; + import org.apache.avro.util.Utf8; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.connector.ConnectRecord; @@ -55,13 +55,15 @@ import org.junit.Test; import java.io.IOException; -import java.util.Arrays; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -83,11 +85,19 @@ import io.confluent.connect.storage.partitioner.PartitionerConfig; import io.confluent.connect.storage.partitioner.TimeBasedPartitioner; import io.confluent.connect.storage.partitioner.TimestampExtractor; +import org.junit.experimental.theories.DataPoints; +import org.junit.experimental.theories.FromDataPoints; +import org.junit.experimental.theories.Theories; +import org.junit.experimental.theories.Theory; +import org.junit.runner.RunWith; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; +import static io.confluent.connect.s3.S3SinkConnectorConfig.SCHEMA_PARTITION_AFFIX_TYPE_CONFIG; import static io.confluent.connect.s3.util.Utils.sinkRecordToLoggableString; import static io.confluent.connect.storage.StorageSinkConnectorConfig.FLUSH_SIZE_CONFIG; +import static io.confluent.connect.storage.StorageSinkConnectorConfig.FORMAT_CLASS_CONFIG; +import static io.confluent.connect.storage.common.StorageCommonConfig.DIRECTORY_DELIM_CONFIG; import static io.confluent.connect.storage.partitioner.PartitionerConfig.PARTITION_FIELD_NAME_CONFIG; import static org.apache.kafka.common.utils.Time.SYSTEM; import static org.hamcrest.CoreMatchers.is; @@ -97,6 +107,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; +@RunWith(Theories.class) public class TopicPartitionWriterTest extends TestWithMockedS3 { // The default private static final String ZERO_PAD_FMT = "%010d"; @@ -623,6 +634,26 @@ public void testWallclockUsesBatchTimePartitionBoundary() throws Exception { topicsDir, dirPrefix, TOPIC_PARTITION, 0, extension, ZERO_PAD_FMT)); verify(expectedFiles, 6, schema, records); } + @DataPoints("affixType") + public static S3SinkConnectorConfig.AffixType[] affixTypeValues(){ + return new S3SinkConnectorConfig.AffixType[]{ + S3SinkConnectorConfig.AffixType.PREFIX, S3SinkConnectorConfig.AffixType.SUFFIX + }; + } + + @DataPoints("testWithSchemaData") + public static boolean[] testWithSchemaDataValues(){ + return new boolean[]{true, false}; + } + + @Theory + public void testWriteSchemaPartitionerWithAffix( + @FromDataPoints("affixType")S3SinkConnectorConfig.AffixType affixType, + @FromDataPoints("testWithSchemaData") boolean testWithSchemaData + ) throws Exception { + testWriteSchemaPartitionerWithAffix(testWithSchemaData, affixType); + } + @Test public void testWriteRecordsAfterScheduleRotationExpiryButNoResetShouldGoToSameFile() @@ -1453,6 +1484,246 @@ private RecordWriterProvider getKeyHeaderValueProviderJso ); } + public void testWriteSchemaPartitionerWithAffix( + boolean testWithSchemaData, S3SinkConnectorConfig.AffixType affixType + ) throws Exception { + localProps.put(FLUSH_SIZE_CONFIG, "9"); + localProps.put(FORMAT_CLASS_CONFIG, JsonFormat.class.getName()); + setUp(); + + Format myFormat = new JsonFormat(storage); + writerProvider = myFormat.getRecordWriterProvider(); + extension = writerProvider.getExtension(); + + parsedConfig.put(SCHEMA_PARTITION_AFFIX_TYPE_CONFIG, affixType.name()); + // Define the partitioner + Partitioner basePartitioner = new DefaultPartitioner<>(); + Partitioner partitioner = new SchemaPartitioner<>(basePartitioner); + partitioner.configure(parsedConfig); + + TopicPartitionWriter topicPartitionWriter = new TopicPartitionWriter( + TOPIC_PARTITION, storage, writerProvider, partitioner, connectorConfig, context, null + ); + + List testData; + if (testWithSchemaData) { + testData = generateTestDataWithSchema(partitioner, affixType); + } else { + testData = generateTestDataWithoutSchema(partitioner, affixType); + } + + String key = (String) testData.get(0); + List actualRecords = (List) testData.get(1); + List expectedRecords = (List) testData.get(2); + List expectedFiles = (List) testData.get(3); + + + for (SinkRecord actualRecord : actualRecords) { + topicPartitionWriter.buffer(actualRecord); + } + // Test actual write + topicPartitionWriter.write(); + topicPartitionWriter.close(); + + verifyWithJsonOutput( + expectedFiles, expectedRecords.size() / expectedFiles.size(), expectedRecords, CompressionType.NONE + ); + } + + private List generateTestDataWithSchema( + Partitioner partitioner, S3SinkConnectorConfig.AffixType affixType + ) { + String key = "key"; + + Schema schema1 = SchemaBuilder.struct() + .name(null) + .version(null) + .field("boolean", Schema.BOOLEAN_SCHEMA) + .field("int", Schema.INT32_SCHEMA) + .field("long", Schema.INT64_SCHEMA) + .field("float", Schema.FLOAT32_SCHEMA) + .field("double", Schema.FLOAT64_SCHEMA) + .build(); + List records1 = createRecordBatches(schema1, 3, 6); + + Schema schema2 = SchemaBuilder.struct() + .name("record1") + .version(1) + .field("boolean", Schema.BOOLEAN_SCHEMA) + .field("int", Schema.INT32_SCHEMA) + .field("long", Schema.INT64_SCHEMA) + .field("float", Schema.FLOAT32_SCHEMA) + .field("double", Schema.FLOAT64_SCHEMA) + .build(); + + List records2 = createRecordBatches(schema2, 3, 6); + + Schema schema3 = SchemaBuilder.struct() + .name("record2") + .version(1) + .field("boolean", Schema.BOOLEAN_SCHEMA) + .field("int", Schema.INT32_SCHEMA) + .field("long", Schema.INT64_SCHEMA) + .field("float", Schema.FLOAT32_SCHEMA) + .field("double", Schema.FLOAT64_SCHEMA) + .build(); + + List records3 = createRecordBatches(schema3, 3, 6); + + ArrayList actualData = new ArrayList<>(); + int offset = 0; + for (int i = 0; i < records1.size(); i++) { + actualData.add( + new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema1, records1.get(i), + offset++ + ) + ); + actualData.add( + new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema2, records2.get(i), + offset++ + ) + ); + actualData.add( + new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema3, records3.get(i), + offset++ + ) + ); + } + List expectedRecords = new ArrayList<>(); + int ibase = 16; + float fbase = 12.2f; + offset = 0; + // The expected sequence of records is constructed taking into account that sorting of files occurs in verify + + for (int i = 0; i < 6; ++i) { + for (int j = 0; j < 3; ++j) { + expectedRecords.add( + new SinkRecord( + TOPIC, PARTITION, Schema.STRING_SCHEMA, + key, schema1, createRecord(schema1, ibase + j, fbase + j), + offset++ + ) + ); + } + } + for (int i = 0; i < 6; ++i) { + for (int j = 0; j < 3; ++j) { + expectedRecords.add( + new SinkRecord( + TOPIC, PARTITION, Schema.STRING_SCHEMA, + key, schema2, createRecord(schema2, ibase + j, fbase + j), + offset++ + ) + ); + } + } + for (int i = 0; i < 6; ++i) { + for (int j = 0; j < 3; ++j) { + expectedRecords.add( + new SinkRecord( + TOPIC, PARTITION, Schema.STRING_SCHEMA, + key, schema3, createRecord(schema3, ibase + j, fbase + j), + offset++ + ) + ); + } + } + + String dirPrefix1 = generateS3DirectoryPathWithDefaultPartitioner( + partitioner, affixType, PARTITION, TOPIC, "null" + ); + String dirPrefix2 = generateS3DirectoryPathWithDefaultPartitioner( + partitioner, affixType, PARTITION, TOPIC, "record1" + ); + String dirPrefix3 = generateS3DirectoryPathWithDefaultPartitioner( + partitioner, affixType, PARTITION, TOPIC, "record2" + ); + List expectedFiles = new ArrayList<>(); + for (int i = 0; i < 54; i += 9) { + expectedFiles.add(FileUtils.fileKeyToCommit( + topicsDir, dirPrefix1, TOPIC_PARTITION, i, extension, ZERO_PAD_FMT + )); + expectedFiles.add(FileUtils.fileKeyToCommit( + topicsDir, dirPrefix2, TOPIC_PARTITION, i + 1, extension, ZERO_PAD_FMT + )); + expectedFiles.add(FileUtils.fileKeyToCommit( + topicsDir, dirPrefix3, TOPIC_PARTITION, i + 2, extension, ZERO_PAD_FMT + )); + } + return Arrays.asList(key, actualData, expectedRecords, expectedFiles); + } + + private List generateTestDataWithoutSchema( + Partitioner partitioner, S3SinkConnectorConfig.AffixType affixType + ) { + String key = "key"; + List records = createJsonRecordsWithoutSchema(18); + + ArrayList actualData = new ArrayList<>(); + int offset = 0; + for (String record : records) { + actualData.add( + new SinkRecord( + TOPIC, PARTITION, Schema.STRING_SCHEMA, key, null, record, offset++ + ) + ); + } + List expectedRecords = new ArrayList<>(actualData); + + String dirPrefix = generateS3DirectoryPathWithDefaultPartitioner( + partitioner, affixType, PARTITION, TOPIC, "null" + ); + List expectedFiles = new ArrayList<>(); + for (int i = 0; i < 18; i += 9) { + expectedFiles.add(FileUtils.fileKeyToCommit( + topicsDir, dirPrefix, TOPIC_PARTITION, i, extension, ZERO_PAD_FMT + )); + } + return Arrays.asList(key, actualData, expectedRecords, expectedFiles); + } + + private String generateS3DirectoryPathWithDefaultPartitioner( + Partitioner basePartitioner, + S3SinkConnectorConfig.AffixType affixType, int partition, + String topic, String schema_name + ) { + if (affixType == S3SinkConnectorConfig.AffixType.SUFFIX) { + return basePartitioner.generatePartitionedPath(topic, + "partition=" + partition + parsedConfig.get(DIRECTORY_DELIM_CONFIG) + + "schema_name" + "=" + schema_name); + } else if (affixType == S3SinkConnectorConfig.AffixType.PREFIX) { + return basePartitioner.generatePartitionedPath(topic, + "schema_name" + "=" + schema_name + + parsedConfig.get(DIRECTORY_DELIM_CONFIG) + "partition=" + partition); + } else { + return basePartitioner.generatePartitionedPath(topic, + "partition=" + partition); + } + } + + protected List createJsonRecordsWithoutSchema(int size) { + ArrayList records = new ArrayList<>(); + int ibase = 16; + float fbase = 12.2f; + for (int i = 0; i < size; ++i) { + String record = "{\"schema\":{\"type\":\"struct\",\"fields\":[ " + + "{\"type\":\"boolean\",\"optional\":true,\"field\":\"booleanField\"}," + + "{\"type\":\"int\",\"optional\":true,\"field\":\"intField\"}," + + "{\"type\":\"long\",\"optional\":true,\"field\":\"longField\"}," + + "{\"type\":\"float\",\"optional\":true,\"field\":\"floatField\"}," + + "{\"type\":\"double\",\"optional\":true,\"field\":\"doubleField\"}]," + + "\"payload\":" + + "{\"booleanField\":\"true\"," + + "\"intField\":" + String.valueOf(ibase + i) + "," + + "\"longField\":" + String.valueOf((long) ibase + i) + "," + + "\"floatField\":" + String.valueOf(fbase + i) + "," + + "\"doubleField\":" + String.valueOf((double) (fbase + i)) + + "}}"; + records.add(record); + } + return records; + } + private Struct createRecord(Schema schema, int ibase, float fbase) { return new Struct(schema) .put("boolean", true) @@ -1549,12 +1820,49 @@ private void verify(List expectedFileKeys, int expectedSize, Schema sche Collection actualRecords = readRecordsAvro(S3_TEST_BUCKET_NAME, fileKey, s3); assertEquals(expectedSize, actualRecords.size()); for (Object avroRecord : actualRecords) { - Object expectedRecord = format.getAvroData().fromConnectData(schema, records.get(index++)); + Object expectedRecord = format.getAvroData().fromConnectData(records.get(index).schema(), records.get(index++)); assertEquals(expectedRecord, avroRecord); } } } + private void verifyWithJsonOutput( + List expectedFileKeys, int expectedSize, + List expectedRecords, CompressionType compressionType + ) throws IOException { + List summaries = listObjects(S3_TEST_BUCKET_NAME, null, s3); + List actualFiles = new ArrayList<>(); + for (S3ObjectSummary summary : summaries) { + String fileKey = summary.getKey(); + actualFiles.add(fileKey); + } + + Collections.sort(actualFiles); + Collections.sort(expectedFileKeys); + assertThat(actualFiles, is(expectedFileKeys)); + + int index = 0; + for (String fileKey : actualFiles) { + Collection actualRecords = readRecordsJson( + S3_TEST_BUCKET_NAME, fileKey, + s3, compressionType + ); + assertEquals(expectedSize, actualRecords.size()); + for (Object currentRecord : actualRecords) { + SinkRecord expectedRecord = expectedRecords.get(index++); + Object expectedValue = expectedRecord.value(); + JsonConverter converter = new JsonConverter(); + converter.configure(Collections.singletonMap("schemas.enable", "false"), false); + ObjectMapper mapper = new ObjectMapper(); + if (expectedValue instanceof Struct) { + byte[] expectedBytes = converter.fromConnectData(TOPIC, expectedRecord.valueSchema(), expectedRecord.value()); + expectedValue = mapper.readValue(expectedBytes, Object.class); + } + assertEquals(expectedValue, currentRecord); + } + } + } + // based on verify() private void verifyRecordElement(List expectedFileKeys, int expectedSize, List records, RecordElement fileType) throws IOException { diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/storage/S3StorageTest.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/storage/S3StorageTest.java index c19e7cec0..31f947967 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/storage/S3StorageTest.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/storage/S3StorageTest.java @@ -142,7 +142,7 @@ public void testUserDefinedCredentialsProvider() throws Exception { String configPrefix = S3SinkConnectorConfig.CREDENTIALS_PROVIDER_CONFIG_PREFIX; localProps.put(configPrefix.concat(DummyAssertiveCredentialsProvider.ACCESS_KEY_NAME), "foo_key"); localProps.put(configPrefix.concat(DummyAssertiveCredentialsProvider.SECRET_KEY_NAME), "bar_secret"); - localProps.put(configPrefix.concat(DummyAssertiveCredentialsProvider.CONFIGS_NUM_KEY_NAME), "3"); + localProps.put(configPrefix.concat(DummyAssertiveCredentialsProvider.CONFIGS_NUM_KEY_NAME), "5"); localProps.put( S3SinkConnectorConfig.CREDENTIALS_PROVIDER_CLASS_CONFIG, DummyAssertiveCredentialsProvider.class.getName() diff --git a/pom.xml b/pom.xml index f4a5e3716..5c70dec39 100644 --- a/pom.xml +++ b/pom.xml @@ -251,9 +251,14 @@ junit test + + org.junit.vintage + junit-vintage-engine + test + org.junit.jupiter - junit-jupiter-engine + junit-jupiter-api test From e09fca5d279023eecdc890facb5b01266346eaf0 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Thu, 17 Nov 2022 15:18:12 +0530 Subject: [PATCH 056/169] CCMSG-2361 | Support writing of tombstone records. --- .../connect/s3/S3SinkConnectorConfig.java | 100 +++++++++++--- .../connect/s3/S3SinkConnectorValidator.java | 11 ++ .../io/confluent/connect/s3/S3SinkTask.java | 18 ++- .../connect/s3/TopicPartitionWriter.java | 15 ++- .../KeyValueHeaderRecordWriterProvider.java | 90 ++++++++----- .../util/TombstoneSupportedPartitioner.java | 55 ++++++++ .../s3/util/TombstoneTimestampExtractor.java | 46 +++++++ .../s3/integration/BaseConnectorIT.java | 38 ++++++ .../s3/integration/S3SinkConnectorIT.java | 127 +++++++++++++++++- 9 files changed, 437 insertions(+), 63 deletions(-) create mode 100644 kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/TombstoneSupportedPartitioner.java create mode 100644 kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/TombstoneTimestampExtractor.java diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java index 007cda838..c3d4d6fa2 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java @@ -30,6 +30,7 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; +import org.apache.kafka.common.config.ConfigDef.Validator; import org.apache.kafka.common.config.ConfigDef.Width; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.types.Password; @@ -160,7 +161,7 @@ public class S3SinkConnectorConfig extends StorageSinkConnectorConfig { ClientConfiguration.DEFAULT_USE_EXPECT_CONTINUE; public static final String BEHAVIOR_ON_NULL_VALUES_CONFIG = "behavior.on.null.values"; - public static final String BEHAVIOR_ON_NULL_VALUES_DEFAULT = IgnoreOrFailBehavior.FAIL.toString(); + public static final String BEHAVIOR_ON_NULL_VALUES_DEFAULT = OutputWriteBehavior.FAIL.toString(); /** * Maximum back-off time when retrying failed requests. @@ -190,6 +191,9 @@ public class S3SinkConnectorConfig extends StorageSinkConnectorConfig { public static final String ELASTIC_BUFFER_INIT_CAPACITY = "s3.elastic.buffer.init.capacity"; public static final int ELASTIC_BUFFER_INIT_CAPACITY_DEFAULT = 128 * 1024; // 128KB + public static final String TOMBSTONE_ENCODED_PARTITION = "tombstone.encoded.partition"; + public static final String TOMBSTONE_ENCODED_PARTITION_DEFAULT = "tombstone"; + /** * Append schema name in s3-path */ @@ -629,16 +633,35 @@ public static ConfigDef newConfigDef() { BEHAVIOR_ON_NULL_VALUES_CONFIG, Type.STRING, BEHAVIOR_ON_NULL_VALUES_DEFAULT, - IgnoreOrFailBehavior.VALIDATOR, + OutputWriteBehavior.VALIDATOR, Importance.LOW, "How to handle records with a null value (i.e. Kafka tombstone records)." - + " Valid options are 'ignore' and 'fail'.", + + " Valid options are 'ignore', 'fail' and 'write'." + + " In case of the write tombstone option, the connector redirects tombstone records" + + " to a separate directory mentioned in the config tombstone.encoded.partition." + + " The storage of keys is mandatory when this option is selected and the file for" + + " values is not generated.", group, ++orderInGroup, Width.SHORT, "Behavior for null-valued records" ); + // This is done to avoid aggressive schema based rotations resulting out of interleaving + // of tombstones with regular records. + configDef.define( + TOMBSTONE_ENCODED_PARTITION, + Type.STRING, + TOMBSTONE_ENCODED_PARTITION_DEFAULT, + Importance.LOW, + "Output partition to write the tombstone records to.", + group, + ++orderInGroup, + Width.SHORT, + "Tombstone Encoded Partition" + ); + + configDef.define( SCHEMA_PARTITION_AFFIX_TYPE_CONFIG, Type.STRING, @@ -663,7 +686,8 @@ public static ConfigDef newConfigDef() { Type.BOOLEAN, false, Importance.LOW, - "Enable or disable writing keys to storage.", + "Enable or disable writing keys to storage. " + + "This config is mandatory when the writing of tombstone records is enabled.", group, ++orderInGroup, Width.SHORT, @@ -926,6 +950,14 @@ public int getElasticBufferInitCap() { return getInt(ELASTIC_BUFFER_INIT_CAPACITY); } + public boolean isTombstoneWriteEnabled() { + return OutputWriteBehavior.WRITE.toString().equalsIgnoreCase(nullValueBehavior()); + } + + public String getTombstoneEncodedPartition() { + return getString(TOMBSTONE_ENCODED_PARTITION); + } + protected static String parseName(Map props) { String nameProp = props.get("name"); return nameProp != null ? nameProp : "S3-sink"; @@ -1219,27 +1251,34 @@ public enum IgnoreOrFailBehavior { IGNORE, FAIL; - public static final ConfigDef.Validator VALIDATOR = new ConfigDef.Validator() { - private final ConfigDef.ValidString validator = ConfigDef.ValidString.in(names()); + public static final ConfigDef.Validator VALIDATOR = new EnumValidator(names()); - @Override - public void ensureValid(String name, Object value) { - if (value instanceof String) { - value = ((String) value).toLowerCase(Locale.ROOT); - } - validator.ensureValid(name, value); - } + public static String[] names() { + IgnoreOrFailBehavior[] behaviors = values(); + String[] result = new String[behaviors.length]; - // Overridden here so that ConfigDef.toEnrichedRst shows possible values correctly - @Override - public String toString() { - return validator.toString(); + for (int i = 0; i < behaviors.length; i++) { + result[i] = behaviors[i].toString(); } - }; + return result; + } + + @Override + public String toString() { + return name().toLowerCase(Locale.ROOT); + } + } + + public enum OutputWriteBehavior { + IGNORE, + FAIL, + WRITE; + + public static final ConfigDef.Validator VALIDATOR = new EnumValidator(names()); public static String[] names() { - IgnoreOrFailBehavior[] behaviors = values(); + OutputWriteBehavior[] behaviors = values(); String[] result = new String[behaviors.length]; for (int i = 0; i < behaviors.length; i++) { @@ -1255,6 +1294,29 @@ public String toString() { } } + private static class EnumValidator implements Validator { + + private final ConfigDef.ValidString validator; + + private EnumValidator(String[] validValues) { + this.validator = ConfigDef.ValidString.in(validValues); + } + + @Override + public void ensureValid(String name, Object value) { + if (value instanceof String) { + value = ((String) value).toLowerCase(Locale.ROOT); + } + validator.ensureValid(name, value); + } + + // Overridden here so that ConfigDef.toEnrichedRst shows possible values correctly + @Override + public String toString() { + return validator.toString(); + } + } + public enum AffixType { SUFFIX, PREFIX, diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorValidator.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorValidator.java index 52a23859d..434af1be6 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorValidator.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorValidator.java @@ -36,6 +36,7 @@ import java.util.Objects; import java.util.Set; +import static io.confluent.connect.s3.S3SinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG; import static io.confluent.connect.s3.S3SinkConnectorConfig.COMPRESSION_TYPE_CONFIG; import static io.confluent.connect.s3.S3SinkConnectorConfig.HEADERS_FORMAT_CLASS_CONFIG; import static io.confluent.connect.s3.S3SinkConnectorConfig.KEYS_FORMAT_CLASS_CONFIG; @@ -90,6 +91,8 @@ public Config validate() { s3SinkConnectorConfig.storeKafkaKeys(), s3SinkConnectorConfig.keysFormatClass(), s3SinkConnectorConfig.storeKafkaHeaders(), s3SinkConnectorConfig.headersFormatClass() ); + validateTombstoneWriter(s3SinkConnectorConfig.isTombstoneWriteEnabled(), + s3SinkConnectorConfig.storeKafkaKeys()); } return new Config(new ArrayList<>(this.valuesByKey.values())); @@ -128,6 +131,14 @@ public void validateCompression(CompressionType compressionType, Class formatCla } } + public void validateTombstoneWriter(boolean isTombstoneWriteEnabled, boolean isStoreKeysEnabled) { + if (isTombstoneWriteEnabled && !isStoreKeysEnabled) { + recordErrors( + "Writing keys to storage is mandatory when tombstone writing is enabled.", + STORE_KAFKA_KEYS_CONFIG, BEHAVIOR_ON_NULL_VALUES_CONFIG); + } + } + private void recordErrors(String message, String... keys) { log.error("Validation Failed with error: " + message); for (String key: keys) { diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java index 6d05d4755..4d52e8e88 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java @@ -16,6 +16,8 @@ package io.confluent.connect.s3; import com.amazonaws.AmazonClientException; +import io.confluent.connect.s3.S3SinkConnectorConfig.OutputWriteBehavior; +import io.confluent.connect.s3.util.TombstoneSupportedPartitioner; import io.confluent.connect.s3.S3SinkConnectorConfig.IgnoreOrFailBehavior; import io.confluent.connect.s3.util.SchemaPartitioner; import org.apache.kafka.clients.consumer.OffsetAndMetadata; @@ -208,6 +210,10 @@ private Partitioner newPartitioner(S3SinkConnectorConfig config) if (config.getSchemaPartitionAffixType() != S3SinkConnectorConfig.AffixType.NONE) { partitioner = new SchemaPartitioner<>(partitioner); } + if (config.isTombstoneWriteEnabled()) { + String tomebstonePartition = config.getTombstoneEncodedPartition(); + partitioner = new TombstoneSupportedPartitioner<>(partitioner, tomebstonePartition); + } partitioner.configure(plainValues); return partitioner; } @@ -252,7 +258,7 @@ public void put(Collection records) throws ConnectException { private boolean maybeSkipOnNullValue(SinkRecord record) { if (record.value() == null) { if (connectorConfig.nullValueBehavior() - .equalsIgnoreCase(IgnoreOrFailBehavior.IGNORE.toString())) { + .equalsIgnoreCase(OutputWriteBehavior.IGNORE.toString())) { log.debug( "Null valued record from topic '{}', partition {} and offset {} was skipped.", record.topic(), @@ -260,7 +266,17 @@ private boolean maybeSkipOnNullValue(SinkRecord record) { record.kafkaOffset() ); return true; + } else if (connectorConfig.nullValueBehavior() + .equalsIgnoreCase(OutputWriteBehavior.WRITE.toString())) { + log.debug( + "Null valued record from topic '{}', partition {} and offset {} was written.", + record.topic(), + record.kafkaPartition(), + record.kafkaOffset() + ); + return false; } else { + // Fail throw new ConnectException("Null valued records are not writeable with current " + S3SinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG + " 'settings."); } diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/TopicPartitionWriter.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/TopicPartitionWriter.java index b22e8a1f8..0b3a391d9 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/TopicPartitionWriter.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/TopicPartitionWriter.java @@ -18,6 +18,7 @@ import com.amazonaws.SdkClientException; import io.confluent.connect.s3.storage.S3Storage; import io.confluent.connect.s3.util.RetryUtil; +import io.confluent.connect.s3.util.TombstoneTimestampExtractor; import io.confluent.connect.storage.errors.PartitionException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.connect.data.Schema; @@ -66,7 +67,7 @@ public class TopicPartitionWriter { private final TopicPartition tp; private final S3Storage storage; private final Partitioner partitioner; - private final TimestampExtractor timestampExtractor; + private TimestampExtractor timestampExtractor; private String topicsDir; private State state; private final Queue buffer; @@ -128,9 +129,15 @@ public TopicPartitionWriter(TopicPartition tp, this.writerProvider = writerProvider; this.partitioner = partitioner; this.reporter = reporter; - this.timestampExtractor = partitioner instanceof TimeBasedPartitioner - ? ((TimeBasedPartitioner) partitioner).getTimestampExtractor() - : null; + this.timestampExtractor = null; + + if (partitioner instanceof TimeBasedPartitioner) { + this.timestampExtractor = ((TimeBasedPartitioner) partitioner).getTimestampExtractor(); + if (connectorConfig.isTombstoneWriteEnabled()) { + this.timestampExtractor = new TombstoneTimestampExtractor(timestampExtractor); + } + } + isTaggingEnabled = connectorConfig.getBoolean(S3SinkConnectorConfig.S3_OBJECT_TAGGING_CONFIG); ignoreTaggingErrors = connectorConfig.getString( S3SinkConnectorConfig.S3_OBJECT_BEHAVIOR_ON_TAGGING_ERROR_CONFIG) diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/KeyValueHeaderRecordWriterProvider.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/KeyValueHeaderRecordWriterProvider.java index d5312f449..9475cbb22 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/KeyValueHeaderRecordWriterProvider.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/KeyValueHeaderRecordWriterProvider.java @@ -17,20 +17,20 @@ package io.confluent.connect.s3.format; -import javax.annotation.Nullable; -import javax.validation.constraints.NotNull; -import org.apache.kafka.connect.errors.DataException; -import org.apache.kafka.connect.sink.SinkRecord; +import static io.confluent.connect.s3.util.Utils.sinkRecordToLoggableString; +import static java.util.Objects.requireNonNull; import io.confluent.connect.s3.S3SinkConnectorConfig; import io.confluent.connect.storage.format.RecordWriter; import io.confluent.connect.storage.format.RecordWriterProvider; +import java.util.Optional; +import javax.annotation.Nullable; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.sink.SinkRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static io.confluent.connect.s3.util.Utils.sinkRecordToLoggableString; -import static java.util.Objects.requireNonNull; - /** * A class that adds a record writer layer to manage writing values, keys and headers * with a single call. It provides an abstraction for writing, committing and @@ -42,7 +42,7 @@ public class KeyValueHeaderRecordWriterProvider private static final Logger log = LoggerFactory.getLogger(KeyValueHeaderRecordWriterProvider.class); - @NotNull + @Nullable private final RecordWriterProvider valueProvider; @Nullable @@ -75,20 +75,26 @@ public RecordWriter getRecordWriter(S3SinkConnectorConfig conf, String filename) ? filename.substring(0, filename.length() - valueProvider.getExtension().length()) : filename; - RecordWriter valueWriter = valueProvider.getRecordWriter(conf, strippedFilename); - RecordWriter keyWriter = - keyProvider == null ? null : keyProvider.getRecordWriter(conf, strippedFilename); - RecordWriter headerWriter = - headerProvider == null ? null : headerProvider.getRecordWriter(conf, strippedFilename); + Optional valueWriter = filename.contains(conf.getTombstoneEncodedPartition()) + ? Optional.empty() : Optional.of(valueProvider.getRecordWriter(conf, strippedFilename)); + Optional keyWriter = Optional.ofNullable(keyProvider) + .map(keyProvider -> keyProvider.getRecordWriter(conf, strippedFilename)); + Optional headerWriter = Optional.ofNullable(headerProvider) + .map(headerProvider -> headerProvider.getRecordWriter(conf, strippedFilename)); return new RecordWriter() { @Override public void write(SinkRecord sinkRecord) { + if (conf.isTombstoneWriteEnabled() && !keyWriter.isPresent()) { + throw new ConnectException( + "Key Writer must be configured when writing tombstone records is enabled."); + } + // The two data exceptions below must be caught before writing the value // to avoid misaligned K/V/H files. // keyWriter != null means writing keys is turned on - if (keyWriter != null && sinkRecord.key() == null) { + if (keyWriter.isPresent() && sinkRecord.key() == null) { throw new DataException( String.format("Key cannot be null for SinkRecord: %s", sinkRecordToLoggableString(sinkRecord)) @@ -96,7 +102,7 @@ public void write(SinkRecord sinkRecord) { } // headerWriter != null means writing headers is turned on - if (headerWriter != null + if (headerWriter.isPresent() && (sinkRecord.headers() == null || sinkRecord.headers().isEmpty())) { throw new DataException( String.format("Headers cannot be null for SinkRecord: %s", @@ -104,35 +110,51 @@ public void write(SinkRecord sinkRecord) { ); } - valueWriter.write(sinkRecord); // null check happens in sink task - if (keyWriter != null) { - keyWriter.write(sinkRecord); + if (sinkRecord.value() == null) { + validateTombstoneWriteEnabled(sinkRecord); + } else { + writeValue(sinkRecord); } - if (headerWriter != null) { - headerWriter.write(sinkRecord); + keyWriter.ifPresent(writer -> writer.write(sinkRecord)); + headerWriter.ifPresent(writer -> writer.write(sinkRecord)); + } + + private void writeValue(SinkRecord sinkRecord) { + if (valueWriter.isPresent()) { + valueWriter.get().write(sinkRecord); + } else { + throw new ConnectException( + String.format("Value writer not configured for SinkRecord: %s." + + " fileName: %s, tombstonePartition: %s", + sinkRecordToLoggableString(sinkRecord), filename, + conf.getTombstoneEncodedPartition()) + ); + } + } + + private void validateTombstoneWriteEnabled(SinkRecord sinkRecord) { + if (conf.isTombstoneWriteEnabled()) { + // Skip the record value writing, the corresponding key should be written. + } else { + throw new ConnectException( + String.format("Tombstone write must be enabled to sink null valued SinkRecord: %s", + sinkRecordToLoggableString(sinkRecord)) + ); } } @Override public void close() { - valueWriter.close(); - if (keyWriter != null) { - keyWriter.close(); - } - if (headerWriter != null) { - headerWriter.close(); - } + valueWriter.ifPresent(RecordWriter::close); + keyWriter.ifPresent(RecordWriter::close); + headerWriter.ifPresent(RecordWriter::close); } @Override public void commit() { - valueWriter.commit(); - if (keyWriter != null) { - keyWriter.commit(); - } - if (headerWriter != null) { - headerWriter.commit(); - } + valueWriter.ifPresent(RecordWriter::commit); + keyWriter.ifPresent(RecordWriter::commit); + headerWriter.ifPresent(RecordWriter::commit); } }; } diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/TombstoneSupportedPartitioner.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/TombstoneSupportedPartitioner.java new file mode 100644 index 000000000..5db33fee0 --- /dev/null +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/TombstoneSupportedPartitioner.java @@ -0,0 +1,55 @@ +/* + * Copyright 2018 Confluent Inc. + * + * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.connect.s3.util; + +import io.confluent.connect.storage.partitioner.DefaultPartitioner; +import io.confluent.connect.storage.partitioner.Partitioner; +import java.util.List; +import java.util.Map; +import org.apache.kafka.connect.sink.SinkRecord; + +public class TombstoneSupportedPartitioner extends DefaultPartitioner { + + private final Partitioner delegatePartitioner; + private final String tombstonePartition; + + public TombstoneSupportedPartitioner(Partitioner delegatePartitioner, + String tombstonePartition) { + this.delegatePartitioner = delegatePartitioner; + this.tombstonePartition = tombstonePartition; + } + + @Override + public void configure(Map map) { + delegatePartitioner.configure(map); + } + + @Override + public String encodePartition(SinkRecord sinkRecord) { + return sinkRecord.value() == null ? this.tombstonePartition + : delegatePartitioner.encodePartition(sinkRecord); + } + + @Override + public String generatePartitionedPath(String s, String s1) { + return delegatePartitioner.generatePartitionedPath(s, s1); + } + + @Override + public List partitionFields() { + return delegatePartitioner.partitionFields(); + } +} diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/TombstoneTimestampExtractor.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/TombstoneTimestampExtractor.java new file mode 100644 index 000000000..f855adc9b --- /dev/null +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/util/TombstoneTimestampExtractor.java @@ -0,0 +1,46 @@ +/* + * Copyright 2018 Confluent Inc. + * + * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.connect.s3.util; + +import io.confluent.connect.storage.partitioner.TimeBasedPartitioner.RecordTimestampExtractor; +import io.confluent.connect.storage.partitioner.TimestampExtractor; +import java.util.Map; +import org.apache.kafka.connect.connector.ConnectRecord; + +public class TombstoneTimestampExtractor implements TimestampExtractor { + + private final TimestampExtractor delegateTimestampExtractor; + private static final TimestampExtractor recordTimestampExtractor + = new RecordTimestampExtractor(); + + public TombstoneTimestampExtractor(TimestampExtractor delegateTimestampExtractor) { + this.delegateTimestampExtractor = delegateTimestampExtractor; + } + + @Override + public void configure(Map map) { + delegateTimestampExtractor.configure(map); + recordTimestampExtractor.configure(map); + } + + @Override + public Long extract(ConnectRecord connectRecord) { + if (connectRecord.value() == null) { + return recordTimestampExtractor.extract(connectRecord); + } + return delegateTimestampExtractor.extract(connectRecord); + } +} diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/BaseConnectorIT.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/BaseConnectorIT.java index 4afc2141a..4f2b97b6b 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/BaseConnectorIT.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/BaseConnectorIT.java @@ -114,6 +114,44 @@ protected List getExpectedFilenames( return expectedFiles; } + /** + * Get a list of the expected filenames containing keys for the tombstone records for the bucket. + *

+ * Format: topics/s3_topic/tombstone/s3_topic+97+0000000001.keys.avro + * + * @param topic the test kafka topic + * @param partition the expected partition for the tests + * @param flushSize the flush size connector config + * @param numRecords the number of records produced in the test + * @param extension the expected extensions of the files including compression (snappy.parquet) + * @param tombstonePartition the expected directory for tombstone records + * @return the list of expected filenames + */ + protected List getExpectedTombstoneFilenames( + String topic, + int partition, + int flushSize, + long numRecords, + String extension, + String tombstonePartition + ) { + int expectedFileCount = (int) numRecords / flushSize; + List expectedFiles = new ArrayList<>(); + for (int offset = 0; offset < expectedFileCount * flushSize; offset += flushSize) { + String filepath = String.format( + "topics/%s/%s/%s+%d+%010d.keys.%s", + topic, + tombstonePartition, + topic, + partition, + offset, + extension + ); + expectedFiles.add(filepath); + } + return expectedFiles; + } + /** * Check if the file names in the bucket have the expected namings. * diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java index a4bb0ca7c..c97419dbf 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/integration/S3SinkConnectorIT.java @@ -16,11 +16,13 @@ package io.confluent.connect.s3.integration; import static io.confluent.connect.s3.S3SinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG; +import static io.confluent.connect.s3.S3SinkConnectorConfig.KEYS_FORMAT_CLASS_CONFIG; import static io.confluent.connect.s3.S3SinkConnectorConfig.S3_BUCKET_CONFIG; import static io.confluent.connect.s3.S3SinkConnectorConfig.STORE_KAFKA_HEADERS_CONFIG; import static io.confluent.connect.s3.S3SinkConnectorConfig.STORE_KAFKA_KEYS_CONFIG; import static io.confluent.connect.s3.S3SinkConnectorConfig.AWS_ACCESS_KEY_ID_CONFIG; import static io.confluent.connect.s3.S3SinkConnectorConfig.AWS_SECRET_ACCESS_KEY_CONFIG; +import static io.confluent.connect.s3.S3SinkConnectorConfig.TOMBSTONE_ENCODED_PARTITION; import static io.confluent.connect.storage.StorageSinkConnectorConfig.FLUSH_SIZE_CONFIG; import static io.confluent.connect.storage.StorageSinkConnectorConfig.FORMAT_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; @@ -43,6 +45,7 @@ import io.confluent.connect.s3.S3SinkConnector; import io.confluent.connect.s3.S3SinkConnectorConfig.IgnoreOrFailBehavior; +import io.confluent.connect.s3.S3SinkConnectorConfig.OutputWriteBehavior; import io.confluent.connect.s3.format.avro.AvroFormat; import io.confluent.connect.s3.format.json.JsonFormat; import io.confluent.connect.s3.format.parquet.ParquetFormat; @@ -129,6 +132,8 @@ public class S3SinkConnectorIT extends BaseConnectorIT { private static final String DLQ_TOPIC_CONFIG = "errors.deadletterqueue.topic.name"; private static final String DLQ_TOPIC_NAME = "DLQ-topic"; + private static final String TOMBSTONE_PARTITION = "TOMBSTONE_PARTITION"; + private static final List KAFKA_TOPICS = Collections.singletonList(DEFAULT_TEST_TOPIC_NAME); private static final long CONSUME_MAX_DURATION_MS = TimeUnit.SECONDS.toMillis(10); private static final int NUM_RECORDS_INSERT = 30; @@ -211,6 +216,17 @@ public void testBasicRecordsWrittenJson() throws Throwable { } @Test + public void testTombstoneRecordsWrittenJson() throws Throwable { + //add test specific props + props.put(FORMAT_CLASS_CONFIG, JsonFormat.class.getName()); + props.put(BEHAVIOR_ON_NULL_VALUES_CONFIG, OutputWriteBehavior.WRITE.toString()); + props.put(STORE_KAFKA_KEYS_CONFIG, "true"); + props.put(KEYS_FORMAT_CLASS_CONFIG, "io.confluent.connect.s3.format.json.JsonFormat"); + props.put(TOMBSTONE_ENCODED_PARTITION, TOMBSTONE_PARTITION); + testTombstoneRecordsWritten(JSON_EXTENSION, false); + } + + public void testFilesWrittenToBucketAvroWithExtInTopic() throws Throwable { //add test specific props props.put(FORMAT_CLASS_CONFIG, AvroFormat.class.getName()); @@ -299,6 +315,60 @@ private void testBasicRecordsWritten( assertTrue(fileContentsAsExpected(TEST_BUCKET_NAME, FLUSH_SIZE_STANDARD, recordValueStruct)); } + private void testTombstoneRecordsWritten( + String expectedFileExtension, + boolean addExtensionInTopic + ) throws Throwable { + final String topicNameWithExt = "other." + expectedFileExtension + ".topic." + expectedFileExtension; + + // Add an extra topic with this extension inside of the name + // Use a TreeSet for test determinism + Set topicNames = new TreeSet<>(KAFKA_TOPICS); + + if (addExtensionInTopic) { + topicNames.add(topicNameWithExt); + connect.kafka().createTopic(topicNameWithExt, 1); + props.replace( + "topics", + props.get("topics") + "," + topicNameWithExt + ); + } + + // start sink connector + connect.configureConnector(CONNECTOR_NAME, props); + // wait for tasks to spin up + EmbeddedConnectUtils.waitForConnectorToStart(connect, CONNECTOR_NAME, Math.min(topicNames.size(), MAX_TASKS)); + + for (String thisTopicName : topicNames) { + // Create and send records to Kafka using the topic name in the current 'thisTopicName' + SinkRecord sampleRecord = getSampleTopicRecord(thisTopicName, null, null); + produceRecordsWithHeadersNoValue(thisTopicName, NUM_RECORDS_INSERT, sampleRecord); + } + + log.info("Waiting for files in S3..."); + int countPerTopic = NUM_RECORDS_INSERT / FLUSH_SIZE_STANDARD; + int expectedTotalFileCount = countPerTopic * topicNames.size(); + waitForFilesInBucket(TEST_BUCKET_NAME, expectedTotalFileCount); + + Set expectedTopicFilenames = new TreeSet<>(); + for (String thisTopicName : topicNames) { + List theseFiles = getExpectedTombstoneFilenames( + thisTopicName, + TOPIC_PARTITION, + FLUSH_SIZE_STANDARD, + NUM_RECORDS_INSERT, + expectedFileExtension, + TOMBSTONE_PARTITION + ); + assertEquals(theseFiles.size(), countPerTopic); + expectedTopicFilenames.addAll(theseFiles); + } + // This check will catch any duplications + assertEquals(expectedTopicFilenames.size(), expectedTotalFileCount); + assertFileNamesValid(TEST_BUCKET_NAME, new ArrayList<>(expectedTopicFilenames)); + assertTrue(keyfileContentsAsExpected(TEST_BUCKET_NAME, FLUSH_SIZE_STANDARD, "\"key\"")); + } + @Test public void testFaultyRecordsReportedToDLQ() throws Throwable { props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); @@ -425,7 +495,7 @@ private void produceRecords( } } - private SinkRecord getSampleTopicRecord(String topicName, Schema recordValueSchema, Struct recordValueStruct ) { + private SinkRecord getSampleTopicRecord(String topicName, Schema recordValueSchema, Struct recordValueStruct) { return new SinkRecord( topicName, TOPIC_PARTITION, @@ -437,7 +507,7 @@ private SinkRecord getSampleTopicRecord(String topicName, Schema recordValueSche ); } - private SinkRecord getSampleRecord(Schema recordValueSchema, Struct recordValueStruct ) { + private SinkRecord getSampleRecord(Schema recordValueSchema, Struct recordValueStruct) { return getSampleTopicRecord(DEFAULT_TEST_TOPIC_NAME, recordValueSchema, recordValueStruct); } @@ -535,6 +605,46 @@ private boolean fileContentsAsExpected( return true; } + private boolean keyfileContentsAsExpected( + String bucketName, + int expectedRowsPerFile, + String expectedKey + ) { + log.info("expectedKey: {}", expectedKey); + for (String fileName : + getS3KeyFileList(S3Client.listObjectsV2(bucketName).getObjectSummaries())) { + String destinationPath = TEST_DOWNLOAD_PATH + fileName; + File downloadedFile = new File(destinationPath); + log.info("Saving file to : {}", destinationPath); + S3Client.getObject(new GetObjectRequest(bucketName, fileName), downloadedFile); + List keyContent = new ArrayList<>(); + try (FileReader fileReader = new FileReader(destinationPath); + BufferedReader bufferedReader = new BufferedReader(fileReader)) { + String line; + while ((line = bufferedReader.readLine()) != null) { + keyContent.add(line); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + if (keyContent.size() != expectedRowsPerFile) { + log.error("Actual number of records in the key file {}, Expected number of records {}", + keyContent.size(), expectedRowsPerFile); + return false; + } + for (String actualKey: keyContent) { + if (!expectedKey.equals(actualKey)) { + log.error("Key {} did not match the contents in the key file {}", expectedKey, actualKey); + return false; + } else { + log.info("Key {} matched the contents in the key file {}", expectedKey, actualKey); + } + } + downloadedFile.delete(); + } + return true; + } + /** * Check if the contents of a downloaded file match the expected row. * @@ -637,9 +747,8 @@ private static List getContentsFromParquet(String filePath) { * @return the rows of the file as JsonNodes */ private static List getContentsFromJson(String filePath) { - try { - FileReader fileReader = new FileReader(new File(filePath)); - BufferedReader bufferedReader = new BufferedReader(fileReader); + try (FileReader fileReader = new FileReader(filePath); + BufferedReader bufferedReader = new BufferedReader(fileReader)) { List fileRows = new ArrayList<>(); String line; while ((line = bufferedReader.readLine()) != null) { @@ -706,6 +815,14 @@ private List getS3FileListValues(List summaries) { .collect(Collectors.toList()); } + private List getS3KeyFileList(List summaries) { + final String includeExtensions = ".keys."; + return summaries.stream() + .filter(summary -> summary.getKey().contains(includeExtensions)) + .map(S3ObjectSummary::getKey) + .collect(Collectors.toList()); + } + private void initializeCustomProducer() { Map producerProps = new HashMap<>(); producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, connect.kafka().bootstrapServers()); From 4b502568b6fac41a0a7edc36914301ec26c2cb4f Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 2 Dec 2022 10:53:09 +0530 Subject: [PATCH 057/169] CCMSG-2361 | Incorporated review comments. --- .../connect/s3/S3SinkConnectorConfig.java | 9 +++-- .../connect/s3/S3SinkConnectorValidator.java | 3 +- .../io/confluent/connect/s3/S3SinkTask.java | 7 ++-- .../KeyValueHeaderRecordWriterProvider.java | 38 ++++++------------- 4 files changed, 23 insertions(+), 34 deletions(-) diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java index c3d4d6fa2..4ab843906 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorConfig.java @@ -637,10 +637,12 @@ public static ConfigDef newConfigDef() { Importance.LOW, "How to handle records with a null value (i.e. Kafka tombstone records)." + " Valid options are 'ignore', 'fail' and 'write'." + + " Ignore would skip the tombstone record and fail would cause the connector task to" + + " throw an exception." + " In case of the write tombstone option, the connector redirects tombstone records" + " to a separate directory mentioned in the config tombstone.encoded.partition." - + " The storage of keys is mandatory when this option is selected and the file for" - + " values is not generated.", + + " The storage of Kafka record keys is mandatory when this option is selected and" + + " the file for values is not generated for tombstone records.", group, ++orderInGroup, Width.SHORT, @@ -654,7 +656,8 @@ public static ConfigDef newConfigDef() { Type.STRING, TOMBSTONE_ENCODED_PARTITION_DEFAULT, Importance.LOW, - "Output partition to write the tombstone records to.", + "Output s3 folder to write the tombstone records to. The configured" + + " partitioner would map tombstone records to this output folder.", group, ++orderInGroup, Width.SHORT, diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorValidator.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorValidator.java index 434af1be6..c1afd9b5a 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorValidator.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorValidator.java @@ -134,7 +134,8 @@ public void validateCompression(CompressionType compressionType, Class formatCla public void validateTombstoneWriter(boolean isTombstoneWriteEnabled, boolean isStoreKeysEnabled) { if (isTombstoneWriteEnabled && !isStoreKeysEnabled) { recordErrors( - "Writing keys to storage is mandatory when tombstone writing is enabled.", + "Writing Kafka record keys to storage is mandatory when tombstone writing is" + + " enabled.", STORE_KAFKA_KEYS_CONFIG, BEHAVIOR_ON_NULL_VALUES_CONFIG); } } diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java index 4d52e8e88..7d6fade21 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java @@ -18,7 +18,6 @@ import com.amazonaws.AmazonClientException; import io.confluent.connect.s3.S3SinkConnectorConfig.OutputWriteBehavior; import io.confluent.connect.s3.util.TombstoneSupportedPartitioner; -import io.confluent.connect.s3.S3SinkConnectorConfig.IgnoreOrFailBehavior; import io.confluent.connect.s3.util.SchemaPartitioner; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; @@ -269,10 +268,12 @@ private boolean maybeSkipOnNullValue(SinkRecord record) { } else if (connectorConfig.nullValueBehavior() .equalsIgnoreCase(OutputWriteBehavior.WRITE.toString())) { log.debug( - "Null valued record from topic '{}', partition {} and offset {} was written.", + "Null valued record from topic '{}', partition {} and offset {} was written in the" + + "partition {}.", record.topic(), record.kafkaPartition(), - record.kafkaOffset() + record.kafkaOffset(), + connectorConfig.getTombstoneEncodedPartition() ); return false; } else { diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/KeyValueHeaderRecordWriterProvider.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/KeyValueHeaderRecordWriterProvider.java index 9475cbb22..d6a30cca4 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/KeyValueHeaderRecordWriterProvider.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/KeyValueHeaderRecordWriterProvider.java @@ -110,37 +110,21 @@ public void write(SinkRecord sinkRecord) { ); } - if (sinkRecord.value() == null) { - validateTombstoneWriteEnabled(sinkRecord); - } else { - writeValue(sinkRecord); - } - keyWriter.ifPresent(writer -> writer.write(sinkRecord)); - headerWriter.ifPresent(writer -> writer.write(sinkRecord)); - } - - private void writeValue(SinkRecord sinkRecord) { if (valueWriter.isPresent()) { valueWriter.get().write(sinkRecord); } else { - throw new ConnectException( - String.format("Value writer not configured for SinkRecord: %s." - + " fileName: %s, tombstonePartition: %s", - sinkRecordToLoggableString(sinkRecord), filename, - conf.getTombstoneEncodedPartition()) - ); - } - } - - private void validateTombstoneWriteEnabled(SinkRecord sinkRecord) { - if (conf.isTombstoneWriteEnabled()) { - // Skip the record value writing, the corresponding key should be written. - } else { - throw new ConnectException( - String.format("Tombstone write must be enabled to sink null valued SinkRecord: %s", - sinkRecordToLoggableString(sinkRecord)) - ); + // Should only encounter tombstones here. + if (sinkRecord.value() != null) { + throw new ConnectException( + String.format("Value writer not configured for SinkRecord: %s." + + " fileName: %s, tombstonePartition: %s", + sinkRecordToLoggableString(sinkRecord), filename, + conf.getTombstoneEncodedPartition()) + ); + } } + keyWriter.ifPresent(writer -> writer.write(sinkRecord)); + headerWriter.ifPresent(writer -> writer.write(sinkRecord)); } @Override From 2f7179c854c658c44399da346dbfd924386fa2ea Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 2 Dec 2022 15:14:26 +0530 Subject: [PATCH 058/169] [maven-release-plugin] prepare branch 10.4.x --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 5c70dec39..69422836e 100644 --- a/pom.xml +++ b/pom.xml @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - HEAD + 10.4.x From 650ed94049557f1e9dde329cee3efc5f1414a24e Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 2 Dec 2022 15:14:38 +0530 Subject: [PATCH 059/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index e64fb7d4d..8599c7244 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.0-SNAPSHOT + 10.5.0-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 69422836e..61b2dd4c6 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.0-SNAPSHOT + 10.5.0-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.4.x + HEAD From 15e4c8deba1d95b005acae089c35a387740dc282 Mon Sep 17 00:00:00 2001 From: Siddhartha Agarwal Date: Thu, 15 Dec 2022 16:08:05 +0530 Subject: [PATCH 060/169] CCMSG-2368: Cache short-lived credentials when using AWSAssumeRoleCredentialProvider (#603) * CCMSG-2368: Cache temp credentials when using AWSAssumeRoleCredentialProvider --- .../AwsAssumeRoleCredentialsProvider.java | 39 ++++++++++++------- 1 file changed, 24 insertions(+), 15 deletions(-) diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/auth/AwsAssumeRoleCredentialsProvider.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/auth/AwsAssumeRoleCredentialsProvider.java index 2dc6b23da..880a1045b 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/auth/AwsAssumeRoleCredentialsProvider.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/auth/AwsAssumeRoleCredentialsProvider.java @@ -67,6 +67,10 @@ public class AwsAssumeRoleCredentialsProvider implements AWSCredentialsProvider, private BasicAWSCredentials basicCredentials; + // STSAssumeRoleSessionCredentialsProvider takes care of refreshing short-lived + // credentials 60 seconds before it's expiry + private STSAssumeRoleSessionCredentialsProvider stsCredentialProvider; + @Override public void configure(Map configs) { AbstractConfig config = new AbstractConfig(STS_CONFIG_DEF, configs); @@ -77,32 +81,37 @@ public void configure(Map configs) { final String secretKey = (String) configs.get(AWS_SECRET_ACCESS_KEY_CONFIG); if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKey)) { basicCredentials = new BasicAWSCredentials(accessKeyId, secretKey); + stsCredentialProvider = new STSAssumeRoleSessionCredentialsProvider + .Builder(roleArn, roleSessionName) + .withStsClient(AWSSecurityTokenServiceClientBuilder + .standard() + .withCredentials(new AWSStaticCredentialsProvider(basicCredentials)).build() + ) + .withExternalId(roleExternalId) + .build(); } else { basicCredentials = null; + stsCredentialProvider = new STSAssumeRoleSessionCredentialsProvider + .Builder(roleArn, roleSessionName) + // default sts client will internally use default credentials chain provider + // https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default + .withStsClient(AWSSecurityTokenServiceClientBuilder.defaultClient()) + .withExternalId(roleExternalId) + .build(); } } @Override public AWSCredentials getCredentials() { - if (basicCredentials != null) { - return new STSAssumeRoleSessionCredentialsProvider.Builder(roleArn, roleSessionName) - .withStsClient(AWSSecurityTokenServiceClientBuilder.standard() - .withCredentials(new AWSStaticCredentialsProvider(basicCredentials)).build()) - .withExternalId(roleExternalId) - .build() - .getCredentials(); - } else { - return new STSAssumeRoleSessionCredentialsProvider.Builder(roleArn, roleSessionName) - .withStsClient(AWSSecurityTokenServiceClientBuilder.defaultClient()) - .withExternalId(roleExternalId) - .build() - .getCredentials(); - } + return stsCredentialProvider.getCredentials(); } @Override public void refresh() { - // Nothing to do really, since we acquire a new session every getCredentials() call. + // performs a force refresh of credentials + if (stsCredentialProvider != null) { + stsCredentialProvider.refresh(); + } } } From 4ffa1c1d99e83a4042ecd72e1976e43b33e7a3d4 Mon Sep 17 00:00:00 2001 From: Siddhartha Agarwal Date: Thu, 15 Dec 2022 12:08:54 +0000 Subject: [PATCH 061/169] [maven-release-plugin] prepare release v10.3.1 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index ba4f5997f..ed3f738f9 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.1-SNAPSHOT + 10.3.1 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index cd7e1d602..59364c446 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.1-SNAPSHOT + 10.3.1 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.3.x + v10.3.1 From b342e23092ecd5f3051865b7c37bc178778994c0 Mon Sep 17 00:00:00 2001 From: Siddhartha Agarwal Date: Thu, 15 Dec 2022 12:08:58 +0000 Subject: [PATCH 062/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index ed3f738f9..93354e1a5 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.1 + 10.3.2-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 59364c446..2e37c38ed 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.1 + 10.3.2-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.3.1 + 10.3.x From c1ef89dbf6bcc7f65e26aba1181dd13833f49f86 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Tue, 14 Feb 2023 14:46:59 +0000 Subject: [PATCH 063/169] [maven-release-plugin] prepare release v10.1.6 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index b7cdcb0cd..b784f9676 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.6-SNAPSHOT + 10.1.6 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 09d1c9352..02afa2c27 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.6-SNAPSHOT + 10.1.6 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + v10.1.6 From 926899e7e6a22093cc32667f08333b535dc0a667 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Tue, 14 Feb 2023 14:47:02 +0000 Subject: [PATCH 064/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index b784f9676..21ddbc6e3 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.6 + 10.1.7-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 02afa2c27..705b205a9 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.6 + 10.1.7-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.1.6 + 10.1.x From 832327f2e7f1994efe1f4af0daaa7ce7f2f796ab Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Tue, 14 Feb 2023 15:53:56 +0000 Subject: [PATCH 065/169] [maven-release-plugin] prepare release v10.2.5 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index ec19d56eb..ab7fdd7d6 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.5-SNAPSHOT + 10.2.5 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 8e0b9402b..c3adba12c 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.5-SNAPSHOT + 10.2.5 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.2.x + v10.2.5 From 1f19d194698e55ce3cb1435215b451c0e599f0e1 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Tue, 14 Feb 2023 15:54:00 +0000 Subject: [PATCH 066/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index ab7fdd7d6..f8c66f641 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.5 + 10.2.6-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index c3adba12c..af597921c 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.5 + 10.2.6-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.2.5 + 10.2.x From 95c668ace48a46e16f92150f17222e5a53da626e Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Tue, 14 Feb 2023 15:55:28 +0000 Subject: [PATCH 067/169] [maven-release-plugin] prepare release v10.3.2 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 93354e1a5..380be6fe7 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.2-SNAPSHOT + 10.3.2 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 28cf81dc5..60a36eb42 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.2-SNAPSHOT + 10.3.2 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.3.x + v10.3.2 From 00829beedeacbd75b4799798d96b0678e9b8ff17 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Tue, 14 Feb 2023 15:55:32 +0000 Subject: [PATCH 068/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 380be6fe7..0155976cc 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.2 + 10.3.3-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 60a36eb42..477b193c9 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.2 + 10.3.3-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.3.2 + 10.3.x From dcd9784fcc08737c1d41b859c6cd2002fb2f3389 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Tue, 14 Feb 2023 17:20:45 +0000 Subject: [PATCH 069/169] [maven-release-plugin] prepare release v10.4.0 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index e64fb7d4d..c4b8fdbd1 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.0-SNAPSHOT + 10.4.0 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index ea4351884..500cd3a36 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.0-SNAPSHOT + 10.4.0 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.4.x + v10.4.0 From f68ca1093eeb9fbe9672f76fbe339f0e636534c0 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Tue, 14 Feb 2023 17:20:50 +0000 Subject: [PATCH 070/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index c4b8fdbd1..0f9538f27 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.0 + 10.4.1-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 500cd3a36..55e628113 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.0 + 10.4.1-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.4.0 + 10.4.x From 9e68ec7854db4c6ed01880d0e2910db00cfb3dfc Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 17 Feb 2023 15:04:10 +0000 Subject: [PATCH 071/169] [maven-release-plugin] prepare release v10.1.7 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 21ddbc6e3..154a96e67 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.7-SNAPSHOT + 10.1.7 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 725bd7496..d0e146cc3 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.7-SNAPSHOT + 10.1.7 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + v10.1.7 From cb05199b0caa1b2139b1bf1f6c4e35ec363ae4bf Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 17 Feb 2023 15:04:14 +0000 Subject: [PATCH 072/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 154a96e67..65f8fb540 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.7 + 10.1.8-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index d0e146cc3..d9a56dddc 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.7 + 10.1.8-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.1.7 + 10.1.x From 267fb7011423a8e4fe41e43fdee5293acc7e4cb9 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 17 Feb 2023 15:05:42 +0000 Subject: [PATCH 073/169] [maven-release-plugin] prepare release v10.2.6 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index f8c66f641..6bd8523c2 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.6-SNAPSHOT + 10.2.6 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index a685288d2..f10b57528 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.6-SNAPSHOT + 10.2.6 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.2.x + v10.2.6 From 9e84ec0db9bdfb7123df9b72c5481678c9aa0847 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 17 Feb 2023 15:05:46 +0000 Subject: [PATCH 074/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 6bd8523c2..7ca4b3a85 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.6 + 10.2.7-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index f10b57528..3f95aeb77 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.6 + 10.2.7-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.2.6 + 10.2.x From e839ad51928671c6c96e56409a3eff75c27a1b6a Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 17 Feb 2023 16:14:55 +0000 Subject: [PATCH 075/169] [maven-release-plugin] prepare release v10.3.3 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 0155976cc..995b10d33 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.3-SNAPSHOT + 10.3.3 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 14880ef1f..f3fa15c36 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.3-SNAPSHOT + 10.3.3 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.3.x + v10.3.3 From 537944683ed1ed0fbe053edce6305de6da889459 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 17 Feb 2023 16:14:58 +0000 Subject: [PATCH 076/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 995b10d33..40f4ef283 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.3 + 10.3.4-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index f3fa15c36..f27587879 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.3 + 10.3.4-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.3.3 + 10.3.x From a94cd4f8c2a60ab2a5200c67aab25efcb7f787f2 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 17 Feb 2023 16:16:24 +0000 Subject: [PATCH 077/169] [maven-release-plugin] prepare release v10.4.1 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 0f9538f27..2aad5d206 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.1-SNAPSHOT + 10.4.1 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 51a6c3cc8..6dc22a8af 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.1-SNAPSHOT + 10.4.1 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.4.x + v10.4.1 From 2bd52f49a62bcfc3fe109bffe08d9adf52fa6646 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 17 Feb 2023 16:16:27 +0000 Subject: [PATCH 078/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 2aad5d206..b2140084e 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.1 + 10.4.2-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 6dc22a8af..81025eeff 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.1 + 10.4.2-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.4.1 + 10.4.x From 743adb159c0f61634a7ab71d8bb7d604fe4ea99c Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 24 Feb 2023 07:45:39 +0000 Subject: [PATCH 079/169] [maven-release-plugin] prepare release v10.1.8 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 65f8fb540..b9a4b8d6d 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.8-SNAPSHOT + 10.1.8 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index a7562a0c7..e95773211 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.8-SNAPSHOT + 10.1.8 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + v10.1.8 From aacb505060a88f8ef3adb917a94c85c22c74b05e Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 24 Feb 2023 07:45:43 +0000 Subject: [PATCH 080/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index b9a4b8d6d..baff17717 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.8 + 10.1.9-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index e95773211..a14a48f15 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.8 + 10.1.9-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.1.8 + 10.1.x From cabc62c7c4279aa426e82ab4a5fd6c4405adab21 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 24 Feb 2023 09:16:07 +0000 Subject: [PATCH 081/169] [maven-release-plugin] prepare release v10.2.7 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 7ca4b3a85..d49a91506 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.7-SNAPSHOT + 10.2.7 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index d7fec347b..cd1db7634 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.7-SNAPSHOT + 10.2.7 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.2.x + v10.2.7 From c066a6ee2ae96831b16968ed6c47810700218921 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 24 Feb 2023 09:16:10 +0000 Subject: [PATCH 082/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index d49a91506..9799f514c 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.7 + 10.2.8-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index cd1db7634..bb405ab68 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.7 + 10.2.8-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.2.7 + 10.2.x From a32e597c8a78c54e2f81d04711a9e2da6fc51a1d Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 24 Feb 2023 09:17:32 +0000 Subject: [PATCH 083/169] [maven-release-plugin] prepare release v10.3.4 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 40f4ef283..8cdb50ce7 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.4-SNAPSHOT + 10.3.4 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 723c478c0..50b000059 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.4-SNAPSHOT + 10.3.4 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.3.x + v10.3.4 From a19ecdaf2937ad1b7f95a0b866dee47215e870bb Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 24 Feb 2023 09:17:36 +0000 Subject: [PATCH 084/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 8cdb50ce7..e7e97ad5c 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.4 + 10.3.5-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 50b000059..cf0061278 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.4 + 10.3.5-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.3.4 + 10.3.x From 8e0ec73325dea0fb5e45f60db5eea5d9bd847b91 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 24 Feb 2023 10:39:45 +0000 Subject: [PATCH 085/169] [maven-release-plugin] prepare release v10.4.2 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index b2140084e..923e891c7 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.2-SNAPSHOT + 10.4.2 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 11b79f691..e06062283 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.2-SNAPSHOT + 10.4.2 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.4.x + v10.4.2 From 3678bebf64273a8e4d22984b209fcc7b4d65494f Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 24 Feb 2023 10:39:49 +0000 Subject: [PATCH 086/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 923e891c7..5d6a5f7da 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.2 + 10.4.3-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index e06062283..5d95ef6dc 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.2 + 10.4.3-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.4.2 + 10.4.x From 5557bc30813dd24ebb150a9d72be9978377f5019 Mon Sep 17 00:00:00 2001 From: Kapil Chhajer Date: Tue, 28 Feb 2023 23:46:23 +0530 Subject: [PATCH 087/169] added service.yml for setting up semaphore pipeline --- service.yml | 14 ++++++++++++++ 1 file changed, 14 insertions(+) create mode 100644 service.yml diff --git a/service.yml b/service.yml new file mode 100644 index 000000000..8caf9cdf0 --- /dev/null +++ b/service.yml @@ -0,0 +1,14 @@ +name: kafka-connect-storage-cloud +lang: unknown +lang_version: unknown +git: + enable: true +github: + enable: true +semaphore: + enable: true + pipeline_enable: false + triggers: + - tags + - branches + branches: [] From 2533b7942f4989d0ab4fe4265f95b78a3ee677b6 Mon Sep 17 00:00:00 2001 From: srishti-saraswat <98515593+srishti-saraswat@users.noreply.github.com> Date: Tue, 4 Apr 2023 15:25:17 +0530 Subject: [PATCH 088/169] CC-19543 Standard logs (#630) * modified logs --- .../confluent/connect/s3/S3SinkConnectorValidator.java | 2 +- .../main/java/io/confluent/connect/s3/S3SinkTask.java | 4 +++- .../io/confluent/connect/s3/format/RecordViews.java | 7 +++++++ .../connect/s3/format/S3RetriableRecordWriter.java | 5 +++++ .../confluent/connect/s3/format/avro/AvroFormat.java | 5 +++++ .../connect/s3/format/bytearray/ByteArrayFormat.java | 5 +++++ .../confluent/connect/s3/format/json/JsonFormat.java | 5 +++++ .../connect/s3/format/parquet/ParquetFormat.java | 5 +++++ .../connect/s3/storage/ElasticByteBuffer.java | 8 +++++++- .../confluent/connect/s3/storage/S3OutputStream.java | 2 +- .../io/confluent/connect/s3/storage/S3Storage.java | 10 +++++++++- 11 files changed, 53 insertions(+), 5 deletions(-) diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorValidator.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorValidator.java index c1afd9b5a..084598ccf 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorValidator.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkConnectorValidator.java @@ -83,7 +83,7 @@ public Config validate() { try { s3SinkConnectorConfig = new S3SinkConnectorConfig(config, connectorConfigs); } catch (ConfigException exception) { - log.trace("Configuration not ready for cross validation."); + log.error("Configuration not ready for cross validation.", exception); } if (s3SinkConnectorConfig != null) { validateCompression( diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java index 7d6fade21..72799f071 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java @@ -116,7 +116,9 @@ public void start(Map props) { } writerProvider = newRecordWriterProvider(connectorConfig); + log.info("Created S3 sink record writer provider."); partitioner = newPartitioner(connectorConfig); + log.info("Created S3 sink partitioner."); open(context.assignment()); try { @@ -127,7 +129,7 @@ public void start(Map props) { } catch (NoSuchMethodError | NoClassDefFoundError | UnsupportedOperationException e) { // Will occur in Connect runtimes earlier than 2.6 log.warn("Connect versions prior to Apache Kafka 2.6 do not support " - + "the errant record reporter"); + + "the errant record reporter", e); } log.info("Started S3 connector task with assigned partitions: {}", diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/RecordViews.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/RecordViews.java index c8968db3a..9168d2b3d 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/RecordViews.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/RecordViews.java @@ -22,11 +22,16 @@ import org.apache.kafka.connect.data.Values; import org.apache.kafka.connect.sink.SinkRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.stream.Collectors; import java.util.stream.StreamSupport; public final class RecordViews { + private static final Logger log = LoggerFactory.getLogger(RecordViews.class); + public static final class ValueRecordView extends BaseRecordView { @Override public Schema getViewSchema(SinkRecord record, boolean enveloped) { @@ -55,6 +60,7 @@ public Schema getViewSchema(SinkRecord record, boolean enveloped) { keySchema = SchemaBuilder.struct().name(KEY_STRUCT_NAME) .field(KEY_FIELD_NAME, keySchema).build(); } + log.debug("Created key record view schema."); return keySchema; } @@ -90,6 +96,7 @@ public Schema getViewSchema(SinkRecord record, boolean enveloped) { headerSchema = SchemaBuilder.struct().name(HEADER_STRUCT_NAME) .field(HEADER_FIELD_NAME, headerSchema).build(); } + log.debug("Created header record view schema."); return headerSchema; } diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/S3RetriableRecordWriter.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/S3RetriableRecordWriter.java index 6c61ac965..5d0ef4695 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/S3RetriableRecordWriter.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/S3RetriableRecordWriter.java @@ -20,6 +20,8 @@ import org.apache.kafka.connect.sink.SinkRecord; import java.io.IOException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static io.confluent.connect.s3.util.S3ErrorUtils.throwConnectException; @@ -29,10 +31,13 @@ * as determined within `throwConnectException()`. */ public class S3RetriableRecordWriter implements RecordWriter { + + private static final Logger log = LoggerFactory.getLogger(S3RetriableRecordWriter.class); private final IORecordWriter writer; public S3RetriableRecordWriter(IORecordWriter writer) { if (writer == null) { + log.debug("S3 Retriable record writer was passed a null writer (IORecordWriter)"); throw new NullPointerException( "S3 Retriable record writer was passed a null writer (IORecordWriter)" ); diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/avro/AvroFormat.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/avro/AvroFormat.java index 26a150dfa..4797fa6b8 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/avro/AvroFormat.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/avro/AvroFormat.java @@ -21,8 +21,11 @@ import io.confluent.connect.storage.format.Format; import io.confluent.connect.storage.format.RecordWriterProvider; import io.confluent.connect.storage.format.SchemaFileReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class AvroFormat implements Format { + private static final Logger log = LoggerFactory.getLogger(AvroFormat.class); private final S3Storage storage; private final AvroData avroData; @@ -38,12 +41,14 @@ public RecordWriterProvider getRecordWriterProvider() { @Override public SchemaFileReader getSchemaFileReader() { + log.debug("Reading schemas from S3 is not currently supported"); throw new UnsupportedOperationException("Reading schemas from S3 is not currently supported"); } @Override @Deprecated public Object getHiveFactory() { + log.debug("Hive integration is not currently supported in S3 Connector"); throw new UnsupportedOperationException( "Hive integration is not currently supported in S3 Connector" ); diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/bytearray/ByteArrayFormat.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/bytearray/ByteArrayFormat.java index 24d20e189..6249f7568 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/bytearray/ByteArrayFormat.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/bytearray/ByteArrayFormat.java @@ -24,8 +24,11 @@ import java.util.HashMap; import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ByteArrayFormat implements Format { + private static final Logger log = LoggerFactory.getLogger(ByteArrayFormat.class); private final S3Storage storage; private final ByteArrayConverter converter; @@ -43,12 +46,14 @@ public RecordWriterProvider getRecordWriterProvider() { @Override public SchemaFileReader getSchemaFileReader() { + log.debug("Reading schemas from S3 is not currently supported"); throw new UnsupportedOperationException("Reading schemas from S3 is not currently supported"); } @Override @Deprecated public Object getHiveFactory() { + log.debug("Hive integration is not currently supported in S3 Connector"); throw new UnsupportedOperationException( "Hive integration is not currently supported in S3 Connector"); } diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/json/JsonFormat.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/json/JsonFormat.java index e12ed181f..a28e0057b 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/json/JsonFormat.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/json/JsonFormat.java @@ -19,6 +19,8 @@ import java.util.HashMap; import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import io.confluent.connect.s3.S3SinkConnectorConfig; import io.confluent.connect.s3.storage.S3Storage; @@ -27,6 +29,7 @@ import io.confluent.connect.storage.format.SchemaFileReader; public class JsonFormat implements Format { + private static final Logger log = LoggerFactory.getLogger(JsonFormat.class); private final S3Storage storage; private final JsonConverter converter; @@ -49,12 +52,14 @@ public RecordWriterProvider getRecordWriterProvider() { @Override public SchemaFileReader getSchemaFileReader() { + log.debug("Reading schemas from S3 is not currently supported"); throw new UnsupportedOperationException("Reading schemas from S3 is not currently supported"); } @Override @Deprecated public Object getHiveFactory() { + log.debug("Hive integration is not currently supported in S3 Connector"); throw new UnsupportedOperationException( "Hive integration is not currently supported in S3 Connector" ); diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/parquet/ParquetFormat.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/parquet/ParquetFormat.java index 7b18731b7..6cc6ded41 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/parquet/ParquetFormat.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/format/parquet/ParquetFormat.java @@ -23,8 +23,11 @@ import io.confluent.connect.storage.format.Format; import io.confluent.connect.storage.format.RecordWriterProvider; import io.confluent.connect.storage.format.SchemaFileReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ParquetFormat implements Format { + private static final Logger log = LoggerFactory.getLogger(ParquetFormat.class); private final S3Storage storage; private final AvroData avroData; @@ -41,12 +44,14 @@ public RecordWriterProvider getRecordWriterProvider() { @Override public SchemaFileReader getSchemaFileReader() { + log.debug("Reading schemas from S3 is not currently supported"); throw new UnsupportedOperationException("Reading schemas from S3 is not currently supported"); } @Override @Deprecated public Object getHiveFactory() { + log.debug("Hive integration is not currently supported in S3 Connector"); throw new UnsupportedOperationException( "Hive integration is not currently supported in S3 Connector" ); diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/ElasticByteBuffer.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/ElasticByteBuffer.java index 505988ed4..b62d39886 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/ElasticByteBuffer.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/ElasticByteBuffer.java @@ -15,6 +15,9 @@ package io.confluent.connect.s3.storage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.nio.BufferOverflowException; /** @@ -23,6 +26,8 @@ */ public class ElasticByteBuffer implements ByteBuf { + private static final Logger log = LoggerFactory.getLogger(ElasticByteBuffer.class); + public static final int INCREMENT_FACTOR = 1; /* logical capacity */ @@ -43,7 +48,8 @@ public ElasticByteBuffer(int capacity, int initPhysicalCap) { } if (initPhysicalCap <= 0) { - throw new IllegalArgumentException("initial physical capacity must greater than zero"); + log.debug("initial physical capacity must be greater than zero"); + throw new IllegalArgumentException("initial physical capacity must be greater than zero"); } this.capacity = capacity; diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3OutputStream.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3OutputStream.java index a8afac74f..97d037f2f 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3OutputStream.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3OutputStream.java @@ -93,7 +93,7 @@ public S3OutputStream(String key, S3SinkConnectorConfig conf, AmazonS3 s3) { this.compressionType = conf.getCompressionType(); this.compressionLevel = conf.getCompressionLevel(); this.position = 0L; - log.debug("Create S3OutputStream for bucket '{}' key '{}'", bucket, key); + log.info("Create S3OutputStream for bucket '{}' key '{}'", bucket, key); } @Override diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3Storage.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3Storage.java index 6e01c4559..16d759dc7 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3Storage.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/storage/S3Storage.java @@ -86,6 +86,7 @@ public S3Storage(S3SinkConnectorConfig conf, String url) { * @return S3 client */ public AmazonS3 newS3Client(S3SinkConnectorConfig config) { + log.info("Creating S3 client."); ClientConfiguration clientConfiguration = newClientConfiguration(config); AmazonS3ClientBuilder builder = AmazonS3ClientBuilder.standard() .withAccelerateModeEnabled(config.getBoolean(WAN_MODE_CONFIG)) @@ -103,7 +104,7 @@ public AmazonS3 newS3Client(S3SinkConnectorConfig config) { new AwsClientBuilder.EndpointConfiguration(url, region) ); } - + log.info("S3 client created"); return builder.build(); } @@ -166,6 +167,7 @@ protected RetryPolicy newFullJitterRetryPolicy(S3SinkConnectorConfig config) { conf.getS3PartRetries(), false ); + log.info("Created a retry policy for the connector"); return retryPolicy; } @@ -195,17 +197,22 @@ public OutputStream create(String path, S3SinkConnectorConfig conf, boolean over } public S3OutputStream create(String path, boolean overwrite, Class formatClass) { + log.info("Creating S3 output stream."); if (!overwrite) { + log.debug("Creating a file without overwriting is not currently supported in S3 Connector"); throw new UnsupportedOperationException( "Creating a file without overwriting is not currently supported in S3 Connector" ); } if (StringUtils.isBlank(path)) { + log.debug("Path can not be empty!"); throw new IllegalArgumentException("Path can not be empty!"); } if (ParquetFormat.class.isAssignableFrom(formatClass)) { + log.info("Create S3ParquetOutputStream for bucket '{}' key '{}'", + this.conf.getBucketName(), path); return new S3ParquetOutputStream(path, this.conf, s3); } else { // currently ignore what is passed as method argument. @@ -256,6 +263,7 @@ public String url() { @Override public SeekableInput open(String path, S3SinkConnectorConfig conf) { + log.debug("File reading is not currently supported in S3 Connector"); throw new UnsupportedOperationException( "File reading is not currently supported in S3 Connector" ); From a5cc26b68fe0ffcb57c80d504bd593c2b93096b2 Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Thu, 13 Apr 2023 10:46:59 +0000 Subject: [PATCH 089/169] [maven-release-plugin] prepare release v10.1.9 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index dc80b6427..28f931462 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.9-SNAPSHOT + 10.1.9 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index ffb887ca0..edd35a568 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.9-SNAPSHOT + 10.1.9 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + v10.1.9 From 4d7fd42ea49d849fd20490ac3815de10e5024922 Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Thu, 13 Apr 2023 10:47:03 +0000 Subject: [PATCH 090/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 28f931462..b8f3aca65 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.9 + 10.1.10-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index edd35a568..79bf44342 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.9 + 10.1.10-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.1.9 + 10.1.x From 663e2edcf1facaa3fde5e4c374a557f538a995b6 Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Thu, 13 Apr 2023 11:15:52 +0000 Subject: [PATCH 091/169] [maven-release-plugin] prepare release v10.2.8 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 919f9e3e6..08e7c5d4b 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.8-SNAPSHOT + 10.2.8 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index c68baac9d..c2559d9b7 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.8-SNAPSHOT + 10.2.8 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.2.x + v10.2.8 From 40775d0208295627eb8850b9a21bfdcc27cdd77b Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Thu, 13 Apr 2023 11:15:55 +0000 Subject: [PATCH 092/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 08e7c5d4b..0fe5341f3 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.8 + 10.2.9-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index c2559d9b7..586b3980f 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.8 + 10.2.9-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.2.8 + 10.2.x From 604c667fdf31923a6bfca91d283d70f748316886 Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Thu, 13 Apr 2023 11:19:14 +0000 Subject: [PATCH 093/169] [maven-release-plugin] prepare release v10.3.5 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index ba52e789d..62aeda367 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.5-SNAPSHOT + 10.3.5 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index ee22b23fc..90773ef6e 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.5-SNAPSHOT + 10.3.5 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.3.x + v10.3.5 From ebe4023ff2f370e79436321c212f087655317cfe Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Thu, 13 Apr 2023 11:19:18 +0000 Subject: [PATCH 094/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 62aeda367..48579b965 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.5 + 10.3.6-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 90773ef6e..ec5502bf8 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.5 + 10.3.6-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.3.5 + 10.3.x From 0516038ddfcd866982d66cbc3f32893d2ccfc4bb Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Thu, 13 Apr 2023 11:34:11 +0000 Subject: [PATCH 095/169] [maven-release-plugin] prepare release v10.4.3 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 59242317b..bc62e5f82 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.3-SNAPSHOT + 10.4.3 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index b8f3c9b03..4f34d05cb 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.3-SNAPSHOT + 10.4.3 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.4.x + v10.4.3 From 1243094fb750ec1fa59dbfac5254ce442c4b261c Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Thu, 13 Apr 2023 11:34:14 +0000 Subject: [PATCH 096/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index bc62e5f82..34d98aa24 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.3 + 10.4.4-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 4f34d05cb..15a2943c0 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.3 + 10.4.4-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.4.3 + 10.4.x From b3d356d0c0c4dd33399c2548766e7fe3522d514c Mon Sep 17 00:00:00 2001 From: kapilchhajer <91519406+kapilchhajer@users.noreply.github.com> Date: Tue, 18 Apr 2023 15:43:25 +0530 Subject: [PATCH 097/169] Revert "added service.yml for setting up semaphore pipeline" --- service.yml | 14 -------------- 1 file changed, 14 deletions(-) delete mode 100644 service.yml diff --git a/service.yml b/service.yml deleted file mode 100644 index 8caf9cdf0..000000000 --- a/service.yml +++ /dev/null @@ -1,14 +0,0 @@ -name: kafka-connect-storage-cloud -lang: unknown -lang_version: unknown -git: - enable: true -github: - enable: true -semaphore: - enable: true - pipeline_enable: false - triggers: - - tags - - branches - branches: [] From 53c4c4dfbb86643b7304460a0c6f128550da748b Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Tue, 16 May 2023 11:42:21 +0530 Subject: [PATCH 098/169] [maven-release-plugin] prepare branch 10.5.x --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index aeb63735f..228d0d197 100644 --- a/pom.xml +++ b/pom.xml @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - HEAD + 10.5.x From 92f491a4d540489453b11f1b34e43b3a1ade8b2d Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Tue, 16 May 2023 07:08:13 +0000 Subject: [PATCH 099/169] [maven-release-plugin] prepare release v10.5.0 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 3545362e5..1c8af18ea 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.5.0-SNAPSHOT + 10.5.0 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 228d0d197..2e544efee 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.5.0-SNAPSHOT + 10.5.0 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.5.x + v10.5.0 From 68f73770af2377677348310a6d9bfdbdbb4ca16b Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Tue, 16 May 2023 07:08:17 +0000 Subject: [PATCH 100/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 1c8af18ea..13949a92a 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.5.0 + 10.5.1-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 2e544efee..4e19b13f5 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.5.0 + 10.5.1-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.5.0 + 10.5.x From 9e6214a6e9708c9a19a8ce09856c0a557e40c8e2 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Thu, 1 Jun 2023 12:31:14 +0000 Subject: [PATCH 101/169] [maven-release-plugin] prepare release v10.1.10 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index b8f3aca65..0ffb1b939 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.10-SNAPSHOT + 10.1.10 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index eb5c30c8b..7a5e0eae5 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.10-SNAPSHOT + 10.1.10 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + v10.1.10 From 4af2454c44f294f11aad2d343250a0fae0eea946 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Thu, 1 Jun 2023 12:31:17 +0000 Subject: [PATCH 102/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 0ffb1b939..25d286b03 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.10 + 10.1.11-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 7a5e0eae5..e9c59e5f7 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.10 + 10.1.11-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.1.10 + 10.1.x From 2811693263f26e18f1b2d7849ded33729e0f284a Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Thu, 1 Jun 2023 15:13:40 +0000 Subject: [PATCH 103/169] [maven-release-plugin] prepare release v10.2.9 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 0fe5341f3..ad31ecc75 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.9-SNAPSHOT + 10.2.9 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index d2736c9ef..dc5d1e160 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.9-SNAPSHOT + 10.2.9 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.2.x + v10.2.9 From a5085a45cef962d29d1a2a9105948365c00d72d0 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Thu, 1 Jun 2023 15:13:43 +0000 Subject: [PATCH 104/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index ad31ecc75..8773bbd2e 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.9 + 10.2.10-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index dc5d1e160..0a8e27029 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.9 + 10.2.10-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.2.9 + 10.2.x From a59584ac209552b532deccd3faffd6c5274cc932 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Thu, 1 Jun 2023 15:15:14 +0000 Subject: [PATCH 105/169] [maven-release-plugin] prepare release v10.3.6 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 48579b965..8b52afa36 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.6-SNAPSHOT + 10.3.6 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index a5f07fc91..785117ef1 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.6-SNAPSHOT + 10.3.6 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.3.x + v10.3.6 From e4bd7e87699f337ad13d59ea27ee3bc2c46fd464 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Thu, 1 Jun 2023 15:15:17 +0000 Subject: [PATCH 106/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 8b52afa36..b78d61404 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.6 + 10.3.7-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 785117ef1..7eaeaa92a 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.6 + 10.3.7-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.3.6 + 10.3.x From 8bc360f23df2f8c4f4c3e66ca908d6a5fd0ff821 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 2 Jun 2023 06:25:37 +0000 Subject: [PATCH 107/169] [maven-release-plugin] prepare release v10.4.4 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 34d98aa24..e1f291333 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.4-SNAPSHOT + 10.4.4 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 08e3b6a16..8f2b5c077 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.4-SNAPSHOT + 10.4.4 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.4.x + v10.4.4 From 59228c3a764fd4e7e1983ee3154691f108afc0b1 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 2 Jun 2023 06:25:41 +0000 Subject: [PATCH 108/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index e1f291333..336fdf9cc 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.4 + 10.4.5-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 8f2b5c077..62cbafbf6 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.4 + 10.4.5-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.4.4 + 10.4.x From be6d9c35d2f58a660e257710381a2b93d8b88de2 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 2 Jun 2023 06:27:13 +0000 Subject: [PATCH 109/169] [maven-release-plugin] prepare release v10.5.1 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 13949a92a..47d618bd8 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.5.1-SNAPSHOT + 10.5.1 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index d184d191a..f6c9c20b9 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.5.1-SNAPSHOT + 10.5.1 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.5.x + v10.5.1 From 59a396fd6b94aec1f7db776786f9791a7bc81eb2 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 2 Jun 2023 06:27:16 +0000 Subject: [PATCH 110/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 47d618bd8..2918b2be1 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.5.1 + 10.5.2-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index f6c9c20b9..8a9b989af 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.5.1 + 10.5.2-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.5.1 + 10.5.x From cf45891d0810a274a610f9080445458c7cf7d73d Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Fri, 11 Aug 2023 09:34:32 +0000 Subject: [PATCH 111/169] [maven-release-plugin] prepare release v10.1.11 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 25d286b03..b3a9e8451 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.11-SNAPSHOT + 10.1.11 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index b3e7037d4..90e49ea41 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.11-SNAPSHOT + 10.1.11 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + v10.1.11 From e97ceaf336a5f856bacca842b4f733e104e4ca42 Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Fri, 11 Aug 2023 09:34:35 +0000 Subject: [PATCH 112/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index b3a9e8451..a01e34b38 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.11 + 10.1.12-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 90e49ea41..003926deb 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.11 + 10.1.12-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.1.11 + 10.1.x From ee285c71200b8305c6e4e8c8618223b9309162dc Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Fri, 11 Aug 2023 09:35:53 +0000 Subject: [PATCH 113/169] [maven-release-plugin] prepare release v10.2.10 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 8773bbd2e..a4ddcdd5a 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.10-SNAPSHOT + 10.2.10 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 2607eef0f..e8f437818 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.10-SNAPSHOT + 10.2.10 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.2.x + v10.2.10 From 2fad7aa5ba1cfff38720c8a08c50d7c7e654986d Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Fri, 11 Aug 2023 09:35:57 +0000 Subject: [PATCH 114/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index a4ddcdd5a..68d92b8a1 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.10 + 10.2.11-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index e8f437818..15e37ab62 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.10 + 10.2.11-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.2.10 + 10.2.x From 212707b4dde8d386e84e75c7e0bce8cb5142816b Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Fri, 11 Aug 2023 12:57:39 +0000 Subject: [PATCH 115/169] [maven-release-plugin] prepare release v10.5.2 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 2918b2be1..4473199f8 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.5.2-SNAPSHOT + 10.5.2 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 90896ada9..b26b04f43 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.5.2-SNAPSHOT + 10.5.2 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.5.x + v10.5.2 From a2d51d3cdf665b1c96c53290cc9355300fc1741d Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Fri, 11 Aug 2023 12:57:42 +0000 Subject: [PATCH 116/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 4473199f8..1ec388be3 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.5.2 + 10.5.3-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index b26b04f43..248d7a387 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.5.2 + 10.5.3-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.5.2 + 10.5.x From 42a5fb10a329ecaa0ddf52bcf4d7d2c0e3826a66 Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Wed, 16 Aug 2023 13:22:02 +0000 Subject: [PATCH 117/169] [maven-release-plugin] prepare release v10.1.12 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 68e4991b5..863a24b63 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.12-SNAPSHOT + 10.1.12 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 2049a8073..e47ad5fa4 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.12-SNAPSHOT + 10.1.12 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + v10.1.12 From 1dba657282421895795c2365825fd976bf567402 Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Wed, 16 Aug 2023 13:22:05 +0000 Subject: [PATCH 118/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 863a24b63..dc421c60d 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.12 + 10.1.13-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index e47ad5fa4..ad88f727c 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.12 + 10.1.13-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.1.12 + 10.1.x From fc99bb00b9eb1fd0afb0e8cd9d9459e9c22132f8 Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Wed, 16 Aug 2023 14:08:25 +0000 Subject: [PATCH 119/169] [maven-release-plugin] prepare release v10.2.11 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 6f32c887c..ce1f07b1a 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.11-SNAPSHOT + 10.2.11 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 890c2614c..ca9e9bf17 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.11-SNAPSHOT + 10.2.11 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.2.x + v10.2.11 From bff56976f539cf9facb4742e20505b805999e553 Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Wed, 16 Aug 2023 14:08:28 +0000 Subject: [PATCH 120/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index ce1f07b1a..85b515495 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.11 + 10.2.12-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index ca9e9bf17..82d68b860 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.11 + 10.2.12-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.2.11 + 10.2.x From 6661adb8781b5ea8e7849f3ad9446df18664e548 Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Wed, 16 Aug 2023 14:09:44 +0000 Subject: [PATCH 121/169] [maven-release-plugin] prepare release v10.3.7 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 3cea7274a..72ee8d1c7 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.7-SNAPSHOT + 10.3.7 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 3f6891118..e20c5c870 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.7-SNAPSHOT + 10.3.7 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.3.x + v10.3.7 From 9a4163a9f933d8a92dcba0e8556974d775b96dbb Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Wed, 16 Aug 2023 14:09:47 +0000 Subject: [PATCH 122/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 72ee8d1c7..769d8f9db 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.7 + 10.3.8-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index e20c5c870..d3cf77e04 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.7 + 10.3.8-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.3.7 + 10.3.x From 53adb6fdddfe15266c6161df75c3b7f55f5fc32c Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Wed, 16 Aug 2023 15:50:57 +0000 Subject: [PATCH 123/169] [maven-release-plugin] prepare release v10.4.5 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 04d4759e4..5d2e55d9d 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.5-SNAPSHOT + 10.4.5 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 95637e79a..388ecce7a 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.5-SNAPSHOT + 10.4.5 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.4.x + v10.4.5 From 38095fc19128256f36c373e731f2831d41d6af8e Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Wed, 16 Aug 2023 15:51:01 +0000 Subject: [PATCH 124/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 5d2e55d9d..2167f8076 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.5 + 10.4.6-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 388ecce7a..c46a7c4da 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.5 + 10.4.6-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.4.5 + 10.4.x From dea0d575665c08e53eaae9b0a4bb5afa7dcfeea7 Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Wed, 16 Aug 2023 15:52:14 +0000 Subject: [PATCH 125/169] [maven-release-plugin] prepare release v10.5.3 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 18303f3ba..4aab81a40 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.5.3-SNAPSHOT + 10.5.3 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 047629d5f..68e361e7a 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.5.3-SNAPSHOT + 10.5.3 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.5.x + v10.5.3 From 7fe4367face16de942abcb129457af4b74a5e8e7 Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Wed, 16 Aug 2023 15:52:18 +0000 Subject: [PATCH 126/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 4aab81a40..2990adc2a 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.5.3 + 10.5.4-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 68e361e7a..40f20fb49 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.5.3 + 10.5.4-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.5.3 + 10.5.x From 6266561c96800203257c0800bff9e32eb775d426 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 18 Aug 2023 10:38:26 +0000 Subject: [PATCH 127/169] [maven-release-plugin] prepare release v10.1.13 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index dc421c60d..78a4f3e2a 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.13-SNAPSHOT + 10.1.13 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index ad88f727c..faa7a8f14 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.13-SNAPSHOT + 10.1.13 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + v10.1.13 From ec800a623bd26112d650726053d8649bfa8bc96c Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 18 Aug 2023 10:38:29 +0000 Subject: [PATCH 128/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 78a4f3e2a..f3b74b725 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.13 + 10.1.14-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index faa7a8f14..78a437449 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.13 + 10.1.14-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.1.13 + 10.1.x From 007cf69c5f7ed93423a582c96c11c232e99584b4 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 18 Aug 2023 11:33:13 +0000 Subject: [PATCH 129/169] [maven-release-plugin] prepare release v10.2.12 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 85b515495..7f09c1b4d 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.12-SNAPSHOT + 10.2.12 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 82d68b860..ccfa67b5a 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.12-SNAPSHOT + 10.2.12 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.2.x + v10.2.12 From dbe69bc8f805ad6079b10959962360cb54cf1866 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 18 Aug 2023 11:33:17 +0000 Subject: [PATCH 130/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 7f09c1b4d..8d0fab7a8 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.12 + 10.2.13-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index ccfa67b5a..d23b1d67e 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.12 + 10.2.13-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.2.12 + 10.2.x From ac208b3e185c9b7c4a51378a7232bde7fcbcbed1 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 18 Aug 2023 11:34:41 +0000 Subject: [PATCH 131/169] [maven-release-plugin] prepare release v10.3.8 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 769d8f9db..d5099bdec 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.8-SNAPSHOT + 10.3.8 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index d3cf77e04..55348dace 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.8-SNAPSHOT + 10.3.8 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.3.x + v10.3.8 From 3299a084bc842cd62d1da1fd50f70ffa37e52dd7 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 18 Aug 2023 11:34:45 +0000 Subject: [PATCH 132/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index d5099bdec..2d80e5382 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.8 + 10.3.9-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 55348dace..9283cdec6 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.8 + 10.3.9-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.3.8 + 10.3.x From 8a1ddcf54f2d69bc887c3fd9fb3a9d5ea02dc36a Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 18 Aug 2023 13:13:00 +0000 Subject: [PATCH 133/169] [maven-release-plugin] prepare release v10.4.6 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 2167f8076..7dcc12ea9 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.6-SNAPSHOT + 10.4.6 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index c46a7c4da..09aac1274 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.6-SNAPSHOT + 10.4.6 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.4.x + v10.4.6 From 0a4c6008a0cfe74628095ab16144b3d58e936f04 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 18 Aug 2023 13:13:03 +0000 Subject: [PATCH 134/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 7dcc12ea9..5b9956dbb 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.6 + 10.4.7-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 09aac1274..122ef15cb 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.6 + 10.4.7-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.4.6 + 10.4.x From 2b3715990d8490fca0873fef5161837e5551190e Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 18 Aug 2023 13:14:30 +0000 Subject: [PATCH 135/169] [maven-release-plugin] prepare release v10.5.4 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 2990adc2a..afbaa58c4 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.5.4-SNAPSHOT + 10.5.4 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 40f20fb49..dae6b3df3 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.5.4-SNAPSHOT + 10.5.4 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.5.x + v10.5.4 From cfe2437bcf71f85270149bbba9281677e7c7a870 Mon Sep 17 00:00:00 2001 From: Parag Badani Date: Fri, 18 Aug 2023 13:14:34 +0000 Subject: [PATCH 136/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index afbaa58c4..181a292a4 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.5.4 + 10.5.5-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index dae6b3df3..e43b708c6 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.5.4 + 10.5.5-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.5.4 + 10.5.x From 666f00d368d2e54ea19bcf0873e0016c44933993 Mon Sep 17 00:00:00 2001 From: Ashok Date: Tue, 12 Sep 2023 12:34:09 +0000 Subject: [PATCH 137/169] [maven-release-plugin] prepare release v10.1.14 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 77a7d26fa..5bf859d64 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.14-SNAPSHOT + 10.1.14 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 30b120a47..2d3680a29 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.14-SNAPSHOT + 10.1.14 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + v10.1.14 From d19367823e715963640042bf54d3b093b6eba1b2 Mon Sep 17 00:00:00 2001 From: Ashok Date: Tue, 12 Sep 2023 12:34:13 +0000 Subject: [PATCH 138/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 5bf859d64..55b1913b4 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.14 + 10.1.15-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 2d3680a29..5f95a13f9 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.14 + 10.1.15-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.1.14 + 10.1.x From 0c932f62ff4cd3ef24e122b5cad881251e1c3646 Mon Sep 17 00:00:00 2001 From: Ashok Date: Tue, 12 Sep 2023 15:04:47 +0000 Subject: [PATCH 139/169] [maven-release-plugin] prepare release v10.3.9 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 72512c7a9..4f775048c 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.9-SNAPSHOT + 10.3.9 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index b56b60163..7946715ea 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.9-SNAPSHOT + 10.3.9 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.3.x + v10.3.9 From 02d2b301751c628c8113789e3d3d3c406716147a Mon Sep 17 00:00:00 2001 From: Ashok Date: Tue, 12 Sep 2023 15:04:48 +0000 Subject: [PATCH 140/169] [maven-release-plugin] prepare release v10.5.5 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 9cc8b3949..fa6608d00 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.5.5-SNAPSHOT + 10.5.5 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 3520a94d2..2c5279691 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.5.5-SNAPSHOT + 10.5.5 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.5.x + v10.5.5 From ac91c484d2052f916aaa7374d40b5d5e843c16ea Mon Sep 17 00:00:00 2001 From: Ashok Date: Tue, 12 Sep 2023 15:04:51 +0000 Subject: [PATCH 141/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 4f775048c..82940f8e4 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.9 + 10.3.10-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 7946715ea..b213b408c 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.9 + 10.3.10-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.3.9 + 10.3.x From 0b74b267f831a1c148bb91eaa574f43aab43a5a4 Mon Sep 17 00:00:00 2001 From: Ashok Date: Tue, 12 Sep 2023 15:04:52 +0000 Subject: [PATCH 142/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index fa6608d00..6dd04d50b 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.5.5 + 10.5.6-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 2c5279691..cbceebe70 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.5.5 + 10.5.6-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.5.5 + 10.5.x From 98800292369b46ca92f1a2701e7c0a9796f23381 Mon Sep 17 00:00:00 2001 From: Ashok Date: Tue, 12 Sep 2023 16:19:25 +0000 Subject: [PATCH 143/169] [maven-release-plugin] prepare release v10.4.7 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 587ed7b42..7894da1c3 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.7-SNAPSHOT + 10.4.7 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 00cb58b4a..99eba7ded 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.7-SNAPSHOT + 10.4.7 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.4.x + v10.4.7 From 92604079af24ead6cace672db846803f3ceae5f5 Mon Sep 17 00:00:00 2001 From: Ashok Date: Tue, 12 Sep 2023 16:19:29 +0000 Subject: [PATCH 144/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 7894da1c3..8450e4de0 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.7 + 10.4.8-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 99eba7ded..8d0ac2b86 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.7 + 10.4.8-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.4.7 + 10.4.x From e9ff424c84b8e8d1120e5c875b75d3e46ff72dd3 Mon Sep 17 00:00:00 2001 From: Ashok Date: Tue, 12 Sep 2023 16:22:01 +0000 Subject: [PATCH 145/169] [maven-release-plugin] prepare release v10.2.13 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 944c29114..9c83be5da 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.13-SNAPSHOT + 10.2.13 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 548390b8a..7b067efbd 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.13-SNAPSHOT + 10.2.13 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.2.x + v10.2.13 From 38610a635be7651fdf3e369fa69e4291179dd34b Mon Sep 17 00:00:00 2001 From: Ashok Date: Tue, 12 Sep 2023 16:22:05 +0000 Subject: [PATCH 146/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 9c83be5da..e55839ae1 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.13 + 10.2.14-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 7b067efbd..f781fe147 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.13 + 10.2.14-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.2.13 + 10.2.x From 8f6ecc4c35b4acab60c7cb24a655c239bf12559f Mon Sep 17 00:00:00 2001 From: Nikhil Srivastava Date: Wed, 27 Sep 2023 11:51:54 +0000 Subject: [PATCH 147/169] [maven-release-plugin] prepare release v10.1.15 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 55b1913b4..770c83ec7 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.15-SNAPSHOT + 10.1.15 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 5f95a13f9..9da013827 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.15-SNAPSHOT + 10.1.15 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + v10.1.15 From 71b2516734657e77c48b671f35d9b5ebea97ee2b Mon Sep 17 00:00:00 2001 From: Nikhil Srivastava Date: Wed, 27 Sep 2023 11:51:58 +0000 Subject: [PATCH 148/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 770c83ec7..3d0b7fc50 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.15 + 10.1.16-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 9da013827..5d053f510 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.15 + 10.1.16-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.1.15 + 10.1.x From b4b232df8c8490365b386797441ddf452b497393 Mon Sep 17 00:00:00 2001 From: Nikhil Srivastava Date: Wed, 27 Sep 2023 13:00:50 +0000 Subject: [PATCH 149/169] [maven-release-plugin] prepare release v10.2.14 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index e55839ae1..5ccd1523c 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.14-SNAPSHOT + 10.2.14 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index f781fe147..966a311d6 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.14-SNAPSHOT + 10.2.14 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.2.x + v10.2.14 From b7230981801146d2023932b4132b2861cd40eb38 Mon Sep 17 00:00:00 2001 From: Nikhil Srivastava Date: Wed, 27 Sep 2023 13:00:54 +0000 Subject: [PATCH 150/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 5ccd1523c..ce21791cc 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.14 + 10.2.15-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 966a311d6..d861bb3d7 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.14 + 10.2.15-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.2.14 + 10.2.x From ee638c9b23991ffa12275bacac93b4f0d37ea3ea Mon Sep 17 00:00:00 2001 From: Nikhil Srivastava Date: Wed, 27 Sep 2023 13:02:19 +0000 Subject: [PATCH 151/169] [maven-release-plugin] prepare release v10.3.10 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 82940f8e4..712ca0a96 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.10-SNAPSHOT + 10.3.10 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index b213b408c..3d80e6183 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.10-SNAPSHOT + 10.3.10 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.3.x + v10.3.10 From 11ce78a109573af6cb7218a5b232b9460a179b2f Mon Sep 17 00:00:00 2001 From: Nikhil Srivastava Date: Wed, 27 Sep 2023 13:02:22 +0000 Subject: [PATCH 152/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 712ca0a96..7473d2c11 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.10 + 10.3.11-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 3d80e6183..369b9f3b7 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.10 + 10.3.11-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.3.10 + 10.3.x From d7b95fd2aade513811fb0165c1633a5ddfaece8e Mon Sep 17 00:00:00 2001 From: Nikhil Srivastava Date: Thu, 28 Sep 2023 10:50:07 +0000 Subject: [PATCH 153/169] [maven-release-plugin] prepare release v10.4.8 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 8450e4de0..97cb30768 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.8-SNAPSHOT + 10.4.8 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 8d0ac2b86..359599fa4 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.8-SNAPSHOT + 10.4.8 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.4.x + v10.4.8 From 92e78b624e10b807ef6767d0f1d9607f264076db Mon Sep 17 00:00:00 2001 From: Nikhil Srivastava Date: Thu, 28 Sep 2023 10:50:11 +0000 Subject: [PATCH 154/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 97cb30768..189108f06 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.8 + 10.4.9-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 359599fa4..3b29e730e 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.8 + 10.4.9-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.4.8 + 10.4.x From d7cb7d779754de2dc32dd0220aa096d8f50e68cb Mon Sep 17 00:00:00 2001 From: Nikhil Srivastava Date: Thu, 28 Sep 2023 10:51:33 +0000 Subject: [PATCH 155/169] [maven-release-plugin] prepare release v10.5.6 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 6dd04d50b..a2f333cc8 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.5.6-SNAPSHOT + 10.5.6 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index cbceebe70..be264852e 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.5.6-SNAPSHOT + 10.5.6 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.5.x + v10.5.6 From f4d287d2b99fbbe8f89bebe3b0f686cce29522ec Mon Sep 17 00:00:00 2001 From: Nikhil Srivastava Date: Thu, 28 Sep 2023 10:51:37 +0000 Subject: [PATCH 156/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index a2f333cc8..503eafc7f 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.5.6 + 10.5.7-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index be264852e..f31bc8478 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.5.6 + 10.5.7-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.5.6 + 10.5.x From d02ca6b9ee0497698144fb3161fceebe98080054 Mon Sep 17 00:00:00 2001 From: sp-gupta Date: Sun, 29 Oct 2023 09:34:02 +0530 Subject: [PATCH 157/169] [CC-21863] Resolve plugin.discovery error --- .../io/confluent/connect/s3/S3SinkConnectorFaultyS3Test.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorFaultyS3Test.java b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorFaultyS3Test.java index 343122123..8228804c5 100644 --- a/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorFaultyS3Test.java +++ b/kafka-connect-s3/src/test/java/io/confluent/connect/s3/S3SinkConnectorFaultyS3Test.java @@ -32,6 +32,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.UUID; import java.util.concurrent.TimeUnit; @@ -134,8 +135,10 @@ public void tearDown() throws Exception { @BeforeClass public static void startConnect() { + Map workerProps = new HashMap<>(); + workerProps.put("plugin.discovery","hybrid_warn"); connect = new EmbeddedConnectCluster.Builder() - .name("s3-connect-cluster") + .name("s3-connect-cluster").workerProps(workerProps) .build(); connect.start(); kafkaAdmin = connect.kafka().createAdminClient(); From 94bd08c2e019cca548e7c3f62dff43131ae6a821 Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Sun, 29 Oct 2023 04:15:34 +0000 Subject: [PATCH 158/169] [maven-release-plugin] prepare release v10.1.16 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 3c12fdc6c..c8bf4cc80 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.16-SNAPSHOT + 10.1.16 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 32ba93c77..b7a4436b3 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.16-SNAPSHOT + 10.1.16 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.1.x + v10.1.16 From c370809731d9b29035220039c337e765ce87e43e Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Sun, 29 Oct 2023 04:15:37 +0000 Subject: [PATCH 159/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index c8bf4cc80..2f533260a 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.1.16 + 10.1.17-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index b7a4436b3..fd0525cd4 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.1.16 + 10.1.17-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.1.16 + 10.1.x From e42b430fca5b115a202e0a0037e1f33a70b5758c Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Sun, 29 Oct 2023 05:05:35 +0000 Subject: [PATCH 160/169] [maven-release-plugin] prepare release v10.2.15 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 3d9c7b17d..6df0f167b 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.15-SNAPSHOT + 10.2.15 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index d7d17d3a2..c0cce58e1 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.15-SNAPSHOT + 10.2.15 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.2.x + v10.2.15 From 6f952b448fe22b05f51ef43b9f55d8abbf699d12 Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Sun, 29 Oct 2023 05:05:38 +0000 Subject: [PATCH 161/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 6df0f167b..dc8147a32 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.2.15 + 10.2.16-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index c0cce58e1..186da213e 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.2.15 + 10.2.16-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.2.15 + 10.2.x From dfa1b46e5729005f1473e25658837425140fb624 Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Sun, 29 Oct 2023 05:07:28 +0000 Subject: [PATCH 162/169] [maven-release-plugin] prepare release v10.3.11 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 91743b623..ccca4d10d 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.11-SNAPSHOT + 10.3.11 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 95f9950f2..b7ca67ba9 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.11-SNAPSHOT + 10.3.11 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.3.x + v10.3.11 From e75b6314321b87f86df4cddec9df046aa3418d6c Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Sun, 29 Oct 2023 05:07:32 +0000 Subject: [PATCH 163/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index ccca4d10d..24963b5b1 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.3.11 + 10.3.12-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index b7ca67ba9..7ecfb042b 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.3.11 + 10.3.12-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.3.11 + 10.3.x From 702e288f6620f69331af029a724083c1e9b65b6a Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Mon, 30 Oct 2023 06:01:51 +0000 Subject: [PATCH 164/169] [maven-release-plugin] prepare release v10.4.9 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 63be34a96..9470c0b5e 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.9-SNAPSHOT + 10.4.9 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 43bb49daf..46cf5608a 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.9-SNAPSHOT + 10.4.9 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.4.x + v10.4.9 From e26f66df9c95ebe90b83a797b89d042c4e0610fc Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Mon, 30 Oct 2023 06:01:54 +0000 Subject: [PATCH 165/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 9470c0b5e..9e6ba0733 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.4.9 + 10.4.10-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 46cf5608a..c1cfd3136 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.4.9 + 10.4.10-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.4.9 + 10.4.x From d23c3b74b9acd7d1406b15c8afcb77bca8f54a39 Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Mon, 30 Oct 2023 06:03:50 +0000 Subject: [PATCH 166/169] [maven-release-plugin] prepare release v10.5.7 --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 1884db3ea..71444a0d9 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.5.7-SNAPSHOT + 10.5.7 kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 423f37eca..118dc94de 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.5.7-SNAPSHOT + 10.5.7 kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - 10.5.x + v10.5.7 From 7d07124f48552a101dcb791c437a061755d9703b Mon Sep 17 00:00:00 2001 From: Sparsh Gupta Date: Mon, 30 Oct 2023 06:03:53 +0000 Subject: [PATCH 167/169] [maven-release-plugin] prepare for next development iteration --- kafka-connect-s3/pom.xml | 2 +- pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka-connect-s3/pom.xml b/kafka-connect-s3/pom.xml index 71444a0d9..0e541227c 100644 --- a/kafka-connect-s3/pom.xml +++ b/kafka-connect-s3/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-cloud - 10.5.7 + 10.5.8-SNAPSHOT kafka-connect-s3 diff --git a/pom.xml b/pom.xml index 118dc94de..a8bebe03f 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ io.confluent kafka-connect-storage-cloud pom - 10.5.7 + 10.5.8-SNAPSHOT kafka-connect-storage-cloud Confluent, Inc. @@ -50,7 +50,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-storage-cloud.git scm:git:git@github.com:confluentinc/kafka-connect-storage-cloud.git https://github.com/confluentinc/kafka-connect-storage-cloud - v10.5.7 + 10.5.x From 11300e9e71ff79cdfa0a5a1d77a6e3e55c639509 Mon Sep 17 00:00:00 2001 From: anupamaggarwal Date: Wed, 8 Nov 2023 13:14:54 +0530 Subject: [PATCH 168/169] MINOR: Add diagnostic info for topic partition assignments --- .../src/main/java/io/confluent/connect/s3/S3SinkTask.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java index 590c9c5e8..8255f955d 100644 --- a/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java +++ b/kafka-connect-s3/src/main/java/io/confluent/connect/s3/S3SinkTask.java @@ -153,6 +153,9 @@ public void open(Collection partitions) { for (TopicPartition tp : partitions) { topicPartitionWriters.put(tp, newTopicPartitionWriter(tp)); } + log.info("Assigned topic partitions: {}", + topicPartitionWriters.keySet() + ); } @SuppressWarnings("unchecked") From 41b2dc037007d1320aced1f9bc9b6e3a763f1c6c Mon Sep 17 00:00:00 2001 From: naveenmall11 Date: Tue, 20 Feb 2024 19:16:58 +0530 Subject: [PATCH 169/169] Fix reporting modified record --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a8bebe03f..17fd96cd0 100644 --- a/pom.xml +++ b/pom.xml @@ -21,7 +21,7 @@ io.confluent kafka-connect-storage-common-parent - 11.2.4 + 11.2.5 io.confluent