From 29b360d3e43c2de360048257bee995e95b8001f0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Fri, 27 Dec 2024 11:31:37 +0000 Subject: [PATCH 01/44] intial changes --- .../common/config/SourceCommonConfig.java | 11 + .../common/config/SourceConfigFragment.java | 30 +- .../common/config/enums/ErrorsTolerance.java | 44 +++ .../common/source/AbstractSourceTask.java | 328 ++++++++++++++++++ .../source/input/TransformerFactory.java | 38 +- .../kafka/connect/s3/source/S3SourceTask.java | 221 +++++------- 6 files changed, 530 insertions(+), 142 deletions(-) create mode 100644 commons/src/main/java/io/aiven/kafka/connect/common/config/enums/ErrorsTolerance.java create mode 100644 commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/config/SourceCommonConfig.java b/commons/src/main/java/io/aiven/kafka/connect/common/config/SourceCommonConfig.java index 44575e5e0..954c9151d 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/config/SourceCommonConfig.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/config/SourceCommonConfig.java @@ -20,7 +20,10 @@ import org.apache.kafka.common.config.ConfigDef; +import io.aiven.kafka.connect.common.config.enums.ErrorsTolerance; import io.aiven.kafka.connect.common.source.input.InputFormat; +import io.aiven.kafka.connect.common.source.input.Transformer; +import io.aiven.kafka.connect.common.source.input.TransformerFactory; public class SourceCommonConfig extends CommonConfig { @@ -62,8 +65,16 @@ public String getTargetTopicPartitions() { return sourceConfigFragment.getTargetTopicPartitions(); } + public ErrorsTolerance getErrorsTolerance() { + return sourceConfigFragment.getErrorsTolerance(); + } + public int getMaxPollRecords() { return sourceConfigFragment.getMaxPollRecords(); } + public Transformer getTransformer() { + return TransformerFactory.getTransformer(schemaRegistryFragment.getInputFormat()); + } + } diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/config/SourceConfigFragment.java b/commons/src/main/java/io/aiven/kafka/connect/common/config/SourceConfigFragment.java index 568610da7..58befa60e 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/config/SourceConfigFragment.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/config/SourceConfigFragment.java @@ -19,6 +19,10 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; +import io.aiven.kafka.connect.common.config.enums.ErrorsTolerance; + +import org.codehaus.plexus.util.StringUtils; + public final class SourceConfigFragment extends ConfigFragment { private static final String GROUP_OTHER = "OTHER_CFG"; public static final String MAX_POLL_RECORDS = "max.poll.records"; @@ -26,6 +30,7 @@ public final class SourceConfigFragment extends ConfigFragment { private static final String GROUP_OFFSET_TOPIC = "OFFSET_TOPIC"; public static final String TARGET_TOPIC_PARTITIONS = "topic.partitions"; public static final String TARGET_TOPICS = "topics"; + public static final String ERRORS_TOLERANCE = "errors.tolerance"; /** * Construct the ConfigFragment.. @@ -41,9 +46,14 @@ public static ConfigDef update(final ConfigDef configDef) { int sourcePollingConfigCounter = 0; configDef.define(MAX_POLL_RECORDS, ConfigDef.Type.INT, 500, ConfigDef.Range.atLeast(1), - ConfigDef.Importance.MEDIUM, "Max poll records", GROUP_OTHER, sourcePollingConfigCounter++, // NOPMD - // UnusedAssignment + ConfigDef.Importance.MEDIUM, "Max poll records", GROUP_OTHER, sourcePollingConfigCounter++, ConfigDef.Width.NONE, MAX_POLL_RECORDS); + // KIP-298 Error Handling in Connect + configDef.define(ERRORS_TOLERANCE, ConfigDef.Type.STRING, ErrorsTolerance.NONE.name(), + new ErrorsToleranceValidator(), ConfigDef.Importance.MEDIUM, + "Indicates to the connector what level of exceptions are allowed before the connector stops, supported values : none,all", + GROUP_OTHER, sourcePollingConfigCounter++, ConfigDef.Width.NONE, ERRORS_TOLERANCE); + configDef.define(EXPECTED_MAX_MESSAGE_BYTES, ConfigDef.Type.INT, 1_048_588, ConfigDef.Importance.MEDIUM, "The largest record batch size allowed by Kafka config max.message.bytes", GROUP_OTHER, sourcePollingConfigCounter++, // NOPMD @@ -58,6 +68,7 @@ public static ConfigDef update(final ConfigDef configDef) { configDef.define(TARGET_TOPICS, ConfigDef.Type.STRING, null, new ConfigDef.NonEmptyString(), ConfigDef.Importance.MEDIUM, "eg : connect-storage-offsets", GROUP_OFFSET_TOPIC, offsetStorageGroupCounter++, ConfigDef.Width.NONE, TARGET_TOPICS); // NOPMD + return configDef; } @@ -77,4 +88,19 @@ public int getExpectedMaxMessageBytes() { return cfg.getInt(EXPECTED_MAX_MESSAGE_BYTES); } + public ErrorsTolerance getErrorsTolerance() { + return ErrorsTolerance.forName(cfg.getString(ERRORS_TOLERANCE)); + } + + private static class ErrorsToleranceValidator implements ConfigDef.Validator { + @Override + public void ensureValid(final String name, final Object value) { + final String errorsTolerance = (String) value; + if (StringUtils.isNotBlank(errorsTolerance)) { + // This will throw an Exception if not a valid value. + ErrorsTolerance.forName(errorsTolerance); + } + } + } + } diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/config/enums/ErrorsTolerance.java b/commons/src/main/java/io/aiven/kafka/connect/common/config/enums/ErrorsTolerance.java new file mode 100644 index 000000000..9c42c46d9 --- /dev/null +++ b/commons/src/main/java/io/aiven/kafka/connect/common/config/enums/ErrorsTolerance.java @@ -0,0 +1,44 @@ +/* + * Copyright 2024 Aiven Oy + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.aiven.kafka.connect.common.config.enums; + +import java.util.Arrays; +import java.util.Objects; + +import org.apache.kafka.common.config.ConfigException; + +public enum ErrorsTolerance { + + NONE("none"), ALL("all"); + + private final String name; + + ErrorsTolerance(final String name) { + this.name = name; + } + + public static ErrorsTolerance forName(final String name) { + Objects.requireNonNull(name, "name cannot be null"); + for (final ErrorsTolerance errorsTolerance : ErrorsTolerance.values()) { + if (errorsTolerance.name.equalsIgnoreCase(name)) { + return errorsTolerance; + } + } + throw new ConfigException(String.format("Unknown errors.tolerance type: %s, allowed values %s ", name, + Arrays.toString(ErrorsTolerance.values()))); + } +} diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java new file mode 100644 index 000000000..3b064dba2 --- /dev/null +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -0,0 +1,328 @@ +/* + * Copyright 2024 Aiven Oy + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.aiven.kafka.connect.common.source; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.kafka.connect.source.SourceRecord; +import org.apache.kafka.connect.source.SourceTask; + +import io.aiven.kafka.connect.common.config.SourceCommonConfig; +import io.aiven.kafka.connect.common.config.enums.ErrorsTolerance; + +import org.apache.commons.lang3.time.StopWatch; +import org.slf4j.Logger; + +/** + * This class handles extracting records from an iterator and returning them to Kafka. It uses an exponential backoff + * with jitter to reduce the number of calls to the backend when there is no data. This solution: + * + * + * + */ +public abstract class AbstractSourceTask extends SourceTask { + + /** + * The maximum time to spend polling. This is set to 5 seconds as that is the time that is allotted to a system for + * shutdown. + */ + protected static final Duration MAX_POLL_TIME = Duration.ofSeconds(5); + /** + * The boolean that indicates the connector is stopped. + */ + private final AtomicBoolean connectorStopped; + + /** + * The logger to use. Set from the class implementing AbstractSourceTask. + */ + private final Logger logger; + + /** + * The maximum number of records to put in a poll. Specified in the configuration. + */ + private int maxPollRecords; + + /** + * The Backoff implementation that executes the delay in the poll loop. + */ + private final Backoff backoff; + + private final Timer timer; + + /** + * The configuration + */ + private SourceCommonConfig config; + + private Iterator sourceRecordIterator; + + /** + * Constructor. + * + * @param logger + * the logger to use. + */ + protected AbstractSourceTask(final Logger logger) { + super(); + this.logger = logger; + connectorStopped = new AtomicBoolean(); + backoff = new Backoff(MAX_POLL_TIME); + timer = new Timer(MAX_POLL_TIME); + } + + /** + * Gets the iterator of SourceRecords. The iterator that SourceRecords are extracted from during a poll event. When + * this iterator runs out of records it should attempt to reset and read more records from the backend on the next + * {@code hasNext()} call. In this way it should detect when new data has been added to the backend and continue + * processing. + *

+ * This method should handle any backend exception that can be retried. Any runtime exceptions that are thrown when + * this iterator executes may cause the task to abort. + *

+ * + * @return The iterator of SourceRecords. + */ + abstract protected Iterator getIterator(); + + /** + * Called by {@link #start} to allows the concrete implementation to configure itself based on properties. + * + * @param props + * the properties to use for configuration. + */ + abstract protected SourceCommonConfig configure(Map props); + + @Override + public final void start(final Map props) { + logger.debug("Starting"); + config = configure(props); + maxPollRecords = config.getMaxPollRecords(); + sourceRecordIterator = getIterator(); + } + + /** + * Try to add a SourceRecord to the results. + * + * @param results + * the result to add the recrod to. + * @param sourceRecordIterator + * the source record iterator. + * @return true if successful, false if the iterator is empty. + */ + private boolean tryAdd(final List results, final Iterator sourceRecordIterator) { + if (sourceRecordIterator.hasNext()) { + backoff.reset(); + results.add(sourceRecordIterator.next()); + return true; + } + return false; + } + + /** + * Returns {@code true} if the connector is not stopped and the timer has not expired. + * + * @return {@code true} if the connector is not stopped and the timer has not expired. + */ + protected boolean stillPolling() { + return !connectorStopped.get() && !timer.expired(); + } + + @Override + public final List poll() { + logger.debug("Polling"); + if (connectorStopped.get()) { + closeResources(); + return Collections.emptyList(); + } else { + timer.start(); + try { + return populateList(); + } finally { + timer.stop(); + } + } + } + + /** + * Attempts to populate the return list. Will read as many records into the list as it can until the timer expires + * or the task is shut down. + * + * @return A list SourceRecords or {@code null} if the system hit a runtime exception. + */ + private List populateList() { + final List results = new ArrayList<>(); + try { + while (stillPolling() && results.size() < maxPollRecords) { + // if we could not get a record and the results are not empty return them + if (!tryAdd(results, sourceRecordIterator) && !results.isEmpty()) { + break; + } + // attempt a backoff + backoff.cleanDelay(); + } + } catch (RuntimeException e) { // NOPMD must catch runtime here. + logger.error("Error during poll(): {}", e.getMessage(), e); + if (config.getErrorsTolerance() == ErrorsTolerance.NONE) { + logger.error("Stopping Task"); + return null; // NOPMD must return null in this case. + } + } + return results; + } + + @Override + public final void stop() { + logger.debug("Stopping"); + connectorStopped.set(true); + } + + /** + * Returns the running state of the task. + * + * @return {@code true} if the connector is running, {@code false} otherwise. + */ + public final boolean isRunning() { + return !connectorStopped.get(); + } + + /** + * Close any resources the source has open. Called by the IteratorRunnable when it is stopping. + */ + abstract protected void closeResources(); + + /** + * Calculates elapsed time and flags when expired. + */ + protected static class Timer extends StopWatch { + /** + * The length of time that the timer should run. + */ + private final long duration; + + /** + * Constructor. + * + * @param duration + * the length of time the timer should run. + */ + private Timer(final Duration duration) { + super(); + this.duration = duration.toMillis(); + } + + /** + * Returns {@code true} if the timer has expired. + * + * @return {@code true} if the timer has expired. + */ + public boolean expired() { + return super.getTime() >= duration; + } + } + + /** + * Calculates the amount of time to sleep during a backoff performs the sleep. Backoff calculation uses an + * expenantially increasing delay until the maxDelay is reached. Then all delays are maxDelay length. + */ + protected static class Backoff { + /** + * The maximum wait time. + */ + private final long maxWait; + /** + * The maximum number of times {@link #delay()} will be called before maxWait is reached. + */ + private final int maxCount; + /** + * The number of times {@link #delay()} has been called. + */ + private int waitCount; + + /** + * A random number generator to construct jitter. + */ + Random random = new Random(); + + /** + * Constructor. + * + * @param maxDelay + * The maximum delay that this instance will use. + */ + public Backoff(final Duration maxDelay) { + // calculate the approx wait time. + maxWait = maxDelay.toMillis(); + maxCount = (int) (Math.log10(maxWait) / Math.log10(2)); + waitCount = 0; + } + + /** + * Reset the backoff time so that delay is again at the minimum. + */ + public void reset() { + waitCount = 0; + } + + /** + * Delay execution based on the number of times this method has been called. + * + * @throws InterruptedException + * If any thread interrupts this thread. + */ + public void delay() throws InterruptedException { + // power of 2 next int is faster and so we generate approx +/- 0.512 seconds of jitter + final int jitter = random.nextInt(1024) - 512; + + if (waitCount < maxCount) { + waitCount++; + final long sleep = (long) Math.pow(2, waitCount) + jitter; + // don't allow jitter to set sleep argument negative. + Thread.sleep(Math.max(0, sleep)); + } else { + Thread.sleep(maxWait + jitter); + } + } + + /** + * Like {@link #delay} but swallows the {@link InterruptedException}. + */ + public void cleanDelay() { + try { + delay(); + } catch (InterruptedException exception) { + // do nothing return results below + } + } + } +} diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/input/TransformerFactory.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/input/TransformerFactory.java index f868d7328..574604306 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/input/TransformerFactory.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/input/TransformerFactory.java @@ -16,30 +16,46 @@ package io.aiven.kafka.connect.common.source.input; -import static io.aiven.kafka.connect.common.config.SchemaRegistryFragment.INPUT_FORMAT_KEY; +import static io.aiven.kafka.connect.common.config.SchemaRegistryFragment.SCHEMAS_ENABLE; -import io.aiven.kafka.connect.common.config.SchemaRegistryFragment; -import io.aiven.kafka.connect.common.config.SourceCommonConfig; +import java.util.Map; +import org.apache.kafka.connect.json.JsonConverter; + +import io.confluent.connect.avro.AvroData; + +/** + * A factory to create Transformers. + */ public final class TransformerFactory { + /** The cache size for systems that read Avro data */ + public static final int CACHE_SIZE = 100; private TransformerFactory() { // hidden } - public static Transformer getTransformer(final SourceCommonConfig sourceConfig) { - final InputFormat inputFormatEnum = new SchemaRegistryFragment(sourceConfig).getInputFormat(); - switch (inputFormatEnum) { + + /** + * Gets a configured Transformer. + * + * @param inputFormat + * The input format for the transformer. + * @return the Transformer for the specified input format. + */ + public static Transformer getTransformer(final InputFormat inputFormat) { + switch (inputFormat) { case AVRO : - return new AvroTransformer(); + return new AvroTransformer(new AvroData(CACHE_SIZE)); case PARQUET : - return new ParquetTransformer(); + return new ParquetTransformer(new AvroData(CACHE_SIZE)); case JSONL : - return new JsonTransformer(); + final JsonConverter jsonConverter = new JsonConverter(); + jsonConverter.configure(Map.of(SCHEMAS_ENABLE, "false"), false); + return new JsonTransformer(jsonConverter); case BYTES : return new ByteArrayTransformer(); default : - throw new IllegalArgumentException( - "Unknown input format in configuration: " + sourceConfig.getString(INPUT_FORMAT_KEY)); + throw new IllegalArgumentException("Unknown input format in configuration: " + inputFormat); } } } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index be3d89618..49de2e06b 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -16,35 +16,24 @@ package io.aiven.kafka.connect.s3.source; -import static io.aiven.kafka.connect.common.config.SourceConfigFragment.MAX_POLL_RECORDS; - -import java.lang.reflect.InvocationTargetException; -import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; -import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.errors.DataException; +import io.aiven.kafka.connect.common.source.input.TransformerFactory; +import io.aiven.kafka.connect.s3.source.utils.OffsetManager; import org.apache.kafka.connect.source.SourceRecord; -import org.apache.kafka.connect.source.SourceTask; -import org.apache.kafka.connect.storage.Converter; +import io.aiven.kafka.connect.common.config.SourceCommonConfig; +import io.aiven.kafka.connect.common.source.AbstractSourceTask; import io.aiven.kafka.connect.common.source.input.Transformer; -import io.aiven.kafka.connect.common.source.input.TransformerFactory; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; import io.aiven.kafka.connect.s3.source.utils.AWSV2SourceClient; -import io.aiven.kafka.connect.s3.source.utils.OffsetManager; -import io.aiven.kafka.connect.s3.source.utils.RecordProcessor; import io.aiven.kafka.connect.s3.source.utils.S3SourceRecord; import io.aiven.kafka.connect.s3.source.utils.SourceRecordIterator; import io.aiven.kafka.connect.s3.source.utils.Version; -import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.model.AmazonS3Exception; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,9 +42,8 @@ * S3SourceTask is a Kafka Connect SourceTask implementation that reads from source-s3 buckets and generates Kafka * Connect records. */ -@SuppressWarnings({ "PMD.TooManyMethods", "PMD.ExcessiveImports" }) -public class S3SourceTask extends SourceTask { - +public class S3SourceTask extends AbstractSourceTask { + /** The logger to write to */ private static final Logger LOGGER = LoggerFactory.getLogger(S3SourceTask.class); public static final String BUCKET = "bucket"; @@ -64,33 +52,24 @@ public class S3SourceTask extends SourceTask { public static final String OBJECT_KEY = "object_key"; public static final String PARTITION = "topicPartition"; - private static final long S_3_POLL_INTERVAL_MS = 10_000L; - private static final long ERROR_BACKOFF = 1000L; - - private S3SourceConfig s3SourceConfig; - private AmazonS3 s3Client; - - private Iterator sourceRecordIterator; - private Optional keyConverter; - - private Converter valueConverter; + /** An iterator or S3SourceRecords */ + private Iterator s3SourceRecordIterator; + /** + * The transformer that we are using TODO move this to AbstractSourceTask + */ private Transformer transformer; - + /** The task initialized flag */ private boolean taskInitialized; - - private final AtomicBoolean connectorStopped = new AtomicBoolean(); - - private final Object pollLock = new Object(); + /** The AWS Source client */ private AWSV2SourceClient awsv2SourceClient; + /** The list of failed object keys */ private final Set failedObjectKeys = new HashSet<>(); - private final Set inProcessObjectKeys = new HashSet<>(); - + /** The offset manager this task uses */ private OffsetManager offsetManager; - @SuppressWarnings("PMD.UnnecessaryConstructor") public S3SourceTask() { - super(); + super(LOGGER); } @Override @@ -99,124 +78,108 @@ public String version() { } @Override - public void start(final Map props) { + protected Iterator getIterator() { // NOPMD cognatavie complexity + return new Iterator() { + /** The backoff for Amazon retryable exceptions */ + final Backoff backoff = new Backoff(MAX_POLL_TIME); + @Override + public boolean hasNext() { + try { + // this timer is the master timer from the AbstractSourceTask. + while (stillPolling()) { + try { + return s3SourceRecordIterator.hasNext(); + } catch (AmazonS3Exception exception) { + if (exception.isRetryable()) { + LOGGER.warn("Retryable error encountered during polling. Waiting before retrying...", + exception); + try { + backoff.delay(); + } catch (InterruptedException e) { + LOGGER.warn("Backoff delay was interrupted. Throwing original exception: {}", + exception.getMessage()); + throw exception; + } + } else { + // TODO validate that the iterator does not lose an S3Object. Add test to + // S3ObjectIterator. + throw exception; + } + } + } + return false; + } finally { + backoff.reset(); + } + } + + @Override + public SourceRecord next() { + final S3SourceRecord s3SourceRecord = s3SourceRecordIterator.next(); + offsetManager.incrementAndUpdateOffsetMap(s3SourceRecord.getPartitionMap(), s3SourceRecord.getObjectKey(), 1L); + return s3SourceRecord.getSourceRecord(s3SourceRecord.getTopic()); + } + }; + } + + @Override + protected SourceCommonConfig configure(final Map props) { LOGGER.info("S3 Source task started."); - s3SourceConfig = new S3SourceConfig(props); - initializeConverters(); + final S3SourceConfig s3SourceConfig = new S3SourceConfig(props); this.transformer = TransformerFactory.getTransformer(s3SourceConfig); offsetManager = new OffsetManager(context, s3SourceConfig); awsv2SourceClient = new AWSV2SourceClient(s3SourceConfig, failedObjectKeys); - prepareReaderFromOffsetStorageReader(); + setS3SourceRecordIterator( + new SourceRecordIterator(s3SourceConfig, offsetManager, this.transformer, awsv2SourceClient)); this.taskInitialized = true; - } - - private void initializeConverters() { - try { - keyConverter = Optional - .of((Converter) Class.forName((String) s3SourceConfig.originals().get("key.converter")) - .getDeclaredConstructor() - .newInstance()); - valueConverter = (Converter) Class.forName((String) s3SourceConfig.originals().get("value.converter")) - .getDeclaredConstructor() - .newInstance(); - } catch (ClassNotFoundException | InstantiationException | IllegalAccessException | InvocationTargetException - | NoSuchMethodException e) { - throw new ConnectException("Connect converters could not be instantiated.", e); - } - } - - private void prepareReaderFromOffsetStorageReader() { - sourceRecordIterator = new SourceRecordIterator(s3SourceConfig, offsetManager, this.transformer, - awsv2SourceClient); + return s3SourceConfig; } @Override - public List poll() throws InterruptedException { - LOGGER.info("Polling for new records..."); - synchronized (pollLock) { - final List results = new ArrayList<>(s3SourceConfig.getInt(MAX_POLL_RECORDS)); - - if (connectorStopped.get()) { - LOGGER.info("Connector has been stopped. Returning empty result list."); - return results; - } - - while (!connectorStopped.get()) { - try { - extractSourceRecords(results); - LOGGER.info("Number of records extracted and sent: {}", results.size()); - return results; - } catch (AmazonS3Exception exception) { - if (exception.isRetryable()) { - LOGGER.warn("Retryable error encountered during polling. Waiting before retrying...", - exception); - pollLock.wait(ERROR_BACKOFF); - - prepareReaderFromOffsetStorageReader(); - } else { - LOGGER.warn("Non-retryable AmazonS3Exception occurred. Stopping polling.", exception); - return null; // NOPMD - } - } catch (DataException exception) { - LOGGER.warn("DataException occurred during polling. No retries will be attempted.", exception); - } catch (final Throwable t) { // NOPMD - LOGGER.error("Unexpected error encountered. Closing resources and stopping task.", t); - closeResources(); - throw t; - } - } - return results; - } + public void commit() throws InterruptedException { + LOGGER.info("Committed all records through last poll()"); } - private List extractSourceRecords(final List results) throws InterruptedException { - waitForObjects(); - if (connectorStopped.get()) { - return results; + @Override + public void commitRecord(final SourceRecord record) throws InterruptedException { + if (LOGGER.isInfoEnabled()) { + final Map map = (Map) record.sourceOffset(); + LOGGER.info("Committed individual record {} {} {} committed", map.get(BUCKET), map.get(OBJECT_KEY), offsetManager.recordsProcessedForObjectKey((Map)record.sourcePartition(), map.get(OBJECT_KEY).toString())); } - return RecordProcessor.processRecords(sourceRecordIterator, results, s3SourceConfig, keyConverter, - valueConverter, connectorStopped, this.transformer, awsv2SourceClient, offsetManager); } - private void waitForObjects() throws InterruptedException { - while (!sourceRecordIterator.hasNext() && !connectorStopped.get()) { - LOGGER.debug("Blocking until new S3 files are available."); - Thread.sleep(S_3_POLL_INTERVAL_MS); - prepareReaderFromOffsetStorageReader(); - } + /** + * Set the S3 source record iterator that this task is using. protected to be overridden in testing impl. + * + * @param iterator + * The S3SourceRecord iterator to use. + */ + protected void setS3SourceRecordIterator(final Iterator iterator) { + s3SourceRecordIterator = iterator; } @Override - public void stop() { - this.taskInitialized = false; - this.connectorStopped.set(true); - synchronized (pollLock) { - closeResources(); - } - } - - private void closeResources() { + protected void closeResources() { awsv2SourceClient.shutdown(); } // below for visibility in tests - public Optional getKeyConverter() { - return keyConverter; - } - - public Converter getValueConverter() { - return valueConverter; - } + /** + * Get the transformer that we are using. + * + * @return the transformer that we are using. + */ public Transformer getTransformer() { return transformer; } + /** + * Get the initialized flag. + * + * @return {@code true} if the task has been initialized, {@code false} otherwise. + */ public boolean isTaskInitialized() { return taskInitialized; } - - public AtomicBoolean getConnectorStopped() { - return new AtomicBoolean(connectorStopped.get()); - } } From b6f2d08f479e7fdf09b7af99968b25434fa1fbff Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Fri, 27 Dec 2024 14:38:42 +0000 Subject: [PATCH 02/44] fixed test cases --- .../common/source/AbstractSourceTask.java | 85 ++++++--- .../kafka/connect/s3/source/S3SourceTask.java | 51 +++--- .../connect/s3/source/utils/ConnectUtils.java | 2 +- .../s3/source/utils/RecordProcessor.java | 6 +- .../s3/source/utils/S3SourceRecord.java | 8 + .../connect/s3/source/S3SourceTaskTest.java | 172 ++++++++++++------ .../s3/source/utils/RecordProcessorTest.java | 13 +- 7 files changed, 214 insertions(+), 123 deletions(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java index 3b064dba2..990d9e65b 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -57,7 +57,7 @@ public abstract class AbstractSourceTask extends SourceTask { * The maximum time to spend polling. This is set to 5 seconds as that is the time that is allotted to a system for * shutdown. */ - protected static final Duration MAX_POLL_TIME = Duration.ofSeconds(5); + public static final Duration MAX_POLL_TIME = Duration.ofSeconds(5); /** * The boolean that indicates the connector is stopped. */ @@ -97,8 +97,8 @@ protected AbstractSourceTask(final Logger logger) { super(); this.logger = logger; connectorStopped = new AtomicBoolean(); - backoff = new Backoff(MAX_POLL_TIME); timer = new Timer(MAX_POLL_TIME); + backoff = new Backoff(timer::millisecondsRemaining); } /** @@ -111,9 +111,11 @@ protected AbstractSourceTask(final Logger logger) { * this iterator executes may cause the task to abort. *

* + * @param timer + * a SupplierOfLong that provides the amount of time remaining before the polling expires. * @return The iterator of SourceRecords. */ - abstract protected Iterator getIterator(); + abstract protected Iterator getIterator(SupplierOfLong timer); /** * Called by {@link #start} to allows the concrete implementation to configure itself based on properties. @@ -128,7 +130,7 @@ public final void start(final Map props) { logger.debug("Starting"); config = configure(props); maxPollRecords = config.getMaxPollRecords(); - sourceRecordIterator = getIterator(); + sourceRecordIterator = getIterator(timer::millisecondsRemaining); } /** @@ -184,13 +186,16 @@ private List populateList() { final List results = new ArrayList<>(); try { while (stillPolling() && results.size() < maxPollRecords) { - // if we could not get a record and the results are not empty return them - if (!tryAdd(results, sourceRecordIterator) && !results.isEmpty()) { - break; + if (!tryAdd(results, sourceRecordIterator)) { + if (!results.isEmpty()) { + // if we could not get a record and the results are not empty return them + break; + } + // attempt a backoff + backoff.cleanDelay(); } - // attempt a backoff - backoff.cleanDelay(); } + } catch (RuntimeException e) { // NOPMD must catch runtime here. logger.error("Error during poll(): {}", e.getMessage(), e); if (config.getErrorsTolerance() == ErrorsTolerance.NONE) { @@ -241,6 +246,15 @@ private Timer(final Duration duration) { this.duration = duration.toMillis(); } + /** + * Gets the maximum duration for this timer. + * + * @return the maximum duration for the timer. + */ + public long millisecondsRemaining() { + return super.isStarted() ? super.getTime() - duration : duration; + } + /** * Returns {@code true} if the timer has expired. * @@ -252,18 +266,19 @@ public boolean expired() { } /** - * Calculates the amount of time to sleep during a backoff performs the sleep. Backoff calculation uses an - * expenantially increasing delay until the maxDelay is reached. Then all delays are maxDelay length. + * Performs a delay based on the number of successive {@link #delay()} or {@link #cleanDelay()} calls without a + * {@link #reset()}. Delay increases exponentially but never exceeds the time remaining by more than 0.512 seconds. */ protected static class Backoff { /** - * The maximum wait time. + * A supplier of the time remaining (in milliseconds) on the overriding timer. */ - private final long maxWait; + private final SupplierOfLong timeRemaining; + /** * The maximum number of times {@link #delay()} will be called before maxWait is reached. */ - private final int maxCount; + private int maxCount; /** * The number of times {@link #delay()} has been called. */ @@ -277,23 +292,28 @@ protected static class Backoff { /** * Constructor. * - * @param maxDelay - * The maximum delay that this instance will use. + * @param timeRemaining + * A supplier of long as milliseconds remaining before time expires. */ - public Backoff(final Duration maxDelay) { - // calculate the approx wait time. - maxWait = maxDelay.toMillis(); - maxCount = (int) (Math.log10(maxWait) / Math.log10(2)); - waitCount = 0; + public Backoff(final SupplierOfLong timeRemaining) { + this.timeRemaining = timeRemaining; + reset(); } /** * Reset the backoff time so that delay is again at the minimum. */ - public void reset() { + public final void reset() { + // calculate the approx wait count. + maxCount = (int) (Math.log10(timeRemaining.get()) / Math.log10(2)); waitCount = 0; } + private long timeWithJitter() { + // generate approx +/- 0.512 seconds of jitter + final int jitter = random.nextInt(1024) - 512; + return (long) Math.pow(2, waitCount) + jitter; + } /** * Delay execution based on the number of times this method has been called. * @@ -301,16 +321,15 @@ public void reset() { * If any thread interrupts this thread. */ public void delay() throws InterruptedException { - // power of 2 next int is faster and so we generate approx +/- 0.512 seconds of jitter - final int jitter = random.nextInt(1024) - 512; + long sleepTime = timeRemaining.get(); if (waitCount < maxCount) { waitCount++; - final long sleep = (long) Math.pow(2, waitCount) + jitter; - // don't allow jitter to set sleep argument negative. - Thread.sleep(Math.max(0, sleep)); - } else { - Thread.sleep(maxWait + jitter); + sleepTime = Math.min(sleepTime, timeWithJitter()); + } + // don't sleep negative time. + if (sleepTime > 0) { + Thread.sleep(sleepTime); } } @@ -325,4 +344,12 @@ public void cleanDelay() { } } } + + /** + * A functional interface to return long values. + */ + @FunctionalInterface + public interface SupplierOfLong { + long get(); + } } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index 49de2e06b..bbdd78b13 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -16,13 +16,14 @@ package io.aiven.kafka.connect.s3.source; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Set; -import io.aiven.kafka.connect.common.source.input.TransformerFactory; -import io.aiven.kafka.connect.s3.source.utils.OffsetManager; import org.apache.kafka.connect.source.SourceRecord; import io.aiven.kafka.connect.common.config.SourceCommonConfig; @@ -30,6 +31,8 @@ import io.aiven.kafka.connect.common.source.input.Transformer; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; import io.aiven.kafka.connect.s3.source.utils.AWSV2SourceClient; +import io.aiven.kafka.connect.s3.source.utils.OffsetManager; +import io.aiven.kafka.connect.s3.source.utils.RecordProcessor; import io.aiven.kafka.connect.s3.source.utils.S3SourceRecord; import io.aiven.kafka.connect.s3.source.utils.SourceRecordIterator; import io.aiven.kafka.connect.s3.source.utils.Version; @@ -52,21 +55,19 @@ public class S3SourceTask extends AbstractSourceTask { public static final String OBJECT_KEY = "object_key"; public static final String PARTITION = "topicPartition"; - /** An iterator or S3SourceRecords */ private Iterator s3SourceRecordIterator; /** * The transformer that we are using TODO move this to AbstractSourceTask */ - private Transformer transformer; - /** The task initialized flag */ - private boolean taskInitialized; + private Transformer transformer; /** The AWS Source client */ private AWSV2SourceClient awsv2SourceClient; /** The list of failed object keys */ private final Set failedObjectKeys = new HashSet<>(); /** The offset manager this task uses */ private OffsetManager offsetManager; + private S3SourceConfig s3SourceConfig; public S3SourceTask() { super(LOGGER); @@ -78,10 +79,10 @@ public String version() { } @Override - protected Iterator getIterator() { // NOPMD cognatavie complexity - return new Iterator() { + protected Iterator getIterator(SupplierOfLong timer) { // NOPMD cognatavie complexity + return new Iterator<>() { /** The backoff for Amazon retryable exceptions */ - final Backoff backoff = new Backoff(MAX_POLL_TIME); + final Backoff backoff = new Backoff(timer); @Override public boolean hasNext() { try { @@ -116,8 +117,12 @@ public boolean hasNext() { @Override public SourceRecord next() { final S3SourceRecord s3SourceRecord = s3SourceRecordIterator.next(); - offsetManager.incrementAndUpdateOffsetMap(s3SourceRecord.getPartitionMap(), s3SourceRecord.getObjectKey(), 1L); - return s3SourceRecord.getSourceRecord(s3SourceRecord.getTopic()); + offsetManager.incrementAndUpdateOffsetMap(s3SourceRecord.getPartitionMap(), + s3SourceRecord.getObjectKey(), 1L); + final List result = RecordProcessor.processRecords( + Collections.singletonList(s3SourceRecord).iterator(), new ArrayList<>(), s3SourceConfig, + S3SourceTask.this::stillPolling, awsv2SourceClient, offsetManager); + return result.get(0); } }; } @@ -125,26 +130,27 @@ public SourceRecord next() { @Override protected SourceCommonConfig configure(final Map props) { LOGGER.info("S3 Source task started."); - final S3SourceConfig s3SourceConfig = new S3SourceConfig(props); - this.transformer = TransformerFactory.getTransformer(s3SourceConfig); + this.s3SourceConfig = new S3SourceConfig(props); + this.transformer = s3SourceConfig.getTransformer(); offsetManager = new OffsetManager(context, s3SourceConfig); awsv2SourceClient = new AWSV2SourceClient(s3SourceConfig, failedObjectKeys); setS3SourceRecordIterator( new SourceRecordIterator(s3SourceConfig, offsetManager, this.transformer, awsv2SourceClient)); - this.taskInitialized = true; return s3SourceConfig; } @Override - public void commit() throws InterruptedException { + public void commit() { LOGGER.info("Committed all records through last poll()"); } @Override - public void commitRecord(final SourceRecord record) throws InterruptedException { + public void commitRecord(final SourceRecord record) { if (LOGGER.isInfoEnabled()) { final Map map = (Map) record.sourceOffset(); - LOGGER.info("Committed individual record {} {} {} committed", map.get(BUCKET), map.get(OBJECT_KEY), offsetManager.recordsProcessedForObjectKey((Map)record.sourcePartition(), map.get(OBJECT_KEY).toString())); + LOGGER.info("Committed individual record {} {} {} committed", map.get(BUCKET), map.get(OBJECT_KEY), + offsetManager.recordsProcessedForObjectKey((Map) record.sourcePartition(), + map.get(OBJECT_KEY).toString())); } } @@ -170,16 +176,7 @@ protected void closeResources() { * * @return the transformer that we are using. */ - public Transformer getTransformer() { + public Transformer getTransformer() { return transformer; } - - /** - * Get the initialized flag. - * - * @return {@code true} if the task has been initialized, {@code false} otherwise. - */ - public boolean isTaskInitialized() { - return taskInitialized; - } } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/ConnectUtils.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/ConnectUtils.java index f401c4e1f..6c60bb8ed 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/ConnectUtils.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/ConnectUtils.java @@ -28,7 +28,7 @@ final public class ConnectUtils { private ConnectUtils() { // hidden } - static Map getPartitionMap(final String topicName, final Integer defaultPartitionId, + public static Map getPartitionMap(final String topicName, final Integer defaultPartitionId, final String bucketName) { final Map partitionMap = new HashMap<>(); partitionMap.put(BUCKET, bucketName); diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessor.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessor.java index bdf265338..7ec803820 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessor.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessor.java @@ -18,7 +18,7 @@ import java.util.Iterator; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.DataException; @@ -39,12 +39,12 @@ private RecordProcessor() { } public static List processRecords(final Iterator sourceRecordIterator, - final List results, final S3SourceConfig s3SourceConfig, final AtomicBoolean connectorStopped, + final List results, final S3SourceConfig s3SourceConfig, final Supplier stillPolling, final AWSV2SourceClient sourceClient, final OffsetManager offsetManager) { final int maxPollRecords = s3SourceConfig.getMaxPollRecords(); - for (int i = 0; sourceRecordIterator.hasNext() && i < maxPollRecords && !connectorStopped.get(); i++) { + for (int i = 0; sourceRecordIterator.hasNext() && i < maxPollRecords && stillPolling.get(); i++) { final S3SourceRecord s3SourceRecord = sourceRecordIterator.next(); if (s3SourceRecord != null) { final SourceRecord sourceRecord = createSourceRecord(s3SourceRecord, s3SourceConfig, sourceClient, diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java index c4be50217..403dc3748 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java @@ -66,6 +66,14 @@ public String getObjectKey() { return objectKey; } + public SchemaAndValue getKey() { + return new SchemaAndValue(keyData.schema(), keyData.value()); + } + + public SchemaAndValue getValue() { + return new SchemaAndValue(valueData.schema(), valueData.value()); + } + public void setOffsetMap(final Map offsetMap) { this.offsetMap = new HashMap<>(offsetMap); } diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java index 590ad23bb..695e873fa 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java @@ -23,8 +23,11 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import java.lang.reflect.Field; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Random; @@ -35,13 +38,14 @@ import org.apache.kafka.connect.source.SourceTaskContext; import org.apache.kafka.connect.storage.OffsetStorageReader; +import io.aiven.kafka.connect.common.source.AbstractSourceTask; import io.aiven.kafka.connect.common.source.input.ByteArrayTransformer; import io.aiven.kafka.connect.common.source.input.InputFormat; -import io.aiven.kafka.connect.common.source.input.Transformer; import io.aiven.kafka.connect.config.s3.S3ConfigFragment; +import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; import io.aiven.kafka.connect.s3.source.testutils.BucketAccessor; +import io.aiven.kafka.connect.s3.source.utils.ConnectUtils; import io.aiven.kafka.connect.s3.source.utils.S3SourceRecord; -import io.aiven.kafka.connect.s3.source.utils.SourceRecordIterator; import com.amazonaws.auth.AWSStaticCredentialsProvider; import com.amazonaws.auth.BasicAWSCredentials; @@ -49,35 +53,36 @@ import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.AmazonS3ClientBuilder; import io.findify.s3mock.S3Mock; +import org.apache.commons.lang3.time.StopWatch; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.Mock; -import org.mockito.junit.jupiter.MockitoExtension; -@ExtendWith(MockitoExtension.class) final class S3SourceTaskTest { + /** + * The amount of extra time that we will allow for timing errors. + */ + private static final long TIMING_DELTA = 500; + private static final Random RANDOM = new Random(); private Map properties; - private static BucketAccessor testBucketAccessor; private static final String TEST_BUCKET = "test-bucket"; + private static final String TOPIC = "TOPIC1"; + + private static final int PARTITION = 1; + + private static final String OBJECT_KEY = "object_key"; + private static S3Mock s3Api; private static AmazonS3 s3Client; private static Map commonProperties; - @Mock - private SourceTaskContext mockedSourceTaskContext; - - @Mock - private OffsetStorageReader mockedOffsetStorageReader; - @BeforeAll public static void setUpClass() { final int s3Port = RANDOM.nextInt(10_000) + 10_000; @@ -102,7 +107,7 @@ public static void setUpClass() { s3Client = builder.build(); - testBucketAccessor = new BucketAccessor(s3Client, TEST_BUCKET); + final BucketAccessor testBucketAccessor = new BucketAccessor(s3Client, TEST_BUCKET); testBucketAccessor.createBucket(); } @@ -115,8 +120,6 @@ public static void tearDownClass() { public void setUp() { properties = new HashMap<>(commonProperties); s3Client.createBucket(TEST_BUCKET); - mockedSourceTaskContext = mock(SourceTaskContext.class); - mockedOffsetStorageReader = mock(OffsetStorageReader.class); } @AfterEach @@ -129,29 +132,9 @@ void testS3SourceTaskInitialization() { final S3SourceTask s3SourceTask = new S3SourceTask(); startSourceTask(s3SourceTask); - final Transformer transformer = s3SourceTask.getTransformer(); - assertThat(transformer).isInstanceOf(ByteArrayTransformer.class); + assertThat(s3SourceTask.getTransformer()).isInstanceOf(ByteArrayTransformer.class); - final boolean taskInitialized = s3SourceTask.isTaskInitialized(); - assertThat(taskInitialized).isTrue(); - } - - @Test - void testPoll() throws Exception { - final S3SourceTask s3SourceTask = new S3SourceTask(); - startSourceTask(s3SourceTask); - - SourceRecordIterator mockSourceRecordIterator; - - mockSourceRecordIterator = mock(SourceRecordIterator.class); - setPrivateField(s3SourceTask, "sourceRecordIterator", mockSourceRecordIterator); - when(mockSourceRecordIterator.hasNext()).thenReturn(true).thenReturn(true).thenReturn(false); - - final S3SourceRecord s3SourceRecordList = getAivenS3SourceRecord(); - when(mockSourceRecordIterator.next()).thenReturn(s3SourceRecordList); - - final List sourceRecordList = s3SourceTask.poll(); - assertThat(sourceRecordList).isNotEmpty(); + assertThat(s3SourceTask.isRunning()).isTrue(); } @Test @@ -160,29 +143,22 @@ void testStop() { startSourceTask(s3SourceTask); s3SourceTask.stop(); - final boolean taskInitialized = s3SourceTask.isTaskInitialized(); - assertThat(taskInitialized).isFalse(); - assertThat(s3SourceTask.getConnectorStopped()).isTrue(); - } - - private static S3SourceRecord getAivenS3SourceRecord() { - return new S3SourceRecord(new HashMap<>(), new HashMap<>(), "testtopic", 0, "", - new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, new byte[0]), - new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, new byte[0])); + assertThat(s3SourceTask.isRunning()).isFalse(); } - @SuppressWarnings("PMD.AvoidAccessibilityAlteration") - private void setPrivateField(final Object object, final String fieldName, final Object value) - throws NoSuchFieldException, IllegalAccessException { - Field field; - field = object.getClass().getDeclaredField(fieldName); - field.setAccessible(true); - field.set(object, value); + private static S3SourceRecord createS3SourceRecord(final String topicName, final Integer defaultPartitionId, + final String bucketName, final String objectKey, final byte[] key, final byte[] value) { + return new S3SourceRecord(ConnectUtils.getPartitionMap(topicName, defaultPartitionId, bucketName), + new HashMap<>(), topicName, defaultPartitionId, objectKey, + new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, key), + new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, value)); } private void startSourceTask(final S3SourceTask s3SourceTask) { - s3SourceTask.initialize(mockedSourceTaskContext); + final SourceTaskContext mockedSourceTaskContext = mock(SourceTaskContext.class); + final OffsetStorageReader mockedOffsetStorageReader = mock(OffsetStorageReader.class); when(mockedSourceTaskContext.offsetStorageReader()).thenReturn(mockedOffsetStorageReader); + s3SourceTask.initialize(mockedSourceTaskContext); setBasicProperties(); s3SourceTask.start(properties); @@ -199,4 +175,88 @@ private void setBasicProperties() { properties.put(TARGET_TOPICS, "testtopic"); } + + @Test + void testPollWithEmptyIterator() { + final S3SourceConfig s3SourceConfig = mock(S3SourceConfig.class); + when(s3SourceConfig.getMaxPollRecords()).thenReturn(5); + final Iterator sourceRecordIterator = Collections.emptyIterator(); + final S3SourceTask s3SourceTask = new TestingS3SourceTask(sourceRecordIterator); + + startSourceTask(s3SourceTask); + final StopWatch stopWatch = new StopWatch(); + stopWatch.start(); + final List results = s3SourceTask.poll(); + stopWatch.stop(); + assertThat(results).isEmpty(); + assertThat(stopWatch.getTime()).isLessThan(AbstractSourceTask.MAX_POLL_TIME.toMillis() + TIMING_DELTA); + } + + private void assertEquals(final S3SourceRecord s3Record, final SourceRecord sourceRecord) { + assertThat(sourceRecord).isNotNull(); + assertThat(sourceRecord.sourcePartition()).isEqualTo(s3Record.getPartitionMap()); + assertThat(sourceRecord.sourceOffset()).isEqualTo(s3Record.getOffsetMap()); + assertThat(sourceRecord.key()).isEqualTo(s3Record.getKey().value()); + assertThat(sourceRecord.value()).isEqualTo(s3Record.getValue().value()); + } + + @Test + void testPollsWithRecords() { + final S3SourceConfig s3SourceConfig = mock(S3SourceConfig.class); + when(s3SourceConfig.getMaxPollRecords()).thenReturn(5); + final List lst = new ArrayList<>(); + lst.add(createS3SourceRecord(TOPIC, PARTITION, TEST_BUCKET, OBJECT_KEY, + "Hello".getBytes(StandardCharsets.UTF_8), "Hello World".getBytes(StandardCharsets.UTF_8))); + lst.add(createS3SourceRecord(TOPIC, PARTITION, TEST_BUCKET, OBJECT_KEY + "a", + "Goodbye".getBytes(StandardCharsets.UTF_8), "Goodbye cruel World".getBytes(StandardCharsets.UTF_8))); + final Iterator sourceRecordIterator = lst.iterator(); + final S3SourceTask s3SourceTask = new TestingS3SourceTask(sourceRecordIterator); + + startSourceTask(s3SourceTask); + final StopWatch stopWatch = new StopWatch(); + stopWatch.start(); + final List results = s3SourceTask.poll(); + stopWatch.stop(); + + assertThat(results).hasSize(2); + assertEquals(lst.get(0), results.get(0)); + assertEquals(lst.get(1), results.get(1)); + assertThat(stopWatch.getTime()).isLessThan(AbstractSourceTask.MAX_POLL_TIME.toMillis()); + } + + @Test + void testPollWhenConnectorStopped() { + final S3SourceConfig s3SourceConfig = mock(S3SourceConfig.class); + when(s3SourceConfig.getMaxPollRecords()).thenReturn(5); + final List lst = new ArrayList<>(); + lst.add(createS3SourceRecord(TOPIC, PARTITION, TEST_BUCKET, OBJECT_KEY, + "Hello".getBytes(StandardCharsets.UTF_8), "Hello World".getBytes(StandardCharsets.UTF_8))); + lst.add(createS3SourceRecord(TOPIC, PARTITION, TEST_BUCKET, OBJECT_KEY + "a", + "Goodbye".getBytes(StandardCharsets.UTF_8), "Goodbye cruel World".getBytes(StandardCharsets.UTF_8))); + final Iterator sourceRecordIterator = lst.iterator(); + final S3SourceTask s3SourceTask = new TestingS3SourceTask(sourceRecordIterator); + + startSourceTask(s3SourceTask); + s3SourceTask.stop(); + final StopWatch stopWatch = new StopWatch(); + stopWatch.start(); + final List results = s3SourceTask.poll(); + stopWatch.stop(); + assertThat(results).isEmpty(); + assertThat(stopWatch.getTime()).isLessThan(TIMING_DELTA); + + } + + private static class TestingS3SourceTask extends S3SourceTask { // NOPMD not a test class + + TestingS3SourceTask(final Iterator realIterator) { + super(); + super.setS3SourceRecordIterator(realIterator); + } + + @Override + protected void setS3SourceRecordIterator(final Iterator iterator) { + // do nothing. + } + } } diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java index e02135d18..cbf8b1325 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java @@ -28,7 +28,7 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.source.SourceRecord; @@ -61,12 +61,12 @@ class RecordProcessorTest { @Mock private AWSV2SourceClient sourceClient; - private AtomicBoolean connectorStopped; + private static final Supplier TRUE = () -> true; + private static final Supplier FALSE = () -> false; private Iterator sourceRecordIterator; @BeforeEach void setUp() { - connectorStopped = new AtomicBoolean(false); sourceRecordIterator = mock(Iterator.class); } @@ -80,7 +80,7 @@ void testProcessRecordsNoRecords() { sourceRecordIterator, results, s3SourceConfig, - connectorStopped, + TRUE, sourceClient, offsetManager ); @@ -100,7 +100,7 @@ void testProcessRecordsWithRecords() throws ConnectException { sourceRecordIterator, results, s3SourceConfig, - connectorStopped, + TRUE, sourceClient, offsetManager ); @@ -111,14 +111,13 @@ void testProcessRecordsWithRecords() throws ConnectException { @Test void testProcessRecordsConnectorStopped() { when(s3SourceConfig.getMaxPollRecords()).thenReturn(5); - connectorStopped.set(true); // Simulate connector stopped final List results = new ArrayList<>(); final List processedRecords = RecordProcessor.processRecords( sourceRecordIterator, results, s3SourceConfig, - connectorStopped, + FALSE, sourceClient, offsetManager ); From 0a106065f12de0dac12122f65b3db55595f8aa0e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Mon, 30 Dec 2024 23:58:08 +0000 Subject: [PATCH 03/44] Added more complete tests --- commons/build.gradle.kts | 1 + .../common/source/AbstractSourceTask.java | 104 ++++++++++-- .../common/source/AbstractSourceTaskTest.java | 128 +++++++++++++++ s3-source-connector/build.gradle.kts | 1 + .../kafka/connect/s3/source/S3SourceTask.java | 58 ++++--- .../s3/source/utils/RecordProcessor.java | 28 +--- .../connect/s3/source/S3SourceTaskTest.java | 152 +++++++++++++++--- .../s3/source/utils/RecordProcessorTest.java | 80 ++++----- settings.gradle.kts | 4 + 9 files changed, 412 insertions(+), 144 deletions(-) create mode 100644 commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java diff --git a/commons/build.gradle.kts b/commons/build.gradle.kts index 232404466..101ef8db9 100644 --- a/commons/build.gradle.kts +++ b/commons/build.gradle.kts @@ -87,6 +87,7 @@ dependencies { testImplementation(jackson.databind) testImplementation(testinglibs.mockito.core) testImplementation(testinglibs.assertj.core) + testImplementation(testinglibs.awaitility) testImplementation(testFixtures(project(":commons"))) testImplementation(testinglibs.woodstox.stax2.api) testImplementation(apache.hadoop.mapreduce.client.core) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java index 990d9e65b..d3e9a2c04 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -33,6 +33,7 @@ import org.apache.commons.lang3.time.StopWatch; import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * This class handles extracting records from an iterator and returning them to Kafka. It uses an exponential backoff @@ -164,14 +165,20 @@ protected boolean stillPolling() { public final List poll() { logger.debug("Polling"); if (connectorStopped.get()) { + logger.info("Stopping"); closeResources(); return Collections.emptyList(); } else { timer.start(); try { - return populateList(); + final List result = populateList(); + if (logger.isDebugEnabled()) { + logger.debug("Poll() returning {} SourceRecords.", result == null ? null : result.size()); + } + return result; } finally { timer.stop(); + timer.reset(); } } } @@ -188,10 +195,11 @@ private List populateList() { while (stillPolling() && results.size() < maxPollRecords) { if (!tryAdd(results, sourceRecordIterator)) { if (!results.isEmpty()) { + logger.debug("tryAdd() did not add to the list, returning current results."); // if we could not get a record and the results are not empty return them break; } - // attempt a backoff + logger.debug("Attempting {}", backoff); backoff.cleanDelay(); } } @@ -241,7 +249,7 @@ protected static class Timer extends StopWatch { * @param duration * the length of time the timer should run. */ - private Timer(final Duration duration) { + Timer(final Duration duration) { super(); this.duration = duration.toMillis(); } @@ -252,7 +260,7 @@ private Timer(final Duration duration) { * @return the maximum duration for the timer. */ public long millisecondsRemaining() { - return super.isStarted() ? super.getTime() - duration : duration; + return super.isStarted() ? duration - super.getTime() : duration; } /** @@ -263,13 +271,48 @@ public long millisecondsRemaining() { public boolean expired() { return super.getTime() >= duration; } + + @Override + public void start() { + try { + super.start(); + } catch (IllegalStateException e) { + throw new IllegalStateException("Timer: " + e.getMessage()); + } + } + + @Override + public void stop() { + try { + super.stop(); + } catch (IllegalStateException e) { + throw new IllegalStateException("Timer: " + e.getMessage()); + } + } + + @Override + public void reset() { + try { + super.reset(); + } catch (IllegalStateException e) { + throw new IllegalStateException("Timer: " + e.getMessage()); + } + } } /** * Performs a delay based on the number of successive {@link #delay()} or {@link #cleanDelay()} calls without a * {@link #reset()}. Delay increases exponentially but never exceeds the time remaining by more than 0.512 seconds. */ - protected static class Backoff { + public static class Backoff { + /** The logger to write to */ + private static final Logger LOGGER = LoggerFactory.getLogger(Backoff.class); + /** + * The maximum jitter random number. Should be a power of 2 for speed. + */ + public static final int MAX_JITTER = 1024; + + public static final int JITTER_SUBTRAHEND = MAX_JITTER / 2; /** * A supplier of the time remaining (in milliseconds) on the overriding timer. */ @@ -304,14 +347,39 @@ public Backoff(final SupplierOfLong timeRemaining) { * Reset the backoff time so that delay is again at the minimum. */ public final void reset() { - // calculate the approx wait count. - maxCount = (int) (Math.log10(timeRemaining.get()) / Math.log10(2)); + // if the reminaing time is 0 or negative the maxCount will be infinity + // so make sure that it is 0 in that case. + final long remainingTime = timeRemaining.get(); + maxCount = remainingTime < 1L ? 0 : (int) (Math.log10(remainingTime) / Math.log10(2)); waitCount = 0; + LOGGER.debug("Reset {}", this); + } + + /** + * Calculates the delay wihtout jitter. + * + * @return the number of milliseconds the delay will be. + */ + public long estimatedDelay() { + long sleepTime = timeRemaining.get(); + if (sleepTime > 0 && waitCount < maxCount) { + sleepTime = (long) Math.min(sleepTime, Math.pow(2, waitCount + 1)); + } + return sleepTime < 0 ? 0 : sleepTime; + } + + /** + * Calculates the range of jitter in milliseconds. + * + * @return the maximum jitter in milliseconds. jitter is +/- maximum jitter. + */ + public int getMaxJitter() { + return MAX_JITTER - JITTER_SUBTRAHEND; } private long timeWithJitter() { // generate approx +/- 0.512 seconds of jitter - final int jitter = random.nextInt(1024) - 512; + final int jitter = random.nextInt(MAX_JITTER) - JITTER_SUBTRAHEND; return (long) Math.pow(2, waitCount) + jitter; } /** @@ -321,15 +389,16 @@ private long timeWithJitter() { * If any thread interrupts this thread. */ public void delay() throws InterruptedException { - long sleepTime = timeRemaining.get(); - if (waitCount < maxCount) { - waitCount++; - sleepTime = Math.min(sleepTime, timeWithJitter()); - } - // don't sleep negative time. if (sleepTime > 0) { - Thread.sleep(sleepTime); + if (waitCount < maxCount) { + waitCount++; + sleepTime = Math.min(sleepTime, timeWithJitter()); + } + // don't sleep negative time. Jitter can introduce negative tme. + if (sleepTime > 0) { + Thread.sleep(sleepTime); + } } } @@ -343,6 +412,11 @@ public void cleanDelay() { // do nothing return results below } } + + @Override + public String toString() { + return String.format("Backoff %s/%s, %s milliseconds remaining.", waitCount, maxCount, timeRemaining.get()); + } } /** diff --git a/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java b/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java new file mode 100644 index 000000000..8ea7768c5 --- /dev/null +++ b/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java @@ -0,0 +1,128 @@ +/* + * Copyright 2024 Aiven Oy + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.aiven.kafka.connect.common.source; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.awaitility.Awaitility.await; + +import java.time.Duration; + +import org.apache.commons.lang3.time.StopWatch; +import org.junit.jupiter.api.Test; + +class AbstractSourceTaskTest { + + @Test + void timerTest() { + final AbstractSourceTask.Timer timer = new AbstractSourceTask.Timer(Duration.ofSeconds(1)); + assertThat(timer.millisecondsRemaining()).isEqualTo(Duration.ofSeconds(1).toMillis()); + timer.start(); + await().atMost(Duration.ofSeconds(2)).until(timer::expired); + assertThat(timer.millisecondsRemaining()).isLessThan(0); + timer.stop(); + assertThat(timer.millisecondsRemaining()).isEqualTo(Duration.ofSeconds(1).toMillis()); + } + + @Test + void timerSequenceTest() { + final AbstractSourceTask.Timer timer = new AbstractSourceTask.Timer(Duration.ofSeconds(1)); + // stopped state does not allow stop + assertThatExceptionOfType(IllegalStateException.class).as("stop while not running") + .isThrownBy(() -> timer.stop()) + .withMessageStartingWith("Timer: "); + timer.reset(); // verify that an exception is not thrown. + + // started state does not allow start + timer.start(); + assertThatExceptionOfType(IllegalStateException.class).as("start while running") + .isThrownBy(() -> timer.start()) + .withMessageStartingWith("Timer: "); + timer.reset(); + timer.start(); // restart the timer. + timer.stop(); + + // stopped state does not allow stop or start + assertThatExceptionOfType(IllegalStateException.class).as("stop after stop") + .isThrownBy(() -> timer.stop()) + .withMessageStartingWith("Timer: "); + assertThatExceptionOfType(IllegalStateException.class).as("start after stop") + .isThrownBy(() -> timer.start()) + .withMessageStartingWith("Timer: "); + timer.reset(); + + // stopped + reset does not allow stop. + assertThatExceptionOfType(IllegalStateException.class).as("stop after reset (1)") + .isThrownBy(() -> timer.stop()) + .withMessageStartingWith("Timer: "); + timer.start(); + timer.reset(); + + // started + reset does not allow stop; + assertThatExceptionOfType(IllegalStateException.class).as("stop after reset (2)") + .isThrownBy(() -> timer.stop()) + .withMessageStartingWith("Timer: "); + } + + @Test + void backoffTest() throws InterruptedException { + final AbstractSourceTask.Timer timer = new AbstractSourceTask.Timer(Duration.ofSeconds(1)); + final AbstractSourceTask.Backoff backoff = new AbstractSourceTask.Backoff(timer::millisecondsRemaining); + final long estimatedDelay = backoff.estimatedDelay(); + assertThat(estimatedDelay).isLessThan(500); + + // execute delay without timer running. + final StopWatch stopWatch = new StopWatch(); + stopWatch.start(); + backoff.delay(); + stopWatch.stop(); + assertThat(stopWatch.getTime()).as("Result without timer running") + .isBetween(estimatedDelay - backoff.getMaxJitter(), estimatedDelay + backoff.getMaxJitter()); + + timer.start(); + for (int i = 0; i < 10; i++) { + stopWatch.reset(); + timer.reset(); + timer.start(); + stopWatch.start(); + await().atMost(Duration.ofSeconds(2)).until(() -> { + backoff.delay(); + return backoff.estimatedDelay() == 0; + }); + stopWatch.stop(); + timer.stop(); + final int step = i; + assertThat(stopWatch.getTime()).as(() -> String.format("Result with timer running at step %s", step)) + .isBetween(Duration.ofSeconds(1).toMillis() - backoff.getMaxJitter(), + Duration.ofSeconds(1).toMillis() + backoff.getMaxJitter()); + } + } + + @Test + void backoffIncrementalTimeTest() throws InterruptedException { + // delay increases in powers of 2. + final long maxDelay = 1000; // not a power of 2 + final AbstractSourceTask.Backoff backoff = new AbstractSourceTask.Backoff(() -> maxDelay); + long expected = 2; + while (backoff.estimatedDelay() < maxDelay) { + assertThat(backoff.estimatedDelay()).isEqualTo(expected); + backoff.delay(); + expected *= 2; + } + assertThat(backoff.estimatedDelay()).isEqualTo(maxDelay); + } +} diff --git a/s3-source-connector/build.gradle.kts b/s3-source-connector/build.gradle.kts index 3530724e0..24c9cc9cf 100644 --- a/s3-source-connector/build.gradle.kts +++ b/s3-source-connector/build.gradle.kts @@ -65,6 +65,7 @@ dependencies { compileOnly(apache.kafka.connect.api) compileOnly(apache.kafka.connect.runtime) + implementation(apache.commons.collection4) implementation(project(":commons")) implementation(project(":s3-commons")) implementation("com.amazonaws:aws-java-sdk-s3:$amazonS3Version") diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index bbdd78b13..b2e044ae6 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -16,12 +16,10 @@ package io.aiven.kafka.connect.s3.source; -import java.util.ArrayList; -import java.util.Collections; import java.util.HashSet; import java.util.Iterator; -import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import org.apache.kafka.connect.source.SourceRecord; @@ -38,6 +36,7 @@ import io.aiven.kafka.connect.s3.source.utils.Version; import com.amazonaws.services.s3.model.AmazonS3Exception; +import org.apache.commons.collections4.IteratorUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,38 +79,36 @@ public String version() { @Override protected Iterator getIterator(SupplierOfLong timer) { // NOPMD cognatavie complexity - return new Iterator<>() { - /** The backoff for Amazon retryable exceptions */ + Iterator inner = new Iterator<>() { + /** + * The backoff for Amazon retryable exceptions + */ final Backoff backoff = new Backoff(timer); + @Override public boolean hasNext() { - try { - // this timer is the master timer from the AbstractSourceTask. - while (stillPolling()) { - try { - return s3SourceRecordIterator.hasNext(); - } catch (AmazonS3Exception exception) { - if (exception.isRetryable()) { - LOGGER.warn("Retryable error encountered during polling. Waiting before retrying...", - exception); - try { - backoff.delay(); - } catch (InterruptedException e) { - LOGGER.warn("Backoff delay was interrupted. Throwing original exception: {}", - exception.getMessage()); - throw exception; - } - } else { - // TODO validate that the iterator does not lose an S3Object. Add test to - // S3ObjectIterator. + while (stillPolling()) { + try { + return s3SourceRecordIterator.hasNext(); + } catch (AmazonS3Exception exception) { + if (exception.isRetryable()) { + LOGGER.warn("Retryable error encountered during polling. Waiting before retrying...", + exception); + try { + backoff.delay(); + } catch (InterruptedException e) { + LOGGER.warn("Backoff delay was interrupted. Throwing original exception: {}", + exception.getMessage()); throw exception; } + } else { + // TODO validate that the iterator does not lose an S3Object. Add test to + // S3ObjectIterator. + throw exception; } } - return false; - } finally { - backoff.reset(); } + return false; } @Override @@ -119,12 +116,11 @@ public SourceRecord next() { final S3SourceRecord s3SourceRecord = s3SourceRecordIterator.next(); offsetManager.incrementAndUpdateOffsetMap(s3SourceRecord.getPartitionMap(), s3SourceRecord.getObjectKey(), 1L); - final List result = RecordProcessor.processRecords( - Collections.singletonList(s3SourceRecord).iterator(), new ArrayList<>(), s3SourceConfig, - S3SourceTask.this::stillPolling, awsv2SourceClient, offsetManager); - return result.get(0); + return RecordProcessor.createSourceRecord(s3SourceRecord, s3SourceConfig, awsv2SourceClient, + offsetManager); } }; + return IteratorUtils.filteredIterator(inner, Objects::nonNull); } @Override diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessor.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessor.java index 7ec803820..9760ffeee 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessor.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessor.java @@ -16,10 +16,6 @@ package io.aiven.kafka.connect.s3.source.utils; -import java.util.Iterator; -import java.util.List; -import java.util.function.Supplier; - import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.source.SourceRecord; @@ -35,29 +31,11 @@ public final class RecordProcessor { private static final Logger LOGGER = LoggerFactory.getLogger(RecordProcessor.class); private RecordProcessor() { - - } - - public static List processRecords(final Iterator sourceRecordIterator, - final List results, final S3SourceConfig s3SourceConfig, final Supplier stillPolling, - final AWSV2SourceClient sourceClient, final OffsetManager offsetManager) { - - final int maxPollRecords = s3SourceConfig.getMaxPollRecords(); - - for (int i = 0; sourceRecordIterator.hasNext() && i < maxPollRecords && stillPolling.get(); i++) { - final S3SourceRecord s3SourceRecord = sourceRecordIterator.next(); - if (s3SourceRecord != null) { - final SourceRecord sourceRecord = createSourceRecord(s3SourceRecord, s3SourceConfig, sourceClient, - offsetManager); - results.add(sourceRecord); - } - } - - return results; } - static SourceRecord createSourceRecord(final S3SourceRecord s3SourceRecord, final S3SourceConfig s3SourceConfig, - final AWSV2SourceClient sourceClient, final OffsetManager offsetManager) { + public static SourceRecord createSourceRecord(final S3SourceRecord s3SourceRecord, + final S3SourceConfig s3SourceConfig, final AWSV2SourceClient sourceClient, + final OffsetManager offsetManager) { try { offsetManager.updateCurrentOffsets(s3SourceRecord.getPartitionMap(), s3SourceRecord.getOffsetMap()); s3SourceRecord.setOffsetMap(offsetManager.getOffsets().get(s3SourceRecord.getPartitionMap())); diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java index 695e873fa..f1a0babac 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java @@ -24,6 +24,7 @@ import static org.mockito.Mockito.when; import java.nio.charset.StandardCharsets; +import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -38,6 +39,7 @@ import org.apache.kafka.connect.source.SourceTaskContext; import org.apache.kafka.connect.storage.OffsetStorageReader; +import io.aiven.kafka.connect.common.config.SourceConfigFragment; import io.aiven.kafka.connect.common.source.AbstractSourceTask; import io.aiven.kafka.connect.common.source.input.ByteArrayTransformer; import io.aiven.kafka.connect.common.source.input.InputFormat; @@ -165,19 +167,19 @@ private void startSourceTask(final S3SourceTask s3SourceTask) { } private void setBasicProperties() { - properties.put(INPUT_FORMAT_KEY, InputFormat.BYTES.getValue()); - properties.put("name", "test_source_connector"); - properties.put("key.converter", "org.apache.kafka.connect.converters.ByteArrayConverter"); - properties.put("value.converter", "org.apache.kafka.connect.converters.ByteArrayConverter"); - properties.put("tasks.max", "1"); - properties.put("connector.class", AivenKafkaConnectS3SourceConnector.class.getName()); - properties.put(TARGET_TOPIC_PARTITIONS, "0,1"); - properties.put(TARGET_TOPICS, "testtopic"); + properties.putIfAbsent(INPUT_FORMAT_KEY, InputFormat.BYTES.getValue()); + properties.putIfAbsent("name", "test_source_connector"); + properties.putIfAbsent("key.converter", "org.apache.kafka.connect.converters.ByteArrayConverter"); + properties.putIfAbsent("value.converter", "org.apache.kafka.connect.converters.ByteArrayConverter"); + properties.putIfAbsent("tasks.max", "1"); + properties.putIfAbsent("connector.class", AivenKafkaConnectS3SourceConnector.class.getName()); + properties.putIfAbsent(TARGET_TOPIC_PARTITIONS, "0,1"); + properties.putIfAbsent(TARGET_TOPICS, "testtopic"); } @Test - void testPollWithEmptyIterator() { + void testPollWithNoDataReturned() { final S3SourceConfig s3SourceConfig = mock(S3SourceConfig.class); when(s3SourceConfig.getMaxPollRecords()).thenReturn(5); final Iterator sourceRecordIterator = Collections.emptyIterator(); @@ -202,13 +204,7 @@ private void assertEquals(final S3SourceRecord s3Record, final SourceRecord sour @Test void testPollsWithRecords() { - final S3SourceConfig s3SourceConfig = mock(S3SourceConfig.class); - when(s3SourceConfig.getMaxPollRecords()).thenReturn(5); - final List lst = new ArrayList<>(); - lst.add(createS3SourceRecord(TOPIC, PARTITION, TEST_BUCKET, OBJECT_KEY, - "Hello".getBytes(StandardCharsets.UTF_8), "Hello World".getBytes(StandardCharsets.UTF_8))); - lst.add(createS3SourceRecord(TOPIC, PARTITION, TEST_BUCKET, OBJECT_KEY + "a", - "Goodbye".getBytes(StandardCharsets.UTF_8), "Goodbye cruel World".getBytes(StandardCharsets.UTF_8))); + final List lst = createS3SourceRecords(2); final Iterator sourceRecordIterator = lst.iterator(); final S3SourceTask s3SourceTask = new TestingS3SourceTask(sourceRecordIterator); @@ -224,15 +220,125 @@ void testPollsWithRecords() { assertThat(stopWatch.getTime()).isLessThan(AbstractSourceTask.MAX_POLL_TIME.toMillis()); } + private List createS3SourceRecords(int count) { + final List lst = new ArrayList<>(); + if (count > 0) { + lst.add(createS3SourceRecord(TOPIC, PARTITION, TEST_BUCKET, OBJECT_KEY, + "Hello".getBytes(StandardCharsets.UTF_8), "Hello World".getBytes(StandardCharsets.UTF_8))); + for (int i = 1; i < count; i++) { + lst.add(createS3SourceRecord(TOPIC, PARTITION, TEST_BUCKET, OBJECT_KEY + i, + "Goodbye".getBytes(StandardCharsets.UTF_8), + String.format("Goodbye cruel World (%s)", i).getBytes(StandardCharsets.UTF_8))); + } + } + return lst; + } + + @Test + void testPollWithInterruptedIterator() { + List lst = createS3SourceRecords(3); + + Iterator inner1 = lst.subList(0, 2).iterator(); + Iterator inner2 = lst.subList(2, 3).iterator(); + Iterator sourceRecordIterator = new Iterator<>() { + Iterator inner = inner1; + @Override + public boolean hasNext() { + if (inner == null) { + inner = inner2; + return false; + } + return inner.hasNext(); + } + + @Override + public S3SourceRecord next() { + S3SourceRecord result = inner.next(); + if (!inner.hasNext()) { + inner = null; + } + return result; + } + }; + + final S3SourceTask s3SourceTask = new TestingS3SourceTask(sourceRecordIterator); + startSourceTask(s3SourceTask); + final StopWatch stopWatch = new StopWatch(); + stopWatch.start(); + List results = s3SourceTask.poll(); + stopWatch.stop(); + + assertThat(results).hasSize(2); + assertEquals(lst.get(0), results.get(0)); + assertEquals(lst.get(1), results.get(1)); + + results = s3SourceTask.poll(); + assertThat(results).hasSize(1); + + assertThat(stopWatch.getTime()).isLessThan(AbstractSourceTask.MAX_POLL_TIME.toMillis()); + + } + + @Test + void testPollWithSlowProducer() { + List lst = createS3SourceRecords(3); + + Iterator sourceRecordIterator = new Iterator<>() { + Iterator inner = lst.iterator(); + @Override + public boolean hasNext() { + return inner.hasNext(); + } + + @Override + public S3SourceRecord next() { + try { + Thread.sleep(Duration.ofSeconds(5).toMillis()); + } catch (InterruptedException e) { + // do nothing. + } + return inner.next(); + } + }; + + final S3SourceTask s3SourceTask = new TestingS3SourceTask(sourceRecordIterator); + startSourceTask(s3SourceTask); + final StopWatch stopWatch = new StopWatch(); + stopWatch.start(); + List results = s3SourceTask.poll(); + assertThat(results).hasSize(1); + results = s3SourceTask.poll(); + assertThat(results).hasSize(1); + results = s3SourceTask.poll(); + assertThat(results).hasSize(1); + results = s3SourceTask.poll(); + assertThat(results).isEmpty(); + } + + @Test + void testPollsWithExcessRecords() { + // test that multiple polls to get all records succeeds. + properties.put(SourceConfigFragment.MAX_POLL_RECORDS, "2"); + + final List lst = createS3SourceRecords(3); + + final Iterator sourceRecordIterator = lst.iterator(); + final S3SourceTask s3SourceTask = new TestingS3SourceTask(sourceRecordIterator); + + startSourceTask(s3SourceTask); + final StopWatch stopWatch = new StopWatch(); + stopWatch.start(); + List results = s3SourceTask.poll(); + assertThat(results).hasSize(2); + results = s3SourceTask.poll(); + assertThat(results).hasSize(1); + stopWatch.stop(); + assertThat(stopWatch.getTime()).isLessThan(AbstractSourceTask.MAX_POLL_TIME.toMillis() * 2); + } + @Test void testPollWhenConnectorStopped() { - final S3SourceConfig s3SourceConfig = mock(S3SourceConfig.class); - when(s3SourceConfig.getMaxPollRecords()).thenReturn(5); - final List lst = new ArrayList<>(); - lst.add(createS3SourceRecord(TOPIC, PARTITION, TEST_BUCKET, OBJECT_KEY, - "Hello".getBytes(StandardCharsets.UTF_8), "Hello World".getBytes(StandardCharsets.UTF_8))); - lst.add(createS3SourceRecord(TOPIC, PARTITION, TEST_BUCKET, OBJECT_KEY + "a", - "Goodbye".getBytes(StandardCharsets.UTF_8), "Goodbye cruel World".getBytes(StandardCharsets.UTF_8))); + final List lst = createS3SourceRecords(3); final Iterator sourceRecordIterator = lst.iterator(); final S3SourceTask s3SourceTask = new TestingS3SourceTask(sourceRecordIterator); diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java index cbf8b1325..7d4a6624f 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java @@ -17,19 +17,18 @@ package io.aiven.kafka.connect.s3.source.utils; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.mockito.internal.verification.VerificationModeFactory.times; -import java.net.ConnectException; -import java.util.ArrayList; import java.util.Iterator; -import java.util.List; import java.util.function.Supplier; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.storage.Converter; @@ -71,58 +70,39 @@ void setUp() { } @Test - void testProcessRecordsNoRecords() { - when(s3SourceConfig.getMaxPollRecords()).thenReturn(5); - when(sourceRecordIterator.hasNext()).thenReturn(false); - - final List results = new ArrayList<>(); - final List processedRecords = RecordProcessor.processRecords( - sourceRecordIterator, - results, - s3SourceConfig, - TRUE, - sourceClient, offsetManager - ); - - assertThat(processedRecords).as("Processed records should be empty when there are no records.").isEmpty(); + void testCreateSourceRecord() throws ConnectException { + + final SourceRecord mockSourceRecord = mock(SourceRecord.class); + final S3SourceRecord mockRecord = mock(S3SourceRecord.class); + when(mockRecord.getSourceRecord()).thenReturn(mockSourceRecord); + + SourceRecord result = RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, + offsetManager); + + verify(offsetManager, times(1)).updateCurrentOffsets(any(), any()); + verify(mockRecord, times(1)).setOffsetMap(any()); + verify(mockRecord, times(1)).getOffsetMap(); + verify(mockRecord, times(2)).getPartitionMap(); + assertThat(result).isEqualTo(mockSourceRecord); + } @Test - void testProcessRecordsWithRecords() throws ConnectException { - when(s3SourceConfig.getMaxPollRecords()).thenReturn(5); - when(sourceRecordIterator.hasNext()).thenReturn(true, false); // One iteration with records + void testCreateSourceRecordWithDataError() throws ConnectException { final S3SourceRecord mockRecord = mock(S3SourceRecord.class); - when(sourceRecordIterator.next()).thenReturn(mockRecord); - - final List results = new ArrayList<>(); - RecordProcessor.processRecords( - sourceRecordIterator, - results, - s3SourceConfig, - TRUE, - sourceClient, offsetManager - ); - - assertThat(results).hasSize(1); - verify(sourceRecordIterator, times(1)).next(); - } + when(mockRecord.getSourceRecord()).thenThrow(new DataException("Testing exception")); - @Test - void testProcessRecordsConnectorStopped() { - when(s3SourceConfig.getMaxPollRecords()).thenReturn(5); - - final List results = new ArrayList<>(); - final List processedRecords = RecordProcessor.processRecords( - sourceRecordIterator, - results, - s3SourceConfig, - FALSE, - sourceClient, offsetManager - ); - - assertThat(processedRecords).as("Processed records should be empty when connector is stopped.").isEmpty(); - verify(sourceRecordIterator, never()).next(); + when(s3SourceConfig.getErrorsTolerance()).thenReturn(ErrorsTolerance.NONE); + + assertThatExceptionOfType(ConnectException.class).as("Errors tolerance: NONE") + .isThrownBy(() -> RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, + offsetManager)); + + when(s3SourceConfig.getErrorsTolerance()).thenReturn(ErrorsTolerance.ALL); + SourceRecord result = RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, + offsetManager); + assertThat(result).isNull(); } @Test diff --git a/settings.gradle.kts b/settings.gradle.kts index 21aca87b9..a4451cb5e 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -6,6 +6,7 @@ val avroConverterVersion by extra("7.2.2") val avroDataVersion by extra("7.2.2") val awaitilityVersion by extra("4.2.1") val commonsTextVersion by extra("1.11.0") +val commonsCollections4Version by extra("4.4") val hadoopVersion by extra("3.4.0") val hamcrestVersion by extra("2.2") val jacksonVersion by extra("2.15.3") @@ -30,6 +31,9 @@ dependencyResolutionManagement { create("apache") { library("avro", "org.apache.avro:avro:$avroVersion") library("commons-text", "org.apache.commons:commons-text:$commonsTextVersion") + library( + "commons-collection4", + "org.apache.commons:commons-collections4:$commonsCollections4Version") library("kafka-connect-api", "org.apache.kafka:connect-api:$kafkaVersion") library("kafka-connect-json", "org.apache.kafka:connect-json:$kafkaVersion") library("kafka-connect-runtime", "org.apache.kafka:connect-runtime:$kafkaVersion") From 9ccb1a8ff3dd68129e06ef7d5effc6a9b2e63e47 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Tue, 31 Dec 2024 08:49:51 +0000 Subject: [PATCH 04/44] Migrate to new AWS client --- .../kafka/connect/s3/source/S3SourceTask.java | 5 ++--- .../connect/s3/source/S3SourceTaskTest.java | 18 +++++------------- 2 files changed, 7 insertions(+), 16 deletions(-) diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index 73653659e..6baccdb94 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -39,7 +39,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.core.exception.SdkException; -import software.amazon.awssdk.services.s3.S3Client; /** * S3SourceTask is a Kafka Connect SourceTask implementation that reads from source-s3 buckets and generates Kafka @@ -92,8 +91,8 @@ public boolean hasNext() { while (stillPolling()) { try { return s3SourceRecordIterator.hasNext(); - } catch (AmazonS3Exception exception) { - if (exception.isRetryable()) { + } catch (SdkException exception) { + if (exception.retryable()) { LOGGER.warn("Retryable error encountered during polling. Waiting before retrying...", exception); try { diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java index 682468089..580af7dfc 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java @@ -23,7 +23,8 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; - +import java.net.URI; +import java.net.URISyntaxException; import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.ArrayList; @@ -45,9 +46,8 @@ import io.aiven.kafka.connect.common.source.input.ByteArrayTransformer; import io.aiven.kafka.connect.common.source.input.InputFormat; import io.aiven.kafka.connect.config.s3.S3ConfigFragment; - +import io.aiven.kafka.connect.iam.AwsCredentialProviderFactory; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; -import io.aiven.kafka.connect.s3.source.testutils.BucketAccessor; import io.aiven.kafka.connect.s3.source.utils.ConnectUtils; import io.aiven.kafka.connect.s3.source.utils.S3SourceRecord; @@ -58,16 +58,11 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; - -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.Mock; -import org.mockito.junit.jupiter.MockitoExtension; import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; import software.amazon.awssdk.core.retry.RetryMode; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.S3Configuration; - final class S3SourceTaskTest { /** @@ -80,7 +75,6 @@ final class S3SourceTaskTest { private static final String TEST_BUCKET = "test-bucket"; - private static final String TOPIC = "TOPIC1"; private static final int PARTITION = 1; @@ -89,7 +83,6 @@ final class S3SourceTaskTest { // TODO S3Mock has not been maintained in 4 years // Adobe have an alternative we can move to. - private static S3Mock s3Api; private static S3Client s3Client; @@ -120,9 +113,6 @@ public static void setUpClass() throws URISyntaxException { .serviceConfiguration(S3Configuration.builder().pathStyleAccessEnabled(true).build()) .credentialsProvider(credentialFactory.getAwsV2Provider(config.getS3ConfigFragment())) .build(); - - final BucketAccessor testBucketAccessor = new BucketAccessor(s3Client, TEST_BUCKET); - testBucketAccessor.createBucket(); } @AfterAll @@ -134,6 +124,8 @@ public static void tearDownClass() { public void setUp() { properties = new HashMap<>(commonProperties); s3Client.createBucket(create -> create.bucket(TEST_BUCKET).build()); + // mockedSourceTaskContext = mock(SourceTaskContext.class); + // mockedOffsetStorageReader = mock(OffsetStorageReader.class); } @AfterEach From d3ce578eebaa27bc6b26547e4cd2a727022560be Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Tue, 31 Dec 2024 12:16:34 +0000 Subject: [PATCH 05/44] added AWS Integration test --- .../common/source/AbstractSourceTask.java | 44 +++-- .../connect/s3/source/AwsIntegrationTest.java | 158 ++++++++++++++++++ .../connect/s3/source/IntegrationBase.java | 20 +++ .../connect/s3/source/IntegrationTest.java | 46 ++--- .../kafka/connect/s3/source/S3SourceTask.java | 7 +- .../s3/source/utils/AWSV2SourceClient.java | 46 ++++- .../s3/source/utils/SourceRecordIterator.java | 14 +- 7 files changed, 276 insertions(+), 59 deletions(-) create mode 100644 s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java index d3e9a2c04..1f9d55a79 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -149,6 +149,7 @@ private boolean tryAdd(final List results, final Iterator results, final Iterator poll() { - logger.debug("Polling"); - if (connectorStopped.get()) { - logger.info("Stopping"); - closeResources(); - return Collections.emptyList(); - } else { - timer.start(); - try { - final List result = populateList(); - if (logger.isDebugEnabled()) { - logger.debug("Poll() returning {} SourceRecords.", result == null ? null : result.size()); + try { + logger.debug("Polling"); + if (connectorStopped.get()) { + logger.info("Stopping"); + closeResources(); + return Collections.emptyList(); + } else { + timer.start(); + try { + final List result = populateList(); + if (logger.isInfoEnabled()) { // TODO reset this to debug + logger.info("Poll() returning {} SourceRecords.", result == null ? null : result.size()); + } + return result; + } finally { + timer.stop(); + timer.reset(); } - return result; - } finally { - timer.stop(); - timer.reset(); } + } catch (RuntimeException e) { + logger.error("******************** " + e.getMessage(), e); + throw e; } } @@ -195,11 +203,11 @@ private List populateList() { while (stillPolling() && results.size() < maxPollRecords) { if (!tryAdd(results, sourceRecordIterator)) { if (!results.isEmpty()) { - logger.debug("tryAdd() did not add to the list, returning current results."); + logger.info("tryAdd() did not add to the list, returning current results."); // if we could not get a record and the results are not empty return them break; } - logger.debug("Attempting {}", backoff); + logger.info("Attempting {}", backoff); backoff.cleanDelay(); } } diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java new file mode 100644 index 000000000..9027f91fe --- /dev/null +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java @@ -0,0 +1,158 @@ +package io.aiven.kafka.connect.s3.source; + +import io.aiven.kafka.connect.common.source.input.InputFormat; +import io.aiven.kafka.connect.common.source.input.TransformerFactory; +import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; +import io.aiven.kafka.connect.s3.source.testutils.BucketAccessor; +import io.aiven.kafka.connect.s3.source.utils.AWSV2SourceClient; +import io.aiven.kafka.connect.s3.source.utils.OffsetManager; +import io.aiven.kafka.connect.s3.source.utils.S3SourceRecord; +import io.aiven.kafka.connect.s3.source.utils.SourceRecordIterator; +import org.apache.kafka.connect.source.SourceTaskContext; +import org.apache.kafka.connect.storage.OffsetStorageReader; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; +import org.testcontainers.containers.localstack.LocalStackContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; +import software.amazon.awssdk.services.s3.S3Client; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; + +import static io.aiven.kafka.connect.common.config.SchemaRegistryFragment.INPUT_FORMAT_KEY; +import static io.aiven.kafka.connect.common.config.SourceConfigFragment.TARGET_TOPICS; +import static io.aiven.kafka.connect.common.config.SourceConfigFragment.TARGET_TOPIC_PARTITIONS; +import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_ACCESS_KEY_ID_CONFIG; +import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_S3_BUCKET_NAME_CONFIG; +import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_S3_ENDPOINT_CONFIG; +import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_S3_PREFIX_CONFIG; +import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_SECRET_ACCESS_KEY_CONFIG; +import static io.aiven.kafka.connect.s3.source.S3SourceTask.OBJECT_KEY; +import static io.aiven.kafka.connect.s3.source.utils.OffsetManager.SEPARATOR; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@Testcontainers +public class AwsIntegrationTest implements IntegrationBase { + + private static final String COMMON_PREFIX = "s3-source-connector-for-apache-kafka-AWS-test-"; + + @Container + public static final LocalStackContainer LOCALSTACK = IntegrationBase.createS3Container(); + + private static String s3Prefix; + + private S3Client s3Client; + private String s3Endpoint; + + private BucketAccessor testBucketAccessor; + + + @Override + public String getS3Prefix() { + return s3Prefix; + } + + @Override + public S3Client getS3Client() { + return s3Client; + } + + @BeforeAll + static void setUpAll() throws IOException, InterruptedException { + s3Prefix = COMMON_PREFIX + ZonedDateTime.now().format(DateTimeFormatter.ISO_LOCAL_DATE_TIME) + "/"; + } + + @BeforeEach + void setupAWS() { + s3Client = IntegrationBase.createS3Client(LOCALSTACK); + s3Endpoint = LOCALSTACK.getEndpoint().toString(); + testBucketAccessor = new BucketAccessor(s3Client, TEST_BUCKET_NAME); + testBucketAccessor.createBucket(); + } + + @AfterEach + void tearDownAWS() { + testBucketAccessor.removeBucket(); + s3Client.close(); + } + + private Map getConfig(final String topics, final int maxTasks) { + final Map config = new HashMap<>(); + config.put(AWS_ACCESS_KEY_ID_CONFIG, S3_ACCESS_KEY_ID); + config.put(AWS_SECRET_ACCESS_KEY_CONFIG, S3_SECRET_ACCESS_KEY); + config.put(AWS_S3_ENDPOINT_CONFIG, s3Endpoint); + config.put(AWS_S3_BUCKET_NAME_CONFIG, TEST_BUCKET_NAME); + config.put(AWS_S3_PREFIX_CONFIG, getS3Prefix()); + config.put(TARGET_TOPIC_PARTITIONS, "0,1"); + config.put(TARGET_TOPICS, topics); + config.put("key.converter", "org.apache.kafka.connect.converters.ByteArrayConverter"); + config.put(VALUE_CONVERTER_KEY, "org.apache.kafka.connect.converters.ByteArrayConverter"); + config.put("tasks.max", String.valueOf(maxTasks)); + return config; + } + /** + * Test the integration with the Amazon connector + * @param testInfo + */ + @Test + void sourceRecordIteratorTest(final TestInfo testInfo) { + final var topicName = IntegrationBase.topicName(testInfo); + final Map configData = getConfig(topicName, 1); + + configData.put(INPUT_FORMAT_KEY, InputFormat.BYTES.getValue()); + + final String testData1 = "Hello, Kafka Connect S3 Source! object 1"; + final String testData2 = "Hello, Kafka Connect S3 Source! object 2"; + + final List offsetKeys = new ArrayList<>(); + final List expectedKeys = new ArrayList<>(); + // write 2 objects to s3 + expectedKeys.add(writeToS3(topicName, testData1.getBytes(StandardCharsets.UTF_8), "00000")); + expectedKeys.add(writeToS3(topicName, testData2.getBytes(StandardCharsets.UTF_8), "00000")); + expectedKeys.add(writeToS3(topicName, testData1.getBytes(StandardCharsets.UTF_8), "00001")); + expectedKeys.add(writeToS3(topicName, testData2.getBytes(StandardCharsets.UTF_8), "00001")); + + // we don't expext the empty one. + offsetKeys.addAll(expectedKeys); + offsetKeys.add(writeToS3(topicName, new byte[0], "00003")); + + assertThat(testBucketAccessor.listObjects()).hasSize(5); + + S3SourceConfig s3SourceConfig = new S3SourceConfig(configData); + SourceTaskContext context = mock(SourceTaskContext.class); + OffsetStorageReader offsetStorageReader = mock(OffsetStorageReader.class); + when(context.offsetStorageReader()).thenReturn(offsetStorageReader); + when(offsetStorageReader.offsets(any())).thenReturn(new HashMap<>()); + + OffsetManager offsetManager = new OffsetManager(context, s3SourceConfig); + + AWSV2SourceClient sourceClient = new AWSV2SourceClient(s3SourceConfig, new HashSet<>()); + + SourceRecordIterator sourceRecordIterator = new SourceRecordIterator(s3SourceConfig, offsetManager, + TransformerFactory.getTransformer(InputFormat.BYTES), sourceClient); + + HashSet seenKeys = new HashSet<>(); + while (sourceRecordIterator.hasNext()) { + S3SourceRecord s3SourceRecord = sourceRecordIterator.next(); + String key = OBJECT_KEY + SEPARATOR + s3SourceRecord.getObjectKey(); + assertThat(offsetKeys).contains(key); + seenKeys.add(key); + } + assertThat(seenKeys).containsAll(expectedKeys); + } +} diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java index 6b505b996..0f6a85767 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java @@ -16,6 +16,8 @@ package io.aiven.kafka.connect.s3.source; +import static io.aiven.kafka.connect.s3.source.S3SourceTask.OBJECT_KEY; +import static io.aiven.kafka.connect.s3.source.utils.OffsetManager.SEPARATOR; import static org.assertj.core.api.Assertions.assertThat; import static org.awaitility.Awaitility.await; @@ -59,13 +61,31 @@ import org.testcontainers.utility.DockerImageName; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; +import software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; public interface IntegrationBase { String PLUGINS_S3_SOURCE_CONNECTOR_FOR_APACHE_KAFKA = "plugins/s3-source-connector-for-apache-kafka/"; String S3_SOURCE_CONNECTOR_FOR_APACHE_KAFKA_TEST = "s3-source-connector-for-apache-kafka-test-"; ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + String TEST_BUCKET_NAME = "test-bucket0"; + String S3_ACCESS_KEY_ID = "test-key-id0"; + String VALUE_CONVERTER_KEY = "value.converter"; + String S3_SECRET_ACCESS_KEY = "test_secret_key0"; + + S3Client getS3Client(); + + String getS3Prefix(); + + default String writeToS3(final String topicName, final byte[] testDataBytes, final String partitionId) { + final String objectKey = org.apache.commons.lang3.StringUtils.defaultIfBlank(getS3Prefix(), "") + topicName + "-" + partitionId + "-" + + System.currentTimeMillis() + ".txt"; + final PutObjectRequest request = PutObjectRequest.builder().bucket(IntegrationTest.TEST_BUCKET_NAME).key(objectKey).build(); + getS3Client().putObject(request, RequestBody.fromBytes(testDataBytes)); + return OBJECT_KEY + SEPARATOR + objectKey; + } default AdminClient newAdminClient(final String bootstrapServers) { final Properties adminClientConfig = new Properties(); diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java index 884051e30..766b709a5 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java @@ -32,6 +32,9 @@ import static java.util.Map.entry; import static org.assertj.core.api.Assertions.assertThat; import static org.awaitility.Awaitility.await; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -53,6 +56,12 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import io.aiven.kafka.connect.common.source.input.TransformerFactory; +import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; +import io.aiven.kafka.connect.s3.source.utils.AWSV2SourceClient; +import io.aiven.kafka.connect.s3.source.utils.OffsetManager; +import io.aiven.kafka.connect.s3.source.utils.S3SourceRecord; +import io.aiven.kafka.connect.s3.source.utils.SourceRecordIterator; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.serialization.ByteArrayDeserializer; @@ -68,18 +77,18 @@ import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; import org.apache.avro.io.DatumWriter; +import org.apache.kafka.connect.source.SourceTaskContext; +import org.apache.kafka.connect.storage.OffsetStorageReader; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInfo; -import org.junit.platform.commons.util.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.localstack.LocalStackContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; -import software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.PutObjectRequest; @@ -92,13 +101,6 @@ final class IntegrationTest implements IntegrationBase { private static final String COMMON_PREFIX = "s3-source-connector-for-apache-kafka-test-"; private static final int OFFSET_FLUSH_INTERVAL_MS = 500; - private static final String S3_ACCESS_KEY_ID = "test-key-id0"; - private static final String S3_SECRET_ACCESS_KEY = "test_secret_key0"; - - private static final String VALUE_CONVERTER_KEY = "value.converter"; - - private static final String TEST_BUCKET_NAME = "test-bucket0"; - private static String s3Endpoint; private static String s3Prefix; private static BucketAccessor testBucketAccessor; @@ -112,6 +114,16 @@ final class IntegrationTest implements IntegrationBase { private static S3Client s3Client; + public S3Client getS3Client() { + return s3Client; + } + + public String getS3Prefix() { + return s3Prefix; + } + + public + @BeforeAll static void setUpAll() throws IOException, InterruptedException { s3Prefix = COMMON_PREFIX + ZonedDateTime.now().format(DateTimeFormatter.ISO_LOCAL_DATE_TIME) + "/"; @@ -159,7 +171,7 @@ void tearDown() { @Test void bytesTest(final TestInfo testInfo) { final var topicName = IntegrationBase.topicName(testInfo); - final Map connectorConfig = getConfig(CONNECTOR_NAME, topicName, 2); + final Map connectorConfig = getConfig(CONNECTOR_NAME, topicName, 1); connectorConfig.put(INPUT_FORMAT_KEY, InputFormat.BYTES.getValue()); connectRunner.configureConnector(CONNECTOR_NAME, connectorConfig); @@ -253,7 +265,7 @@ void parquetTest(final TestInfo testInfo) throws IOException { final var topicName = IntegrationBase.topicName(testInfo); final String partition = "00000"; - final String fileName = addPrefixOrDefault("") + topicName + "-" + partition + "-" + System.currentTimeMillis() + final String fileName = org.apache.commons.lang3.StringUtils.defaultIfBlank(getS3Prefix(), "") + topicName + "-" + partition + "-" + System.currentTimeMillis() + ".txt"; final String name = "testuser"; @@ -337,18 +349,6 @@ private static byte[] generateNextAvroMessagesStartingFromId(final int messageId } } - private static String writeToS3(final String topicName, final byte[] testDataBytes, final String partitionId) { - final String objectKey = addPrefixOrDefault("") + topicName + "-" + partitionId + "-" - + System.currentTimeMillis() + ".txt"; - final PutObjectRequest request = PutObjectRequest.builder().bucket(TEST_BUCKET_NAME).key(objectKey).build(); - s3Client.putObject(request, RequestBody.fromBytes(testDataBytes)); - return OBJECT_KEY + SEPARATOR + objectKey; - } - - private static String addPrefixOrDefault(final String defaultValue) { - return StringUtils.isNotBlank(s3Prefix) ? s3Prefix : defaultValue; - } - private Map getConfig(final String connectorName, final String topics, final int maxTasks) { final Map config = new HashMap<>(basicS3ConnectorConfig()); config.put("name", connectorName); diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index 6baccdb94..43ed82409 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -146,9 +146,10 @@ public void commit() { public void commitRecord(final SourceRecord record) { if (LOGGER.isInfoEnabled()) { final Map map = (Map) record.sourceOffset(); - LOGGER.info("Committed individual record {} {} {} committed", map.get(BUCKET), map.get(OBJECT_KEY), - offsetManager.recordsProcessedForObjectKey((Map) record.sourcePartition(), - map.get(OBJECT_KEY).toString())); +// LOGGER.info("Committed individual record {} {} {} committed", map.get(BUCKET), map.get(OBJECT_KEY), +// offsetManager.recordsProcessedForObjectKey((Map) record.sourcePartition(), +// map.get(OBJECT_KEY).toString())); + LOGGER.info("Committed individual record {} committed", map); } } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java index 44e28dfa7..655e5ebf2 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java @@ -29,6 +29,8 @@ import org.apache.commons.io.function.IOSupplier; import org.codehaus.plexus.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import software.amazon.awssdk.core.ResponseBytes; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.GetObjectRequest; @@ -42,6 +44,7 @@ */ public class AWSV2SourceClient { + private static final Logger LOGGER = LoggerFactory.getLogger(AWSV2SourceClient.class); public static final int PAGE_SIZE_FACTOR = 2; private final S3SourceConfig s3SourceConfig; private final S3Client s3Client; @@ -50,6 +53,9 @@ public class AWSV2SourceClient { private Predicate filterPredicate = s3Object -> s3Object.size() > 0; private final Set failedObjectKeys; + private final int taskId; + private final int maxTasks; + /** * @param s3SourceConfig * configuration for Source connector @@ -57,11 +63,7 @@ public class AWSV2SourceClient { * all objectKeys which have already been tried but have been unable to process. */ public AWSV2SourceClient(final S3SourceConfig s3SourceConfig, final Set failedObjectKeys) { - this.s3SourceConfig = s3SourceConfig; - final S3ClientFactory s3ClientFactory = new S3ClientFactory(); - this.s3Client = s3ClientFactory.createAmazonS3Client(s3SourceConfig); - this.bucketName = s3SourceConfig.getAwsS3BucketName(); - this.failedObjectKeys = new HashSet<>(failedObjectKeys); + this(new S3ClientFactory().createAmazonS3Client(s3SourceConfig), s3SourceConfig, failedObjectKeys); } /** @@ -80,6 +82,38 @@ public AWSV2SourceClient(final S3SourceConfig s3SourceConfig, final Set this.s3Client = s3Client; this.bucketName = s3SourceConfig.getAwsS3BucketName(); this.failedObjectKeys = new HashSet<>(failedObjectKeys); + + // TODO the code below should be configured in some sort of taks assignement method/process/call. + int maxTasks; + try { + final Object value = s3SourceConfig.originals().get("tasks.max"); + if (value == null) { + LOGGER.info("Setting tasks.max to 1"); + maxTasks = 1; + } else { + maxTasks = Integer.parseInt(value.toString()); + } + } catch (NumberFormatException e) { // NOPMD catch null pointer + LOGGER.warn("Invalid tasks.max: {}", e.getMessage()); + LOGGER.info("Setting tasks.max to 1"); + maxTasks = 1; + } + this.maxTasks = maxTasks; + int taskId; + try { + final Object value = s3SourceConfig.originals().get("task.id"); + if (value == null) { + LOGGER.info("Setting task.id to 0"); + taskId = 0; + } else { + taskId = Integer.parseInt(value.toString()) % maxTasks; + } + } catch (NumberFormatException e) { // NOPMD catch null pointer + LOGGER.warn("Invalid task.id: {}", e.getMessage()); + LOGGER.info("Setting task.id to 0"); + taskId = 0; + } + this.taskId = taskId; } public Iterator getListOfObjectKeys(final String startToken) { @@ -133,8 +167,6 @@ public void setFilterPredicate(final Predicate predicate) { } private boolean assignObjectToTask(final String objectKey) { - final int maxTasks = Integer.parseInt(s3SourceConfig.originals().get("tasks.max").toString()); - final int taskId = Integer.parseInt(s3SourceConfig.originals().get("task.id").toString()) % maxTasks; final int taskAssignment = Math.floorMod(objectKey.hashCode(), maxTasks); return taskAssignment == taskId; } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java index 26f3c03cf..4df03bde0 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java @@ -23,6 +23,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.NoSuchElementException; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Stream; @@ -207,20 +208,17 @@ public S3SourceRecord next() { @Override public boolean hasNext() { - return recordIterator.hasNext() || objectListIterator.hasNext(); - } - - @Override - public S3SourceRecord next() { if (!recordIterator.hasNext()) { nextS3Object(); } + return recordIterator.hasNext(); + } + @Override + public S3SourceRecord next() { if (!recordIterator.hasNext()) { - // If there are still no records, return null or throw an exception - return null; // Or throw new NoSuchElementException(); + throw new NoSuchElementException(); } - return recordIterator.next(); } From e744e32524fca918230ac2ff55d03d988e9e1ace Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Thu, 2 Jan 2025 13:37:21 +0000 Subject: [PATCH 06/44] attempts to fix polling --- .../common/source/AbstractSourceTask.java | 10 +- .../common/source/input/Transformer.java | 12 +- .../connect/s3/source/AwsIntegrationTest.java | 92 ++++- .../connect/s3/source/IntegrationBase.java | 24 ++ .../connect/s3/source/IntegrationTest.java | 51 +-- .../kafka/connect/s3/source/S3SourceTask.java | 4 +- .../s3/source/utils/AWSV2SourceClient.java | 15 +- .../s3/source/utils/OffsetManager.java | 14 +- .../s3/source/utils/RecordProcessor.java | 4 +- .../s3/source/utils/S3ObjectIterator.java | 98 +++++ .../s3/source/utils/S3SourceRecord.java | 19 +- .../s3/source/utils/SourceRecordIterator.java | 385 +++++++++++------- .../connect/s3/source/S3SourceTaskTest.java | 7 +- .../source/utils/AWSV2SourceClientTest.java | 1 + .../s3/source/utils/RecordProcessorTest.java | 12 +- .../utils/SourceRecordIteratorTest.java | 5 +- 16 files changed, 519 insertions(+), 234 deletions(-) create mode 100644 s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3ObjectIterator.java diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java index 1f9d55a79..0273bd3b2 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -58,7 +58,7 @@ public abstract class AbstractSourceTask extends SourceTask { * The maximum time to spend polling. This is set to 5 seconds as that is the time that is allotted to a system for * shutdown. */ - public static final Duration MAX_POLL_TIME = Duration.ofSeconds(5); + public static final Duration MAX_POLL_TIME = Duration.ofMinutes(5); // TODO reset this to 5 seconds /** * The boolean that indicates the connector is stopped. */ @@ -146,7 +146,9 @@ public final void start(final Map props) { private boolean tryAdd(final List results, final Iterator sourceRecordIterator) { if (sourceRecordIterator.hasNext()) { backoff.reset(); - results.add(sourceRecordIterator.next()); + SourceRecord sr = sourceRecordIterator.next(); + logger.info("tryAdd() : read record "+sr.sourceOffset()); + results.add(sr); return true; } logger.info("No records found in tryAdd call"); @@ -160,7 +162,7 @@ private boolean tryAdd(final List results, final Iterator poll() { try { final List result = populateList(); if (logger.isInfoEnabled()) { // TODO reset this to debug - logger.info("Poll() returning {} SourceRecords.", result == null ? null : result.size()); + logger.info("********************************** Poll() returning {} SourceRecords.", result == null ? null : result.size()); } return result; } finally { diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/input/Transformer.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/input/Transformer.java index 196d9ae3c..5bc353f6c 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/input/Transformer.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/input/Transformer.java @@ -42,6 +42,15 @@ public final Stream getRecords(final IOSupplier inputStreamIOSup return StreamSupport.stream(spliterator, false).onClose(spliterator::close).skip(skipRecords); } + public final Stream getValues(final IOSupplier inputStreamIOSupplier, final String topic, + final int topicPartition, final AbstractConfig sourceConfig, final long skipRecords) { + + final StreamSpliterator spliterator = createSpliterator(inputStreamIOSupplier, topic, topicPartition, + sourceConfig); + return StreamSupport.stream(spliterator, false).onClose(spliterator::close).skip(skipRecords) + .map(t -> getValueData(t, topic, sourceConfig)); + } + /** * Creates the stream spliterator for this transformer. * @@ -121,6 +130,7 @@ public final void close() { try { if (inputStream != null) { inputStream.close(); + inputStream = null; closed = true; } } catch (IOException e) { @@ -146,7 +156,7 @@ public final void close() { public final boolean tryAdvance(final Consumer action) { boolean result = false; if (closed) { - logger.error("Attempt to advance after closed"); + return false; } try { if (inputStream == null) { diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java index 9027f91fe..80ba203a7 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java @@ -8,6 +8,7 @@ import io.aiven.kafka.connect.s3.source.utils.OffsetManager; import io.aiven.kafka.connect.s3.source.utils.S3SourceRecord; import io.aiven.kafka.connect.s3.source.utils.SourceRecordIterator; +import org.apache.avro.Schema; import org.apache.kafka.connect.source.SourceTaskContext; import org.apache.kafka.connect.storage.OffsetStorageReader; import org.junit.jupiter.api.AfterEach; @@ -22,15 +23,17 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.nio.file.Path; import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; +import static io.aiven.kafka.connect.common.config.SchemaRegistryFragment.AVRO_VALUE_SERIALIZER; import static io.aiven.kafka.connect.common.config.SchemaRegistryFragment.INPUT_FORMAT_KEY; import static io.aiven.kafka.connect.common.config.SourceConfigFragment.TARGET_TOPICS; import static io.aiven.kafka.connect.common.config.SourceConfigFragment.TARGET_TOPIC_PARTITIONS; @@ -73,7 +76,7 @@ public S3Client getS3Client() { } @BeforeAll - static void setUpAll() throws IOException, InterruptedException { + static void setUpAll() { s3Prefix = COMMON_PREFIX + ZonedDateTime.now().format(DateTimeFormatter.ISO_LOCAL_DATE_TIME) + "/"; } @@ -105,12 +108,13 @@ private Map getConfig(final String topics, final int maxTasks) { config.put("tasks.max", String.valueOf(maxTasks)); return config; } + /** * Test the integration with the Amazon connector - * @param testInfo + * @param testInfo The testing configuration. */ @Test - void sourceRecordIteratorTest(final TestInfo testInfo) { + void sourceRecordIteratorBytesTest(final TestInfo testInfo) { final var topicName = IntegrationBase.topicName(testInfo); final Map configData = getConfig(topicName, 1); @@ -143,7 +147,7 @@ void sourceRecordIteratorTest(final TestInfo testInfo) { AWSV2SourceClient sourceClient = new AWSV2SourceClient(s3SourceConfig, new HashSet<>()); - SourceRecordIterator sourceRecordIterator = new SourceRecordIterator(s3SourceConfig, offsetManager, + Iterator sourceRecordIterator = new SourceRecordIterator(s3SourceConfig, offsetManager, TransformerFactory.getTransformer(InputFormat.BYTES), sourceClient); HashSet seenKeys = new HashSet<>(); @@ -155,4 +159,82 @@ void sourceRecordIteratorTest(final TestInfo testInfo) { } assertThat(seenKeys).containsAll(expectedKeys); } + + @Test + void sourceRecordIteratorAvroTest(final TestInfo testInfo) throws IOException { + final var topicName = IntegrationBase.topicName(testInfo); + + final Map configData = getConfig(topicName, 1); + + configData.put(INPUT_FORMAT_KEY, InputFormat.AVRO.getValue()); + configData.put(VALUE_CONVERTER_KEY, "io.confluent.connect.avro.AvroConverter"); + configData.put(AVRO_VALUE_SERIALIZER, "io.confluent.kafka.serializers.KafkaAvroSerializer"); + + // Define Avro schema + final String schemaJson = "{\n" + " \"type\": \"record\",\n" + " \"name\": \"TestRecord\",\n" + + " \"fields\": [\n" + " {\"name\": \"message\", \"type\": \"string\"},\n" + + " {\"name\": \"id\", \"type\": \"int\"}\n" + " ]\n" + "}"; + final Schema.Parser parser = new Schema.Parser(); + final Schema schema = parser.parse(schemaJson); + + final int numOfRecsFactor = 5000; + + final byte[] outputStream1 = IntegrationBase.generateNextAvroMessagesStartingFromId(1, numOfRecsFactor, schema); + final byte[] outputStream2 = IntegrationBase.generateNextAvroMessagesStartingFromId(numOfRecsFactor + 1, numOfRecsFactor, + schema); + final byte[] outputStream3 = IntegrationBase.generateNextAvroMessagesStartingFromId(2 * numOfRecsFactor + 1, numOfRecsFactor, + schema); + final byte[] outputStream4 = IntegrationBase.generateNextAvroMessagesStartingFromId(3 * numOfRecsFactor + 1, numOfRecsFactor, + schema); + final byte[] outputStream5 = IntegrationBase.generateNextAvroMessagesStartingFromId(4 * numOfRecsFactor + 1, numOfRecsFactor, + schema); + + final Set offsetKeys = new HashSet<>(); + + offsetKeys.add(writeToS3(topicName, outputStream1, "00001")); + offsetKeys.add(writeToS3(topicName, outputStream2, "00001")); + + offsetKeys.add(writeToS3(topicName, outputStream3, "00002")); + offsetKeys.add(writeToS3(topicName, outputStream4, "00002")); + offsetKeys.add(writeToS3(topicName, outputStream5, "00002")); + + assertThat(testBucketAccessor.listObjects()).hasSize(5); + + S3SourceConfig s3SourceConfig = new S3SourceConfig(configData); + SourceTaskContext context = mock(SourceTaskContext.class); + OffsetStorageReader offsetStorageReader = mock(OffsetStorageReader.class); + when(context.offsetStorageReader()).thenReturn(offsetStorageReader); + when(offsetStorageReader.offsets(any())).thenReturn(new HashMap<>()); + + OffsetManager offsetManager = new OffsetManager(context, s3SourceConfig); + + AWSV2SourceClient sourceClient = new AWSV2SourceClient(s3SourceConfig, new HashSet<>()); + + Iterator sourceRecordIterator = new SourceRecordIterator(s3SourceConfig, offsetManager, + TransformerFactory.getTransformer(InputFormat.AVRO), sourceClient); + + HashSet seenKeys = new HashSet<>(); + Map> seenRecords = new HashMap<>(); + while (sourceRecordIterator.hasNext()) { + S3SourceRecord s3SourceRecord = sourceRecordIterator.next(); + String key = OBJECT_KEY + SEPARATOR + s3SourceRecord.getObjectKey(); + seenRecords.compute(key, (k, v) -> { + List lst = v == null ? new ArrayList<>() : v; + lst.add(s3SourceRecord.getRecordNumber()); + return lst; + }); + assertThat(offsetKeys).contains(key); + seenKeys.add(key); + } + assertThat(seenKeys).containsAll(offsetKeys); + assertThat(seenRecords).hasSize(5); + List expected = new ArrayList<>(); + for (long l=0; l < numOfRecsFactor; l++) { + expected.add(l+1); + } + for (String key : offsetKeys) { + List seen = seenRecords.get(key); + assertThat(seen).as("Count for "+key).containsExactlyInAnyOrderElementsOf(expected); + } + } } diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java index 0f6a85767..e3fdc1f87 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.awaitility.Awaitility.await; +import java.io.ByteArrayOutputStream; import java.io.File; import java.io.IOException; import java.net.ServerSocket; @@ -38,6 +39,11 @@ import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.io.DatumWriter; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.NewTopic; @@ -75,6 +81,24 @@ public interface IntegrationBase { String VALUE_CONVERTER_KEY = "value.converter"; String S3_SECRET_ACCESS_KEY = "test_secret_key0"; + static byte[] generateNextAvroMessagesStartingFromId(final int messageId, final int noOfAvroRecs, + final Schema schema) throws IOException { + final DatumWriter datumWriter = new GenericDatumWriter<>(schema); + try (DataFileWriter dataFileWriter = new DataFileWriter<>(datumWriter); + ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) { + dataFileWriter.create(schema, outputStream); + for (int i = messageId; i < messageId + noOfAvroRecs; i++) { + final GenericRecord avroRecord = new GenericData.Record(schema); // NOPMD + avroRecord.put("message", "Hello, Kafka Connect S3 Source! object " + i); + avroRecord.put("id", i); + dataFileWriter.append(avroRecord); + } + + dataFileWriter.flush(); + return outputStream.toByteArray(); + } + } + S3Client getS3Client(); String getS3Prefix(); diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java index 766b709a5..33459c2e0 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java @@ -27,16 +27,12 @@ import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_S3_ENDPOINT_CONFIG; import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_S3_PREFIX_CONFIG; import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_SECRET_ACCESS_KEY_CONFIG; -import static io.aiven.kafka.connect.s3.source.S3SourceTask.OBJECT_KEY; -import static io.aiven.kafka.connect.s3.source.utils.OffsetManager.SEPARATOR; import static java.util.Map.entry; import static org.assertj.core.api.Assertions.assertThat; import static org.awaitility.Awaitility.await; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.nio.file.Files; @@ -56,12 +52,6 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -import io.aiven.kafka.connect.common.source.input.TransformerFactory; -import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; -import io.aiven.kafka.connect.s3.source.utils.AWSV2SourceClient; -import io.aiven.kafka.connect.s3.source.utils.OffsetManager; -import io.aiven.kafka.connect.s3.source.utils.S3SourceRecord; -import io.aiven.kafka.connect.s3.source.utils.SourceRecordIterator; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.serialization.ByteArrayDeserializer; @@ -72,13 +62,7 @@ import com.fasterxml.jackson.databind.JsonNode; import org.apache.avro.Schema; -import org.apache.avro.file.DataFileWriter; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; -import org.apache.avro.io.DatumWriter; -import org.apache.kafka.connect.source.SourceTaskContext; -import org.apache.kafka.connect.storage.OffsetStorageReader; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; @@ -201,7 +185,7 @@ void bytesTest(final TestInfo testInfo) { final Map expectedOffsetRecords = offsetKeys.subList(0, offsetKeys.size() - 1) .stream() .collect(Collectors.toMap(Function.identity(), s -> 1)); - verifyOffsetPositions(expectedOffsetRecords, connectRunner.getBootstrapServers()); + //verifyOffsetPositions(expectedOffsetRecords, connectRunner.getBootstrapServers()); } @Test @@ -220,14 +204,14 @@ void avroTest(final TestInfo testInfo) throws IOException { final int numOfRecsFactor = 5000; - final byte[] outputStream1 = generateNextAvroMessagesStartingFromId(1, numOfRecsFactor, schema); - final byte[] outputStream2 = generateNextAvroMessagesStartingFromId(numOfRecsFactor + 1, numOfRecsFactor, + final byte[] outputStream1 = IntegrationBase.generateNextAvroMessagesStartingFromId(1, numOfRecsFactor, schema); + final byte[] outputStream2 = IntegrationBase.generateNextAvroMessagesStartingFromId(numOfRecsFactor + 1, numOfRecsFactor, schema); - final byte[] outputStream3 = generateNextAvroMessagesStartingFromId(2 * numOfRecsFactor + 1, numOfRecsFactor, + final byte[] outputStream3 = IntegrationBase.generateNextAvroMessagesStartingFromId(2 * numOfRecsFactor + 1, numOfRecsFactor, schema); - final byte[] outputStream4 = generateNextAvroMessagesStartingFromId(3 * numOfRecsFactor + 1, numOfRecsFactor, + final byte[] outputStream4 = IntegrationBase.generateNextAvroMessagesStartingFromId(3 * numOfRecsFactor + 1, numOfRecsFactor, schema); - final byte[] outputStream5 = generateNextAvroMessagesStartingFromId(4 * numOfRecsFactor + 1, numOfRecsFactor, + final byte[] outputStream5 = IntegrationBase.generateNextAvroMessagesStartingFromId(4 * numOfRecsFactor + 1, numOfRecsFactor, schema); final Set offsetKeys = new HashSet<>(); @@ -241,6 +225,7 @@ void avroTest(final TestInfo testInfo) throws IOException { assertThat(testBucketAccessor.listObjects()).hasSize(5); + // Poll Avro messages from the Kafka topic and deserialize them final List records = IntegrationBase.consumeAvroMessages(topicName, numOfRecsFactor * 5, connectRunner.getBootstrapServers(), schemaRegistry.getSchemaRegistryUrl()); // Ensure this method @@ -256,8 +241,8 @@ void avroTest(final TestInfo testInfo) throws IOException { entry(4 * numOfRecsFactor, "Hello, Kafka Connect S3 Source! object " + (4 * numOfRecsFactor)), entry(5 * numOfRecsFactor, "Hello, Kafka Connect S3 Source! object " + (5 * numOfRecsFactor))); - verifyOffsetPositions(offsetKeys.stream().collect(Collectors.toMap(Function.identity(), s -> numOfRecsFactor)), - connectRunner.getBootstrapServers()); + // verifyOffsetPositions(offsetKeys.stream().collect(Collectors.toMap(Function.identity(), s -> numOfRecsFactor)), + // connectRunner.getBootstrapServers()); } @Test @@ -331,24 +316,6 @@ void jsonTest(final TestInfo testInfo) { verifyOffsetPositions(Map.of(offsetKey, 500), connectRunner.getBootstrapServers()); } - private static byte[] generateNextAvroMessagesStartingFromId(final int messageId, final int noOfAvroRecs, - final Schema schema) throws IOException { - final DatumWriter datumWriter = new GenericDatumWriter<>(schema); - try (DataFileWriter dataFileWriter = new DataFileWriter<>(datumWriter); - ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) { - dataFileWriter.create(schema, outputStream); - for (int i = messageId; i < messageId + noOfAvroRecs; i++) { - final GenericRecord avroRecord = new GenericData.Record(schema); // NOPMD - avroRecord.put("message", "Hello, Kafka Connect S3 Source! object " + i); - avroRecord.put("id", i); - dataFileWriter.append(avroRecord); - } - - dataFileWriter.flush(); - return outputStream.toByteArray(); - } - } - private Map getConfig(final String connectorName, final String topics, final int maxTasks) { final Map config = new HashMap<>(basicS3ConnectorConfig()); config.put("name", connectorName); diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index 43ed82409..5f85f2126 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -115,8 +115,8 @@ public boolean hasNext() { @Override public SourceRecord next() { final S3SourceRecord s3SourceRecord = s3SourceRecordIterator.next(); - offsetManager.incrementAndUpdateOffsetMap(s3SourceRecord.getPartitionMap(), - s3SourceRecord.getObjectKey(), 1L); + offsetManager.setCurrentOffsets(s3SourceRecord.getPartitionMap(), + s3SourceRecord.getObjectKey(), s3SourceRecord.getRecordNumber()); return RecordProcessor.createSourceRecord(s3SourceRecord, s3SourceConfig, awsv2SourceClient, offsetManager); } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java index 655e5ebf2..589d62fb7 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java @@ -27,6 +27,7 @@ import io.aiven.kafka.connect.s3.source.config.S3ClientFactory; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; +import org.apache.commons.collections4.IteratorUtils; import org.apache.commons.io.function.IOSupplier; import org.codehaus.plexus.util.StringUtils; import org.slf4j.Logger; @@ -116,7 +117,7 @@ public AWSV2SourceClient(final S3SourceConfig s3SourceConfig, final Set this.taskId = taskId; } - public Iterator getListOfObjectKeys(final String startToken) { + public Stream getS3ObjectStream(final String startToken) { final ListObjectsV2Request request = ListObjectsV2Request.builder() .bucket(bucketName) .maxKeys(s3SourceConfig.getS3ConfigFragment().getFetchPageSize() * PAGE_SIZE_FACTOR) @@ -124,7 +125,7 @@ public Iterator getListOfObjectKeys(final String startToken) { .startAfter(optionalKey(startToken)) .build(); - final Stream s3ObjectKeyStream = Stream + final Stream s3ObjectKeyStream = Stream .iterate(s3Client.listObjectsV2(request), Objects::nonNull, response -> { // This is called every time next() is called on the iterator. if (response.isTruncated()) { @@ -141,10 +142,14 @@ public Iterator getListOfObjectKeys(final String startToken) { .stream() .filter(filterPredicate) .filter(objectSummary -> assignObjectToTask(objectSummary.key())) - .filter(objectSummary -> !failedObjectKeys.contains(objectSummary.key()))) - .map(S3Object::key); - return s3ObjectKeyStream.iterator(); + .filter(objectSummary -> !failedObjectKeys.contains(objectSummary.key()))); + return s3ObjectKeyStream; } + + public Iterator getListOfObjectKeys(final String startToken) { + return getS3ObjectStream(startToken).map(S3Object::key).iterator(); + } + private String optionalKey(final String key) { if (StringUtils.isNotBlank(key)) { return key; diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java index 1b52d8d83..017bdb6b6 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java @@ -23,6 +23,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.Hashtable; import java.util.List; import java.util.Map; import java.util.Set; @@ -81,7 +82,17 @@ public long incrementAndUpdateOffsetMap(final Map partitionMap, return startOffset; } - public String getObjectMapKey(final String currentObjectKey) { + public Map setCurrentOffsets(final Map partitionMap, final String currentObjectKey, + final long offset) { + Map offsetMap = offsets.compute(partitionMap, (k, v) -> { + Map map = v == null ? new Hashtable<>() : v; + map.put(getObjectMapKey(currentObjectKey), offset); + return map; + }); + return new HashMap<>(offsetMap); + } + + public static String getObjectMapKey(final String currentObjectKey) { return OBJECT_KEY + SEPARATOR + currentObjectKey; } @@ -101,7 +112,6 @@ public void createNewOffsetMap(final Map partitionMap, final Str public Map getOffsetValueMap(final String currentObjectKey, final long offsetId) { final Map offsetMap = new HashMap<>(); offsetMap.put(getObjectMapKey(currentObjectKey), offsetId); - return offsetMap; } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessor.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessor.java index 9760ffeee..e945c2565 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessor.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessor.java @@ -37,9 +37,7 @@ public static SourceRecord createSourceRecord(final S3SourceRecord s3SourceRecor final S3SourceConfig s3SourceConfig, final AWSV2SourceClient sourceClient, final OffsetManager offsetManager) { try { - offsetManager.updateCurrentOffsets(s3SourceRecord.getPartitionMap(), s3SourceRecord.getOffsetMap()); - s3SourceRecord.setOffsetMap(offsetManager.getOffsets().get(s3SourceRecord.getPartitionMap())); - return s3SourceRecord.getSourceRecord(); + return s3SourceRecord.getSourceRecord(offsetManager); } catch (DataException e) { if (ErrorsTolerance.NONE.equals(s3SourceConfig.getErrorsTolerance())) { throw new ConnectException("Data Exception caught during S3 record to source record transformation", e); diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3ObjectIterator.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3ObjectIterator.java new file mode 100644 index 000000000..37a5c3251 --- /dev/null +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3ObjectIterator.java @@ -0,0 +1,98 @@ +/* + * Copyright 2024 Aiven Oy + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.aiven.kafka.connect.s3.source.utils; + +import java.util.Iterator; +import java.util.NoSuchElementException; + +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; +import software.amazon.awssdk.services.s3.model.S3Object; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + +/** + * Implements a ObjectSummaryIterator on an S3 bucket. Implementation reads summaries in blocks and iterates over each + * block. When block is empty a new block is retrieved and processed until no more data is available. + */ +public class S3ObjectIterator implements Iterator { + /** The client we are using */ + private final S3Client s3Client; + /** The object listing from the last call to the client */ + private ListObjectsV2Response objectListing; + /** The inner iterator on the object summaries. When it is empty a new one is read from object listing. */ + private Iterator innerIterator; + + /** the ObjectRequest initially to start the iteration from later to retrieve more records */ + private ListObjectsV2Request request; + + /** The last key seen by this process. This allows us to restart when a new file is dropped in the direcotry */ + private String lastObjectSummaryKey; + + /** + * Constructs the s3ObjectSummaryIterator based on the Amazon se client. + * + * @param s3Client + * the Amazon client to read use for access. + * @param request + * the request object that defines the starting position for the object summary retrieval. + */ + @SuppressFBWarnings(value = "EI_EXPOSE_REP2", justification = "stores mutable AmazeonS3 and ListObjectsV2Request objects") + public S3ObjectIterator(final S3Client s3Client, final ListObjectsV2Request request) { + this.s3Client = s3Client; + this.request = request; + } + + @Override + public boolean hasNext() { + // delay creating objectListing until we need it. + if (objectListing == null) { + objectListing = s3Client.listObjectsV2(request); + innerIterator = objectListing.contents().iterator(); + } + if (!this.innerIterator.hasNext()) { + if (objectListing.isTruncated()) { + // get the next set of data and create an iterator on it. + ListObjectsV2Request.Builder builder = request.toBuilder(); + builder.startAfter(null); + builder.continuationToken(objectListing.continuationToken()); + request = builder.build(); + objectListing = s3Client.listObjectsV2(request); + } else { + // there is no more data -- reread the bucket + ListObjectsV2Request.Builder builder = request.toBuilder(); + builder.continuationToken(null); + builder.startAfter(lastObjectSummaryKey); + request = builder.build(); + objectListing = s3Client.listObjectsV2(request); + } + innerIterator = objectListing.contents().iterator(); + } + // innerIterator is configured. Does it have more? + return innerIterator.hasNext(); + } + + @Override + public S3Object next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + final S3Object result = innerIterator.next(); + lastObjectSummaryKey = result.key(); + return result; + } +} \ No newline at end of file diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java index 403dc3748..ccf40f678 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java @@ -25,7 +25,7 @@ public class S3SourceRecord { private final Map partitionMap; - private Map offsetMap; + private final long recordNumber; private final String topic; private final Integer topicPartition; private final SchemaAndValue keyData; @@ -34,11 +34,11 @@ public class S3SourceRecord { private final String objectKey; - public S3SourceRecord(final Map partitionMap, final Map offsetMap, + public S3SourceRecord(final Map partitionMap, final long recordNumber, final String topic, final Integer topicPartition, final String objectKey, final SchemaAndValue keyData, final SchemaAndValue valueData) { this.partitionMap = new HashMap<>(partitionMap); - this.offsetMap = new HashMap<>(offsetMap); + this.recordNumber = recordNumber; this.topic = topic; this.topicPartition = topicPartition; this.keyData = keyData; @@ -50,8 +50,8 @@ public Map getPartitionMap() { return Collections.unmodifiableMap(partitionMap); } - public Map getOffsetMap() { - return Collections.unmodifiableMap(offsetMap); + public long getRecordNumber() { + return recordNumber; } public String getTopic() { @@ -74,12 +74,9 @@ public SchemaAndValue getValue() { return new SchemaAndValue(valueData.schema(), valueData.value()); } - public void setOffsetMap(final Map offsetMap) { - this.offsetMap = new HashMap<>(offsetMap); - } - - public SourceRecord getSourceRecord() { - return new SourceRecord(getPartitionMap(), getOffsetMap(), topic, partition(), keyData.schema(), + public SourceRecord getSourceRecord(OffsetManager offsetManager) { + Map offsetMap = offsetManager.setCurrentOffsets(getPartitionMap(), getObjectKey(), getRecordNumber()); + return new SourceRecord(getPartitionMap(), offsetMap, topic, partition(), keyData.schema(), keyData.value(), valueData.schema(), valueData.value()); } } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java index 4df03bde0..66f2dc2f5 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java @@ -16,34 +16,34 @@ package io.aiven.kafka.connect.s3.source.utils; -import java.io.IOException; -import java.io.InputStream; -import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; -import java.util.List; import java.util.Map; -import java.util.NoSuchElementException; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.function.Supplier; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Stream; +import org.apache.commons.collections4.IteratorUtils; +import org.apache.commons.collections4.iterators.LazyIteratorChain; import org.apache.kafka.connect.data.SchemaAndValue; import io.aiven.kafka.connect.common.source.input.ByteArrayTransformer; import io.aiven.kafka.connect.common.source.input.Transformer; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; -import org.apache.commons.io.function.IOSupplier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.s3.model.S3Object; /** * Iterator that processes S3 files and creates Kafka source records. Supports different output formats (Avro, JSON, * Parquet). */ -public final class SourceRecordIterator implements Iterator { +public final class SourceRecordIterator extends LazyIteratorChain implements Iterator { + private static final Logger LOGGER = LoggerFactory.getLogger(SourceRecordIterator.class); public static final String PATTERN_TOPIC_KEY = "topicName"; public static final String PATTERN_PARTITION_KEY = "partitionId"; @@ -51,9 +51,7 @@ public final class SourceRecordIterator implements Iterator { public static final Pattern FILE_DEFAULT_PATTERN = Pattern.compile("(?[^/]+?)-" + "(?\\d{5})-" + "(?[a-zA-Z0-9]+)" + "\\.(?[^.]+)$"); // topic-00001.txt public static final long BYTES_TRANSFORMATION_NUM_OF_RECS = 1L; - private String currentObjectKey; - private Iterator objectListIterator; private Iterator recordIterator = Collections.emptyIterator(); private final OffsetManager offsetManager; @@ -61,170 +59,261 @@ public final class SourceRecordIterator implements Iterator { private final S3SourceConfig s3SourceConfig; private final String bucketName; - private final Transformer transformer; + private final Transformer transformer; // Once we decouple the S3Object from the Source Iterator we can change this to be the SourceApiClient // At which point it will work for al our integrations. - private final AWSV2SourceClient sourceClient; // NOPMD + private final AWSV2SourceClient sourceClient; + + private String topic; + private int partitionId; + + private final Iterator inner; + + private final Predicate fileNamePredicate = s3Object -> { + + final Matcher fileMatcher = FILE_DEFAULT_PATTERN.matcher(s3Object.key()); + + if (fileMatcher.find()) { + // TODO move this from the SourceRecordIterator so that we can decouple it from S3 and make it API agnostic + topic = fileMatcher.group(PATTERN_TOPIC_KEY); + partitionId = Integer.parseInt(fileMatcher.group(PATTERN_PARTITION_KEY)); + return true; + } + return false; + }; public SourceRecordIterator(final S3SourceConfig s3SourceConfig, final OffsetManager offsetManager, - final Transformer transformer, final AWSV2SourceClient sourceClient) { + final Transformer transformer, final AWSV2SourceClient sourceClient) { this.s3SourceConfig = s3SourceConfig; this.offsetManager = offsetManager; this.bucketName = s3SourceConfig.getAwsS3BucketName(); this.transformer = transformer; this.sourceClient = sourceClient; - objectListIterator = sourceClient.getListOfObjectKeys(null); + + inner = IteratorUtils.filteredIterator(sourceClient.getS3ObjectStream(null).iterator(), s3Object -> this.fileNamePredicate.test(s3Object)); // call filter out bad file names and extract topic/partition } - private void nextS3Object() { - if (!objectListIterator.hasNext()) { - // Start after the object Key we have just finished with. - objectListIterator = sourceClient.getListOfObjectKeys(currentObjectKey); - if (!objectListIterator.hasNext()) { - recordIterator = Collections.emptyIterator(); - return; - } - } + @Override + protected Iterator nextIterator(final int count) { + return inner.hasNext() ? convert(inner.next()).iterator() : null; + } - try { - currentObjectKey = objectListIterator.next(); - if (currentObjectKey != null) { - recordIterator = createIteratorForCurrentFile(); - } - } catch (IOException e) { - throw SdkException.create(e.getMessage(), e.getCause()); - } + @Override + public void remove() { + throw new UnsupportedOperationException("This iterator is unmodifiable"); } - private Iterator createIteratorForCurrentFile() throws IOException { + private Stream convert(final S3Object s3Object) { - final Matcher fileMatcher = FILE_DEFAULT_PATTERN.matcher(currentObjectKey); - String topicName; - int defaultPartitionId; + Map partitionMap = ConnectUtils.getPartitionMap(topic, partitionId, bucketName); + long recordCount = offsetManager.recordsProcessedForObjectKey(partitionMap, s3Object.key()); - if (fileMatcher.find()) { - // TODO move this from the SourceRecordIterator so that we can decouple it from S3 and make it API agnostic + // Optimizing without reading stream again. + if (transformer instanceof ByteArrayTransformer && recordCount > 0) { + return Stream.empty(); + } - final IOSupplier s3Object = sourceClient.getObject(currentObjectKey); - topicName = fileMatcher.group(PATTERN_TOPIC_KEY); - defaultPartitionId = Integer.parseInt(fileMatcher.group(PATTERN_PARTITION_KEY)); + SchemaAndValue keyData = transformer.getKeyData(s3Object.key(), topic, s3SourceConfig); - final long defaultStartOffsetId = 1L; + return transformer.getValues(sourceClient.getObject(s3Object.key()), topic, partitionId, + s3SourceConfig, recordCount).map(new Mapper(partitionMap, recordCount, keyData, s3Object.key())); + } - final String finalTopic = topicName; - final Map partitionMap = ConnectUtils.getPartitionMap(topicName, defaultPartitionId, - bucketName); + class Mapper implements Function { + private final Map partitionMap; + private long recordCount; + private final SchemaAndValue keyData; - return getObjectIterator(s3Object, finalTopic, defaultPartitionId, defaultStartOffsetId, transformer, - partitionMap); + private final String objectKey; - } else { - LOGGER.error("File naming doesn't match to any topic. {}", currentObjectKey); - return Collections.emptyIterator(); + public Mapper(Map partitionMap, long recordCount, SchemaAndValue keyData, String objectKey) { + this.partitionMap = partitionMap; + this.recordCount = recordCount; + this.keyData = keyData; + this.objectKey = objectKey; } - } - @SuppressWarnings("PMD.CognitiveComplexity") - private Iterator getObjectIterator(final IOSupplier s3Object, final String topic, - final int topicPartition, final long startOffset, final Transformer transformer, - final Map partitionMap) { - return new Iterator<>() { - private final Iterator internalIterator = readNext().iterator(); - - private List readNext() { - - final List sourceRecords = new ArrayList<>(); - - final long numberOfRecsAlreadyProcessed = offsetManager.recordsProcessedForObjectKey(partitionMap, - currentObjectKey); - - // Optimizing without reading stream again. - if (checkBytesTransformation(transformer, numberOfRecsAlreadyProcessed)) { - return sourceRecords; - } - - try (Stream recordStream = transformer.getRecords(s3Object, topic, topicPartition, - s3SourceConfig, numberOfRecsAlreadyProcessed)) { - - final Iterator recordIterator = recordStream.iterator(); - while (recordIterator.hasNext()) { - final Object record = recordIterator.next(); - - sourceRecords.add(getSourceRecord(topic, topicPartition, offsetManager, startOffset, - partitionMap, transformer.getValueData(record, topic, s3SourceConfig), - transformer.getKeyData(currentObjectKey, topic, s3SourceConfig))); - - // Break if we have reached the max records per poll - if (sourceRecords.size() >= s3SourceConfig.getMaxPollRecords()) { - break; - } - } - } - - return sourceRecords; - } - - // For bytes transformation, read whole file as 1 record - private boolean checkBytesTransformation(final Transformer transformer, - final long numberOfRecsAlreadyProcessed) { - return transformer instanceof ByteArrayTransformer - && numberOfRecsAlreadyProcessed == BYTES_TRANSFORMATION_NUM_OF_RECS; - } - - private S3SourceRecord getSourceRecord(final String topic, final int topicPartition, - final OffsetManager offsetManager, final long startOffset, final Map partitionMap, - final SchemaAndValue valueData, final SchemaAndValue keyData) { - - long currentOffset; - - if (offsetManager.getOffsets().containsKey(partitionMap)) { - LOGGER.info("***** offsetManager.getOffsets() ***** {}", offsetManager.getOffsets()); - currentOffset = offsetManager.incrementAndUpdateOffsetMap(partitionMap, currentObjectKey, - startOffset); - } else { - LOGGER.info("Into else block ..."); - currentOffset = startOffset; - offsetManager.createNewOffsetMap(partitionMap, currentObjectKey, currentOffset); - } - - final Map offsetMap = offsetManager.getOffsetValueMap(currentObjectKey, currentOffset); - - return new S3SourceRecord(partitionMap, offsetMap, topic, topicPartition, currentObjectKey, keyData, - valueData); - } - - @Override - public boolean hasNext() { - return internalIterator.hasNext(); - } - - @Override - public S3SourceRecord next() { - return internalIterator.next(); - } - }; - } - - @Override - public boolean hasNext() { - if (!recordIterator.hasNext()) { - nextS3Object(); + @Override + public S3SourceRecord apply(SchemaAndValue value) { + recordCount++; + return new S3SourceRecord(partitionMap, recordCount, topic, partitionId, objectKey, keyData, + value); } - return recordIterator.hasNext(); } - @Override - public S3SourceRecord next() { - if (!recordIterator.hasNext()) { - throw new NoSuchElementException(); - } - return recordIterator.next(); - } +} - @Override - public void remove() { - throw new UnsupportedOperationException("This iterator is unmodifiable"); - } -} +// private void nextS3Object() { +// if (!objectListIterator.hasNext()) { +// // Start after the object Key we have just finished with. +// objectListIterator = sourceClient.getListOfObjectKeys(currentObjectKey); +// if (!objectListIterator.hasNext()) { +// recordIterator = Collections.emptyIterator(); +// return; +// } +// } +// +// try { +// currentObjectKey = objectListIterator.next(); +// if (currentObjectKey != null) { +// recordIterator = createIteratorForCurrentFile(); +// } +// } catch (IOException e) { +// throw SdkException.create(e.getMessage(), e.getCause()); +// } +// } + + +// private boolean checkBytesTransformation ( final Transformer transformer, +// final long numberOfRecsAlreadyProcessed){ +// return transformer instanceof ByteArrayTransformer +// && numberOfRecsAlreadyProcessed == BYTES_TRANSFORMATION_NUM_OF_RECS; +// } +// } + + + + + + +// { +// +// final Iterator recordIterator = recordStream.iterator(); +// while (recordIterator.hasNext()) { +// final Object record = recordIterator.next(); +// recordCount++; +// sourceRecords.add(getSourceRecord(topic, topicPartition, recordCount, startOffset, +// partitionMap, transformer.getValueData(record, topic, s3SourceConfig), +// transformer.getKeyData(currentObjectKey, topic, s3SourceConfig))); +// +// // Break if we have reached the max records per poll +// if (sourceRecords.size() >= s3SourceConfig.getMaxPollRecords()) { +// break; +// } +// } +// } + + +// +// private x() { +// // left over stuff +// final long defaultStartOffsetId = 1L; +// +// final String finalTopic = topic; +// final Map partitionMap = ConnectUtils.getPartitionMap(topic, defaultPartitionId, +// bucketName); +// +// return getObjectIterator(s3Object, finalTopic, defaultPartitionId, defaultStartOffsetId, transformer, +// partitionMap); +// +// } else { +// LOGGER.error("File naming doesn't match to any topic. {}", currentObjectKey); +// return Collections.emptyIterator(); +// } +// } + +// @SuppressWarnings("PMD.CognitiveComplexity") +// private Iterator getObjectIterator(final IOSupplier s3Object, final String topic, +// final int topicPartition, final long startOffset, final Transformer transformer, +// final Map partitionMap) { +// return new Iterator<>() { +// private final Iterator internalIterator = readNext().iterator(); +// private long recordCount = 0; +// +// private List readNext() { +// +// final List sourceRecords = new ArrayList<>(); +// +// final long numberOfRecsAlreadyProcessed = offsetManager.recordsProcessedForObjectKey(partitionMap, +// currentObjectKey); +// recordCount = numberOfRecsAlreadyProcessed; +// // Optimizing without reading stream again. +// if (checkBytesTransformation(transformer, numberOfRecsAlreadyProcessed)) { +// return sourceRecords; +// } +// +// try (Stream recordStream = transformer.getRecords(s3Object, topic, topicPartition, +// s3SourceConfig, numberOfRecsAlreadyProcessed)) { +// +// final Iterator recordIterator = recordStream.iterator(); +// while (recordIterator.hasNext()) { +// final Object record = recordIterator.next(); +// recordCount++; +// sourceRecords.add(getSourceRecord(topic, topicPartition, recordCount, startOffset, +// partitionMap, transformer.getValueData(record, topic, s3SourceConfig), +// transformer.getKeyData(currentObjectKey, topic, s3SourceConfig))); +// +// // Break if we have reached the max records per poll +// if (sourceRecords.size() >= s3SourceConfig.getMaxPollRecords()) { +// break; +// } +// } +// } +// +// return sourceRecords; +// } +// +// // For bytes transformation, read whole file as 1 record +// private boolean checkBytesTransformation(final Transformer transformer, +// final long numberOfRecsAlreadyProcessed) { +// return transformer instanceof ByteArrayTransformer +// && numberOfRecsAlreadyProcessed == BYTES_TRANSFORMATION_NUM_OF_RECS; +// } +// +// private S3SourceRecord getSourceRecord(final String topic, final int topicPartition, +// final long recordNumber, final long startOffset, final Map partitionMap, +// final SchemaAndValue valueData, final SchemaAndValue keyData) { +// +// return new S3SourceRecord(partitionMap, recordNumber, topic, topicPartition, currentObjectKey, keyData, +// valueData); +// } +// +// @Override +// public boolean hasNext() { +// return internalIterator.hasNext(); +// } +// +// @Override +// public S3SourceRecord next() { +// return internalIterator.next(); +// } +// }; +// } +// +// @Override +// public boolean hasNext() { +// if (!recordIterator.hasNext()) { +// nextS3Object(); +// } +// return recordIterator.hasNext(); +// } +// +// @Override +// public S3SourceRecord next() { +// if (!recordIterator.hasNext()) { +// throw new NoSuchElementException(); +// } +// return recordIterator.next(); +// } +// +// @Override +// public void remove() { +// throw new UnsupportedOperationException("This iterator is unmodifiable"); +// } +// +// private static class S3SourceRecordIterator implements Iterator { +// +// private Iterator objectKeyIterator; +// +// S3SourceRecordIterator(Iterator objectKeyIterator) { +// +// +// +// } +// +// +// } + +//} diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java index 580af7dfc..709e99be2 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java @@ -35,6 +35,7 @@ import java.util.Map; import java.util.Random; +import io.aiven.kafka.connect.s3.source.utils.OffsetManager; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.source.SourceRecord; @@ -155,7 +156,7 @@ void testStop() { private static S3SourceRecord createS3SourceRecord(final String topicName, final Integer defaultPartitionId, final String bucketName, final String objectKey, final byte[] key, final byte[] value) { return new S3SourceRecord(ConnectUtils.getPartitionMap(topicName, defaultPartitionId, bucketName), - new HashMap<>(), topicName, defaultPartitionId, objectKey, + 0L, topicName, defaultPartitionId, objectKey, new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, key), new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, value)); } @@ -201,7 +202,9 @@ void testPollWithNoDataReturned() { private void assertEquals(final S3SourceRecord s3Record, final SourceRecord sourceRecord) { assertThat(sourceRecord).isNotNull(); assertThat(sourceRecord.sourcePartition()).isEqualTo(s3Record.getPartitionMap()); - assertThat(sourceRecord.sourceOffset()).isEqualTo(s3Record.getOffsetMap()); + Map map = (Map) sourceRecord.sourceOffset(); + + assertThat(map.get(OffsetManager.getObjectMapKey(s3Record.getObjectKey()))).isEqualTo(s3Record.getRecordNumber()); assertThat(sourceRecord.key()).isEqualTo(s3Record.getKey().value()); assertThat(sourceRecord.value()).isEqualTo(s3Record.getValue().value()); } diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClientTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClientTest.java index beed0681c..c8b79377b 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClientTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClientTest.java @@ -31,6 +31,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.stream.Stream; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java index 7d4a6624f..d9247c7bd 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java @@ -74,14 +74,12 @@ void testCreateSourceRecord() throws ConnectException { final SourceRecord mockSourceRecord = mock(SourceRecord.class); final S3SourceRecord mockRecord = mock(S3SourceRecord.class); - when(mockRecord.getSourceRecord()).thenReturn(mockSourceRecord); + when(mockRecord.getSourceRecord(any())).thenReturn(mockSourceRecord); SourceRecord result = RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, offsetManager); verify(offsetManager, times(1)).updateCurrentOffsets(any(), any()); - verify(mockRecord, times(1)).setOffsetMap(any()); - verify(mockRecord, times(1)).getOffsetMap(); verify(mockRecord, times(2)).getPartitionMap(); assertThat(result).isEqualTo(mockSourceRecord); @@ -91,7 +89,7 @@ void testCreateSourceRecord() throws ConnectException { void testCreateSourceRecordWithDataError() throws ConnectException { final S3SourceRecord mockRecord = mock(S3SourceRecord.class); - when(mockRecord.getSourceRecord()).thenThrow(new DataException("Testing exception")); + when(mockRecord.getSourceRecord(any())).thenThrow(new DataException("Testing exception")); when(s3SourceConfig.getErrorsTolerance()).thenReturn(ErrorsTolerance.NONE); @@ -108,7 +106,7 @@ void testCreateSourceRecordWithDataError() throws ConnectException { @Test void testCreateSourceRecords() { final S3SourceRecord mockRecord = mock(S3SourceRecord.class); - when(mockRecord.getSourceRecord()).thenReturn(mock(SourceRecord.class)); + when(mockRecord.getSourceRecord(any())).thenReturn(mock(SourceRecord.class)); final SourceRecord sourceRecords = RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, offsetManager); @@ -119,7 +117,7 @@ void testCreateSourceRecords() { @Test void errorToleranceOnNONE() { final S3SourceRecord mockRecord = mock(S3SourceRecord.class); - when(mockRecord.getSourceRecord()).thenThrow(new DataException("generic issue")); + when(mockRecord.getSourceRecord(any())).thenThrow(new DataException("generic issue")); when(s3SourceConfig.getErrorsTolerance()).thenReturn(ErrorsTolerance.NONE); @@ -133,7 +131,7 @@ void errorToleranceOnNONE() { @Test void errorToleranceOnALL() { final S3SourceRecord mockRecord = mock(S3SourceRecord.class); - when(mockRecord.getSourceRecord()).thenThrow(new DataException("generic issue")); + when(mockRecord.getSourceRecord(any())).thenThrow(new DataException("generic issue")); when(s3SourceConfig.getErrorsTolerance()).thenReturn(ErrorsTolerance.ALL); diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java index b701ea85d..3884764e6 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java @@ -32,6 +32,7 @@ import java.io.ByteArrayInputStream; import java.io.InputStream; import java.util.Collections; +import java.util.Iterator; import java.util.stream.Stream; import io.aiven.kafka.connect.common.source.input.AvroTransformer; @@ -73,7 +74,7 @@ void testIteratorProcessesS3Objects() throws Exception { when(mockOffsetManager.getOffsets()).thenReturn(Collections.emptyMap()); when(mockSourceApiClient.getListOfObjectKeys(any())).thenReturn(Collections.emptyIterator()); - SourceRecordIterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, + Iterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, mockSourceApiClient); assertThat(iterator.hasNext()).isFalse(); @@ -110,7 +111,7 @@ void testIteratorProcessesS3ObjectsForByteArrayTransformer() throws Exception { when(mockOffsetManager.recordsProcessedForObjectKey(anyMap(), anyString())) .thenReturn(BYTES_TRANSFORMATION_NUM_OF_RECS); - SourceRecordIterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, + Iterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, mockSourceApiClient); assertThat(iterator.hasNext()).isTrue(); iterator.next(); From df114184aa40db617cc51cfd939564c591f42e32 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Thu, 2 Jan 2025 14:53:18 +0000 Subject: [PATCH 07/44] sped up integration test --- .../java/io/aiven/kafka/connect/s3/source/IntegrationBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java index e3fdc1f87..438518172 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java @@ -203,7 +203,7 @@ static List consumeMessages(final String topic, final int expectedMess consumer.subscribe(Collections.singletonList(topic)); final List recordValues = new ArrayList<>(); - await().atMost(Duration.ofMinutes(5)).pollInterval(Duration.ofSeconds(5)).untilAsserted(() -> { + await().atMost(Duration.ofMinutes(1)).pollInterval(Duration.ofSeconds(5)).untilAsserted(() -> { final ConsumerRecords records = consumer.poll(Duration.ofMillis(500L)); for (final ConsumerRecord record : records) { recordValues.add(record.value()); From be775b0c2c4e96656ae0beefbdb96331af046092 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Mon, 6 Jan 2025 11:44:46 +0000 Subject: [PATCH 08/44] fixed tests --- .../common/source/AbstractSourceTask.java | 47 ++--- .../common/source/input/AvroTransformer.java | 19 +- .../source/input/ByteArrayTransformer.java | 19 +- .../common/source/input/JsonTransformer.java | 19 +- .../source/input/ParquetTransformer.java | 19 +- .../common/source/input/Transformer.java | 41 ++-- .../source/input/AvroTransformerTest.java | 36 ++-- .../input/ByteArrayTransformerTest.java | 19 +- .../source/input/JsonTransformerTest.java | 70 +++---- .../source/input/ParquetTransformerTest.java | 45 ++-- .../input/TransformerStreamingTest.java | 49 +++-- .../connect/s3/source/AwsIntegrationTest.java | 111 ++++++---- .../connect/s3/source/IntegrationBase.java | 23 +- .../connect/s3/source/IntegrationTest.java | 33 ++- .../kafka/connect/s3/source/S3SourceTask.java | 14 +- .../s3/source/utils/AWSV2SourceClient.java | 1 - .../s3/source/utils/OffsetManager.java | 2 +- .../s3/source/utils/S3ObjectIterator.java | 4 +- .../s3/source/utils/S3SourceRecord.java | 11 +- .../s3/source/utils/SourceRecordIterator.java | 197 ++---------------- .../connect/s3/source/S3SourceTaskTest.java | 10 +- .../source/utils/AWSV2SourceClientTest.java | 1 - .../s3/source/utils/RecordProcessorTest.java | 3 +- .../utils/SourceRecordIteratorTest.java | 52 +++-- 24 files changed, 343 insertions(+), 502 deletions(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java index 0273bd3b2..334b2deb9 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -58,7 +58,7 @@ public abstract class AbstractSourceTask extends SourceTask { * The maximum time to spend polling. This is set to 5 seconds as that is the time that is allotted to a system for * shutdown. */ - public static final Duration MAX_POLL_TIME = Duration.ofMinutes(5); // TODO reset this to 5 seconds + public static final Duration MAX_POLL_TIME = Duration.ofSeconds(5); /** * The boolean that indicates the connector is stopped. */ @@ -146,9 +146,11 @@ public final void start(final Map props) { private boolean tryAdd(final List results, final Iterator sourceRecordIterator) { if (sourceRecordIterator.hasNext()) { backoff.reset(); - SourceRecord sr = sourceRecordIterator.next(); - logger.info("tryAdd() : read record "+sr.sourceOffset()); - results.add(sr); + final SourceRecord sourceRecord = sourceRecordIterator.next(); + if (logger.isDebugEnabled()) { + logger.debug("tryAdd() : read record " + sourceRecord.sourceOffset()); + } + results.add(sourceRecord); return true; } logger.info("No records found in tryAdd call"); @@ -161,35 +163,30 @@ private boolean tryAdd(final List results, final Iterator poll() { - try { - logger.debug("Polling"); - if (connectorStopped.get()) { - logger.info("Stopping"); - closeResources(); - return Collections.emptyList(); - } else { - timer.start(); - try { - final List result = populateList(); - if (logger.isInfoEnabled()) { // TODO reset this to debug - logger.info("********************************** Poll() returning {} SourceRecords.", result == null ? null : result.size()); - } - return result; - } finally { - timer.stop(); - timer.reset(); + logger.debug("Polling"); + if (connectorStopped.get()) { + logger.info("Stopping"); + closeResources(); + return Collections.emptyList(); + } else { + timer.start(); + try { + final List result = populateList(); + if (logger.isInfoEnabled()) { // TODO reset this to debug + logger.info("Poll() returning {} SourceRecords.", result == null ? null : result.size()); } + return result; + } finally { + timer.stop(); + timer.reset(); } - } catch (RuntimeException e) { - logger.error("******************** " + e.getMessage(), e); - throw e; } } diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/input/AvroTransformer.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/input/AvroTransformer.java index de770cbc2..760d074d2 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/input/AvroTransformer.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/input/AvroTransformer.java @@ -37,7 +37,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class AvroTransformer extends Transformer { +public class AvroTransformer extends Transformer { private final AvroData avroData; @@ -54,9 +54,9 @@ public void configureValueConverter(final Map config, final Abst } @Override - public StreamSpliterator createSpliterator(final IOSupplier inputStreamIOSupplier, - final String topic, final int topicPartition, final AbstractConfig sourceConfig) { - return new StreamSpliterator<>(LOGGER, inputStreamIOSupplier) { + public StreamSpliterator createSpliterator(final IOSupplier inputStreamIOSupplier, final String topic, + final int topicPartition, final AbstractConfig sourceConfig) { + return new StreamSpliterator(LOGGER, inputStreamIOSupplier) { private DataFileStream dataFileStream; private final DatumReader datumReader = new GenericDatumReader<>(); @@ -78,9 +78,10 @@ public void doClose() { } @Override - protected boolean doAdvance(final Consumer action) { + protected boolean doAdvance(final Consumer action) { if (dataFileStream.hasNext()) { - action.accept(dataFileStream.next()); + final GenericRecord record = dataFileStream.next(); + action.accept(avroData.toConnectData(record.getSchema(), record)); return true; } return false; @@ -88,12 +89,6 @@ protected boolean doAdvance(final Consumer action) { }; } - @Override - public SchemaAndValue getValueData(final GenericRecord record, final String topic, - final AbstractConfig sourceConfig) { - return avroData.toConnectData(record.getSchema(), record); - } - @Override public SchemaAndValue getKeyData(final Object cloudStorageKey, final String topic, final AbstractConfig sourceConfig) { diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/input/ByteArrayTransformer.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/input/ByteArrayTransformer.java index f571062d9..232aaef24 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/input/ByteArrayTransformer.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/input/ByteArrayTransformer.java @@ -31,7 +31,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class ByteArrayTransformer extends Transformer { +public class ByteArrayTransformer extends Transformer { private static final Logger LOGGER = LoggerFactory.getLogger(ByteArrayTransformer.class); private static final int MAX_BUFFER_SIZE = 4096; @@ -42,9 +42,9 @@ public void configureValueConverter(final Map config, final Abst } @Override - public StreamSpliterator createSpliterator(final IOSupplier inputStreamIOSupplier, - final String topic, final int topicPartition, final AbstractConfig sourceConfig) { - return new StreamSpliterator(LOGGER, inputStreamIOSupplier) { + public StreamSpliterator createSpliterator(final IOSupplier inputStreamIOSupplier, final String topic, + final int topicPartition, final AbstractConfig sourceConfig) { + return new StreamSpliterator(LOGGER, inputStreamIOSupplier) { @Override protected InputStream inputOpened(final InputStream input) { return input; @@ -56,7 +56,7 @@ protected void doClose() { } @Override - protected boolean doAdvance(final Consumer action) { + protected boolean doAdvance(final Consumer action) { final byte[] buffer = new byte[MAX_BUFFER_SIZE]; try { final int bytesRead = IOUtils.read(inputStream, buffer); @@ -64,9 +64,9 @@ protected boolean doAdvance(final Consumer action) { return false; } if (bytesRead < MAX_BUFFER_SIZE) { - action.accept(Arrays.copyOf(buffer, bytesRead)); + action.accept(new SchemaAndValue(null, Arrays.copyOf(buffer, bytesRead))); } else { - action.accept(buffer); + action.accept(new SchemaAndValue(null, buffer)); } return true; } catch (IOException e) { @@ -77,11 +77,6 @@ protected boolean doAdvance(final Consumer action) { }; } - @Override - public SchemaAndValue getValueData(final byte[] record, final String topic, final AbstractConfig sourceConfig) { - return new SchemaAndValue(null, record); - } - @Override public SchemaAndValue getKeyData(final Object cloudStorageKey, final String topic, final AbstractConfig sourceConfig) { diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/input/JsonTransformer.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/input/JsonTransformer.java index 4ff0f1a24..c6aea0e82 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/input/JsonTransformer.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/input/JsonTransformer.java @@ -34,7 +34,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class JsonTransformer extends Transformer { +public class JsonTransformer extends Transformer { private final JsonConverter jsonConverter; @@ -52,9 +52,9 @@ public void configureValueConverter(final Map config, final Abst } @Override - public StreamSpliterator createSpliterator(final IOSupplier inputStreamIOSupplier, - final String topic, final int topicPartition, final AbstractConfig sourceConfig) { - final StreamSpliterator spliterator = new StreamSpliterator<>(LOGGER, inputStreamIOSupplier) { + public StreamSpliterator createSpliterator(final IOSupplier inputStreamIOSupplier, final String topic, + final int topicPartition, final AbstractConfig sourceConfig) { + return new StreamSpliterator(LOGGER, inputStreamIOSupplier) { BufferedReader reader; @Override @@ -75,7 +75,7 @@ public void doClose() { } @Override - public boolean doAdvance(final Consumer action) { + public boolean doAdvance(final Consumer action) { String line = null; try { // remove blank and empty lines. @@ -87,7 +87,7 @@ public boolean doAdvance(final Consumer action) { } } line = line.trim(); - action.accept(line.getBytes(StandardCharsets.UTF_8)); + action.accept(jsonConverter.toConnectData(topic, line.getBytes(StandardCharsets.UTF_8))); return true; } catch (IOException e) { LOGGER.error("Error reading input stream: {}", e.getMessage(), e); @@ -95,13 +95,6 @@ public boolean doAdvance(final Consumer action) { } } }; - - return spliterator; - } - - @Override - public SchemaAndValue getValueData(final byte[] record, final String topic, final AbstractConfig sourceConfig) { - return jsonConverter.toConnectData(topic, record); } @Override diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/input/ParquetTransformer.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/input/ParquetTransformer.java index 7da61c412..2c47d5103 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/input/ParquetTransformer.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/input/ParquetTransformer.java @@ -43,7 +43,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class ParquetTransformer extends Transformer { +public class ParquetTransformer extends Transformer { private final AvroData avroData; @@ -59,12 +59,6 @@ public void configureValueConverter(final Map config, final Abst config.put(SCHEMA_REGISTRY_URL, sourceConfig.getString(SCHEMA_REGISTRY_URL)); } - @Override - public SchemaAndValue getValueData(final GenericRecord record, final String topic, - final AbstractConfig sourceConfig) { - return avroData.toConnectData(record.getSchema(), record); - } - @Override public SchemaAndValue getKeyData(final Object cloudStorageKey, final String topic, final AbstractConfig sourceConfig) { @@ -72,10 +66,10 @@ public SchemaAndValue getKeyData(final Object cloudStorageKey, final String topi } @Override - public StreamSpliterator createSpliterator(final IOSupplier inputStreamIOSupplier, - final String topic, final int topicPartition, final AbstractConfig sourceConfig) { + public StreamSpliterator createSpliterator(final IOSupplier inputStreamIOSupplier, final String topic, + final int topicPartition, final AbstractConfig sourceConfig) { - final StreamSpliterator spliterator = new StreamSpliterator<>(LOGGER, inputStreamIOSupplier) { + return new StreamSpliterator(LOGGER, inputStreamIOSupplier) { private ParquetReader reader; private File parquetFile; @@ -114,11 +108,11 @@ protected void doClose() { } @Override - protected boolean doAdvance(final Consumer action) { + protected boolean doAdvance(final Consumer action) { try { final GenericRecord record = reader.read(); if (record != null) { - action.accept(record); // Pass record to the stream + action.accept(avroData.toConnectData(record.getSchema(), record)); // Pass record to the stream return true; } } catch (IOException e) { @@ -127,7 +121,6 @@ protected boolean doAdvance(final Consumer action) { return false; } }; - return spliterator; } static void deleteTmpFile(final Path parquetFile) { diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/input/Transformer.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/input/Transformer.java index 5bc353f6c..09e8c0ca5 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/input/Transformer.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/input/Transformer.java @@ -30,27 +30,18 @@ import org.apache.commons.io.function.IOSupplier; import org.slf4j.Logger; -public abstract class Transformer { +public abstract class Transformer { public abstract void configureValueConverter(Map config, AbstractConfig sourceConfig); - public final Stream getRecords(final IOSupplier inputStreamIOSupplier, final String topic, - final int topicPartition, final AbstractConfig sourceConfig, final long skipRecords) { + public final Stream getRecords(final IOSupplier inputStreamIOSupplier, + final String topic, final int topicPartition, final AbstractConfig sourceConfig, final long skipRecords) { - final StreamSpliterator spliterator = createSpliterator(inputStreamIOSupplier, topic, topicPartition, + final StreamSpliterator spliterator = createSpliterator(inputStreamIOSupplier, topic, topicPartition, sourceConfig); return StreamSupport.stream(spliterator, false).onClose(spliterator::close).skip(skipRecords); } - public final Stream getValues(final IOSupplier inputStreamIOSupplier, final String topic, - final int topicPartition, final AbstractConfig sourceConfig, final long skipRecords) { - - final StreamSpliterator spliterator = createSpliterator(inputStreamIOSupplier, topic, topicPartition, - sourceConfig); - return StreamSupport.stream(spliterator, false).onClose(spliterator::close).skip(skipRecords) - .map(t -> getValueData(t, topic, sourceConfig)); - } - /** * Creates the stream spliterator for this transformer. * @@ -64,20 +55,15 @@ public final Stream getValues(final IOSupplier inpu * the source configuraiton. * @return a StreamSpliterator instance. */ - protected abstract StreamSpliterator createSpliterator(IOSupplier inputStreamIOSupplier, - String topic, int topicPartition, AbstractConfig sourceConfig); - - public abstract SchemaAndValue getValueData(T record, String topic, AbstractConfig sourceConfig); + protected abstract StreamSpliterator createSpliterator(IOSupplier inputStreamIOSupplier, String topic, + int topicPartition, AbstractConfig sourceConfig); public abstract SchemaAndValue getKeyData(Object cloudStorageKey, String topic, AbstractConfig sourceConfig); /** * A Spliterator that performs various checks on the opening/closing of the input stream. - * - * @param - * the type of item created by this Spliterator. */ - protected abstract static class StreamSpliterator implements Spliterator { + protected abstract static class StreamSpliterator implements Spliterator { /** * The input stream supplier. */ @@ -118,7 +104,7 @@ protected StreamSpliterator(final Logger logger, final IOSupplier i * the Consumer to call if record is created. * @return {@code true} if a record was processed, {@code false} otherwise. */ - abstract protected boolean doAdvance(Consumer action); + abstract protected boolean doAdvance(Consumer action); /** * Method to close additional inputs if needed. @@ -130,7 +116,7 @@ public final void close() { try { if (inputStream != null) { inputStream.close(); - inputStream = null; + inputStream = null; // NOPMD setting null to release resources closed = true; } } catch (IOException e) { @@ -153,15 +139,16 @@ public final void close() { abstract protected InputStream inputOpened(InputStream input) throws IOException; @Override - public final boolean tryAdvance(final Consumer action) { - boolean result = false; + public final boolean tryAdvance(final Consumer action) { if (closed) { return false; } + boolean result = false; try { if (inputStream == null) { try { - inputStream = inputOpened(inputStreamIOSupplier.get()); + inputStream = inputStreamIOSupplier.get(); + inputOpened(inputStream); } catch (IOException e) { logger.error("Error trying to open inputStream: {}", e.getMessage(), e); close(); @@ -179,7 +166,7 @@ public final boolean tryAdvance(final Consumer action) { } @Override - public final Spliterator trySplit() { // NOPMD returning null is reqruied by API + public final Spliterator trySplit() { // NOPMD returning null is reqruied by API return null; } diff --git a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/AvroTransformerTest.java b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/AvroTransformerTest.java index 50e54a284..5c78084eb 100644 --- a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/AvroTransformerTest.java +++ b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/AvroTransformerTest.java @@ -32,6 +32,9 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.data.Struct; + import io.aiven.kafka.connect.common.config.SourceCommonConfig; import io.confluent.connect.avro.AvroData; @@ -75,7 +78,7 @@ void testConfigureValueConverter() { void testReadAvroRecordsInvalidData() { final InputStream inputStream = new ByteArrayInputStream("mock-avro-data".getBytes(StandardCharsets.UTF_8)); - final Stream records = avroTransformer.getRecords(() -> inputStream, "", 0, sourceCommonConfig, + final Stream records = avroTransformer.getRecords(() -> inputStream, "", 0, sourceCommonConfig, 0); final List recs = records.collect(Collectors.toList()); @@ -87,11 +90,17 @@ void testReadAvroRecords() throws Exception { final ByteArrayOutputStream avroData = generateMockAvroData(25); final InputStream inputStream = new ByteArrayInputStream(avroData.toByteArray()); - final Stream records = avroTransformer.getRecords(() -> inputStream, "", 0, sourceCommonConfig, + List expected = new ArrayList<>(); + for (int i = 0; i < 25; i++) { + expected.add("Hello, Kafka Connect S3 Source! object " + i); + } + + final Stream records = avroTransformer.getRecords(() -> inputStream, "", 0, sourceCommonConfig, 0); - final List recs = records.collect(Collectors.toList()); - assertThat(recs).hasSize(25); + assertThat(records).extracting(SchemaAndValue::value) + .extracting(sv -> ((Struct) sv).getString("message")) + .containsExactlyElementsOf(expected); } @Test @@ -99,14 +108,16 @@ void testReadAvroRecordsSkipFew() throws Exception { final ByteArrayOutputStream avroData = generateMockAvroData(20); final InputStream inputStream = new ByteArrayInputStream(avroData.toByteArray()); - final Stream records = avroTransformer.getRecords(() -> inputStream, "", 0, sourceCommonConfig, + List expected = new ArrayList<>(); + for (int i = 5; i < 20; i++) { + expected.add("Hello, Kafka Connect S3 Source! object " + i); + } + final Stream records = avroTransformer.getRecords(() -> inputStream, "", 0, sourceCommonConfig, 5); - final List recs = records.collect(Collectors.toList()); - assertThat(recs).hasSize(15); - // get first rec - assertThat(((GenericRecord) recs.get(0)).get("message").toString()) - .isEqualTo("Hello, Kafka Connect S3 Source! object 5"); + assertThat(records).extracting(SchemaAndValue::value) + .extracting(sv -> ((Struct) sv).getString("message")) + .containsExactlyElementsOf(expected); } @Test @@ -114,11 +125,10 @@ void testReadAvroRecordsSkipMoreRecordsThanExist() throws Exception { final ByteArrayOutputStream avroData = generateMockAvroData(20); final InputStream inputStream = new ByteArrayInputStream(avroData.toByteArray()); - final Stream records = avroTransformer.getRecords(() -> inputStream, "", 0, sourceCommonConfig, + final Stream records = avroTransformer.getRecords(() -> inputStream, "", 0, sourceCommonConfig, 25); - final List recs = records.collect(Collectors.toList()); - assertThat(recs).hasSize(0); + assertThat(records).isEmpty(); } static ByteArrayOutputStream generateMockAvroData(final int numRecs) throws IOException { diff --git a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/ByteArrayTransformerTest.java b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/ByteArrayTransformerTest.java index ee6b76001..80820e13b 100644 --- a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/ByteArrayTransformerTest.java +++ b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/ByteArrayTransformerTest.java @@ -24,6 +24,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.kafka.connect.data.SchemaAndValue; + import io.aiven.kafka.connect.common.config.SourceCommonConfig; import org.apache.commons.io.function.IOSupplier; @@ -53,12 +55,12 @@ void testGetRecordsSingleChunk() { final InputStream inputStream = new ByteArrayInputStream(data); final IOSupplier inputStreamIOSupplier = () -> inputStream; - final Stream records = byteArrayTransformer.getRecords(inputStreamIOSupplier, TEST_TOPIC, 0, + final Stream records = byteArrayTransformer.getRecords(inputStreamIOSupplier, TEST_TOPIC, 0, sourceCommonConfig, 0); - final List recs = records.collect(Collectors.toList()); + final List recs = records.collect(Collectors.toList()); assertThat(recs).hasSize(1); - assertThat((byte[]) recs.get(0)).isEqualTo(data); + assertThat(recs.get(0).value()).isEqualTo(data); } @Test @@ -67,18 +69,9 @@ void testGetRecordsEmptyInputStream() { final IOSupplier inputStreamIOSupplier = () -> inputStream; - final Stream records = byteArrayTransformer.getRecords(inputStreamIOSupplier, TEST_TOPIC, 0, + final Stream records = byteArrayTransformer.getRecords(inputStreamIOSupplier, TEST_TOPIC, 0, sourceCommonConfig, 0); assertThat(records).hasSize(0); } - - @Test - void testGetValueBytes() { - final byte[] record = { 1, 2, 3 }; - final byte[] result = (byte[]) byteArrayTransformer.getValueData(record, TEST_TOPIC, sourceCommonConfig) - .value(); - - assertThat(result).containsExactlyInAnyOrder(record); - } } diff --git a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/JsonTransformerTest.java b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/JsonTransformerTest.java index a38a2bc8a..4a9f615a3 100644 --- a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/JsonTransformerTest.java +++ b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/JsonTransformerTest.java @@ -18,7 +18,6 @@ import static io.aiven.kafka.connect.common.config.SchemaRegistryFragment.SCHEMAS_ENABLE; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -26,13 +25,13 @@ import java.io.IOException; import java.io.InputStream; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import java.util.stream.Stream; -import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.json.JsonConverter; import io.aiven.kafka.connect.common.config.SourceCommonConfig; @@ -77,31 +76,38 @@ void destroy() { @Test void testHandleValueDataWithValidJson() { final InputStream validJsonInputStream = new ByteArrayInputStream( - "{\"key\":\"value\"}".getBytes(StandardCharsets.UTF_8)); - final IOSupplier inputStreamIOSupplier = () -> validJsonInputStream; - final Stream jsonNodes = jsonTransformer.getRecords(inputStreamIOSupplier, TESTTOPIC, 1, + getJsonRecs(100).getBytes(StandardCharsets.UTF_8)); + + List expected = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + expected.add("value" + i); + } + + final Stream records = jsonTransformer.getRecords(() -> validJsonInputStream, TESTTOPIC, 1, sourceCommonConfig, 0); - assertThat(jsonNodes).hasSize(1); + assertThat(records).extracting(SchemaAndValue::value) + .extracting(sv -> ((Map) sv).get("key")) + .containsExactlyElementsOf(expected);; } @Test void testHandleValueDataWithValidJsonSkipFew() { final InputStream validJsonInputStream = new ByteArrayInputStream( getJsonRecs(100).getBytes(StandardCharsets.UTF_8)); - final IOSupplier inputStreamIOSupplier = () -> validJsonInputStream; - final Stream jsonNodes = jsonTransformer.getRecords(inputStreamIOSupplier, TESTTOPIC, 1, + + List expected = new ArrayList<>(); + for (int i = 25; i < 100; i++) { + expected.add("value" + i); + } + + final Stream records = jsonTransformer.getRecords(() -> validJsonInputStream, TESTTOPIC, 1, sourceCommonConfig, 25L); - final List recs = jsonNodes.collect(Collectors.toList()); - assertThat(recs).hasSize(75); - assertThat(recs).extracting(record -> ((Map) jsonTransformer.getValueData(record, "", null).value()).get("key")) - .doesNotContain("value1") - .doesNotContain("value2") - .doesNotContain("value25") - .contains("value26") - .contains("value27") - .contains("value100"); + assertThat(records).extracting(SchemaAndValue::value) + .extracting(sv -> ((Map) sv).get("key")) + .containsExactlyElementsOf(expected); + } @Test @@ -110,35 +116,17 @@ void testHandleValueDataWithInvalidJson() { "invalid-json".getBytes(StandardCharsets.UTF_8)); final IOSupplier inputStreamIOSupplier = () -> invalidJsonInputStream; - final Stream jsonNodes = jsonTransformer.getRecords(inputStreamIOSupplier, TESTTOPIC, 1, + final Stream jsonNodes = jsonTransformer.getRecords(inputStreamIOSupplier, TESTTOPIC, 1, sourceCommonConfig, 0); - assertThatThrownBy(() -> jsonTransformer.getValueData(jsonNodes.findAny().get(), "", null)) - .isInstanceOf(DataException.class) - .hasMessage("Converting byte[] to Kafka Connect data failed due to serialization error: "); - } + assertThat(jsonNodes).isEmpty(); - @Test - void testSerializeJsonDataValid() throws IOException { - final InputStream validJsonInputStream = new ByteArrayInputStream( - "{\"key\":\"value\"}".getBytes(StandardCharsets.UTF_8)); - final IOSupplier inputStreamIOSupplier = () -> validJsonInputStream; - final Stream jsonNodes = jsonTransformer.getRecords(inputStreamIOSupplier, TESTTOPIC, 1, - sourceCommonConfig, 0); - final Object serializedData = jsonTransformer - .getValueData( - jsonNodes.findFirst().orElseThrow(() -> new AssertionError("No records found in stream!")), - TESTTOPIC, sourceCommonConfig) - .value(); - - // Assert: Verify the serialized data - assertThat(serializedData).isInstanceOf(Map.class).extracting("key").isEqualTo("value"); } @Test void testGetRecordsWithIOException() throws IOException { when(inputStreamIOSupplierMock.get()).thenThrow(new IOException("Test IOException")); - final Stream resultStream = jsonTransformer.getRecords(inputStreamIOSupplierMock, "topic", 0, null, 0); + final Stream resultStream = jsonTransformer.getRecords(inputStreamIOSupplierMock, "topic", 0, null, 0); assertThat(resultStream).isEmpty(); } @@ -146,14 +134,14 @@ void testGetRecordsWithIOException() throws IOException { @Test void testCustomSpliteratorWithIOExceptionDuringInitialization() throws IOException { when(inputStreamIOSupplierMock.get()).thenThrow(new IOException("Test IOException during initialization")); - final Stream resultStream = jsonTransformer.getRecords(inputStreamIOSupplierMock, "topic", 0, null, 0); + final Stream resultStream = jsonTransformer.getRecords(inputStreamIOSupplierMock, "topic", 0, null, 0); assertThat(resultStream).isEmpty(); } static String getJsonRecs(final int recordCount) { final StringBuilder jsonRecords = new StringBuilder(); - for (int i = 1; i <= recordCount; i++) { + for (int i = 0; i < recordCount; i++) { jsonRecords.append(String.format("{\"key\":\"value%d\"}", i)); if (i < recordCount) { jsonRecords.append("\n"); // NOPMD AppendCharacterWithChar diff --git a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/ParquetTransformerTest.java b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/ParquetTransformerTest.java index 154baf45a..2f7a405fe 100644 --- a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/ParquetTransformerTest.java +++ b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/ParquetTransformerTest.java @@ -29,14 +29,17 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; +import java.util.ArrayList; import java.util.List; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.data.Struct; + import io.aiven.kafka.connect.common.config.SourceCommonConfig; import io.confluent.connect.avro.AvroData; -import org.apache.avro.generic.GenericRecord; import org.apache.commons.io.IOUtils; import org.apache.commons.io.function.IOSupplier; import org.junit.jupiter.api.BeforeEach; @@ -63,7 +66,7 @@ void testHandleValueDataWithZeroBytes() { final String topic = "test-topic"; final int topicPartition = 0; - final Stream recs = parquetTransformer.getRecords(inputStreamIOSupplier, topic, topicPartition, + final Stream recs = parquetTransformer.getRecords(inputStreamIOSupplier, topic, topicPartition, s3SourceConfig, 0L); assertThat(recs).isEmpty(); @@ -78,15 +81,17 @@ void testGetRecordsWithValidData() throws Exception { final String topic = "test-topic"; final int topicPartition = 0; - - final List records = parquetTransformer + final List expected = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + expected.add("name" + i); + } + final List records = parquetTransformer .getRecords(inputStreamIOSupplier, topic, topicPartition, s3SourceConfig, 0L) .collect(Collectors.toList()); - assertThat(records).hasSize(100); - assertThat(records).extracting(record -> ((GenericRecord) record).get("name").toString()) - .contains("name1") - .contains("name2"); + assertThat(records).extracting(SchemaAndValue::value) + .extracting(sv -> ((Struct) sv).getString("name")) + .containsExactlyElementsOf(expected); } @Test @@ -99,18 +104,18 @@ void testGetRecordsWithValidDataSkipFew() throws Exception { final String topic = "test-topic"; final int topicPartition = 0; - final List records = parquetTransformer + final List expected = new ArrayList<>(); + for (int i = 25; i < 100; i++) { + expected.add("name" + i); + } + + final List records = parquetTransformer .getRecords(inputStreamIOSupplier, topic, topicPartition, s3SourceConfig, 25L) .collect(Collectors.toList()); - assertThat(records).hasSize(75); - assertThat(records).extracting(record -> ((GenericRecord) record).get("name").toString()) - .doesNotContain("name1") - .doesNotContain("name2") - .doesNotContain("name24") - .contains("name25") - .contains("name26") - .contains("name99"); + assertThat(records).extracting(SchemaAndValue::value) + .extracting(sv -> ((Struct) sv).getString("name")) + .containsExactlyElementsOf(expected); } @Test @@ -124,7 +129,7 @@ void testGetRecordsWithInvalidData() { final String topic = "test-topic"; final int topicPartition = 0; - final Stream records = parquetTransformer.getRecords(inputStreamIOSupplier, topic, + final Stream records = parquetTransformer.getRecords(inputStreamIOSupplier, topic, topicPartition, s3SourceConfig, 0L); assertThat(records).isEmpty(); } @@ -150,7 +155,7 @@ void testIOExceptionCreatingTempFile() { .thenThrow(new IOException("Test IOException for temp file")); final IOSupplier inputStreamSupplier = mock(IOSupplier.class); - final Stream resultStream = parquetTransformer.getRecords(inputStreamSupplier, "test-topic", + final Stream resultStream = parquetTransformer.getRecords(inputStreamSupplier, "test-topic", 1, null, 0L); assertThat(resultStream).isEmpty(); @@ -163,7 +168,7 @@ void testIOExceptionDuringDataCopy() throws IOException { when(inputStreamMock.read(any(byte[].class))).thenThrow(new IOException("Test IOException during copy")); final IOSupplier inputStreamSupplier = () -> inputStreamMock; - final Stream resultStream = parquetTransformer.getRecords(inputStreamSupplier, "test-topic", + final Stream resultStream = parquetTransformer.getRecords(inputStreamSupplier, "test-topic", 1, null, 0L); assertThat(resultStream).isEmpty(); diff --git a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/TransformerStreamingTest.java b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/TransformerStreamingTest.java index f61dd9423..7d493a96b 100644 --- a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/TransformerStreamingTest.java +++ b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/TransformerStreamingTest.java @@ -17,6 +17,8 @@ package io.aiven.kafka.connect.common.source.input; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -32,11 +34,10 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.data.SchemaAndValue; import io.aiven.kafka.connect.common.config.CommonConfig; -import io.confluent.connect.avro.AvroData; import org.apache.commons.io.function.IOSupplier; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -49,8 +50,8 @@ class TransformerStreamingTest { @ParameterizedTest @MethodSource("testData") - void verifyExceptionDuringIOOpen(final Transformer transformer, final byte[] testData, - final AbstractConfig config, final int expectedCount) throws IOException { + void verifyExceptionDuringIOOpen(final Transformer transformer, final byte[] testData, final AbstractConfig config, + final int expectedCount) throws IOException { final IOSupplier ioSupplier = mock(IOSupplier.class); when(ioSupplier.get()).thenThrow(new IOException("Test IOException during initialization")); final Stream objStream = transformer.getRecords(ioSupplier, "topic", 1, config, 0); @@ -59,7 +60,25 @@ void verifyExceptionDuringIOOpen(final Transformer transformer, final byte[] @ParameterizedTest @MethodSource("testData") - void verifyCloseCalledAtEnd(final Transformer transformer, final byte[] testData, final AbstractConfig config, + void verifyExceptionDuringRead(final Transformer transformer, final byte[] testData, final AbstractConfig config, + final int expectedCount) throws IOException { + final InputStream inputStream = mock(InputStream.class); + when(inputStream.read()).thenThrow(new IOException("Test IOException during read")); + when(inputStream.read(any())).thenThrow(new IOException("Test IOException during read")); + when(inputStream.read(any(), anyInt(), anyInt())).thenThrow(new IOException("Test IOException during read")); + when(inputStream.readNBytes(any(), anyInt(), anyInt())) + .thenThrow(new IOException("Test IOException during read")); + when(inputStream.readNBytes(anyInt())).thenThrow(new IOException("Test IOException during read")); + when(inputStream.readAllBytes()).thenThrow(new IOException("Test IOException during read")); + final CloseTrackingStream stream = new CloseTrackingStream(inputStream); + final Stream objStream = transformer.getRecords(() -> stream, "topic", 1, config, 0); + assertThat(objStream).isEmpty(); + assertThat(stream.closeCount).isGreaterThan(0); + } + + @ParameterizedTest + @MethodSource("testData") + void verifyCloseCalledAtEnd(final Transformer transformer, final byte[] testData, final AbstractConfig config, final int expectedCount) throws IOException { final CloseTrackingStream stream = new CloseTrackingStream(new ByteArrayInputStream(testData)); final Stream objStream = transformer.getRecords(() -> stream, "topic", 1, config, 0); @@ -70,11 +89,11 @@ void verifyCloseCalledAtEnd(final Transformer transformer, final byte[] testD @ParameterizedTest @MethodSource("testData") - void verifyCloseCalledAtIteratorEnd(final Transformer transformer, final byte[] testData, + void verifyCloseCalledAtIteratorEnd(final Transformer transformer, final byte[] testData, final AbstractConfig config, final int expectedCount) throws IOException { final CloseTrackingStream stream = new CloseTrackingStream(new ByteArrayInputStream(testData)); - final Stream objStream = transformer.getRecords(() -> stream, "topic", 1, config, 0); - final Iterator iter = objStream.iterator(); + final Stream objStream = transformer.getRecords(() -> stream, "topic", 1, config, 0); + final Iterator iter = objStream.iterator(); long count = 0L; while (iter.hasNext()) { count += 1; @@ -86,19 +105,19 @@ void verifyCloseCalledAtIteratorEnd(final Transformer transformer, final byte static Stream testData() throws IOException { final List lst = new ArrayList<>(); - final AvroData avroData = new AvroData(100); - lst.add(Arguments.of(new AvroTransformer(avroData), AvroTransformerTest.generateMockAvroData(100).toByteArray(), + lst.add(Arguments.of(TransformerFactory.getTransformer(InputFormat.AVRO), + AvroTransformerTest.generateMockAvroData(100).toByteArray(), new CommonConfig(new ConfigDef(), new HashMap<>()) { }, 100)); - lst.add(Arguments.of(new ByteArrayTransformer(), "Hello World".getBytes(StandardCharsets.UTF_8), - new CommonConfig(new ConfigDef(), new HashMap<>()) { + lst.add(Arguments.of(TransformerFactory.getTransformer(InputFormat.BYTES), + "Hello World".getBytes(StandardCharsets.UTF_8), new CommonConfig(new ConfigDef(), new HashMap<>()) { }, 1)); - lst.add(Arguments.of(new JsonTransformer(new JsonConverter()), + lst.add(Arguments.of(TransformerFactory.getTransformer(InputFormat.JSONL), JsonTransformerTest.getJsonRecs(100).getBytes(StandardCharsets.UTF_8), new CommonConfig(new ConfigDef(), new HashMap<>()) { }, 100)); - lst.add(Arguments.of(new ParquetTransformer(avroData), ParquetTransformerTest.generateMockParquetData(), - new CommonConfig(new ConfigDef(), new HashMap<>()) { + lst.add(Arguments.of(TransformerFactory.getTransformer(InputFormat.PARQUET), + ParquetTransformerTest.generateMockParquetData(), new CommonConfig(new ConfigDef(), new HashMap<>()) { }, 100)); return lst.stream(); } diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java index 80ba203a7..649ce4b0c 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java @@ -1,5 +1,52 @@ +/* + * Copyright 2025 Aiven Oy + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.aiven.kafka.connect.s3.source; +import static io.aiven.kafka.connect.common.config.SchemaRegistryFragment.AVRO_VALUE_SERIALIZER; +import static io.aiven.kafka.connect.common.config.SchemaRegistryFragment.INPUT_FORMAT_KEY; +import static io.aiven.kafka.connect.common.config.SourceConfigFragment.TARGET_TOPICS; +import static io.aiven.kafka.connect.common.config.SourceConfigFragment.TARGET_TOPIC_PARTITIONS; +import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_ACCESS_KEY_ID_CONFIG; +import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_S3_BUCKET_NAME_CONFIG; +import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_S3_ENDPOINT_CONFIG; +import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_S3_PREFIX_CONFIG; +import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_SECRET_ACCESS_KEY_CONFIG; +import static io.aiven.kafka.connect.s3.source.S3SourceTask.OBJECT_KEY; +import static io.aiven.kafka.connect.s3.source.utils.OffsetManager.SEPARATOR; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.kafka.connect.source.SourceTaskContext; +import org.apache.kafka.connect.storage.OffsetStorageReader; + import io.aiven.kafka.connect.common.source.input.InputFormat; import io.aiven.kafka.connect.common.source.input.TransformerFactory; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; @@ -8,9 +55,8 @@ import io.aiven.kafka.connect.s3.source.utils.OffsetManager; import io.aiven.kafka.connect.s3.source.utils.S3SourceRecord; import io.aiven.kafka.connect.s3.source.utils.SourceRecordIterator; + import org.apache.avro.Schema; -import org.apache.kafka.connect.source.SourceTaskContext; -import org.apache.kafka.connect.storage.OffsetStorageReader; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; @@ -21,34 +67,6 @@ import org.testcontainers.junit.jupiter.Testcontainers; import software.amazon.awssdk.services.s3.S3Client; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.time.ZonedDateTime; -import java.time.format.DateTimeFormatter; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import static io.aiven.kafka.connect.common.config.SchemaRegistryFragment.AVRO_VALUE_SERIALIZER; -import static io.aiven.kafka.connect.common.config.SchemaRegistryFragment.INPUT_FORMAT_KEY; -import static io.aiven.kafka.connect.common.config.SourceConfigFragment.TARGET_TOPICS; -import static io.aiven.kafka.connect.common.config.SourceConfigFragment.TARGET_TOPIC_PARTITIONS; -import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_ACCESS_KEY_ID_CONFIG; -import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_S3_BUCKET_NAME_CONFIG; -import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_S3_ENDPOINT_CONFIG; -import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_S3_PREFIX_CONFIG; -import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_SECRET_ACCESS_KEY_CONFIG; -import static io.aiven.kafka.connect.s3.source.S3SourceTask.OBJECT_KEY; -import static io.aiven.kafka.connect.s3.source.utils.OffsetManager.SEPARATOR; -import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - @Testcontainers public class AwsIntegrationTest implements IntegrationBase { @@ -64,7 +82,6 @@ public class AwsIntegrationTest implements IntegrationBase { private BucketAccessor testBucketAccessor; - @Override public String getS3Prefix() { return s3Prefix; @@ -76,7 +93,7 @@ public S3Client getS3Client() { } @BeforeAll - static void setUpAll() { + static void setUpAll() { s3Prefix = COMMON_PREFIX + ZonedDateTime.now().format(DateTimeFormatter.ISO_LOCAL_DATE_TIME) + "/"; } @@ -111,7 +128,9 @@ private Map getConfig(final String topics, final int maxTasks) { /** * Test the integration with the Amazon connector - * @param testInfo The testing configuration. + * + * @param testInfo + * The testing configuration. */ @Test void sourceRecordIteratorBytesTest(final TestInfo testInfo) { @@ -147,7 +166,7 @@ void sourceRecordIteratorBytesTest(final TestInfo testInfo) { AWSV2SourceClient sourceClient = new AWSV2SourceClient(s3SourceConfig, new HashSet<>()); - Iterator sourceRecordIterator = new SourceRecordIterator(s3SourceConfig, offsetManager, + Iterator sourceRecordIterator = new SourceRecordIterator(s3SourceConfig, offsetManager, TransformerFactory.getTransformer(InputFormat.BYTES), sourceClient); HashSet seenKeys = new HashSet<>(); @@ -180,14 +199,14 @@ void sourceRecordIteratorAvroTest(final TestInfo testInfo) throws IOException { final int numOfRecsFactor = 5000; final byte[] outputStream1 = IntegrationBase.generateNextAvroMessagesStartingFromId(1, numOfRecsFactor, schema); - final byte[] outputStream2 = IntegrationBase.generateNextAvroMessagesStartingFromId(numOfRecsFactor + 1, numOfRecsFactor, - schema); - final byte[] outputStream3 = IntegrationBase.generateNextAvroMessagesStartingFromId(2 * numOfRecsFactor + 1, numOfRecsFactor, - schema); - final byte[] outputStream4 = IntegrationBase.generateNextAvroMessagesStartingFromId(3 * numOfRecsFactor + 1, numOfRecsFactor, - schema); - final byte[] outputStream5 = IntegrationBase.generateNextAvroMessagesStartingFromId(4 * numOfRecsFactor + 1, numOfRecsFactor, - schema); + final byte[] outputStream2 = IntegrationBase.generateNextAvroMessagesStartingFromId(numOfRecsFactor + 1, + numOfRecsFactor, schema); + final byte[] outputStream3 = IntegrationBase.generateNextAvroMessagesStartingFromId(2 * numOfRecsFactor + 1, + numOfRecsFactor, schema); + final byte[] outputStream4 = IntegrationBase.generateNextAvroMessagesStartingFromId(3 * numOfRecsFactor + 1, + numOfRecsFactor, schema); + final byte[] outputStream5 = IntegrationBase.generateNextAvroMessagesStartingFromId(4 * numOfRecsFactor + 1, + numOfRecsFactor, schema); final Set offsetKeys = new HashSet<>(); @@ -214,7 +233,7 @@ void sourceRecordIteratorAvroTest(final TestInfo testInfo) throws IOException { TransformerFactory.getTransformer(InputFormat.AVRO), sourceClient); HashSet seenKeys = new HashSet<>(); - Map> seenRecords = new HashMap<>(); + Map> seenRecords = new HashMap<>(); while (sourceRecordIterator.hasNext()) { S3SourceRecord s3SourceRecord = sourceRecordIterator.next(); String key = OBJECT_KEY + SEPARATOR + s3SourceRecord.getObjectKey(); @@ -229,12 +248,12 @@ void sourceRecordIteratorAvroTest(final TestInfo testInfo) throws IOException { assertThat(seenKeys).containsAll(offsetKeys); assertThat(seenRecords).hasSize(5); List expected = new ArrayList<>(); - for (long l=0; l < numOfRecsFactor; l++) { - expected.add(l+1); + for (long l = 0; l < numOfRecsFactor; l++) { + expected.add(l + 1); } for (String key : offsetKeys) { List seen = seenRecords.get(key); - assertThat(seen).as("Count for "+key).containsExactlyInAnyOrderElementsOf(expected); + assertThat(seen).as("Count for " + key).containsExactlyInAnyOrderElementsOf(expected); } } } diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java index 438518172..8a53acd4e 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java @@ -39,11 +39,6 @@ import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; -import org.apache.avro.Schema; -import org.apache.avro.file.DataFileWriter; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericDatumWriter; -import org.apache.avro.io.DatumWriter; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.NewTopic; @@ -60,7 +55,12 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import io.confluent.kafka.serializers.KafkaAvroDeserializer; +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.DatumWriter; import org.junit.jupiter.api.TestInfo; import org.testcontainers.containers.Container; import org.testcontainers.containers.localstack.LocalStackContainer; @@ -82,10 +82,10 @@ public interface IntegrationBase { String S3_SECRET_ACCESS_KEY = "test_secret_key0"; static byte[] generateNextAvroMessagesStartingFromId(final int messageId, final int noOfAvroRecs, - final Schema schema) throws IOException { + final Schema schema) throws IOException { final DatumWriter datumWriter = new GenericDatumWriter<>(schema); try (DataFileWriter dataFileWriter = new DataFileWriter<>(datumWriter); - ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) { + ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) { dataFileWriter.create(schema, outputStream); for (int i = messageId; i < messageId + noOfAvroRecs; i++) { final GenericRecord avroRecord = new GenericData.Record(schema); // NOPMD @@ -104,9 +104,12 @@ static byte[] generateNextAvroMessagesStartingFromId(final int messageId, final String getS3Prefix(); default String writeToS3(final String topicName, final byte[] testDataBytes, final String partitionId) { - final String objectKey = org.apache.commons.lang3.StringUtils.defaultIfBlank(getS3Prefix(), "") + topicName + "-" + partitionId + "-" - + System.currentTimeMillis() + ".txt"; - final PutObjectRequest request = PutObjectRequest.builder().bucket(IntegrationTest.TEST_BUCKET_NAME).key(objectKey).build(); + final String objectKey = org.apache.commons.lang3.StringUtils.defaultIfBlank(getS3Prefix(), "") + topicName + + "-" + partitionId + "-" + System.currentTimeMillis() + ".txt"; + final PutObjectRequest request = PutObjectRequest.builder() + .bucket(IntegrationTest.TEST_BUCKET_NAME) + .key(objectKey) + .build(); getS3Client().putObject(request, RequestBody.fromBytes(testDataBytes)); return OBJECT_KEY + SEPARATOR + objectKey; } diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java index 33459c2e0..47fadc990 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java @@ -30,8 +30,6 @@ import static java.util.Map.entry; import static org.assertj.core.api.Assertions.assertThat; import static org.awaitility.Awaitility.await; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -108,8 +106,7 @@ public String getS3Prefix() { public - @BeforeAll - static void setUpAll() throws IOException, InterruptedException { + @BeforeAll static void setUpAll() throws IOException, InterruptedException { s3Prefix = COMMON_PREFIX + ZonedDateTime.now().format(DateTimeFormatter.ISO_LOCAL_DATE_TIME) + "/"; s3Client = IntegrationBase.createS3Client(LOCALSTACK); @@ -185,7 +182,7 @@ void bytesTest(final TestInfo testInfo) { final Map expectedOffsetRecords = offsetKeys.subList(0, offsetKeys.size() - 1) .stream() .collect(Collectors.toMap(Function.identity(), s -> 1)); - //verifyOffsetPositions(expectedOffsetRecords, connectRunner.getBootstrapServers()); + // verifyOffsetPositions(expectedOffsetRecords, connectRunner.getBootstrapServers()); } @Test @@ -205,14 +202,14 @@ void avroTest(final TestInfo testInfo) throws IOException { final int numOfRecsFactor = 5000; final byte[] outputStream1 = IntegrationBase.generateNextAvroMessagesStartingFromId(1, numOfRecsFactor, schema); - final byte[] outputStream2 = IntegrationBase.generateNextAvroMessagesStartingFromId(numOfRecsFactor + 1, numOfRecsFactor, - schema); - final byte[] outputStream3 = IntegrationBase.generateNextAvroMessagesStartingFromId(2 * numOfRecsFactor + 1, numOfRecsFactor, - schema); - final byte[] outputStream4 = IntegrationBase.generateNextAvroMessagesStartingFromId(3 * numOfRecsFactor + 1, numOfRecsFactor, - schema); - final byte[] outputStream5 = IntegrationBase.generateNextAvroMessagesStartingFromId(4 * numOfRecsFactor + 1, numOfRecsFactor, - schema); + final byte[] outputStream2 = IntegrationBase.generateNextAvroMessagesStartingFromId(numOfRecsFactor + 1, + numOfRecsFactor, schema); + final byte[] outputStream3 = IntegrationBase.generateNextAvroMessagesStartingFromId(2 * numOfRecsFactor + 1, + numOfRecsFactor, schema); + final byte[] outputStream4 = IntegrationBase.generateNextAvroMessagesStartingFromId(3 * numOfRecsFactor + 1, + numOfRecsFactor, schema); + final byte[] outputStream5 = IntegrationBase.generateNextAvroMessagesStartingFromId(4 * numOfRecsFactor + 1, + numOfRecsFactor, schema); final Set offsetKeys = new HashSet<>(); @@ -225,7 +222,6 @@ void avroTest(final TestInfo testInfo) throws IOException { assertThat(testBucketAccessor.listObjects()).hasSize(5); - // Poll Avro messages from the Kafka topic and deserialize them final List records = IntegrationBase.consumeAvroMessages(topicName, numOfRecsFactor * 5, connectRunner.getBootstrapServers(), schemaRegistry.getSchemaRegistryUrl()); // Ensure this method @@ -241,8 +237,9 @@ void avroTest(final TestInfo testInfo) throws IOException { entry(4 * numOfRecsFactor, "Hello, Kafka Connect S3 Source! object " + (4 * numOfRecsFactor)), entry(5 * numOfRecsFactor, "Hello, Kafka Connect S3 Source! object " + (5 * numOfRecsFactor))); - // verifyOffsetPositions(offsetKeys.stream().collect(Collectors.toMap(Function.identity(), s -> numOfRecsFactor)), - // connectRunner.getBootstrapServers()); + // verifyOffsetPositions(offsetKeys.stream().collect(Collectors.toMap(Function.identity(), s -> + // numOfRecsFactor)), + // connectRunner.getBootstrapServers()); } @Test @@ -250,8 +247,8 @@ void parquetTest(final TestInfo testInfo) throws IOException { final var topicName = IntegrationBase.topicName(testInfo); final String partition = "00000"; - final String fileName = org.apache.commons.lang3.StringUtils.defaultIfBlank(getS3Prefix(), "") + topicName + "-" + partition + "-" + System.currentTimeMillis() - + ".txt"; + final String fileName = org.apache.commons.lang3.StringUtils.defaultIfBlank(getS3Prefix(), "") + topicName + "-" + + partition + "-" + System.currentTimeMillis() + ".txt"; final String name = "testuser"; final Map connectorConfig = getAvroConfig(topicName, InputFormat.PARQUET); diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index 5f85f2126..263b402c6 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -59,7 +59,7 @@ public class S3SourceTask extends AbstractSourceTask { /** * The transformer that we are using TODO move this to AbstractSourceTask */ - private Transformer transformer; + private Transformer transformer; /** The AWS Source client */ private AWSV2SourceClient awsv2SourceClient; @@ -115,8 +115,8 @@ public boolean hasNext() { @Override public SourceRecord next() { final S3SourceRecord s3SourceRecord = s3SourceRecordIterator.next(); - offsetManager.setCurrentOffsets(s3SourceRecord.getPartitionMap(), - s3SourceRecord.getObjectKey(), s3SourceRecord.getRecordNumber()); + offsetManager.setCurrentOffsets(s3SourceRecord.getPartitionMap(), s3SourceRecord.getObjectKey(), + s3SourceRecord.getRecordNumber()); return RecordProcessor.createSourceRecord(s3SourceRecord, s3SourceConfig, awsv2SourceClient, offsetManager); } @@ -146,9 +146,9 @@ public void commit() { public void commitRecord(final SourceRecord record) { if (LOGGER.isInfoEnabled()) { final Map map = (Map) record.sourceOffset(); -// LOGGER.info("Committed individual record {} {} {} committed", map.get(BUCKET), map.get(OBJECT_KEY), -// offsetManager.recordsProcessedForObjectKey((Map) record.sourcePartition(), -// map.get(OBJECT_KEY).toString())); + // LOGGER.info("Committed individual record {} {} {} committed", map.get(BUCKET), map.get(OBJECT_KEY), + // offsetManager.recordsProcessedForObjectKey((Map) record.sourcePartition(), + // map.get(OBJECT_KEY).toString())); LOGGER.info("Committed individual record {} committed", map); } } @@ -175,7 +175,7 @@ protected void closeResources() { * * @return the transformer that we are using. */ - public Transformer getTransformer() { + public Transformer getTransformer() { return transformer; } } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java index 589d62fb7..708ad2a74 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java @@ -27,7 +27,6 @@ import io.aiven.kafka.connect.s3.source.config.S3ClientFactory; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; -import org.apache.commons.collections4.IteratorUtils; import org.apache.commons.io.function.IOSupplier; import org.codehaus.plexus.util.StringUtils; import org.slf4j.Logger; diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java index 017bdb6b6..71d017221 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java @@ -83,7 +83,7 @@ public long incrementAndUpdateOffsetMap(final Map partitionMap, } public Map setCurrentOffsets(final Map partitionMap, final String currentObjectKey, - final long offset) { + final long offset) { Map offsetMap = offsets.compute(partitionMap, (k, v) -> { Map map = v == null ? new Hashtable<>() : v; map.put(getObjectMapKey(currentObjectKey), offset); diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3ObjectIterator.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3ObjectIterator.java index 37a5c3251..459c26464 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3ObjectIterator.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3ObjectIterator.java @@ -19,11 +19,11 @@ import java.util.Iterator; import java.util.NoSuchElementException; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; import software.amazon.awssdk.services.s3.model.S3Object; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; /** * Implements a ObjectSummaryIterator on an S3 bucket. Implementation reads summaries in blocks and iterates over each @@ -95,4 +95,4 @@ public S3Object next() { lastObjectSummaryKey = result.key(); return result; } -} \ No newline at end of file +} diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java index ccf40f678..a997cd6a0 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java @@ -34,8 +34,8 @@ public class S3SourceRecord { private final String objectKey; - public S3SourceRecord(final Map partitionMap, final long recordNumber, - final String topic, final Integer topicPartition, final String objectKey, final SchemaAndValue keyData, + public S3SourceRecord(final Map partitionMap, final long recordNumber, final String topic, + final Integer topicPartition, final String objectKey, final SchemaAndValue keyData, final SchemaAndValue valueData) { this.partitionMap = new HashMap<>(partitionMap); this.recordNumber = recordNumber; @@ -75,8 +75,9 @@ public SchemaAndValue getValue() { } public SourceRecord getSourceRecord(OffsetManager offsetManager) { - Map offsetMap = offsetManager.setCurrentOffsets(getPartitionMap(), getObjectKey(), getRecordNumber()); - return new SourceRecord(getPartitionMap(), offsetMap, topic, partition(), keyData.schema(), - keyData.value(), valueData.schema(), valueData.value()); + Map offsetMap = offsetManager.setCurrentOffsets(getPartitionMap(), getObjectKey(), + getRecordNumber()); + return new SourceRecord(getPartitionMap(), offsetMap, topic, partition(), keyData.schema(), keyData.value(), + valueData.schema(), valueData.value()); } } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java index 66f2dc2f5..7e3942e59 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java @@ -21,19 +21,18 @@ import java.util.Map; import java.util.function.Function; import java.util.function.Predicate; -import java.util.function.Supplier; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Stream; -import org.apache.commons.collections4.IteratorUtils; -import org.apache.commons.collections4.iterators.LazyIteratorChain; import org.apache.kafka.connect.data.SchemaAndValue; import io.aiven.kafka.connect.common.source.input.ByteArrayTransformer; import io.aiven.kafka.connect.common.source.input.Transformer; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; +import org.apache.commons.collections4.IteratorUtils; +import org.apache.commons.collections4.iterators.LazyIteratorChain; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.s3.model.S3Object; @@ -42,7 +41,7 @@ * Iterator that processes S3 files and creates Kafka source records. Supports different output formats (Avro, JSON, * Parquet). */ -public final class SourceRecordIterator extends LazyIteratorChain implements Iterator { +public final class SourceRecordIterator extends LazyIteratorChain implements Iterator { private static final Logger LOGGER = LoggerFactory.getLogger(SourceRecordIterator.class); public static final String PATTERN_TOPIC_KEY = "topicName"; @@ -59,7 +58,7 @@ public final class SourceRecordIterator extends LazyIteratorChain transformer; + private final Transformer transformer; // Once we decouple the S3Object from the Source Iterator we can change this to be the SourceApiClient // At which point it will work for al our integrations. private final AWSV2SourceClient sourceClient; @@ -83,7 +82,7 @@ public final class SourceRecordIterator extends LazyIteratorChain this.fileNamePredicate.test(s3Object)); // call filter out bad file names and extract topic/partition + inner = IteratorUtils.filteredIterator(sourceClient.getS3ObjectStream(null).iterator(), + s3Object -> this.fileNamePredicate.test(s3Object)); // call filter out bad file names and extract + // topic/partition } @Override @@ -116,8 +117,9 @@ private Stream convert(final S3Object s3Object) { SchemaAndValue keyData = transformer.getKeyData(s3Object.key(), topic, s3SourceConfig); - return transformer.getValues(sourceClient.getObject(s3Object.key()), topic, partitionId, - s3SourceConfig, recordCount).map(new Mapper(partitionMap, recordCount, keyData, s3Object.key())); + return transformer + .getRecords(sourceClient.getObject(s3Object.key()), topic, partitionId, s3SourceConfig, recordCount) + .map(new Mapper(partitionMap, recordCount, keyData, s3Object.key())); } class Mapper implements Function { @@ -137,183 +139,8 @@ public Mapper(Map partitionMap, long recordCount, SchemaAndValue @Override public S3SourceRecord apply(SchemaAndValue value) { recordCount++; - return new S3SourceRecord(partitionMap, recordCount, topic, partitionId, objectKey, keyData, - value); + return new S3SourceRecord(partitionMap, recordCount, topic, partitionId, objectKey, keyData, value); } } } - - -// private void nextS3Object() { -// if (!objectListIterator.hasNext()) { -// // Start after the object Key we have just finished with. -// objectListIterator = sourceClient.getListOfObjectKeys(currentObjectKey); -// if (!objectListIterator.hasNext()) { -// recordIterator = Collections.emptyIterator(); -// return; -// } -// } -// -// try { -// currentObjectKey = objectListIterator.next(); -// if (currentObjectKey != null) { -// recordIterator = createIteratorForCurrentFile(); -// } -// } catch (IOException e) { -// throw SdkException.create(e.getMessage(), e.getCause()); -// } -// } - - -// private boolean checkBytesTransformation ( final Transformer transformer, -// final long numberOfRecsAlreadyProcessed){ -// return transformer instanceof ByteArrayTransformer -// && numberOfRecsAlreadyProcessed == BYTES_TRANSFORMATION_NUM_OF_RECS; -// } -// } - - - - - - -// { -// -// final Iterator recordIterator = recordStream.iterator(); -// while (recordIterator.hasNext()) { -// final Object record = recordIterator.next(); -// recordCount++; -// sourceRecords.add(getSourceRecord(topic, topicPartition, recordCount, startOffset, -// partitionMap, transformer.getValueData(record, topic, s3SourceConfig), -// transformer.getKeyData(currentObjectKey, topic, s3SourceConfig))); -// -// // Break if we have reached the max records per poll -// if (sourceRecords.size() >= s3SourceConfig.getMaxPollRecords()) { -// break; -// } -// } -// } - - -// -// private x() { -// // left over stuff -// final long defaultStartOffsetId = 1L; -// -// final String finalTopic = topic; -// final Map partitionMap = ConnectUtils.getPartitionMap(topic, defaultPartitionId, -// bucketName); -// -// return getObjectIterator(s3Object, finalTopic, defaultPartitionId, defaultStartOffsetId, transformer, -// partitionMap); -// -// } else { -// LOGGER.error("File naming doesn't match to any topic. {}", currentObjectKey); -// return Collections.emptyIterator(); -// } -// } - -// @SuppressWarnings("PMD.CognitiveComplexity") -// private Iterator getObjectIterator(final IOSupplier s3Object, final String topic, -// final int topicPartition, final long startOffset, final Transformer transformer, -// final Map partitionMap) { -// return new Iterator<>() { -// private final Iterator internalIterator = readNext().iterator(); -// private long recordCount = 0; -// -// private List readNext() { -// -// final List sourceRecords = new ArrayList<>(); -// -// final long numberOfRecsAlreadyProcessed = offsetManager.recordsProcessedForObjectKey(partitionMap, -// currentObjectKey); -// recordCount = numberOfRecsAlreadyProcessed; -// // Optimizing without reading stream again. -// if (checkBytesTransformation(transformer, numberOfRecsAlreadyProcessed)) { -// return sourceRecords; -// } -// -// try (Stream recordStream = transformer.getRecords(s3Object, topic, topicPartition, -// s3SourceConfig, numberOfRecsAlreadyProcessed)) { -// -// final Iterator recordIterator = recordStream.iterator(); -// while (recordIterator.hasNext()) { -// final Object record = recordIterator.next(); -// recordCount++; -// sourceRecords.add(getSourceRecord(topic, topicPartition, recordCount, startOffset, -// partitionMap, transformer.getValueData(record, topic, s3SourceConfig), -// transformer.getKeyData(currentObjectKey, topic, s3SourceConfig))); -// -// // Break if we have reached the max records per poll -// if (sourceRecords.size() >= s3SourceConfig.getMaxPollRecords()) { -// break; -// } -// } -// } -// -// return sourceRecords; -// } -// -// // For bytes transformation, read whole file as 1 record -// private boolean checkBytesTransformation(final Transformer transformer, -// final long numberOfRecsAlreadyProcessed) { -// return transformer instanceof ByteArrayTransformer -// && numberOfRecsAlreadyProcessed == BYTES_TRANSFORMATION_NUM_OF_RECS; -// } -// -// private S3SourceRecord getSourceRecord(final String topic, final int topicPartition, -// final long recordNumber, final long startOffset, final Map partitionMap, -// final SchemaAndValue valueData, final SchemaAndValue keyData) { -// -// return new S3SourceRecord(partitionMap, recordNumber, topic, topicPartition, currentObjectKey, keyData, -// valueData); -// } -// -// @Override -// public boolean hasNext() { -// return internalIterator.hasNext(); -// } -// -// @Override -// public S3SourceRecord next() { -// return internalIterator.next(); -// } -// }; -// } -// -// @Override -// public boolean hasNext() { -// if (!recordIterator.hasNext()) { -// nextS3Object(); -// } -// return recordIterator.hasNext(); -// } -// -// @Override -// public S3SourceRecord next() { -// if (!recordIterator.hasNext()) { -// throw new NoSuchElementException(); -// } -// return recordIterator.next(); -// } -// -// @Override -// public void remove() { -// throw new UnsupportedOperationException("This iterator is unmodifiable"); -// } -// -// private static class S3SourceRecordIterator implements Iterator { -// -// private Iterator objectKeyIterator; -// -// S3SourceRecordIterator(Iterator objectKeyIterator) { -// -// -// -// } -// -// -// } - -//} diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java index 709e99be2..d809c3559 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java @@ -35,7 +35,6 @@ import java.util.Map; import java.util.Random; -import io.aiven.kafka.connect.s3.source.utils.OffsetManager; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.source.SourceRecord; @@ -50,6 +49,7 @@ import io.aiven.kafka.connect.iam.AwsCredentialProviderFactory; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; import io.aiven.kafka.connect.s3.source.utils.ConnectUtils; +import io.aiven.kafka.connect.s3.source.utils.OffsetManager; import io.aiven.kafka.connect.s3.source.utils.S3SourceRecord; import io.findify.s3mock.S3Mock; @@ -155,9 +155,8 @@ void testStop() { private static S3SourceRecord createS3SourceRecord(final String topicName, final Integer defaultPartitionId, final String bucketName, final String objectKey, final byte[] key, final byte[] value) { - return new S3SourceRecord(ConnectUtils.getPartitionMap(topicName, defaultPartitionId, bucketName), - 0L, topicName, defaultPartitionId, objectKey, - new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, key), + return new S3SourceRecord(ConnectUtils.getPartitionMap(topicName, defaultPartitionId, bucketName), 0L, + topicName, defaultPartitionId, objectKey, new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, key), new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, value)); } @@ -204,7 +203,8 @@ private void assertEquals(final S3SourceRecord s3Record, final SourceRecord sour assertThat(sourceRecord.sourcePartition()).isEqualTo(s3Record.getPartitionMap()); Map map = (Map) sourceRecord.sourceOffset(); - assertThat(map.get(OffsetManager.getObjectMapKey(s3Record.getObjectKey()))).isEqualTo(s3Record.getRecordNumber()); + assertThat(map.get(OffsetManager.getObjectMapKey(s3Record.getObjectKey()))) + .isEqualTo(s3Record.getRecordNumber()); assertThat(sourceRecord.key()).isEqualTo(s3Record.getKey().value()); assertThat(sourceRecord.value()).isEqualTo(s3Record.getValue().value()); } diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClientTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClientTest.java index c8b79377b..beed0681c 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClientTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClientTest.java @@ -31,7 +31,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.stream.Stream; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java index d9247c7bd..59ca6fd41 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java @@ -79,8 +79,7 @@ void testCreateSourceRecord() throws ConnectException { SourceRecord result = RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, offsetManager); - verify(offsetManager, times(1)).updateCurrentOffsets(any(), any()); - verify(mockRecord, times(2)).getPartitionMap(); + verify(mockRecord, times(1)).getSourceRecord(any()); assertThat(result).isEqualTo(mockSourceRecord); } diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java index 3884764e6..a87436ac3 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java @@ -31,17 +31,24 @@ import java.io.ByteArrayInputStream; import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.Collections; import java.util.Iterator; import java.util.stream.Stream; +import org.apache.kafka.connect.data.SchemaAndValue; + import io.aiven.kafka.connect.common.source.input.AvroTransformer; import io.aiven.kafka.connect.common.source.input.ByteArrayTransformer; +import io.aiven.kafka.connect.common.source.input.InputFormat; import io.aiven.kafka.connect.common.source.input.Transformer; +import io.aiven.kafka.connect.common.source.input.TransformerFactory; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import software.amazon.awssdk.services.s3.model.S3Object; final class SourceRecordIteratorTest { @@ -68,8 +75,7 @@ void testIteratorProcessesS3Objects() throws Exception { try (InputStream mockInputStream = new ByteArrayInputStream(new byte[] {})) { when(mockSourceApiClient.getObject(anyString())).thenReturn(() -> mockInputStream); - when(mockTransformer.getRecords(any(), anyString(), anyInt(), any(), anyLong())) - .thenReturn(Stream.of(new Object())); + mockTransformer = TransformerFactory.getTransformer(InputFormat.BYTES); when(mockOffsetManager.getOffsets()).thenReturn(Collections.emptyMap()); @@ -78,15 +84,17 @@ void testIteratorProcessesS3Objects() throws Exception { mockSourceApiClient); assertThat(iterator.hasNext()).isFalse(); - assertThat(iterator.next()).isNull(); - when(mockSourceApiClient.getListOfObjectKeys(any())) - .thenReturn(Collections.singletonList(key).listIterator()); + S3Object obj = S3Object.builder().key(key).build(); + ByteArrayInputStream bais = new ByteArrayInputStream("Hello World".getBytes(StandardCharsets.UTF_8)); + when(mockSourceApiClient.getS3ObjectStream(any())).thenReturn(Arrays.asList(obj).stream()); + when(mockSourceApiClient.getObject(any())).thenReturn(() -> bais); iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, mockSourceApiClient); - assertThat(iterator.hasNext()).isTrue(); + assertThat(iterator).hasNext(); assertThat(iterator.next()).isNotNull(); + assertThat(iterator).isExhausted(); } } @@ -94,15 +102,17 @@ void testIteratorProcessesS3Objects() throws Exception { void testIteratorProcessesS3ObjectsForByteArrayTransformer() throws Exception { final String key = "topic-00001-abc123.txt"; + final S3Object s3Object = S3Object.builder().key(key).build(); - // Mock InputStream - try (InputStream mockInputStream = new ByteArrayInputStream(new byte[] {})) { - when(mockSourceApiClient.getObject(anyString())).thenReturn(() -> mockInputStream); + // With ByteArrayTransformer + try (InputStream inputStream = new ByteArrayInputStream("Hello World".getBytes(StandardCharsets.UTF_8))) { + when(mockSourceApiClient.getObject(key)).thenReturn(() -> inputStream); + + when(mockSourceApiClient.getS3ObjectStream(any())).thenReturn(Arrays.asList(s3Object).stream()); - // With ByteArrayTransformer mockTransformer = mock(ByteArrayTransformer.class); when(mockTransformer.getRecords(any(), anyString(), anyInt(), any(), anyLong())) - .thenReturn(Stream.of(new Object())); + .thenReturn(Stream.of(SchemaAndValue.NULL)); when(mockOffsetManager.getOffsets()).thenReturn(Collections.emptyMap()); @@ -111,24 +121,36 @@ void testIteratorProcessesS3ObjectsForByteArrayTransformer() throws Exception { when(mockOffsetManager.recordsProcessedForObjectKey(anyMap(), anyString())) .thenReturn(BYTES_TRANSFORMATION_NUM_OF_RECS); + // should skip if any records were produced by source record iterator. Iterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, mockSourceApiClient); - assertThat(iterator.hasNext()).isTrue(); - iterator.next(); + assertThat(iterator.hasNext()).isFalse(); + verify(mockSourceApiClient, never()).getObject(any()); verify(mockTransformer, never()).getRecords(any(), anyString(), anyInt(), any(), anyLong()); + } - // With AvroTransformer + // With AvroTransformer + try (InputStream inputStream = new ByteArrayInputStream("Hello World".getBytes(StandardCharsets.UTF_8))) { + when(mockSourceApiClient.getObject(key)).thenReturn(() -> inputStream); + when(mockSourceApiClient.getS3ObjectStream(any())).thenReturn(Arrays.asList(s3Object).stream()); mockTransformer = mock(AvroTransformer.class); when(mockSourceApiClient.getListOfObjectKeys(any())) .thenReturn(Collections.singletonList(key).listIterator()); + when(mockOffsetManager.recordsProcessedForObjectKey(anyMap(), anyString())) .thenReturn(BYTES_TRANSFORMATION_NUM_OF_RECS); - iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, mockSourceApiClient); + when(mockTransformer.getKeyData(anyString(), anyString(), any())).thenReturn(SchemaAndValue.NULL); + when(mockTransformer.getRecords(any(), anyString(), anyInt(), any(), anyLong())) + .thenReturn(Arrays.asList(SchemaAndValue.NULL).stream()); + + Iterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, + mockSourceApiClient); assertThat(iterator.hasNext()).isTrue(); iterator.next(); verify(mockTransformer, times(1)).getRecords(any(), anyString(), anyInt(), any(), anyLong()); } } + } From ebf0e8a3ef217fea94bc2e62c488bf5c8b50270b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Mon, 6 Jan 2025 11:51:02 +0000 Subject: [PATCH 09/44] removed unused class --- .../s3/source/utils/S3ObjectIterator.java | 98 ------------------- 1 file changed, 98 deletions(-) delete mode 100644 s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3ObjectIterator.java diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3ObjectIterator.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3ObjectIterator.java deleted file mode 100644 index 459c26464..000000000 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3ObjectIterator.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * Copyright 2024 Aiven Oy - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.aiven.kafka.connect.s3.source.utils; - -import java.util.Iterator; -import java.util.NoSuchElementException; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -import software.amazon.awssdk.services.s3.S3Client; -import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; -import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; -import software.amazon.awssdk.services.s3.model.S3Object; - -/** - * Implements a ObjectSummaryIterator on an S3 bucket. Implementation reads summaries in blocks and iterates over each - * block. When block is empty a new block is retrieved and processed until no more data is available. - */ -public class S3ObjectIterator implements Iterator { - /** The client we are using */ - private final S3Client s3Client; - /** The object listing from the last call to the client */ - private ListObjectsV2Response objectListing; - /** The inner iterator on the object summaries. When it is empty a new one is read from object listing. */ - private Iterator innerIterator; - - /** the ObjectRequest initially to start the iteration from later to retrieve more records */ - private ListObjectsV2Request request; - - /** The last key seen by this process. This allows us to restart when a new file is dropped in the direcotry */ - private String lastObjectSummaryKey; - - /** - * Constructs the s3ObjectSummaryIterator based on the Amazon se client. - * - * @param s3Client - * the Amazon client to read use for access. - * @param request - * the request object that defines the starting position for the object summary retrieval. - */ - @SuppressFBWarnings(value = "EI_EXPOSE_REP2", justification = "stores mutable AmazeonS3 and ListObjectsV2Request objects") - public S3ObjectIterator(final S3Client s3Client, final ListObjectsV2Request request) { - this.s3Client = s3Client; - this.request = request; - } - - @Override - public boolean hasNext() { - // delay creating objectListing until we need it. - if (objectListing == null) { - objectListing = s3Client.listObjectsV2(request); - innerIterator = objectListing.contents().iterator(); - } - if (!this.innerIterator.hasNext()) { - if (objectListing.isTruncated()) { - // get the next set of data and create an iterator on it. - ListObjectsV2Request.Builder builder = request.toBuilder(); - builder.startAfter(null); - builder.continuationToken(objectListing.continuationToken()); - request = builder.build(); - objectListing = s3Client.listObjectsV2(request); - } else { - // there is no more data -- reread the bucket - ListObjectsV2Request.Builder builder = request.toBuilder(); - builder.continuationToken(null); - builder.startAfter(lastObjectSummaryKey); - request = builder.build(); - objectListing = s3Client.listObjectsV2(request); - } - innerIterator = objectListing.contents().iterator(); - } - // innerIterator is configured. Does it have more? - return innerIterator.hasNext(); - } - - @Override - public S3Object next() { - if (!hasNext()) { - throw new NoSuchElementException(); - } - final S3Object result = innerIterator.next(); - lastObjectSummaryKey = result.key(); - return result; - } -} From 9c73cd4d110b5a47e7a6e416b3f0d18624b3afbb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Mon, 6 Jan 2025 11:56:40 +0000 Subject: [PATCH 10/44] removed unused methods --- .../s3/source/utils/OffsetManager.java | 24 +------------------ 1 file changed, 1 insertion(+), 23 deletions(-) diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java index 71d017221..49f422c5a 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java @@ -102,29 +102,7 @@ public long recordsProcessedForObjectKey(final Map partitionMap, } return 0L; } - - public void createNewOffsetMap(final Map partitionMap, final String objectKey, - final long offsetId) { - final Map offsetMap = getOffsetValueMap(objectKey, offsetId); - offsets.put(partitionMap, offsetMap); - } - - public Map getOffsetValueMap(final String currentObjectKey, final long offsetId) { - final Map offsetMap = new HashMap<>(); - offsetMap.put(getObjectMapKey(currentObjectKey), offsetId); - return offsetMap; - } - - void updateCurrentOffsets(final Map partitionMap, final Map offsetValueMap) { - if (offsets.containsKey(partitionMap)) { - final Map offsetMap = new HashMap<>(offsets.get(partitionMap)); - offsetMap.putAll(offsetValueMap); - offsets.put(partitionMap, offsetMap); - } else { - offsets.put(partitionMap, offsetValueMap); - } - } - + private static Set parsePartitions(final S3SourceConfig s3SourceConfig) { final String partitionString = s3SourceConfig.getTargetTopicPartitions(); return Arrays.stream(partitionString.split(",")).map(Integer::parseInt).collect(Collectors.toSet()); From d7b2f3c92dacda11e27e8f1639ed087de3ec07d8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Mon, 6 Jan 2025 12:53:42 +0000 Subject: [PATCH 11/44] Changes to allow Backoff to abort the timer --- .../common/source/AbstractSourceTask.java | 78 +++++++++++++++---- .../common/source/AbstractSourceTaskTest.java | 37 ++++++--- .../kafka/connect/s3/source/S3SourceTask.java | 4 +- .../s3/source/utils/OffsetManager.java | 2 +- 4 files changed, 90 insertions(+), 31 deletions(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java index 334b2deb9..c18ef5cea 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -99,7 +99,7 @@ protected AbstractSourceTask(final Logger logger) { this.logger = logger; connectorStopped = new AtomicBoolean(); timer = new Timer(MAX_POLL_TIME); - backoff = new Backoff(timer::millisecondsRemaining); + backoff = new Backoff(timer.getBackoffConfig()); } /** @@ -112,11 +112,11 @@ protected AbstractSourceTask(final Logger logger) { * this iterator executes may cause the task to abort. *

* - * @param timer - * a SupplierOfLong that provides the amount of time remaining before the polling expires. + * @param config + * the configuraiton for the Backoff. * @return The iterator of SourceRecords. */ - abstract protected Iterator getIterator(SupplierOfLong timer); + abstract protected Iterator getIterator(BackoffConfig config); /** * Called by {@link #start} to allows the concrete implementation to configure itself based on properties. @@ -131,7 +131,7 @@ public final void start(final Map props) { logger.debug("Starting"); config = configure(props); maxPollRecords = config.getMaxPollRecords(); - sourceRecordIterator = getIterator(timer::millisecondsRemaining); + sourceRecordIterator = getIterator(timer.getBackoffConfig()); } /** @@ -163,7 +163,7 @@ private boolean tryAdd(final List results, final Iterator= duration; + public boolean isExpired() { + return hasAborted || super.getTime() >= duration; + } + + public void abort() { + hasAborted = true; } @Override public void start() { try { + hasAborted = false; super.start(); } catch (IllegalStateException e) { throw new IllegalStateException("Timer: " + e.getMessage()); @@ -300,11 +307,31 @@ public void stop() { @Override public void reset() { try { + hasAborted = false; super.reset(); } catch (IllegalStateException e) { throw new IllegalStateException("Timer: " + e.getMessage()); } } + + /** + * Gets a Backoff Config for this timer. + * @return a backoff Configuration. + */ + public BackoffConfig getBackoffConfig() { + return new BackoffConfig() { + + @Override + public SupplierOfLong getSupplierOfTimeRemaining() { + return Timer.this::millisecondsRemaining; + } + + @Override + public AbortTrigger getAbortTrigger() { + return Timer.this::abort; + } + }; + } } /** @@ -325,6 +352,8 @@ public static class Backoff { */ private final SupplierOfLong timeRemaining; + final AbortTrigger abortTrigger; + /** * The maximum number of times {@link #delay()} will be called before maxWait is reached. */ @@ -342,11 +371,12 @@ public static class Backoff { /** * Constructor. * - * @param timeRemaining - * A supplier of long as milliseconds remaining before time expires. + * @param config + * The configuration for the backoff. */ - public Backoff(final SupplierOfLong timeRemaining) { - this.timeRemaining = timeRemaining; + public Backoff(final BackoffConfig config) { + this.timeRemaining = config.getSupplierOfTimeRemaining(); + this.abortTrigger = config.getAbortTrigger(); reset(); } @@ -400,11 +430,15 @@ public void delay() throws InterruptedException { if (sleepTime > 0) { if (waitCount < maxCount) { waitCount++; - sleepTime = Math.min(sleepTime, timeWithJitter()); - } - // don't sleep negative time. Jitter can introduce negative tme. - if (sleepTime > 0) { - Thread.sleep(sleepTime); + long nextSleep = timeWithJitter(); + // don't sleep negative time. Jitter can introduce negative tme. + if (nextSleep > 0) { + if (nextSleep >= sleepTime) { + abortTrigger.apply(); + } else { + Thread.sleep(nextSleep); + } + } } } } @@ -433,4 +467,14 @@ public String toString() { public interface SupplierOfLong { long get(); } + + @FunctionalInterface + public interface AbortTrigger { + void apply(); + } + + public interface BackoffConfig { + SupplierOfLong getSupplierOfTimeRemaining(); + AbortTrigger getAbortTrigger(); + } } diff --git a/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java b/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java index 8ea7768c5..255f704d4 100644 --- a/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java +++ b/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java @@ -21,6 +21,7 @@ import static org.awaitility.Awaitility.await; import java.time.Duration; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.lang3.time.StopWatch; import org.junit.jupiter.api.Test; @@ -32,7 +33,7 @@ void timerTest() { final AbstractSourceTask.Timer timer = new AbstractSourceTask.Timer(Duration.ofSeconds(1)); assertThat(timer.millisecondsRemaining()).isEqualTo(Duration.ofSeconds(1).toMillis()); timer.start(); - await().atMost(Duration.ofSeconds(2)).until(timer::expired); + await().atMost(Duration.ofSeconds(2)).until(timer::isExpired); assertThat(timer.millisecondsRemaining()).isLessThan(0); timer.stop(); assertThat(timer.millisecondsRemaining()).isEqualTo(Duration.ofSeconds(1).toMillis()); @@ -43,14 +44,14 @@ void timerSequenceTest() { final AbstractSourceTask.Timer timer = new AbstractSourceTask.Timer(Duration.ofSeconds(1)); // stopped state does not allow stop assertThatExceptionOfType(IllegalStateException.class).as("stop while not running") - .isThrownBy(() -> timer.stop()) + .isThrownBy(timer::stop) .withMessageStartingWith("Timer: "); timer.reset(); // verify that an exception is not thrown. // started state does not allow start timer.start(); assertThatExceptionOfType(IllegalStateException.class).as("start while running") - .isThrownBy(() -> timer.start()) + .isThrownBy(timer::start) .withMessageStartingWith("Timer: "); timer.reset(); timer.start(); // restart the timer. @@ -58,30 +59,30 @@ void timerSequenceTest() { // stopped state does not allow stop or start assertThatExceptionOfType(IllegalStateException.class).as("stop after stop") - .isThrownBy(() -> timer.stop()) + .isThrownBy(timer::stop) .withMessageStartingWith("Timer: "); assertThatExceptionOfType(IllegalStateException.class).as("start after stop") - .isThrownBy(() -> timer.start()) + .isThrownBy(timer::start) .withMessageStartingWith("Timer: "); timer.reset(); // stopped + reset does not allow stop. assertThatExceptionOfType(IllegalStateException.class).as("stop after reset (1)") - .isThrownBy(() -> timer.stop()) + .isThrownBy(timer::stop) .withMessageStartingWith("Timer: "); timer.start(); timer.reset(); // started + reset does not allow stop; assertThatExceptionOfType(IllegalStateException.class).as("stop after reset (2)") - .isThrownBy(() -> timer.stop()) + .isThrownBy(timer::stop) .withMessageStartingWith("Timer: "); } @Test void backoffTest() throws InterruptedException { final AbstractSourceTask.Timer timer = new AbstractSourceTask.Timer(Duration.ofSeconds(1)); - final AbstractSourceTask.Backoff backoff = new AbstractSourceTask.Backoff(timer::millisecondsRemaining); + final AbstractSourceTask.Backoff backoff = new AbstractSourceTask.Backoff(timer.getBackoffConfig()); final long estimatedDelay = backoff.estimatedDelay(); assertThat(estimatedDelay).isLessThan(500); @@ -94,14 +95,14 @@ void backoffTest() throws InterruptedException { .isBetween(estimatedDelay - backoff.getMaxJitter(), estimatedDelay + backoff.getMaxJitter()); timer.start(); - for (int i = 0; i < 10; i++) { + for (int i = 0; i < 9; i++) { stopWatch.reset(); timer.reset(); timer.start(); stopWatch.start(); await().atMost(Duration.ofSeconds(2)).until(() -> { backoff.delay(); - return backoff.estimatedDelay() == 0; + return backoff.estimatedDelay() == 0 || timer.isExpired(); }); stopWatch.stop(); timer.stop(); @@ -114,9 +115,22 @@ void backoffTest() throws InterruptedException { @Test void backoffIncrementalTimeTest() throws InterruptedException { + AtomicBoolean abortTrigger = new AtomicBoolean(); // delay increases in powers of 2. final long maxDelay = 1000; // not a power of 2 - final AbstractSourceTask.Backoff backoff = new AbstractSourceTask.Backoff(() -> maxDelay); + AbstractSourceTask.BackoffConfig config = new AbstractSourceTask.BackoffConfig() { + @Override + public AbstractSourceTask.SupplierOfLong getSupplierOfTimeRemaining() { + return () -> maxDelay; + } + + @Override + public AbstractSourceTask.AbortTrigger getAbortTrigger() { + return () -> abortTrigger.set(true); + } + }; + + final AbstractSourceTask.Backoff backoff = new AbstractSourceTask.Backoff(config); long expected = 2; while (backoff.estimatedDelay() < maxDelay) { assertThat(backoff.estimatedDelay()).isEqualTo(expected); @@ -124,5 +138,6 @@ void backoffIncrementalTimeTest() throws InterruptedException { expected *= 2; } assertThat(backoff.estimatedDelay()).isEqualTo(maxDelay); + assertThat(abortTrigger).isFalse();B } } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index 263b402c6..ba00fdb97 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -79,12 +79,12 @@ public String version() { } @Override - protected Iterator getIterator(SupplierOfLong timer) { // NOPMD cognatavie complexity + protected Iterator getIterator(BackoffConfig config) { // NOPMD cognatavie complexity Iterator inner = new Iterator<>() { /** * The backoff for Amazon retryable exceptions */ - final Backoff backoff = new Backoff(timer); + final Backoff backoff = new Backoff(config); @Override public boolean hasNext() { diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java index 49f422c5a..1fbf5f452 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java @@ -102,7 +102,7 @@ public long recordsProcessedForObjectKey(final Map partitionMap, } return 0L; } - + private static Set parsePartitions(final S3SourceConfig s3SourceConfig) { final String partitionString = s3SourceConfig.getTargetTopicPartitions(); return Arrays.stream(partitionString.split(",")).map(Integer::parseInt).collect(Collectors.toSet()); From cf32e7900c8ce3fd08fbc6ef66e9198884225e1a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Mon, 6 Jan 2025 13:00:32 +0000 Subject: [PATCH 12/44] updated javadoc --- .../common/source/AbstractSourceTask.java | 18 +++++++++++++++++- .../common/source/AbstractSourceTaskTest.java | 2 +- 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java index c18ef5cea..df3ccc25b 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -250,6 +250,9 @@ protected static class Timer extends StopWatch { */ private final long duration; + /** + * The flag that indicates the timer has been aborted. + */ private boolean hasAborted; /** @@ -281,6 +284,9 @@ public boolean isExpired() { return hasAborted || super.getTime() >= duration; } + /** + * Aborts the timer. Timer will report that it has expired until reset is called. + */ public void abort() { hasAborted = true; } @@ -352,7 +358,10 @@ public static class Backoff { */ private final SupplierOfLong timeRemaining; - final AbortTrigger abortTrigger; + /** + * A function to call to abort the timer. + */ + private final AbortTrigger abortTrigger; /** * The maximum number of times {@link #delay()} will be called before maxWait is reached. @@ -468,11 +477,18 @@ public interface SupplierOfLong { long get(); } + /** + * A functional interface that will abort the timer. After being called timer will indicate that it is expired, until + * it is reset. + */ @FunctionalInterface public interface AbortTrigger { void apply(); } + /** + * An interface to define the Backoff configuration. Used for convenience with Timer. + */ public interface BackoffConfig { SupplierOfLong getSupplierOfTimeRemaining(); AbortTrigger getAbortTrigger(); diff --git a/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java b/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java index 255f704d4..7753bcbbf 100644 --- a/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java +++ b/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java @@ -138,6 +138,6 @@ public AbstractSourceTask.AbortTrigger getAbortTrigger() { expected *= 2; } assertThat(backoff.estimatedDelay()).isEqualTo(maxDelay); - assertThat(abortTrigger).isFalse();B + assertThat(abortTrigger).isFalse(); } } From 4dd9ad8c9c0e6c32500759a7a6358ab93edc1693 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Mon, 6 Jan 2025 14:48:22 +0000 Subject: [PATCH 13/44] fixed PMD errors --- .../common/source/AbstractSourceTask.java | 22 +++++++--------- .../common/source/AbstractSourceTaskTest.java | 4 +-- .../source/input/AvroTransformerTest.java | 4 +-- .../source/input/JsonTransformerTest.java | 6 ++--- .../input/TransformerStreamingTest.java | 26 ++++++++++--------- .../kafka/connect/s3/source/S3SourceTask.java | 8 +++--- .../s3/source/utils/AWSV2SourceClient.java | 4 +-- .../s3/source/utils/OffsetManager.java | 8 +++--- .../s3/source/utils/S3SourceRecord.java | 4 +-- .../s3/source/utils/SourceRecordIterator.java | 18 +++++-------- .../connect/s3/source/S3SourceTaskTest.java | 20 +++++++------- .../s3/source/utils/RecordProcessorTest.java | 18 ++++--------- .../utils/SourceRecordIteratorTest.java | 8 +++--- 13 files changed, 67 insertions(+), 83 deletions(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java index df3ccc25b..6c31a8647 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -435,18 +435,16 @@ private long timeWithJitter() { * If any thread interrupts this thread. */ public void delay() throws InterruptedException { - long sleepTime = timeRemaining.get(); - if (sleepTime > 0) { - if (waitCount < maxCount) { - waitCount++; - long nextSleep = timeWithJitter(); - // don't sleep negative time. Jitter can introduce negative tme. - if (nextSleep > 0) { - if (nextSleep >= sleepTime) { - abortTrigger.apply(); - } else { - Thread.sleep(nextSleep); - } + final long sleepTime = timeRemaining.get(); + if (sleepTime > 0 && waitCount < maxCount) { + waitCount++; + final long nextSleep = timeWithJitter(); + // don't sleep negative time. Jitter can introduce negative tme. + if (nextSleep > 0) { + if (nextSleep >= sleepTime) { + abortTrigger.apply(); + } else { + Thread.sleep(nextSleep); } } } diff --git a/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java b/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java index 7753bcbbf..b09a4d960 100644 --- a/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java +++ b/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java @@ -115,10 +115,10 @@ void backoffTest() throws InterruptedException { @Test void backoffIncrementalTimeTest() throws InterruptedException { - AtomicBoolean abortTrigger = new AtomicBoolean(); + final AtomicBoolean abortTrigger = new AtomicBoolean(); // delay increases in powers of 2. final long maxDelay = 1000; // not a power of 2 - AbstractSourceTask.BackoffConfig config = new AbstractSourceTask.BackoffConfig() { + final AbstractSourceTask.BackoffConfig config = new AbstractSourceTask.BackoffConfig() { @Override public AbstractSourceTask.SupplierOfLong getSupplierOfTimeRemaining() { return () -> maxDelay; diff --git a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/AvroTransformerTest.java b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/AvroTransformerTest.java index 5c78084eb..617dd290a 100644 --- a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/AvroTransformerTest.java +++ b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/AvroTransformerTest.java @@ -90,7 +90,7 @@ void testReadAvroRecords() throws Exception { final ByteArrayOutputStream avroData = generateMockAvroData(25); final InputStream inputStream = new ByteArrayInputStream(avroData.toByteArray()); - List expected = new ArrayList<>(); + final List expected = new ArrayList<>(); for (int i = 0; i < 25; i++) { expected.add("Hello, Kafka Connect S3 Source! object " + i); } @@ -108,7 +108,7 @@ void testReadAvroRecordsSkipFew() throws Exception { final ByteArrayOutputStream avroData = generateMockAvroData(20); final InputStream inputStream = new ByteArrayInputStream(avroData.toByteArray()); - List expected = new ArrayList<>(); + final List expected = new ArrayList<>(); for (int i = 5; i < 20; i++) { expected.add("Hello, Kafka Connect S3 Source! object " + i); } diff --git a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/JsonTransformerTest.java b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/JsonTransformerTest.java index 4a9f615a3..e482fd61c 100644 --- a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/JsonTransformerTest.java +++ b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/JsonTransformerTest.java @@ -78,7 +78,7 @@ void testHandleValueDataWithValidJson() { final InputStream validJsonInputStream = new ByteArrayInputStream( getJsonRecs(100).getBytes(StandardCharsets.UTF_8)); - List expected = new ArrayList<>(); + final List expected = new ArrayList<>(); for (int i = 0; i < 100; i++) { expected.add("value" + i); } @@ -88,7 +88,7 @@ void testHandleValueDataWithValidJson() { assertThat(records).extracting(SchemaAndValue::value) .extracting(sv -> ((Map) sv).get("key")) - .containsExactlyElementsOf(expected);; + .containsExactlyElementsOf(expected); } @Test @@ -96,7 +96,7 @@ void testHandleValueDataWithValidJsonSkipFew() { final InputStream validJsonInputStream = new ByteArrayInputStream( getJsonRecs(100).getBytes(StandardCharsets.UTF_8)); - List expected = new ArrayList<>(); + final List expected = new ArrayList<>(); for (int i = 25; i < 100; i++) { expected.add("value" + i); } diff --git a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/TransformerStreamingTest.java b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/TransformerStreamingTest.java index 7d493a96b..6aa1283e9 100644 --- a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/TransformerStreamingTest.java +++ b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/TransformerStreamingTest.java @@ -62,18 +62,20 @@ void verifyExceptionDuringIOOpen(final Transformer transformer, final byte[] tes @MethodSource("testData") void verifyExceptionDuringRead(final Transformer transformer, final byte[] testData, final AbstractConfig config, final int expectedCount) throws IOException { - final InputStream inputStream = mock(InputStream.class); - when(inputStream.read()).thenThrow(new IOException("Test IOException during read")); - when(inputStream.read(any())).thenThrow(new IOException("Test IOException during read")); - when(inputStream.read(any(), anyInt(), anyInt())).thenThrow(new IOException("Test IOException during read")); - when(inputStream.readNBytes(any(), anyInt(), anyInt())) - .thenThrow(new IOException("Test IOException during read")); - when(inputStream.readNBytes(anyInt())).thenThrow(new IOException("Test IOException during read")); - when(inputStream.readAllBytes()).thenThrow(new IOException("Test IOException during read")); - final CloseTrackingStream stream = new CloseTrackingStream(inputStream); - final Stream objStream = transformer.getRecords(() -> stream, "topic", 1, config, 0); - assertThat(objStream).isEmpty(); - assertThat(stream.closeCount).isGreaterThan(0); + try (InputStream inputStream = mock(InputStream.class)) { + when(inputStream.read()).thenThrow(new IOException("Test IOException during read")); + when(inputStream.read(any())).thenThrow(new IOException("Test IOException during read")); + when(inputStream.read(any(), anyInt(), anyInt())).thenThrow(new IOException("Test IOException during read")); + when(inputStream.readNBytes(any(), anyInt(), anyInt())) + .thenThrow(new IOException("Test IOException during read")); + when(inputStream.readNBytes(anyInt())).thenThrow(new IOException("Test IOException during read")); + when(inputStream.readAllBytes()).thenThrow(new IOException("Test IOException during read")); + try (CloseTrackingStream stream = new CloseTrackingStream(inputStream)) { + final Stream objStream = transformer.getRecords(() -> stream, "topic", 1, config, 0); + assertThat(objStream).isEmpty(); + assertThat(stream.closeCount).isGreaterThan(0); + } + } } @ParameterizedTest diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index ba00fdb97..612d3a2f3 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -80,7 +80,7 @@ public String version() { @Override protected Iterator getIterator(BackoffConfig config) { // NOPMD cognatavie complexity - Iterator inner = new Iterator<>() { + final Iterator inner = new Iterator<>() { /** * The backoff for Amazon retryable exceptions */ @@ -115,7 +115,7 @@ public boolean hasNext() { @Override public SourceRecord next() { final S3SourceRecord s3SourceRecord = s3SourceRecordIterator.next(); - offsetManager.setCurrentOffsets(s3SourceRecord.getPartitionMap(), s3SourceRecord.getObjectKey(), + offsetManager.updateAndReturnCurrentOffsets(s3SourceRecord.getPartitionMap(), s3SourceRecord.getObjectKey(), s3SourceRecord.getRecordNumber()); return RecordProcessor.createSourceRecord(s3SourceRecord, s3SourceConfig, awsv2SourceClient, offsetManager); @@ -145,11 +145,11 @@ public void commit() { @Override public void commitRecord(final SourceRecord record) { if (LOGGER.isInfoEnabled()) { - final Map map = (Map) record.sourceOffset(); + //final Map map = (Map) record.sourceOffset(); // LOGGER.info("Committed individual record {} {} {} committed", map.get(BUCKET), map.get(OBJECT_KEY), // offsetManager.recordsProcessedForObjectKey((Map) record.sourcePartition(), // map.get(OBJECT_KEY).toString())); - LOGGER.info("Committed individual record {} committed", map); + LOGGER.info("Committed individual record {} committed", (Map) record.sourceOffset()); } } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java index 708ad2a74..2dd5b1f54 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java @@ -124,8 +124,7 @@ public Stream getS3ObjectStream(final String startToken) { .startAfter(optionalKey(startToken)) .build(); - final Stream s3ObjectKeyStream = Stream - .iterate(s3Client.listObjectsV2(request), Objects::nonNull, response -> { + return Stream.iterate(s3Client.listObjectsV2(request), Objects::nonNull, response -> { // This is called every time next() is called on the iterator. if (response.isTruncated()) { return s3Client.listObjectsV2(ListObjectsV2Request.builder() @@ -142,7 +141,6 @@ public Stream getS3ObjectStream(final String startToken) { .filter(filterPredicate) .filter(objectSummary -> assignObjectToTask(objectSummary.key())) .filter(objectSummary -> !failedObjectKeys.contains(objectSummary.key()))); - return s3ObjectKeyStream; } public Iterator getListOfObjectKeys(final String startToken) { diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java index 1fbf5f452..301675079 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java @@ -82,10 +82,10 @@ public long incrementAndUpdateOffsetMap(final Map partitionMap, return startOffset; } - public Map setCurrentOffsets(final Map partitionMap, final String currentObjectKey, - final long offset) { - Map offsetMap = offsets.compute(partitionMap, (k, v) -> { - Map map = v == null ? new Hashtable<>() : v; + public Map updateAndReturnCurrentOffsets(final Map partitionMap, final String currentObjectKey, + final long offset) { + final Map offsetMap = offsets.compute(partitionMap, (k, v) -> { + final Map map = v == null ? new Hashtable<>() : v; map.put(getObjectMapKey(currentObjectKey), offset); return map; }); diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java index a997cd6a0..f3a141aff 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java @@ -74,8 +74,8 @@ public SchemaAndValue getValue() { return new SchemaAndValue(valueData.schema(), valueData.value()); } - public SourceRecord getSourceRecord(OffsetManager offsetManager) { - Map offsetMap = offsetManager.setCurrentOffsets(getPartitionMap(), getObjectKey(), + public SourceRecord getSourceRecord(final OffsetManager offsetManager) { + final Map offsetMap = offsetManager.updateAndReturnCurrentOffsets(getPartitionMap(), getObjectKey(), getRecordNumber()); return new SourceRecord(getPartitionMap(), offsetMap, topic, partition(), keyData.schema(), keyData.value(), valueData.schema(), valueData.value()); diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java index 7e3942e59..a2816611a 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java @@ -16,7 +16,6 @@ package io.aiven.kafka.connect.s3.source.utils; -import java.util.Collections; import java.util.Iterator; import java.util.Map; import java.util.function.Function; @@ -33,8 +32,6 @@ import org.apache.commons.collections4.IteratorUtils; import org.apache.commons.collections4.iterators.LazyIteratorChain; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.s3.model.S3Object; /** @@ -42,8 +39,6 @@ * Parquet). */ public final class SourceRecordIterator extends LazyIteratorChain implements Iterator { - - private static final Logger LOGGER = LoggerFactory.getLogger(SourceRecordIterator.class); public static final String PATTERN_TOPIC_KEY = "topicName"; public static final String PATTERN_PARTITION_KEY = "partitionId"; @@ -51,8 +46,6 @@ public final class SourceRecordIterator extends LazyIteratorChain\\d{5})-" + "(?[a-zA-Z0-9]+)" + "\\.(?[^.]+)$"); // topic-00001.txt public static final long BYTES_TRANSFORMATION_NUM_OF_RECS = 1L; - private Iterator recordIterator = Collections.emptyIterator(); - private final OffsetManager offsetManager; private final S3SourceConfig s3SourceConfig; @@ -83,6 +76,7 @@ public final class SourceRecordIterator extends LazyIteratorChain convert(final S3Object s3Object) { - Map partitionMap = ConnectUtils.getPartitionMap(topic, partitionId, bucketName); - long recordCount = offsetManager.recordsProcessedForObjectKey(partitionMap, s3Object.key()); + final Map partitionMap = ConnectUtils.getPartitionMap(topic, partitionId, bucketName); + final long recordCount = offsetManager.recordsProcessedForObjectKey(partitionMap, s3Object.key()); // Optimizing without reading stream again. if (transformer instanceof ByteArrayTransformer && recordCount > 0) { return Stream.empty(); } - SchemaAndValue keyData = transformer.getKeyData(s3Object.key(), topic, s3SourceConfig); + final SchemaAndValue keyData = transformer.getKeyData(s3Object.key(), topic, s3SourceConfig); return transformer .getRecords(sourceClient.getObject(s3Object.key()), topic, partitionId, s3SourceConfig, recordCount) @@ -129,7 +123,7 @@ class Mapper implements Function { private final String objectKey; - public Mapper(Map partitionMap, long recordCount, SchemaAndValue keyData, String objectKey) { + public Mapper(final Map partitionMap, final long recordCount, final SchemaAndValue keyData, final String objectKey) { this.partitionMap = partitionMap; this.recordCount = recordCount; this.keyData = keyData; @@ -137,7 +131,7 @@ public Mapper(Map partitionMap, long recordCount, SchemaAndValue } @Override - public S3SourceRecord apply(SchemaAndValue value) { + public S3SourceRecord apply(final SchemaAndValue value) { recordCount++; return new S3SourceRecord(partitionMap, recordCount, topic, partitionId, objectKey, keyData, value); } diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java index d809c3559..08846b95f 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java @@ -201,7 +201,7 @@ void testPollWithNoDataReturned() { private void assertEquals(final S3SourceRecord s3Record, final SourceRecord sourceRecord) { assertThat(sourceRecord).isNotNull(); assertThat(sourceRecord.sourcePartition()).isEqualTo(s3Record.getPartitionMap()); - Map map = (Map) sourceRecord.sourceOffset(); + final Map map = (Map) sourceRecord.sourceOffset(); assertThat(map.get(OffsetManager.getObjectMapKey(s3Record.getObjectKey()))) .isEqualTo(s3Record.getRecordNumber()); @@ -227,7 +227,7 @@ void testPollsWithRecords() { assertThat(stopWatch.getTime()).isLessThan(AbstractSourceTask.MAX_POLL_TIME.toMillis()); } - private List createS3SourceRecords(int count) { + private List createS3SourceRecords(final int count) { final List lst = new ArrayList<>(); if (count > 0) { lst.add(createS3SourceRecord(TOPIC, PARTITION, TEST_BUCKET, OBJECT_KEY, @@ -243,11 +243,11 @@ private List createS3SourceRecords(int count) { @Test void testPollWithInterruptedIterator() { - List lst = createS3SourceRecords(3); + final List lst = createS3SourceRecords(3); - Iterator inner1 = lst.subList(0, 2).iterator(); - Iterator inner2 = lst.subList(2, 3).iterator(); - Iterator sourceRecordIterator = new Iterator<>() { + final Iterator inner1 = lst.subList(0, 2).iterator(); + final Iterator inner2 = lst.subList(2, 3).iterator(); + final Iterator sourceRecordIterator = new Iterator<>() { Iterator inner = inner1; @Override public boolean hasNext() { @@ -260,9 +260,9 @@ public boolean hasNext() { @Override public S3SourceRecord next() { - S3SourceRecord result = inner.next(); + final S3SourceRecord result = inner.next(); if (!inner.hasNext()) { - inner = null; + inner = null; //NOPMD null assignment } return result; } @@ -288,9 +288,9 @@ public S3SourceRecord next() { @Test void testPollWithSlowProducer() { - List lst = createS3SourceRecords(3); + final List lst = createS3SourceRecords(3); - Iterator sourceRecordIterator = new Iterator<>() { + final Iterator sourceRecordIterator = new Iterator<>() { Iterator inner = lst.iterator(); @Override public boolean hasNext() { diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java index 59ca6fd41..d62508ada 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java @@ -25,7 +25,6 @@ import static org.mockito.Mockito.when; import static org.mockito.internal.verification.VerificationModeFactory.times; -import java.util.Iterator; import java.util.function.Supplier; import org.apache.kafka.connect.errors.ConnectException; @@ -37,7 +36,6 @@ import io.aiven.kafka.connect.common.source.input.Transformer; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; @@ -62,21 +60,15 @@ class RecordProcessorTest { private static final Supplier TRUE = () -> true; private static final Supplier FALSE = () -> false; - private Iterator sourceRecordIterator; - - @BeforeEach - void setUp() { - sourceRecordIterator = mock(Iterator.class); - } @Test - void testCreateSourceRecord() throws ConnectException { + void testCreateSourceRecord() { final SourceRecord mockSourceRecord = mock(SourceRecord.class); final S3SourceRecord mockRecord = mock(S3SourceRecord.class); when(mockRecord.getSourceRecord(any())).thenReturn(mockSourceRecord); - SourceRecord result = RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, + final SourceRecord result = RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, offsetManager); verify(mockRecord, times(1)).getSourceRecord(any()); @@ -85,7 +77,7 @@ void testCreateSourceRecord() throws ConnectException { } @Test - void testCreateSourceRecordWithDataError() throws ConnectException { + void testCreateSourceRecordWithDataError() { final S3SourceRecord mockRecord = mock(S3SourceRecord.class); when(mockRecord.getSourceRecord(any())).thenThrow(new DataException("Testing exception")); @@ -97,7 +89,7 @@ void testCreateSourceRecordWithDataError() throws ConnectException { offsetManager)); when(s3SourceConfig.getErrorsTolerance()).thenReturn(ErrorsTolerance.ALL); - SourceRecord result = RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, + final SourceRecord result = RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, offsetManager); assertThat(result).isNull(); } @@ -122,7 +114,7 @@ void errorToleranceOnNONE() { assertThatThrownBy( () -> RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, offsetManager)) - .isInstanceOf(org.apache.kafka.connect.errors.ConnectException.class) + .isInstanceOf(ConnectException.class) .hasMessage("Data Exception caught during S3 record to source record transformation"); } diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java index a87436ac3..d307e8407 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java @@ -85,9 +85,9 @@ void testIteratorProcessesS3Objects() throws Exception { assertThat(iterator.hasNext()).isFalse(); - S3Object obj = S3Object.builder().key(key).build(); + final S3Object obj = S3Object.builder().key(key).build(); - ByteArrayInputStream bais = new ByteArrayInputStream("Hello World".getBytes(StandardCharsets.UTF_8)); + final ByteArrayInputStream bais = new ByteArrayInputStream("Hello World".getBytes(StandardCharsets.UTF_8)); when(mockSourceApiClient.getS3ObjectStream(any())).thenReturn(Arrays.asList(obj).stream()); when(mockSourceApiClient.getObject(any())).thenReturn(() -> bais); iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, mockSourceApiClient); @@ -122,7 +122,7 @@ void testIteratorProcessesS3ObjectsForByteArrayTransformer() throws Exception { .thenReturn(BYTES_TRANSFORMATION_NUM_OF_RECS); // should skip if any records were produced by source record iterator. - Iterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, + final Iterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, mockSourceApiClient); assertThat(iterator.hasNext()).isFalse(); verify(mockSourceApiClient, never()).getObject(any()); @@ -144,7 +144,7 @@ void testIteratorProcessesS3ObjectsForByteArrayTransformer() throws Exception { when(mockTransformer.getRecords(any(), anyString(), anyInt(), any(), anyLong())) .thenReturn(Arrays.asList(SchemaAndValue.NULL).stream()); - Iterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, + final Iterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, mockSourceApiClient); assertThat(iterator.hasNext()).isTrue(); iterator.next(); From f4014780ccd896144181693d9b9bc3137c8abb35 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Mon, 6 Jan 2025 16:58:24 +0000 Subject: [PATCH 14/44] fixed testing errors --- .../common/source/AbstractSourceTask.java | 9 +-- .../input/TransformerStreamingTest.java | 3 +- .../connect/s3/source/AwsIntegrationTest.java | 48 +++++++------- .../connect/s3/source/IntegrationBase.java | 1 + .../connect/s3/source/IntegrationTest.java | 9 +-- .../kafka/connect/s3/source/S3SourceTask.java | 6 +- .../s3/source/utils/AWSV2SourceClient.java | 63 +++++++++++++++---- .../s3/source/utils/OffsetManager.java | 4 +- .../s3/source/utils/S3SourceRecord.java | 4 +- .../s3/source/utils/SourceRecordIterator.java | 35 +++++++++-- .../connect/s3/source/S3SourceTaskTest.java | 2 +- .../utils/SourceRecordIteratorTest.java | 16 ++--- 12 files changed, 134 insertions(+), 66 deletions(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java index 6c31a8647..3d187fc53 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -285,7 +285,7 @@ public boolean isExpired() { } /** - * Aborts the timer. Timer will report that it has expired until reset is called. + * Aborts the timer. Timer will report that it has expired until reset is called. */ public void abort() { hasAborted = true; @@ -322,6 +322,7 @@ public void reset() { /** * Gets a Backoff Config for this timer. + * * @return a backoff Configuration. */ public BackoffConfig getBackoffConfig() { @@ -476,8 +477,8 @@ public interface SupplierOfLong { } /** - * A functional interface that will abort the timer. After being called timer will indicate that it is expired, until - * it is reset. + * A functional interface that will abort the timer. After being called timer will indicate that it is expired, + * until it is reset. */ @FunctionalInterface public interface AbortTrigger { @@ -485,7 +486,7 @@ public interface AbortTrigger { } /** - * An interface to define the Backoff configuration. Used for convenience with Timer. + * An interface to define the Backoff configuration. Used for convenience with Timer. */ public interface BackoffConfig { SupplierOfLong getSupplierOfTimeRemaining(); diff --git a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/TransformerStreamingTest.java b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/TransformerStreamingTest.java index 6aa1283e9..73b27b01f 100644 --- a/commons/src/test/java/io/aiven/kafka/connect/common/source/input/TransformerStreamingTest.java +++ b/commons/src/test/java/io/aiven/kafka/connect/common/source/input/TransformerStreamingTest.java @@ -65,7 +65,8 @@ void verifyExceptionDuringRead(final Transformer transformer, final byte[] testD try (InputStream inputStream = mock(InputStream.class)) { when(inputStream.read()).thenThrow(new IOException("Test IOException during read")); when(inputStream.read(any())).thenThrow(new IOException("Test IOException during read")); - when(inputStream.read(any(), anyInt(), anyInt())).thenThrow(new IOException("Test IOException during read")); + when(inputStream.read(any(), anyInt(), anyInt())) + .thenThrow(new IOException("Test IOException during read")); when(inputStream.readNBytes(any(), anyInt(), anyInt())) .thenThrow(new IOException("Test IOException during read")); when(inputStream.readNBytes(anyInt())).thenThrow(new IOException("Test IOException during read")); diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java index 649ce4b0c..3534bc4e3 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java @@ -68,7 +68,7 @@ import software.amazon.awssdk.services.s3.S3Client; @Testcontainers -public class AwsIntegrationTest implements IntegrationBase { +class AwsIntegrationTest implements IntegrationBase { private static final String COMMON_PREFIX = "s3-source-connector-for-apache-kafka-AWS-test-"; @@ -156,23 +156,23 @@ void sourceRecordIteratorBytesTest(final TestInfo testInfo) { assertThat(testBucketAccessor.listObjects()).hasSize(5); - S3SourceConfig s3SourceConfig = new S3SourceConfig(configData); - SourceTaskContext context = mock(SourceTaskContext.class); - OffsetStorageReader offsetStorageReader = mock(OffsetStorageReader.class); + final S3SourceConfig s3SourceConfig = new S3SourceConfig(configData); + final SourceTaskContext context = mock(SourceTaskContext.class); + final OffsetStorageReader offsetStorageReader = mock(OffsetStorageReader.class); when(context.offsetStorageReader()).thenReturn(offsetStorageReader); when(offsetStorageReader.offsets(any())).thenReturn(new HashMap<>()); - OffsetManager offsetManager = new OffsetManager(context, s3SourceConfig); + final OffsetManager offsetManager = new OffsetManager(context, s3SourceConfig); - AWSV2SourceClient sourceClient = new AWSV2SourceClient(s3SourceConfig, new HashSet<>()); + final AWSV2SourceClient sourceClient = new AWSV2SourceClient(s3SourceConfig, new HashSet<>()); - Iterator sourceRecordIterator = new SourceRecordIterator(s3SourceConfig, offsetManager, + final Iterator sourceRecordIterator = new SourceRecordIterator(s3SourceConfig, offsetManager, TransformerFactory.getTransformer(InputFormat.BYTES), sourceClient); - HashSet seenKeys = new HashSet<>(); + final HashSet seenKeys = new HashSet<>(); while (sourceRecordIterator.hasNext()) { - S3SourceRecord s3SourceRecord = sourceRecordIterator.next(); - String key = OBJECT_KEY + SEPARATOR + s3SourceRecord.getObjectKey(); + final S3SourceRecord s3SourceRecord = sourceRecordIterator.next(); + final String key = OBJECT_KEY + SEPARATOR + s3SourceRecord.getObjectKey(); assertThat(offsetKeys).contains(key); seenKeys.add(key); } @@ -219,26 +219,26 @@ void sourceRecordIteratorAvroTest(final TestInfo testInfo) throws IOException { assertThat(testBucketAccessor.listObjects()).hasSize(5); - S3SourceConfig s3SourceConfig = new S3SourceConfig(configData); - SourceTaskContext context = mock(SourceTaskContext.class); - OffsetStorageReader offsetStorageReader = mock(OffsetStorageReader.class); + final S3SourceConfig s3SourceConfig = new S3SourceConfig(configData); + final SourceTaskContext context = mock(SourceTaskContext.class); + final OffsetStorageReader offsetStorageReader = mock(OffsetStorageReader.class); when(context.offsetStorageReader()).thenReturn(offsetStorageReader); when(offsetStorageReader.offsets(any())).thenReturn(new HashMap<>()); - OffsetManager offsetManager = new OffsetManager(context, s3SourceConfig); + final OffsetManager offsetManager = new OffsetManager(context, s3SourceConfig); - AWSV2SourceClient sourceClient = new AWSV2SourceClient(s3SourceConfig, new HashSet<>()); + final AWSV2SourceClient sourceClient = new AWSV2SourceClient(s3SourceConfig, new HashSet<>()); - Iterator sourceRecordIterator = new SourceRecordIterator(s3SourceConfig, offsetManager, + final Iterator sourceRecordIterator = new SourceRecordIterator(s3SourceConfig, offsetManager, TransformerFactory.getTransformer(InputFormat.AVRO), sourceClient); - HashSet seenKeys = new HashSet<>(); - Map> seenRecords = new HashMap<>(); + final HashSet seenKeys = new HashSet<>(); + final Map> seenRecords = new HashMap<>(); while (sourceRecordIterator.hasNext()) { - S3SourceRecord s3SourceRecord = sourceRecordIterator.next(); - String key = OBJECT_KEY + SEPARATOR + s3SourceRecord.getObjectKey(); + final S3SourceRecord s3SourceRecord = sourceRecordIterator.next(); + final String key = OBJECT_KEY + SEPARATOR + s3SourceRecord.getObjectKey(); seenRecords.compute(key, (k, v) -> { - List lst = v == null ? new ArrayList<>() : v; + final List lst = v == null ? new ArrayList<>() : v; // NOPMD new object inside loop lst.add(s3SourceRecord.getRecordNumber()); return lst; }); @@ -247,12 +247,12 @@ void sourceRecordIteratorAvroTest(final TestInfo testInfo) throws IOException { } assertThat(seenKeys).containsAll(offsetKeys); assertThat(seenRecords).hasSize(5); - List expected = new ArrayList<>(); + final List expected = new ArrayList<>(); for (long l = 0; l < numOfRecsFactor; l++) { expected.add(l + 1); } - for (String key : offsetKeys) { - List seen = seenRecords.get(key); + for (final String key : offsetKeys) { + final List seen = seenRecords.get(key); assertThat(seen).as("Count for " + key).containsExactlyInAnyOrderElementsOf(expected); } } diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java index 4101acbb4..2936e8d1e 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java @@ -72,6 +72,7 @@ import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.PutObjectRequest; +@SuppressWarnings("PMD.ExcessiveImports") public interface IntegrationBase { String PLUGINS_S3_SOURCE_CONNECTOR_FOR_APACHE_KAFKA = "plugins/s3-source-connector-for-apache-kafka/"; String S3_SOURCE_CONNECTOR_FOR_APACHE_KAFKA_TEST = "s3-source-connector-for-apache-kafka-test-"; diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java index 519c917a7..083d8627e 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java @@ -96,10 +96,12 @@ final class IntegrationTest implements IntegrationBase { private static S3Client s3Client; + @Override public S3Client getS3Client() { return s3Client; } + @Override public String getS3Prefix() { return s3Prefix; } @@ -182,7 +184,7 @@ void bytesTest(final TestInfo testInfo) { final Map expectedOffsetRecords = offsetKeys.subList(0, offsetKeys.size() - 1) .stream() .collect(Collectors.toMap(Function.identity(), s -> 1)); - // verifyOffsetPositions(expectedOffsetRecords, connectRunner.getBootstrapServers()); + verifyOffsetPositions(expectedOffsetRecords, connectRunner.getBootstrapServers()); } @Test @@ -238,9 +240,8 @@ void avroTest(final TestInfo testInfo) throws IOException { entry(4 * numOfRecsFactor, "Hello, Kafka Connect S3 Source! object " + (4 * numOfRecsFactor)), entry(5 * numOfRecsFactor, "Hello, Kafka Connect S3 Source! object " + (5 * numOfRecsFactor))); - // verifyOffsetPositions(offsetKeys.stream().collect(Collectors.toMap(Function.identity(), s -> - // numOfRecsFactor)), - // connectRunner.getBootstrapServers()); + verifyOffsetPositions(offsetKeys.stream().collect(Collectors.toMap(Function.identity(), s -> numOfRecsFactor)), + connectRunner.getBootstrapServers()); } @Test diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index 612d3a2f3..83b30235f 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -115,8 +115,8 @@ public boolean hasNext() { @Override public SourceRecord next() { final S3SourceRecord s3SourceRecord = s3SourceRecordIterator.next(); - offsetManager.updateAndReturnCurrentOffsets(s3SourceRecord.getPartitionMap(), s3SourceRecord.getObjectKey(), - s3SourceRecord.getRecordNumber()); + offsetManager.updateAndReturnCurrentOffsets(s3SourceRecord.getPartitionMap(), + s3SourceRecord.getObjectKey(), s3SourceRecord.getRecordNumber()); return RecordProcessor.createSourceRecord(s3SourceRecord, s3SourceConfig, awsv2SourceClient, offsetManager); } @@ -145,7 +145,7 @@ public void commit() { @Override public void commitRecord(final SourceRecord record) { if (LOGGER.isInfoEnabled()) { - //final Map map = (Map) record.sourceOffset(); + // final Map map = (Map) record.sourceOffset(); // LOGGER.info("Committed individual record {} {} {} committed", map.get(BUCKET), map.get(OBJECT_KEY), // offsetManager.recordsProcessedForObjectKey((Map) record.sourcePartition(), // map.get(OBJECT_KEY).toString())); diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java index 2dd5b1f54..3cd5fc9fb 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java @@ -116,7 +116,7 @@ public AWSV2SourceClient(final S3SourceConfig s3SourceConfig, final Set this.taskId = taskId; } - public Stream getS3ObjectStream(final String startToken) { + private Stream getS3ObjectStream(final String startToken) { final ListObjectsV2Request request = ListObjectsV2Request.builder() .bucket(bucketName) .maxKeys(s3SourceConfig.getS3ConfigFragment().getFetchPageSize() * PAGE_SIZE_FACTOR) @@ -125,17 +125,17 @@ public Stream getS3ObjectStream(final String startToken) { .build(); return Stream.iterate(s3Client.listObjectsV2(request), Objects::nonNull, response -> { - // This is called every time next() is called on the iterator. - if (response.isTruncated()) { - return s3Client.listObjectsV2(ListObjectsV2Request.builder() - .maxKeys(s3SourceConfig.getS3ConfigFragment().getFetchPageSize() * PAGE_SIZE_FACTOR) - .continuationToken(response.nextContinuationToken()) - .build()); - } else { - return null; - } - - }) + // This is called every time next() is called on the iterator. + if (response.isTruncated()) { + return s3Client.listObjectsV2(ListObjectsV2Request.builder() + .maxKeys(s3SourceConfig.getS3ConfigFragment().getFetchPageSize() * PAGE_SIZE_FACTOR) + .continuationToken(response.nextContinuationToken()) + .build()); + } else { + return null; + } + + }) .flatMap(response -> response.contents() .stream() .filter(filterPredicate) @@ -143,6 +143,10 @@ public Stream getS3ObjectStream(final String startToken) { .filter(objectSummary -> !failedObjectKeys.contains(objectSummary.key()))); } + public Iterator getS3ObjectIterator(final String startToken) { + return new S3ObjectIterator(startToken); + } + public Iterator getListOfObjectKeys(final String startToken) { return getS3ObjectStream(startToken).map(S3Object::key).iterator(); } @@ -177,4 +181,39 @@ public void shutdown() { s3Client.close(); } + /** + * An iterator that reads from + */ + public class S3ObjectIterator implements Iterator { + + /** The current iterator. */ + private Iterator inner; + /** The last object key that was seen. */ + private String lastSeenObjectKey; + + private S3ObjectIterator(final String initialKey) { + lastSeenObjectKey = initialKey; + inner = getS3ObjectStream(lastSeenObjectKey).iterator(); + } + @Override + public boolean hasNext() { + if (!inner.hasNext()) { + inner = getS3ObjectStream(lastSeenObjectKey).iterator(); + } + return inner.hasNext(); + } + + @Override + public S3Object next() { + final S3Object result = inner.next(); + lastSeenObjectKey = result.key(); + return result; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + } + } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java index 301675079..95bc4053d 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java @@ -82,8 +82,8 @@ public long incrementAndUpdateOffsetMap(final Map partitionMap, return startOffset; } - public Map updateAndReturnCurrentOffsets(final Map partitionMap, final String currentObjectKey, - final long offset) { + public Map updateAndReturnCurrentOffsets(final Map partitionMap, + final String currentObjectKey, final long offset) { final Map offsetMap = offsets.compute(partitionMap, (k, v) -> { final Map map = v == null ? new Hashtable<>() : v; map.put(getObjectMapKey(currentObjectKey), offset); diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java index f3a141aff..05ca02ba4 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java @@ -75,8 +75,8 @@ public SchemaAndValue getValue() { } public SourceRecord getSourceRecord(final OffsetManager offsetManager) { - final Map offsetMap = offsetManager.updateAndReturnCurrentOffsets(getPartitionMap(), getObjectKey(), - getRecordNumber()); + final Map offsetMap = offsetManager.updateAndReturnCurrentOffsets(getPartitionMap(), + getObjectKey(), getRecordNumber()); return new SourceRecord(getPartitionMap(), offsetMap, topic, partition(), keyData.schema(), keyData.value(), valueData.schema(), valueData.value()); } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java index a2816611a..ff9a9ef2e 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java @@ -84,13 +84,17 @@ public SourceRecordIterator(final S3SourceConfig s3SourceConfig, final OffsetMan this.transformer = transformer; this.sourceClient = sourceClient; - inner = IteratorUtils.filteredIterator(sourceClient.getS3ObjectStream(null).iterator(), + inner = IteratorUtils.filteredIterator(sourceClient.getS3ObjectIterator(null), s3Object -> this.fileNamePredicate.test(s3Object)); // call filter out bad file names and extract - // topic/partition + // topic/partition } @Override protected Iterator nextIterator(final int count) { + /* + * This code has to get the next iterator from the inner iterator if it exists, otherwise we need to restart + * with the last seen key. + */ return inner.hasNext() ? convert(inner.next()).iterator() : null; } @@ -99,6 +103,13 @@ public void remove() { throw new UnsupportedOperationException("This iterator is unmodifiable"); } + /** + * Converts the S3Object into stream of S3SourceRecords. + * + * @param s3Object + * the S3Object to read data from. + * @return a stream of S3SourceRecords created from the input stream of the S3Object. + */ private Stream convert(final S3Object s3Object) { final Map partitionMap = ConnectUtils.getPartitionMap(topic, partitionId, bucketName); @@ -116,14 +127,29 @@ private Stream convert(final S3Object s3Object) { .map(new Mapper(partitionMap, recordCount, keyData, s3Object.key())); } + /** + * maps the data from the @{link Transformer} stream to an S3SourceRecord given all the additional data required. + */ class Mapper implements Function { + /** + * The partition map + */ private final Map partitionMap; + /** + * The record number for the record being created. + */ private long recordCount; + /** + * The schema and value for the key + */ private final SchemaAndValue keyData; - + /** + * The object key from S3 + */ private final String objectKey; - public Mapper(final Map partitionMap, final long recordCount, final SchemaAndValue keyData, final String objectKey) { + public Mapper(final Map partitionMap, final long recordCount, final SchemaAndValue keyData, + final String objectKey) { this.partitionMap = partitionMap; this.recordCount = recordCount; this.keyData = keyData; @@ -136,5 +162,4 @@ public S3SourceRecord apply(final SchemaAndValue value) { return new S3SourceRecord(partitionMap, recordCount, topic, partitionId, objectKey, keyData, value); } } - } diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java index 08846b95f..383f8b1a9 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java @@ -262,7 +262,7 @@ public boolean hasNext() { public S3SourceRecord next() { final S3SourceRecord result = inner.next(); if (!inner.hasNext()) { - inner = null; //NOPMD null assignment + inner = null; // NOPMD null assignment } return result; } diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java index d307e8407..af9b679fa 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java @@ -79,7 +79,7 @@ void testIteratorProcessesS3Objects() throws Exception { when(mockOffsetManager.getOffsets()).thenReturn(Collections.emptyMap()); - when(mockSourceApiClient.getListOfObjectKeys(any())).thenReturn(Collections.emptyIterator()); + when(mockSourceApiClient.getS3ObjectIterator(any())).thenReturn(Collections.emptyIterator()); Iterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, mockSourceApiClient); @@ -88,7 +88,7 @@ void testIteratorProcessesS3Objects() throws Exception { final S3Object obj = S3Object.builder().key(key).build(); final ByteArrayInputStream bais = new ByteArrayInputStream("Hello World".getBytes(StandardCharsets.UTF_8)); - when(mockSourceApiClient.getS3ObjectStream(any())).thenReturn(Arrays.asList(obj).stream()); + when(mockSourceApiClient.getS3ObjectIterator(any())).thenReturn(Arrays.asList(obj).iterator()); when(mockSourceApiClient.getObject(any())).thenReturn(() -> bais); iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, mockSourceApiClient); @@ -108,7 +108,7 @@ void testIteratorProcessesS3ObjectsForByteArrayTransformer() throws Exception { try (InputStream inputStream = new ByteArrayInputStream("Hello World".getBytes(StandardCharsets.UTF_8))) { when(mockSourceApiClient.getObject(key)).thenReturn(() -> inputStream); - when(mockSourceApiClient.getS3ObjectStream(any())).thenReturn(Arrays.asList(s3Object).stream()); + when(mockSourceApiClient.getS3ObjectIterator(any())).thenReturn(Arrays.asList(s3Object).iterator()); mockTransformer = mock(ByteArrayTransformer.class); when(mockTransformer.getRecords(any(), anyString(), anyInt(), any(), anyLong())) @@ -122,8 +122,8 @@ void testIteratorProcessesS3ObjectsForByteArrayTransformer() throws Exception { .thenReturn(BYTES_TRANSFORMATION_NUM_OF_RECS); // should skip if any records were produced by source record iterator. - final Iterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, - mockSourceApiClient); + final Iterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, + mockTransformer, mockSourceApiClient); assertThat(iterator.hasNext()).isFalse(); verify(mockSourceApiClient, never()).getObject(any()); verify(mockTransformer, never()).getRecords(any(), anyString(), anyInt(), any(), anyLong()); @@ -132,7 +132,7 @@ void testIteratorProcessesS3ObjectsForByteArrayTransformer() throws Exception { // With AvroTransformer try (InputStream inputStream = new ByteArrayInputStream("Hello World".getBytes(StandardCharsets.UTF_8))) { when(mockSourceApiClient.getObject(key)).thenReturn(() -> inputStream); - when(mockSourceApiClient.getS3ObjectStream(any())).thenReturn(Arrays.asList(s3Object).stream()); + when(mockSourceApiClient.getS3ObjectIterator(any())).thenReturn(Arrays.asList(s3Object).iterator()); mockTransformer = mock(AvroTransformer.class); when(mockSourceApiClient.getListOfObjectKeys(any())) .thenReturn(Collections.singletonList(key).listIterator()); @@ -144,8 +144,8 @@ void testIteratorProcessesS3ObjectsForByteArrayTransformer() throws Exception { when(mockTransformer.getRecords(any(), anyString(), anyInt(), any(), anyLong())) .thenReturn(Arrays.asList(SchemaAndValue.NULL).stream()); - final Iterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, - mockSourceApiClient); + final Iterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, + mockTransformer, mockSourceApiClient); assertThat(iterator.hasNext()).isTrue(); iterator.next(); From 1d73e7fb570b5de6bbf62f90c37f9e1e5157ce19 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Mon, 6 Jan 2025 17:10:57 +0000 Subject: [PATCH 15/44] added javadoc --- .../s3/source/utils/AWSV2SourceClient.java | 36 +++++++++++++------ 1 file changed, 26 insertions(+), 10 deletions(-) diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java index 3cd5fc9fb..ed460a500 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java @@ -28,7 +28,7 @@ import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; import org.apache.commons.io.function.IOSupplier; -import org.codehaus.plexus.util.StringUtils; +import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.core.ResponseBytes; @@ -116,12 +116,19 @@ public AWSV2SourceClient(final S3SourceConfig s3SourceConfig, final Set this.taskId = taskId; } + /** + * Creates a stream from which we will create an iterator. + * + * @param startToken + * the beginning key, or {@code null} to start at the beginning. + * @return a Stream of S3Objects for the current state of the S3 storage. + */ private Stream getS3ObjectStream(final String startToken) { final ListObjectsV2Request request = ListObjectsV2Request.builder() .bucket(bucketName) .maxKeys(s3SourceConfig.getS3ConfigFragment().getFetchPageSize() * PAGE_SIZE_FACTOR) - .prefix(optionalKey(s3SourceConfig.getAwsS3Prefix())) - .startAfter(optionalKey(startToken)) + .prefix(StringUtils.defaultIfBlank(s3SourceConfig.getAwsS3Prefix(), null)) + .startAfter(StringUtils.defaultIfBlank(startToken, null)) .build(); return Stream.iterate(s3Client.listObjectsV2(request), Objects::nonNull, response -> { @@ -143,21 +150,30 @@ private Stream getS3ObjectStream(final String startToken) { .filter(objectSummary -> !failedObjectKeys.contains(objectSummary.key()))); } + /** + * Creates an S3Object iterator that will return the objects from the current objects in S3 storage and then try to + * refresh on every {@code hasNext()} that returns false. This should pick up new files as they are dropped on the + * file system. + * + * @param startToken + * the beginning key, or {@code null} to start at the beginning. + * @return an Iterator on the S3Objects. + */ public Iterator getS3ObjectIterator(final String startToken) { return new S3ObjectIterator(startToken); } + /** + * Gets an iterator of keys from the current S3 storage. + * + * @param startToken + * the beginning key, or {@code null} to start at the beginning. + * @return an Iterator on the keys of the current S3Objects. + */ public Iterator getListOfObjectKeys(final String startToken) { return getS3ObjectStream(startToken).map(S3Object::key).iterator(); } - private String optionalKey(final String key) { - if (StringUtils.isNotBlank(key)) { - return key; - } - return null; - } - public IOSupplier getObject(final String objectKey) { final GetObjectRequest getObjectRequest = GetObjectRequest.builder().bucket(bucketName).key(objectKey).build(); final ResponseBytes s3ObjectResponse = s3Client.getObjectAsBytes(getObjectRequest); From 2c0426acf81e36b819ce2ed7e867dd39c61a9d71 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Tue, 7 Jan 2025 08:23:29 +0000 Subject: [PATCH 16/44] fixed backoff test issue --- .../connect/common/source/AbstractSourceTaskTest.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java b/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java index b09a4d960..9b3a581eb 100644 --- a/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java +++ b/commons/src/test/java/io/aiven/kafka/connect/common/source/AbstractSourceTaskTest.java @@ -107,9 +107,11 @@ void backoffTest() throws InterruptedException { stopWatch.stop(); timer.stop(); final int step = i; - assertThat(stopWatch.getTime()).as(() -> String.format("Result with timer running at step %s", step)) - .isBetween(Duration.ofSeconds(1).toMillis() - backoff.getMaxJitter(), - Duration.ofSeconds(1).toMillis() + backoff.getMaxJitter()); + if (!timer.isExpired()) { + assertThat(stopWatch.getTime()).as(() -> String.format("Result with timer running at step %s", step)) + .isBetween(Duration.ofSeconds(1).toMillis() - backoff.getMaxJitter(), + Duration.ofSeconds(1).toMillis() + backoff.getMaxJitter()); + } } } From 332c3f54f6ebc8f8193502e36be0ca0575702181 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Tue, 7 Jan 2025 08:39:28 +0000 Subject: [PATCH 17/44] fixed testPollWithSlowProducer test issue --- .../connect/s3/source/S3SourceTaskTest.java | 27 +++++++++++++------ 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java index 383f8b1a9..742bb5029 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java @@ -308,18 +308,29 @@ public S3SourceRecord next() { } }; + final List results = new ArrayList<>(); + // since the polling is returning data at or near the time limit the 3 record may be returned as follows + // Record Poll1 Poll2 Poll3 Poll4 + // 1 x x + // 2 x x + // 3 x x + // final S3SourceTask s3SourceTask = new TestingS3SourceTask(sourceRecordIterator); startSourceTask(s3SourceTask); final StopWatch stopWatch = new StopWatch(); stopWatch.start(); - List results = s3SourceTask.poll(); - assertThat(results).hasSize(1); - results = s3SourceTask.poll(); - assertThat(results).hasSize(1); - results = s3SourceTask.poll(); - assertThat(results).hasSize(1); - results = s3SourceTask.poll(); - assertThat(results).isEmpty(); + // poll 1 + results.addAll(s3SourceTask.poll()); + assertThat(results).hasSizeLessThanOrEqualTo(1); + // poll 2 + results.addAll(s3SourceTask.poll()); + assertThat(results).hasSizeLessThanOrEqualTo(2); + // poll 3 + results.addAll(s3SourceTask.poll()); + assertThat(results).hasSizeLessThanOrEqualTo(3); + // poll 4 + results.addAll(s3SourceTask.poll()); + assertThat(results).hasSize(3); } @Test From 06a12a36aa1c8e3abb4d8a8892db8335839db394 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Tue, 7 Jan 2025 10:43:03 +0000 Subject: [PATCH 18/44] added test for AWSV2SourceClient rehydration --- .../common/source/AbstractSourceTask.java | 6 +-- .../connect/s3/source/AwsIntegrationTest.java | 51 +++++++++++++++++++ .../connect/s3/source/IntegrationBase.java | 31 +++++++++-- .../kafka/connect/s3/source/S3SourceTask.java | 5 -- .../s3/source/utils/SourceRecordIterator.java | 27 ++++++---- .../s3/source/utils/RecordProcessorTest.java | 10 ++-- 6 files changed, 104 insertions(+), 26 deletions(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java index 3d187fc53..e2050305a 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -1,5 +1,5 @@ /* - * Copyright 2024 Aiven Oy + * Copyright 2024-2025 Aiven Oy * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -179,8 +179,8 @@ public final List poll() { timer.start(); try { final List result = populateList(); - if (logger.isInfoEnabled()) { // TODO reset this to debug - logger.info("Poll() returning {} SourceRecords.", result == null ? null : result.size()); + if (logger.isDebugEnabled()) { + logger.debug("Poll() returning {} SourceRecords.", result == null ? null : result.size()); } return result; } finally { diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java index 3534bc4e3..42d10aad7 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java @@ -66,6 +66,7 @@ import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.S3Object; @Testcontainers class AwsIntegrationTest implements IntegrationBase { @@ -256,4 +257,54 @@ void sourceRecordIteratorAvroTest(final TestInfo testInfo) throws IOException { assertThat(seen).as("Count for " + key).containsExactlyInAnyOrderElementsOf(expected); } } + + @Test + void verifyIteratorRehydration(final TestInfo testInfo) { + // create 2 files. + final var topicName = IntegrationBase.topicName(testInfo); + final Map configData = getConfig(topicName, 1); + + configData.put(INPUT_FORMAT_KEY, InputFormat.BYTES.getValue()); + + final String testData1 = "Hello, Kafka Connect S3 Source! object 1"; + final String testData2 = "Hello, Kafka Connect S3 Source! object 2"; + final String testData3 = "Hello, Kafka Connect S3 Source! object 3"; + + final List expectedKeys = new ArrayList<>(); + + final List actualKeys = new ArrayList<>(); + + // write 2 objects to s3 + expectedKeys.add(writeToS3(topicName, testData1.getBytes(StandardCharsets.UTF_8), "00000") + .substring((OBJECT_KEY + SEPARATOR).length())); + expectedKeys.add(writeToS3(topicName, testData2.getBytes(StandardCharsets.UTF_8), "00000") + .substring((OBJECT_KEY + SEPARATOR).length())); + + assertThat(testBucketAccessor.listObjects()).hasSize(2); + + final S3SourceConfig s3SourceConfig = new S3SourceConfig(configData); + final AWSV2SourceClient sourceClient = new AWSV2SourceClient(s3SourceConfig, new HashSet<>()); + final Iterator iter = sourceClient.getS3ObjectIterator(null); + + assertThat(iter).hasNext(); + S3Object object = iter.next(); + actualKeys.add(object.key()); + assertThat(iter).hasNext(); + object = iter.next(); + actualKeys.add(object.key()); + assertThat(iter).isExhausted(); + assertThat(actualKeys).containsAll(expectedKeys); + + // write 3rd object to s3 + expectedKeys.add(writeToS3(topicName, testData3.getBytes(StandardCharsets.UTF_8), "00000") + .substring((OBJECT_KEY + SEPARATOR).length())); + assertThat(testBucketAccessor.listObjects()).hasSize(3); + + assertThat(iter).hasNext(); + object = iter.next(); + actualKeys.add(object.key()); + assertThat(iter).isExhausted(); + assertThat(actualKeys).containsAll(expectedKeys); + + } } diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java index 2936e8d1e..a8b91a197 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java @@ -104,14 +104,39 @@ static byte[] generateNextAvroMessagesStartingFromId(final int messageId, final String getS3Prefix(); - default String writeToS3(final String topicName, final byte[] testDataBytes, final String partitionId) { - final String objectKey = org.apache.commons.lang3.StringUtils.defaultIfBlank(getS3Prefix(), "") + topicName - + "-" + partitionId + "-" + System.currentTimeMillis() + ".txt"; + /** + * Write file to s3 with the specified key and data. + * + * @param objectKey + * the key. + * @param testDataBytes + * the data. + */ + default void writeToS3WithKey(final String objectKey, final byte[] testDataBytes) { final PutObjectRequest request = PutObjectRequest.builder() .bucket(IntegrationTest.TEST_BUCKET_NAME) .key(objectKey) .build(); getS3Client().putObject(request, RequestBody.fromBytes(testDataBytes)); + + } + + /** + * Writes to S3 using a key of the form {@code [prefix]topicName-partitionId-systemTime.txt}. + * + * @param topicName + * the topic name to use + * @param testDataBytes + * the data. + * @param partitionId + * the partition id. + * @return the key prefixed by {@link S3SourceTask#OBJECT_KEY} and + * {@link io.aiven.kafka.connect.s3.source.utils.OffsetManager#SEPARATOR} + */ + default String writeToS3(final String topicName, final byte[] testDataBytes, final String partitionId) { + final String objectKey = org.apache.commons.lang3.StringUtils.defaultIfBlank(getS3Prefix(), "") + topicName + + "-" + partitionId + "-" + System.currentTimeMillis() + ".txt"; + writeToS3WithKey(objectKey, testDataBytes); return OBJECT_KEY + SEPARATOR + objectKey; } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index 83b30235f..48f9a926e 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -137,7 +137,6 @@ protected SourceCommonConfig configure(final Map props) { } @Override - public void commit() { LOGGER.info("Committed all records through last poll()"); } @@ -145,10 +144,6 @@ public void commit() { @Override public void commitRecord(final SourceRecord record) { if (LOGGER.isInfoEnabled()) { - // final Map map = (Map) record.sourceOffset(); - // LOGGER.info("Committed individual record {} {} {} committed", map.get(BUCKET), map.get(OBJECT_KEY), - // offsetManager.recordsProcessedForObjectKey((Map) record.sourcePartition(), - // map.get(OBJECT_KEY).toString())); LOGGER.info("Committed individual record {} committed", (Map) record.sourceOffset()); } } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java index ff9a9ef2e..319b6fb0e 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java @@ -16,6 +16,7 @@ package io.aiven.kafka.connect.s3.source.utils; +import java.util.Collections; import java.util.Iterator; import java.util.Map; import java.util.function.Function; @@ -31,14 +32,13 @@ import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; import org.apache.commons.collections4.IteratorUtils; -import org.apache.commons.collections4.iterators.LazyIteratorChain; import software.amazon.awssdk.services.s3.model.S3Object; /** * Iterator that processes S3 files and creates Kafka source records. Supports different output formats (Avro, JSON, * Parquet). */ -public final class SourceRecordIterator extends LazyIteratorChain implements Iterator { +public final class SourceRecordIterator implements Iterator { public static final String PATTERN_TOPIC_KEY = "topicName"; public static final String PATTERN_PARTITION_KEY = "partitionId"; @@ -61,6 +61,8 @@ public final class SourceRecordIterator extends LazyIteratorChain inner; + private Iterator outer; + private final Predicate fileNamePredicate = s3Object -> { final Matcher fileMatcher = FILE_DEFAULT_PATTERN.matcher(s3Object.key()); @@ -84,18 +86,23 @@ public SourceRecordIterator(final S3SourceConfig s3SourceConfig, final OffsetMan this.transformer = transformer; this.sourceClient = sourceClient; + // call filters out bad file names and extracts topic/partition inner = IteratorUtils.filteredIterator(sourceClient.getS3ObjectIterator(null), - s3Object -> this.fileNamePredicate.test(s3Object)); // call filter out bad file names and extract - // topic/partition + s3Object -> this.fileNamePredicate.test(s3Object)); + outer = Collections.emptyIterator(); } @Override - protected Iterator nextIterator(final int count) { - /* - * This code has to get the next iterator from the inner iterator if it exists, otherwise we need to restart - * with the last seen key. - */ - return inner.hasNext() ? convert(inner.next()).iterator() : null; + public boolean hasNext() { + while (!outer.hasNext() && inner.hasNext()) { + outer = convert(inner.next()).iterator(); + } + return outer.hasNext(); + } + + @Override + public S3SourceRecord next() { + return outer.next(); } @Override diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java index d62508ada..cc9db65cd 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java @@ -66,7 +66,7 @@ void testCreateSourceRecord() { final SourceRecord mockSourceRecord = mock(SourceRecord.class); final S3SourceRecord mockRecord = mock(S3SourceRecord.class); - when(mockRecord.getSourceRecord(any())).thenReturn(mockSourceRecord); + when(mockRecord.getSourceRecord(any(OffsetManager.class))).thenReturn(mockSourceRecord); final SourceRecord result = RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, offsetManager); @@ -80,7 +80,7 @@ void testCreateSourceRecord() { void testCreateSourceRecordWithDataError() { final S3SourceRecord mockRecord = mock(S3SourceRecord.class); - when(mockRecord.getSourceRecord(any())).thenThrow(new DataException("Testing exception")); + when(mockRecord.getSourceRecord(any(OffsetManager.class))).thenThrow(new DataException("Testing exception")); when(s3SourceConfig.getErrorsTolerance()).thenReturn(ErrorsTolerance.NONE); @@ -97,7 +97,7 @@ void testCreateSourceRecordWithDataError() { @Test void testCreateSourceRecords() { final S3SourceRecord mockRecord = mock(S3SourceRecord.class); - when(mockRecord.getSourceRecord(any())).thenReturn(mock(SourceRecord.class)); + when(mockRecord.getSourceRecord(any(OffsetManager.class))).thenReturn(mock(SourceRecord.class)); final SourceRecord sourceRecords = RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, offsetManager); @@ -108,7 +108,7 @@ void testCreateSourceRecords() { @Test void errorToleranceOnNONE() { final S3SourceRecord mockRecord = mock(S3SourceRecord.class); - when(mockRecord.getSourceRecord(any())).thenThrow(new DataException("generic issue")); + when(mockRecord.getSourceRecord(any(OffsetManager.class))).thenThrow(new DataException("generic issue")); when(s3SourceConfig.getErrorsTolerance()).thenReturn(ErrorsTolerance.NONE); @@ -122,7 +122,7 @@ void errorToleranceOnNONE() { @Test void errorToleranceOnALL() { final S3SourceRecord mockRecord = mock(S3SourceRecord.class); - when(mockRecord.getSourceRecord(any())).thenThrow(new DataException("generic issue")); + when(mockRecord.getSourceRecord(any(OffsetManager.class))).thenThrow(new DataException("generic issue")); when(s3SourceConfig.getErrorsTolerance()).thenReturn(ErrorsTolerance.ALL); From 0edf1145ac496a4bb0d22951eabc8296c3dd7e4e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Tue, 7 Jan 2025 12:47:00 +0000 Subject: [PATCH 19/44] Fix for slow polling test --- .../common/source/AbstractSourceTask.java | 21 ++++++++++++++++--- .../connect/s3/source/S3SourceTaskTest.java | 12 ++++++++++- 2 files changed, 29 insertions(+), 4 deletions(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java index e2050305a..47dea1732 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -202,11 +202,11 @@ private List populateList() { while (stillPolling() && results.size() < maxPollRecords) { if (!tryAdd(results, sourceRecordIterator)) { if (!results.isEmpty()) { - logger.info("tryAdd() did not add to the list, returning current results."); + logger.debug("tryAdd() did not add to the list, returning current results."); // if we could not get a record and the results are not empty return them break; } - logger.info("Attempting {}", backoff); + logger.debug("Attempting {}", backoff); backoff.cleanDelay(); } } @@ -402,6 +402,18 @@ public final void reset() { LOGGER.debug("Reset {}", this); } + /** + * Handle adjustment when maxCount could not be set. + * + * @return the corrected maxCount + */ + private int getMaxCount() { + if (maxCount == 0) { + reset(); + } + return maxCount; + } + /** * Calculates the delay wihtout jitter. * @@ -429,6 +441,7 @@ private long timeWithJitter() { final int jitter = random.nextInt(MAX_JITTER) - JITTER_SUBTRAHEND; return (long) Math.pow(2, waitCount) + jitter; } + /** * Delay execution based on the number of times this method has been called. * @@ -437,14 +450,16 @@ private long timeWithJitter() { */ public void delay() throws InterruptedException { final long sleepTime = timeRemaining.get(); - if (sleepTime > 0 && waitCount < maxCount) { + if (sleepTime > 0 && waitCount < (maxCount == 0 ? getMaxCount() : maxCount)) { waitCount++; final long nextSleep = timeWithJitter(); // don't sleep negative time. Jitter can introduce negative tme. if (nextSleep > 0) { if (nextSleep >= sleepTime) { + LOGGER.debug("Backoff aborting timer"); abortTrigger.apply(); } else { + LOGGER.debug("Backoff sleepiing {}", nextSleep); Thread.sleep(nextSleep); } } diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java index 742bb5029..ea510a226 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java @@ -300,7 +300,7 @@ public boolean hasNext() { @Override public S3SourceRecord next() { try { - Thread.sleep(Duration.ofSeconds(5).toMillis()); + Thread.sleep(Duration.ofSeconds(6).toMillis()); } catch (InterruptedException e) { // do nothing. } @@ -321,15 +321,25 @@ public S3SourceRecord next() { stopWatch.start(); // poll 1 results.addAll(s3SourceTask.poll()); + stopWatch.stop(); assertThat(results).hasSizeLessThanOrEqualTo(1); // poll 2 + stopWatch.reset(); + stopWatch.start(); results.addAll(s3SourceTask.poll()); assertThat(results).hasSizeLessThanOrEqualTo(2); + stopWatch.stop(); // poll 3 + stopWatch.reset(); + stopWatch.start(); results.addAll(s3SourceTask.poll()); assertThat(results).hasSizeLessThanOrEqualTo(3); + stopWatch.stop(); // poll 4 + stopWatch.reset(); + stopWatch.start(); results.addAll(s3SourceTask.poll()); + stopWatch.stop(); assertThat(results).hasSize(3); } From da17e1f742e523c16c7a3091a0f83978c627fce5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Tue, 7 Jan 2025 15:05:11 +0000 Subject: [PATCH 20/44] Changes as per review --- .../java/io/aiven/kafka/connect/s3/source/S3SourceTask.java | 4 ++-- .../io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java | 2 -- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index 48f9a926e..d2b2c44a3 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -143,8 +143,8 @@ public void commit() { @Override public void commitRecord(final SourceRecord record) { - if (LOGGER.isInfoEnabled()) { - LOGGER.info("Committed individual record {} committed", (Map) record.sourceOffset()); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Committed individual record {} committed", (Map) record.sourceOffset()); } } diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java index ea510a226..6a1a81573 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java @@ -125,8 +125,6 @@ public static void tearDownClass() { public void setUp() { properties = new HashMap<>(commonProperties); s3Client.createBucket(create -> create.bucket(TEST_BUCKET).build()); - // mockedSourceTaskContext = mock(SourceTaskContext.class); - // mockedOffsetStorageReader = mock(OffsetStorageReader.class); } @AfterEach From dc35ee4c1e4bc2faac277c4967c9b49d6fc735b9 Mon Sep 17 00:00:00 2001 From: Claude Warren Date: Thu, 9 Jan 2025 08:13:17 +0000 Subject: [PATCH 21/44] Update commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java Co-authored-by: Jarkko Jaakola <91882676+jjaakola-aiven@users.noreply.github.com> --- .../aiven/kafka/connect/common/source/AbstractSourceTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java index 47dea1732..a5476a249 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -39,7 +39,7 @@ * This class handles extracting records from an iterator and returning them to Kafka. It uses an exponential backoff * with jitter to reduce the number of calls to the backend when there is no data. This solution: *
    - *
  • When polled this implementation moves available records from the SsourceRecord iterator to the return array.
  • + *
  • When polled this implementation moves available records from the SourceRecord iterator to the return array.
  • *
  • if there are no records *
      *
    • {@link #poll()} will return null.
    • From 7aa64fc9eb302f5786c29dab792e5a60db1a9df1 Mon Sep 17 00:00:00 2001 From: Claude Warren Date: Thu, 9 Jan 2025 08:13:34 +0000 Subject: [PATCH 22/44] Update commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java Co-authored-by: Jarkko Jaakola <91882676+jjaakola-aiven@users.noreply.github.com> --- .../aiven/kafka/connect/common/source/AbstractSourceTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java index a5476a249..c93d2a84e 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -138,7 +138,7 @@ public final void start(final Map props) { * Try to add a SourceRecord to the results. * * @param results - * the result to add the recrod to. + * the result to add the record to. * @param sourceRecordIterator * the source record iterator. * @return true if successful, false if the iterator is empty. From c25a877c697bcf6659dacd3fea6968c720d33384 Mon Sep 17 00:00:00 2001 From: Claude Warren Date: Thu, 9 Jan 2025 08:13:56 +0000 Subject: [PATCH 23/44] Update commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java Co-authored-by: Jarkko Jaakola <91882676+jjaakola-aiven@users.noreply.github.com> --- .../aiven/kafka/connect/common/source/AbstractSourceTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java index c93d2a84e..2f1f3d0f0 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -148,7 +148,7 @@ private boolean tryAdd(final List results, final Iterator Date: Thu, 9 Jan 2025 08:20:40 +0000 Subject: [PATCH 24/44] Update s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java Co-authored-by: Jarkko Jaakola <91882676+jjaakola-aiven@users.noreply.github.com> --- .../java/io/aiven/kafka/connect/s3/source/S3SourceTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index d2b2c44a3..88b843433 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -79,7 +79,7 @@ public String version() { } @Override - protected Iterator getIterator(BackoffConfig config) { // NOPMD cognatavie complexity + protected Iterator getIterator(BackoffConfig config) { // NOPMD cognitive complexity final Iterator inner = new Iterator<>() { /** * The backoff for Amazon retryable exceptions From 21f713a9dde830109651ef8c71f170e750269e62 Mon Sep 17 00:00:00 2001 From: Claude Warren Date: Thu, 9 Jan 2025 08:21:02 +0000 Subject: [PATCH 25/44] Update commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java Co-authored-by: Jarkko Jaakola <91882676+jjaakola-aiven@users.noreply.github.com> --- .../aiven/kafka/connect/common/source/AbstractSourceTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java index 2f1f3d0f0..560af65be 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -113,7 +113,7 @@ protected AbstractSourceTask(final Logger logger) { *

      * * @param config - * the configuraiton for the Backoff. + * the configuration for the Backoff. * @return The iterator of SourceRecords. */ abstract protected Iterator getIterator(BackoffConfig config); From eb2c051f03dffe46867a84fb4b35902b1e11f2d1 Mon Sep 17 00:00:00 2001 From: Claude Warren Date: Thu, 9 Jan 2025 08:21:18 +0000 Subject: [PATCH 26/44] Update s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java Co-authored-by: Jarkko Jaakola <91882676+jjaakola-aiven@users.noreply.github.com> --- .../java/io/aiven/kafka/connect/s3/source/S3SourceTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index 88b843433..1bfc55580 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -149,7 +149,7 @@ public void commitRecord(final SourceRecord record) { } /** - * Set the S3 source record iterator that this task is using. protected to be overridden in testing impl. + * Set the S3 source record iterator that this task is using. Protected to be overridden in testing implementation. * * @param iterator * The S3SourceRecord iterator to use. From a9d93803810f68e66090b7273248826ccfd0316b Mon Sep 17 00:00:00 2001 From: Claude Warren Date: Thu, 9 Jan 2025 08:21:37 +0000 Subject: [PATCH 27/44] Update s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java Co-authored-by: Murali Basani --- .../kafka/connect/s3/source/utils/SourceRecordIterator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java index 319b6fb0e..3598cb28a 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java @@ -164,7 +164,7 @@ public Mapper(final Map partitionMap, final long recordCount, fi } @Override - public S3SourceRecord apply(final SchemaAndValue value) { + public S3SourceRecord apply(final SchemaAndValue valueData) { recordCount++; return new S3SourceRecord(partitionMap, recordCount, topic, partitionId, objectKey, keyData, value); } From 4f3f89698b9c1215cba0ce40a6cc02144b8eee2c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Thu, 9 Jan 2025 08:56:22 +0000 Subject: [PATCH 28/44] changes as per review requests --- .../common/source/AbstractSourceTask.java | 9 ++--- .../connect/s3/source/S3SourceTaskTest.java | 36 +++++++++++++------ 2 files changed, 31 insertions(+), 14 deletions(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java index 47dea1732..1adc2da59 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -18,7 +18,6 @@ import java.time.Duration; import java.util.ArrayList; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -54,6 +53,8 @@ */ public abstract class AbstractSourceTask extends SourceTask { + public static final List NULL_RESULT = null; + /** * The maximum time to spend polling. This is set to 5 seconds as that is the time that is allotted to a system for * shutdown. @@ -174,7 +175,7 @@ public final List poll() { if (connectorStopped.get()) { logger.info("Stopping"); closeResources(); - return Collections.emptyList(); + return NULL_RESULT; } else { timer.start(); try { @@ -215,10 +216,10 @@ private List populateList() { logger.error("Error during poll(): {}", e.getMessage(), e); if (config.getErrorsTolerance() == ErrorsTolerance.NONE) { logger.error("Stopping Task"); - return null; // NOPMD must return null in this case. + throw e; } } - return results; + return results.isEmpty() ? NULL_RESULT : results; } @Override diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java index 6a1a81573..21ca7807d 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java @@ -192,7 +192,7 @@ void testPollWithNoDataReturned() { stopWatch.start(); final List results = s3SourceTask.poll(); stopWatch.stop(); - assertThat(results).isEmpty(); + assertThat(results).isNull(); assertThat(stopWatch.getTime()).isLessThan(AbstractSourceTask.MAX_POLL_TIME.toMillis() + TIMING_DELTA); } @@ -307,37 +307,53 @@ public S3SourceRecord next() { }; final List results = new ArrayList<>(); - // since the polling is returning data at or near the time limit the 3 record may be returned as follows + // spotless:off + // since the polling is returning data at or near the time limit the 3 record may be returned as follows // // Record Poll1 Poll2 Poll3 Poll4 // 1 x x // 2 x x // 3 x x - // + // spotless:on + final S3SourceTask s3SourceTask = new TestingS3SourceTask(sourceRecordIterator); startSourceTask(s3SourceTask); final StopWatch stopWatch = new StopWatch(); stopWatch.start(); // poll 1 - results.addAll(s3SourceTask.poll()); + List pollResult = s3SourceTask.poll(); stopWatch.stop(); + if (pollResult != null) { + results.addAll(pollResult); + } assertThat(results).hasSizeLessThanOrEqualTo(1); // poll 2 stopWatch.reset(); stopWatch.start(); - results.addAll(s3SourceTask.poll()); - assertThat(results).hasSizeLessThanOrEqualTo(2); + pollResult = s3SourceTask.poll(); stopWatch.stop(); + if (pollResult != null) { + results.addAll(pollResult); + } + assertThat(results).hasSizeLessThanOrEqualTo(2); // poll 3 stopWatch.reset(); stopWatch.start(); - results.addAll(s3SourceTask.poll()); - assertThat(results).hasSizeLessThanOrEqualTo(3); + pollResult = s3SourceTask.poll(); stopWatch.stop(); + if (pollResult != null) { + results.addAll(pollResult); + } + assertThat(results).hasSizeLessThanOrEqualTo(3); // poll 4 stopWatch.reset(); stopWatch.start(); - results.addAll(s3SourceTask.poll()); + pollResult = s3SourceTask.poll(); stopWatch.stop(); + if (results.size() == 3) { + assertThat(pollResult).isNull(); + } else { + results.addAll(pollResult); + } assertThat(results).hasSize(3); } @@ -374,7 +390,7 @@ void testPollWhenConnectorStopped() { stopWatch.start(); final List results = s3SourceTask.poll(); stopWatch.stop(); - assertThat(results).isEmpty(); + assertThat(results).isNull(); assertThat(stopWatch.getTime()).isLessThan(TIMING_DELTA); } From 45bd6fdf1d56d33fa76ee0484afaa3a19ce76191 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Thu, 9 Jan 2025 09:05:36 +0000 Subject: [PATCH 29/44] fix for javadoc --- .../kafka/connect/s3/source/S3SourceTaskTest.java | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java index 21ca7807d..120d178c5 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java @@ -307,13 +307,10 @@ public S3SourceRecord next() { }; final List results = new ArrayList<>(); - // spotless:off - // since the polling is returning data at or near the time limit the 3 record may be returned as follows // - // Record Poll1 Poll2 Poll3 Poll4 - // 1 x x - // 2 x x - // 3 x x - // spotless:on + // since the polling is returning data at or near the time limit the 3 record may be returned as follows + // Record 1 may be returned in Poll1 or Poll2 + // Record 2 may be returned in Poll2 or Poll2 + // Record 3 may be returned in Poll3 or Poll4 final S3SourceTask s3SourceTask = new TestingS3SourceTask(sourceRecordIterator); startSourceTask(s3SourceTask); From 316a824f41e4496be505abb0f3e700aea12109b4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Thu, 9 Jan 2025 09:22:48 +0000 Subject: [PATCH 30/44] initial changes --- .../connect/s3/source/AwsIntegrationTest.java | 24 ++- .../connect/s3/source/IntegrationBase.java | 13 +- .../kafka/connect/s3/source/S3SourceTask.java | 20 +-- .../s3/source/utils/OffsetManager.java | 124 --------------- .../s3/source/utils/RecordProcessor.java | 4 +- .../s3/source/utils/S3SourceRecord.java | 54 ++----- .../s3/source/utils/SourceRecordIterator.java | 80 +++++----- .../connect/s3/source/S3SourceTaskTest.java | 26 ++-- .../s3/source/utils/OffsetManagerTest.java | 147 ------------------ .../s3/source/utils/RecordProcessorTest.java | 28 ++-- .../utils/SourceRecordIteratorTest.java | 18 ++- 11 files changed, 121 insertions(+), 417 deletions(-) delete mode 100644 s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java delete mode 100644 s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/OffsetManagerTest.java diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java index 42d10aad7..fc6e1d40b 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java @@ -25,8 +25,6 @@ import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_S3_ENDPOINT_CONFIG; import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_S3_PREFIX_CONFIG; import static io.aiven.kafka.connect.config.s3.S3ConfigFragment.AWS_SECRET_ACCESS_KEY_CONFIG; -import static io.aiven.kafka.connect.s3.source.S3SourceTask.OBJECT_KEY; -import static io.aiven.kafka.connect.s3.source.utils.OffsetManager.SEPARATOR; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; @@ -47,12 +45,13 @@ import org.apache.kafka.connect.source.SourceTaskContext; import org.apache.kafka.connect.storage.OffsetStorageReader; +import io.aiven.kafka.connect.common.OffsetManager; import io.aiven.kafka.connect.common.source.input.InputFormat; import io.aiven.kafka.connect.common.source.input.TransformerFactory; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; import io.aiven.kafka.connect.s3.source.testutils.BucketAccessor; import io.aiven.kafka.connect.s3.source.utils.AWSV2SourceClient; -import io.aiven.kafka.connect.s3.source.utils.OffsetManager; +import io.aiven.kafka.connect.s3.source.utils.S3OffsetManagerEntry; import io.aiven.kafka.connect.s3.source.utils.S3SourceRecord; import io.aiven.kafka.connect.s3.source.utils.SourceRecordIterator; @@ -163,7 +162,7 @@ void sourceRecordIteratorBytesTest(final TestInfo testInfo) { when(context.offsetStorageReader()).thenReturn(offsetStorageReader); when(offsetStorageReader.offsets(any())).thenReturn(new HashMap<>()); - final OffsetManager offsetManager = new OffsetManager(context, s3SourceConfig); + final OffsetManager offsetManager = new OffsetManager<>(context); final AWSV2SourceClient sourceClient = new AWSV2SourceClient(s3SourceConfig, new HashSet<>()); @@ -173,7 +172,7 @@ void sourceRecordIteratorBytesTest(final TestInfo testInfo) { final HashSet seenKeys = new HashSet<>(); while (sourceRecordIterator.hasNext()) { final S3SourceRecord s3SourceRecord = sourceRecordIterator.next(); - final String key = OBJECT_KEY + SEPARATOR + s3SourceRecord.getObjectKey(); + final String key = s3SourceRecord.getObjectKey(); assertThat(offsetKeys).contains(key); seenKeys.add(key); } @@ -226,7 +225,7 @@ void sourceRecordIteratorAvroTest(final TestInfo testInfo) throws IOException { when(context.offsetStorageReader()).thenReturn(offsetStorageReader); when(offsetStorageReader.offsets(any())).thenReturn(new HashMap<>()); - final OffsetManager offsetManager = new OffsetManager(context, s3SourceConfig); + final OffsetManager offsetManager = new OffsetManager(context); final AWSV2SourceClient sourceClient = new AWSV2SourceClient(s3SourceConfig, new HashSet<>()); @@ -237,10 +236,10 @@ void sourceRecordIteratorAvroTest(final TestInfo testInfo) throws IOException { final Map> seenRecords = new HashMap<>(); while (sourceRecordIterator.hasNext()) { final S3SourceRecord s3SourceRecord = sourceRecordIterator.next(); - final String key = OBJECT_KEY + SEPARATOR + s3SourceRecord.getObjectKey(); + final String key = s3SourceRecord.getObjectKey(); seenRecords.compute(key, (k, v) -> { final List lst = v == null ? new ArrayList<>() : v; // NOPMD new object inside loop - lst.add(s3SourceRecord.getRecordNumber()); + lst.add(s3SourceRecord.getOffsetManagerEntry().getRecordCount()); return lst; }); assertThat(offsetKeys).contains(key); @@ -275,10 +274,8 @@ void verifyIteratorRehydration(final TestInfo testInfo) { final List actualKeys = new ArrayList<>(); // write 2 objects to s3 - expectedKeys.add(writeToS3(topicName, testData1.getBytes(StandardCharsets.UTF_8), "00000") - .substring((OBJECT_KEY + SEPARATOR).length())); - expectedKeys.add(writeToS3(topicName, testData2.getBytes(StandardCharsets.UTF_8), "00000") - .substring((OBJECT_KEY + SEPARATOR).length())); + expectedKeys.add(writeToS3(topicName, testData1.getBytes(StandardCharsets.UTF_8), "00000")); + expectedKeys.add(writeToS3(topicName, testData2.getBytes(StandardCharsets.UTF_8), "00000")); assertThat(testBucketAccessor.listObjects()).hasSize(2); @@ -296,8 +293,7 @@ void verifyIteratorRehydration(final TestInfo testInfo) { assertThat(actualKeys).containsAll(expectedKeys); // write 3rd object to s3 - expectedKeys.add(writeToS3(topicName, testData3.getBytes(StandardCharsets.UTF_8), "00000") - .substring((OBJECT_KEY + SEPARATOR).length())); + expectedKeys.add(writeToS3(topicName, testData3.getBytes(StandardCharsets.UTF_8), "00000")); assertThat(testBucketAccessor.listObjects()).hasSize(3); assertThat(iter).hasNext(); diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java index a8b91a197..04b015f4e 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java @@ -16,8 +16,8 @@ package io.aiven.kafka.connect.s3.source; -import static io.aiven.kafka.connect.s3.source.S3SourceTask.OBJECT_KEY; -import static io.aiven.kafka.connect.s3.source.utils.OffsetManager.SEPARATOR; +import static io.aiven.kafka.connect.s3.source.utils.S3OffsetManagerEntry.OBJECT_KEY; +import static io.aiven.kafka.connect.s3.source.utils.S3OffsetManagerEntry.RECORD_COUNT; import static org.assertj.core.api.Assertions.assertThat; import static org.awaitility.Awaitility.await; @@ -130,14 +130,15 @@ default void writeToS3WithKey(final String objectKey, final byte[] testDataBytes * the data. * @param partitionId * the partition id. - * @return the key prefixed by {@link S3SourceTask#OBJECT_KEY} and - * {@link io.aiven.kafka.connect.s3.source.utils.OffsetManager#SEPARATOR} + * @return the key prefixed by {@link io.aiven.kafka.connect.s3.source.utils.S3OffsetManagerEntry#OBJECT_KEY} and + * {@link io.aiven.kafka.connect.common.OffsetManager} */ default String writeToS3(final String topicName, final byte[] testDataBytes, final String partitionId) { final String objectKey = org.apache.commons.lang3.StringUtils.defaultIfBlank(getS3Prefix(), "") + topicName + "-" + partitionId + "-" + System.currentTimeMillis() + ".txt"; writeToS3WithKey(objectKey, testDataBytes); - return OBJECT_KEY + SEPARATOR + objectKey; + return objectKey; + } default AdminClient newAdminClient(final String bootstrapServers) { @@ -262,7 +263,7 @@ static Map consumeOffsetMessages(KafkaConsumer c for (final ConsumerRecord record : records) { Map offsetRec = OBJECT_MAPPER.readValue(record.value(), new TypeReference<>() { // NOPMD }); - messages.putAll(offsetRec); + messages.put((String) offsetRec.get(OBJECT_KEY), offsetRec.get(RECORD_COUNT)); } return messages; } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index 1bfc55580..8a4f9f4cf 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -24,13 +24,14 @@ import org.apache.kafka.connect.source.SourceRecord; +import io.aiven.kafka.connect.common.OffsetManager; import io.aiven.kafka.connect.common.config.SourceCommonConfig; import io.aiven.kafka.connect.common.source.AbstractSourceTask; import io.aiven.kafka.connect.common.source.input.Transformer; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; import io.aiven.kafka.connect.s3.source.utils.AWSV2SourceClient; -import io.aiven.kafka.connect.s3.source.utils.OffsetManager; import io.aiven.kafka.connect.s3.source.utils.RecordProcessor; +import io.aiven.kafka.connect.s3.source.utils.S3OffsetManagerEntry; import io.aiven.kafka.connect.s3.source.utils.S3SourceRecord; import io.aiven.kafka.connect.s3.source.utils.SourceRecordIterator; import io.aiven.kafka.connect.s3.source.utils.Version; @@ -48,12 +49,6 @@ public class S3SourceTask extends AbstractSourceTask { /** The logger to write to */ private static final Logger LOGGER = LoggerFactory.getLogger(S3SourceTask.class); - public static final String BUCKET = "bucket"; - public static final String TOPIC = "topic"; - - public static final String OBJECT_KEY = "object_key"; - public static final String PARTITION = "topicPartition"; - /** An iterator or S3SourceRecords */ private Iterator s3SourceRecordIterator; /** @@ -66,7 +61,7 @@ public class S3SourceTask extends AbstractSourceTask { /** The list of failed object keys */ private final Set failedObjectKeys = new HashSet<>(); /** The offset manager this task uses */ - private OffsetManager offsetManager; + private OffsetManager offsetManager; private S3SourceConfig s3SourceConfig; public S3SourceTask() { @@ -115,10 +110,9 @@ public boolean hasNext() { @Override public SourceRecord next() { final S3SourceRecord s3SourceRecord = s3SourceRecordIterator.next(); - offsetManager.updateAndReturnCurrentOffsets(s3SourceRecord.getPartitionMap(), - s3SourceRecord.getObjectKey(), s3SourceRecord.getRecordNumber()); - return RecordProcessor.createSourceRecord(s3SourceRecord, s3SourceConfig, awsv2SourceClient, - offsetManager); + final S3OffsetManagerEntry entry = s3SourceRecord.getOffsetManagerEntry(); + offsetManager.updateCurrentOffsets(entry); + return RecordProcessor.createSourceRecord(s3SourceRecord, s3SourceConfig, awsv2SourceClient, entry); } }; return IteratorUtils.filteredIterator(inner, Objects::nonNull); @@ -129,7 +123,7 @@ protected SourceCommonConfig configure(final Map props) { LOGGER.info("S3 Source task started."); this.s3SourceConfig = new S3SourceConfig(props); this.transformer = s3SourceConfig.getTransformer(); - offsetManager = new OffsetManager(context, s3SourceConfig); + offsetManager = new OffsetManager<>(context); awsv2SourceClient = new AWSV2SourceClient(s3SourceConfig, failedObjectKeys); setS3SourceRecordIterator( new SourceRecordIterator(s3SourceConfig, offsetManager, this.transformer, awsv2SourceClient)); diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java deleted file mode 100644 index 95bc4053d..000000000 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/OffsetManager.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * Copyright 2024 Aiven Oy - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.aiven.kafka.connect.s3.source.utils; - -import static io.aiven.kafka.connect.s3.source.S3SourceTask.OBJECT_KEY; -import static java.util.stream.Collectors.toMap; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.Hashtable; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; - -import org.apache.kafka.connect.source.SourceTaskContext; - -import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class OffsetManager { - - private static final Logger LOGGER = LoggerFactory.getLogger(OffsetManager.class); - public static final String SEPARATOR = "_"; - private final Map, Map> offsets; - - public OffsetManager(final SourceTaskContext context, final S3SourceConfig s3SourceConfig) { - final String s3Bucket = s3SourceConfig.getAwsS3BucketName(); - final Set partitions = parsePartitions(s3SourceConfig); - final Set topics = parseTopics(s3SourceConfig); - - // Build the partition keys and fetch offsets from offset storage - final List> partitionKeys = buildPartitionKeys(s3Bucket, partitions, topics); - final Map, Map> offsetMap = context.offsetStorageReader() - .offsets(partitionKeys); - - LOGGER.info(" ********** offsetMap ***** {}", offsetMap); - this.offsets = offsetMap.entrySet() - .stream() - .filter(e -> e.getValue() != null) - .collect(toMap(entry -> new HashMap<>(entry.getKey()), entry -> new HashMap<>(entry.getValue()))); - LOGGER.info(" ********** offsets ***** {}", offsets); - } - - public Map, Map> getOffsets() { - return Collections.unmodifiableMap(offsets); - } - - public long incrementAndUpdateOffsetMap(final Map partitionMap, final String currentObjectKey, - final long startOffset) { - if (offsets.containsKey(partitionMap)) { - final Map offsetValue = new HashMap<>(offsets.get(partitionMap)); - if (offsetValue.containsKey(getObjectMapKey(currentObjectKey))) { - final long newOffsetVal = (long) offsetValue.get(getObjectMapKey(currentObjectKey)) + 1L; - offsetValue.put(getObjectMapKey(currentObjectKey), newOffsetVal); - offsets.put(partitionMap, offsetValue); - return newOffsetVal; - } else { - offsetValue.put(getObjectMapKey(currentObjectKey), startOffset); - offsets.put(partitionMap, offsetValue); - return startOffset; - } - } - return startOffset; - } - - public Map updateAndReturnCurrentOffsets(final Map partitionMap, - final String currentObjectKey, final long offset) { - final Map offsetMap = offsets.compute(partitionMap, (k, v) -> { - final Map map = v == null ? new Hashtable<>() : v; - map.put(getObjectMapKey(currentObjectKey), offset); - return map; - }); - return new HashMap<>(offsetMap); - } - - public static String getObjectMapKey(final String currentObjectKey) { - return OBJECT_KEY + SEPARATOR + currentObjectKey; - } - - public long recordsProcessedForObjectKey(final Map partitionMap, final String currentObjectKey) { - if (offsets.containsKey(partitionMap)) { - return (long) offsets.get(partitionMap).getOrDefault(getObjectMapKey(currentObjectKey), 0L); - } - return 0L; - } - - private static Set parsePartitions(final S3SourceConfig s3SourceConfig) { - final String partitionString = s3SourceConfig.getTargetTopicPartitions(); - return Arrays.stream(partitionString.split(",")).map(Integer::parseInt).collect(Collectors.toSet()); - } - - private static Set parseTopics(final S3SourceConfig s3SourceConfig) { - final String topicString = s3SourceConfig.getTargetTopics(); - return Arrays.stream(topicString.split(",")).collect(Collectors.toSet()); - } - - private static List> buildPartitionKeys(final String bucket, final Set partitions, - final Set topics) { - final List> partitionKeys = new ArrayList<>(); - partitions.forEach(partition -> topics.forEach(topic -> { - partitionKeys.add(ConnectUtils.getPartitionMap(topic, partition, bucket)); - })); - return partitionKeys; - } -} diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessor.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessor.java index e945c2565..45e1e2c6e 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessor.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessor.java @@ -35,9 +35,9 @@ private RecordProcessor() { public static SourceRecord createSourceRecord(final S3SourceRecord s3SourceRecord, final S3SourceConfig s3SourceConfig, final AWSV2SourceClient sourceClient, - final OffsetManager offsetManager) { + final S3OffsetManagerEntry s3OffsetManagerEntry) { try { - return s3SourceRecord.getSourceRecord(offsetManager); + return s3SourceRecord.getSourceRecord(s3OffsetManagerEntry); } catch (DataException e) { if (ErrorsTolerance.NONE.equals(s3SourceConfig.getErrorsTolerance())) { throw new ConnectException("Data Exception caught during S3 record to source record transformation", e); diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java index 05ca02ba4..4c1df43cf 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java @@ -16,54 +16,25 @@ package io.aiven.kafka.connect.s3.source.utils; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; - import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.source.SourceRecord; public class S3SourceRecord { - private final Map partitionMap; - private final long recordNumber; - private final String topic; - private final Integer topicPartition; - private final SchemaAndValue keyData; + private final SchemaAndValue keyData; private final SchemaAndValue valueData; + /** The S3OffsetManagerEntry for this source record */ + private final S3OffsetManagerEntry offsetManagerEntry; - private final String objectKey; - - public S3SourceRecord(final Map partitionMap, final long recordNumber, final String topic, - final Integer topicPartition, final String objectKey, final SchemaAndValue keyData, + public S3SourceRecord(final S3OffsetManagerEntry offsetManagerEntry, final SchemaAndValue keyData, final SchemaAndValue valueData) { - this.partitionMap = new HashMap<>(partitionMap); - this.recordNumber = recordNumber; - this.topic = topic; - this.topicPartition = topicPartition; + this.offsetManagerEntry = offsetManagerEntry.fromProperties(offsetManagerEntry.getProperties()); this.keyData = keyData; this.valueData = valueData; - this.objectKey = objectKey; - } - - public Map getPartitionMap() { - return Collections.unmodifiableMap(partitionMap); - } - - public long getRecordNumber() { - return recordNumber; - } - - public String getTopic() { - return topic; - } - - public Integer partition() { - return topicPartition; } public String getObjectKey() { - return objectKey; + return offsetManagerEntry.getKey(); } public SchemaAndValue getKey() { @@ -74,10 +45,13 @@ public SchemaAndValue getValue() { return new SchemaAndValue(valueData.schema(), valueData.value()); } - public SourceRecord getSourceRecord(final OffsetManager offsetManager) { - final Map offsetMap = offsetManager.updateAndReturnCurrentOffsets(getPartitionMap(), - getObjectKey(), getRecordNumber()); - return new SourceRecord(getPartitionMap(), offsetMap, topic, partition(), keyData.schema(), keyData.value(), - valueData.schema(), valueData.value()); + public S3OffsetManagerEntry getOffsetManagerEntry() { + return offsetManagerEntry.fromProperties(offsetManagerEntry.getProperties()); // return a defensive copy + } + + public SourceRecord getSourceRecord(final S3OffsetManagerEntry offsetManager) { + return new SourceRecord(offsetManagerEntry.getManagerKey().getPartitionMap(), + offsetManagerEntry.getProperties(), offsetManagerEntry.getTopic(), offsetManagerEntry.getPartition(), + keyData.schema(), keyData.value(), valueData.schema(), valueData.value()); } } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java index 3598cb28a..97c229b21 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java @@ -18,7 +18,6 @@ import java.util.Collections; import java.util.Iterator; -import java.util.Map; import java.util.function.Function; import java.util.function.Predicate; import java.util.regex.Matcher; @@ -27,6 +26,7 @@ import org.apache.kafka.connect.data.SchemaAndValue; +import io.aiven.kafka.connect.common.OffsetManager; import io.aiven.kafka.connect.common.source.input.ByteArrayTransformer; import io.aiven.kafka.connect.common.source.input.Transformer; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; @@ -46,10 +46,12 @@ public final class SourceRecordIterator implements Iterator { + "(?\\d{5})-" + "(?[a-zA-Z0-9]+)" + "\\.(?[^.]+)$"); // topic-00001.txt public static final long BYTES_TRANSFORMATION_NUM_OF_RECS = 1L; - private final OffsetManager offsetManager; + /** The OffsetManager that we are using */ + private final OffsetManager offsetManager; + /** The offset manager Entry we are working with */ + private S3OffsetManagerEntry offsetManagerEntry; private final S3SourceConfig s3SourceConfig; - private final String bucketName; private final Transformer transformer; // Once we decouple the S3Object from the Source Iterator we can change this to be the SourceApiClient @@ -58,31 +60,22 @@ public final class SourceRecordIterator implements Iterator { private String topic; private int partitionId; - + private final String bucketName; private final Iterator inner; private Iterator outer; - private final Predicate fileNamePredicate = s3Object -> { - - final Matcher fileMatcher = FILE_DEFAULT_PATTERN.matcher(s3Object.key()); + private final Predicate fileNamePredicate; - if (fileMatcher.find()) { - // TODO move this from the SourceRecordIterator so that we can decouple it from S3 and make it API agnostic - topic = fileMatcher.group(PATTERN_TOPIC_KEY); - partitionId = Integer.parseInt(fileMatcher.group(PATTERN_PARTITION_KEY)); - return true; - } - return false; - }; - - public SourceRecordIterator(final S3SourceConfig s3SourceConfig, final OffsetManager offsetManager, - final Transformer transformer, final AWSV2SourceClient sourceClient) { + public SourceRecordIterator(final S3SourceConfig s3SourceConfig, + final OffsetManager offsetManager, final Transformer transformer, + final AWSV2SourceClient sourceClient) { super(); this.s3SourceConfig = s3SourceConfig; this.offsetManager = offsetManager; - this.bucketName = s3SourceConfig.getAwsS3BucketName(); + this.fileNamePredicate = buildPredicate(); + this.transformer = transformer; this.sourceClient = sourceClient; @@ -92,6 +85,27 @@ public SourceRecordIterator(final S3SourceConfig s3SourceConfig, final OffsetMan outer = Collections.emptyIterator(); } + private Predicate buildPredicate() { + return s3Object -> { + + final Matcher fileMatcher = FILE_DEFAULT_PATTERN.matcher(s3Object.key()); + + if (fileMatcher.find()) { + // TODO move this from the SourceRecordIterator so that we can decouple it from S3 and make it API + // agnostic + topic = fileMatcher.group(PATTERN_TOPIC_KEY); + partitionId = Integer.parseInt(fileMatcher.group(PATTERN_PARTITION_KEY)); + final S3OffsetManagerEntry keyEntry = new S3OffsetManagerEntry(bucketName, s3Object.key(), + fileMatcher.group(PATTERN_TOPIC_KEY), + Integer.parseInt(fileMatcher.group(PATTERN_PARTITION_KEY))); + offsetManagerEntry = offsetManager.getEntry(keyEntry.getManagerKey(), keyEntry::fromProperties) + .orElse(keyEntry); + return true; + } + return false; + }; + } + @Override public boolean hasNext() { while (!outer.hasNext() && inner.hasNext()) { @@ -119,8 +133,7 @@ public void remove() { */ private Stream convert(final S3Object s3Object) { - final Map partitionMap = ConnectUtils.getPartitionMap(topic, partitionId, bucketName); - final long recordCount = offsetManager.recordsProcessedForObjectKey(partitionMap, s3Object.key()); + final long recordCount = offsetManagerEntry.getRecordCount(); // Optimizing without reading stream again. if (transformer instanceof ByteArrayTransformer && recordCount > 0) { @@ -131,42 +144,31 @@ private Stream convert(final S3Object s3Object) { return transformer .getRecords(sourceClient.getObject(s3Object.key()), topic, partitionId, s3SourceConfig, recordCount) - .map(new Mapper(partitionMap, recordCount, keyData, s3Object.key())); + .map(new Mapper(offsetManagerEntry, keyData)); } /** * maps the data from the @{link Transformer} stream to an S3SourceRecord given all the additional data required. */ - class Mapper implements Function { + static class Mapper implements Function { /** * The partition map */ - private final Map partitionMap; - /** - * The record number for the record being created. - */ - private long recordCount; + private final S3OffsetManagerEntry entry; /** * The schema and value for the key */ private final SchemaAndValue keyData; - /** - * The object key from S3 - */ - private final String objectKey; - public Mapper(final Map partitionMap, final long recordCount, final SchemaAndValue keyData, - final String objectKey) { - this.partitionMap = partitionMap; - this.recordCount = recordCount; + public Mapper(final S3OffsetManagerEntry entry, final SchemaAndValue keyData) { + this.entry = entry; this.keyData = keyData; - this.objectKey = objectKey; } @Override public S3SourceRecord apply(final SchemaAndValue valueData) { - recordCount++; - return new S3SourceRecord(partitionMap, recordCount, topic, partitionId, objectKey, keyData, value); + entry.incrementRecordCount(); + return new S3SourceRecord(entry, keyData, valueData); } } } diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java index 120d178c5..0c51665c2 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java @@ -19,6 +19,9 @@ import static io.aiven.kafka.connect.common.config.SchemaRegistryFragment.INPUT_FORMAT_KEY; import static io.aiven.kafka.connect.common.config.SourceConfigFragment.TARGET_TOPICS; import static io.aiven.kafka.connect.common.config.SourceConfigFragment.TARGET_TOPIC_PARTITIONS; +import static io.aiven.kafka.connect.s3.source.utils.S3OffsetManagerEntry.BUCKET; +import static io.aiven.kafka.connect.s3.source.utils.S3OffsetManagerEntry.OBJECT_KEY; +import static io.aiven.kafka.connect.s3.source.utils.S3OffsetManagerEntry.RECORD_COUNT; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -48,8 +51,7 @@ import io.aiven.kafka.connect.config.s3.S3ConfigFragment; import io.aiven.kafka.connect.iam.AwsCredentialProviderFactory; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; -import io.aiven.kafka.connect.s3.source.utils.ConnectUtils; -import io.aiven.kafka.connect.s3.source.utils.OffsetManager; +import io.aiven.kafka.connect.s3.source.utils.S3OffsetManagerEntry; import io.aiven.kafka.connect.s3.source.utils.S3SourceRecord; import io.findify.s3mock.S3Mock; @@ -80,7 +82,7 @@ final class S3SourceTaskTest { private static final int PARTITION = 1; - private static final String OBJECT_KEY = "object_key"; + private static final String TEST_OBJECT_KEY = "object_key"; // TODO S3Mock has not been maintained in 4 years // Adobe have an alternative we can move to. @@ -153,8 +155,8 @@ void testStop() { private static S3SourceRecord createS3SourceRecord(final String topicName, final Integer defaultPartitionId, final String bucketName, final String objectKey, final byte[] key, final byte[] value) { - return new S3SourceRecord(ConnectUtils.getPartitionMap(topicName, defaultPartitionId, bucketName), 0L, - topicName, defaultPartitionId, objectKey, new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, key), + return new S3SourceRecord(new S3OffsetManagerEntry(bucketName, objectKey, topicName, defaultPartitionId), + new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, key), new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, value)); } @@ -198,11 +200,14 @@ void testPollWithNoDataReturned() { private void assertEquals(final S3SourceRecord s3Record, final SourceRecord sourceRecord) { assertThat(sourceRecord).isNotNull(); - assertThat(sourceRecord.sourcePartition()).isEqualTo(s3Record.getPartitionMap()); + + assertThat(sourceRecord.sourcePartition()).hasSize(2); + assertThat(sourceRecord.sourcePartition().get(BUCKET)).isEqualTo(s3Record.getOffsetManagerEntry().getBucket()); + assertThat(sourceRecord.sourcePartition().get(OBJECT_KEY)).isEqualTo(s3Record.getOffsetManagerEntry().getKey()); + final Map map = (Map) sourceRecord.sourceOffset(); - assertThat(map.get(OffsetManager.getObjectMapKey(s3Record.getObjectKey()))) - .isEqualTo(s3Record.getRecordNumber()); + assertThat(map.get(RECORD_COUNT)).isEqualTo(s3Record.getOffsetManagerEntry().getRecordCount()); assertThat(sourceRecord.key()).isEqualTo(s3Record.getKey().value()); assertThat(sourceRecord.value()).isEqualTo(s3Record.getValue().value()); } @@ -228,10 +233,11 @@ void testPollsWithRecords() { private List createS3SourceRecords(final int count) { final List lst = new ArrayList<>(); if (count > 0) { - lst.add(createS3SourceRecord(TOPIC, PARTITION, TEST_BUCKET, OBJECT_KEY, + + lst.add(createS3SourceRecord(TOPIC, PARTITION, TEST_BUCKET, TEST_OBJECT_KEY, "Hello".getBytes(StandardCharsets.UTF_8), "Hello World".getBytes(StandardCharsets.UTF_8))); for (int i = 1; i < count; i++) { - lst.add(createS3SourceRecord(TOPIC, PARTITION, TEST_BUCKET, OBJECT_KEY + i, + lst.add(createS3SourceRecord(TOPIC, PARTITION, TEST_BUCKET, TEST_OBJECT_KEY + i, "Goodbye".getBytes(StandardCharsets.UTF_8), String.format("Goodbye cruel World (%s)", i).getBytes(StandardCharsets.UTF_8))); } diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/OffsetManagerTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/OffsetManagerTest.java deleted file mode 100644 index 1367d71f0..000000000 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/OffsetManagerTest.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * Copyright 2024 Aiven Oy - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.aiven.kafka.connect.s3.source.utils; - -import static io.aiven.kafka.connect.common.config.SourceConfigFragment.TARGET_TOPICS; -import static io.aiven.kafka.connect.common.config.SourceConfigFragment.TARGET_TOPIC_PARTITIONS; -import static io.aiven.kafka.connect.s3.source.S3SourceTask.OBJECT_KEY; -import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; - -import org.apache.kafka.connect.source.SourceTaskContext; -import org.apache.kafka.connect.storage.OffsetStorageReader; - -import io.aiven.kafka.connect.config.s3.S3ConfigFragment; -import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; - -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.Mock; - -final class OffsetManagerTest { - - private Map properties; - private static final String TEST_BUCKET = "test-bucket"; - - @Mock - private SourceTaskContext sourceTaskContext; - - private S3SourceConfig s3SourceConfig; - - private OffsetManager offsetManager; - - @BeforeEach - public void setUp() { - properties = new HashMap<>(); - setBasicProperties(); - s3SourceConfig = new S3SourceConfig(properties); - } - - @Test - void testWithOffsets() { - sourceTaskContext = mock(SourceTaskContext.class); - final OffsetStorageReader offsetStorageReader = mock(OffsetStorageReader.class); - when(sourceTaskContext.offsetStorageReader()).thenReturn(offsetStorageReader); - - final Map partitionKey = new HashMap<>(); - partitionKey.put("topic", "topic1"); - partitionKey.put("partition", 0); - partitionKey.put("bucket", TEST_BUCKET); - - final Map offsetValue = new HashMap<>(); - offsetValue.put("object_key_file", 5L); - final Map, Map> offsets = new HashMap<>(); - offsets.put(partitionKey, offsetValue); - - when(offsetStorageReader.offsets(any())).thenReturn(offsets); - - offsetManager = new OffsetManager(sourceTaskContext, s3SourceConfig); - - final Map, Map> retrievedOffsets = offsetManager.getOffsets(); - assertThat(retrievedOffsets.size()).isEqualTo(1); - assertThat(retrievedOffsets.values().iterator().next().get("object_key_file")).isEqualTo(5L); - } - - @Test - void testIncrementAndUpdateOffsetMapExistingOffset() { - sourceTaskContext = mock(SourceTaskContext.class); - final OffsetStorageReader offsetStorageReader = mock(OffsetStorageReader.class); - when(sourceTaskContext.offsetStorageReader()).thenReturn(offsetStorageReader); - - // Mock partition and offset values - final String objectKey = "testObject"; - final String offsetObjectKey = OBJECT_KEY + "_" + objectKey; - - final Map partitionKey = new HashMap<>(); - partitionKey.put("topic", "topic1"); - partitionKey.put("partition", 0); - partitionKey.put("bucket", "bucket"); - - final Map offsetValue = new HashMap<>(); - offsetValue.put(offsetObjectKey, 1L); // Existing offset value - final Map, Map> offsets = new HashMap<>(); - offsets.put(partitionKey, offsetValue); - - when(offsetStorageReader.offsets(any())).thenReturn(offsets); // Mock offset retrieval - - // Initialize offset manager - offsetManager = new OffsetManager(sourceTaskContext, s3SourceConfig); - - // Invoke method and assert new offset value - final long newOffset = offsetManager.incrementAndUpdateOffsetMap(partitionKey, objectKey, 2L); - - assertThat(newOffset).isEqualTo(2L); // Expect incremented offset - assertThat(offsetManager.getOffsets().get(partitionKey).get(offsetObjectKey)).isEqualTo(2L); // Verify updated - // offset in map - } - - @Test - void testIncrementAndUpdateOffsetMapNonExistingOffset() { - sourceTaskContext = mock(SourceTaskContext.class); - final OffsetStorageReader offsetStorageReader = mock(OffsetStorageReader.class); - when(sourceTaskContext.offsetStorageReader()).thenReturn(offsetStorageReader); - - // Mock partition without any existing offset - final Map partitionKey = new HashMap<>(); - partitionKey.put("topic", "topic1"); - partitionKey.put("partition", 0); - - when(offsetStorageReader.offsets(any())).thenReturn(Collections.emptyMap()); // No existing offset - - // Initialize offset manager - offsetManager = new OffsetManager(sourceTaskContext, s3SourceConfig); - - // Invoke method and assert new offset value - final long startOffset = 5L; - final long newOffset = offsetManager.incrementAndUpdateOffsetMap(partitionKey, "", startOffset); - - // Expect the startOffset to be returned when no existing offset is found - assertThat(newOffset).isEqualTo(startOffset); - } - - private void setBasicProperties() { - properties.put(S3ConfigFragment.AWS_S3_BUCKET_NAME_CONFIG, TEST_BUCKET); - properties.put(TARGET_TOPIC_PARTITIONS, "0,1"); - properties.put(TARGET_TOPICS, "topic1,topic2"); - } -} diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java index cc9db65cd..8d9800577 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/RecordProcessorTest.java @@ -53,7 +53,7 @@ class RecordProcessorTest { @Mock private Converter keyConverter; @Mock - private OffsetManager offsetManager; + private S3OffsetManagerEntry s3OffsetManagerEntry; @Mock private AWSV2SourceClient sourceClient; @@ -66,10 +66,10 @@ void testCreateSourceRecord() { final SourceRecord mockSourceRecord = mock(SourceRecord.class); final S3SourceRecord mockRecord = mock(S3SourceRecord.class); - when(mockRecord.getSourceRecord(any(OffsetManager.class))).thenReturn(mockSourceRecord); + when(mockRecord.getSourceRecord(any(S3OffsetManagerEntry.class))).thenReturn(mockSourceRecord); final SourceRecord result = RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, - offsetManager); + s3OffsetManagerEntry); verify(mockRecord, times(1)).getSourceRecord(any()); assertThat(result).isEqualTo(mockSourceRecord); @@ -80,27 +80,28 @@ void testCreateSourceRecord() { void testCreateSourceRecordWithDataError() { final S3SourceRecord mockRecord = mock(S3SourceRecord.class); - when(mockRecord.getSourceRecord(any(OffsetManager.class))).thenThrow(new DataException("Testing exception")); + when(mockRecord.getSourceRecord(any(S3OffsetManagerEntry.class))) + .thenThrow(new DataException("Testing exception")); when(s3SourceConfig.getErrorsTolerance()).thenReturn(ErrorsTolerance.NONE); assertThatExceptionOfType(ConnectException.class).as("Errors tolerance: NONE") .isThrownBy(() -> RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, - offsetManager)); + s3OffsetManagerEntry)); when(s3SourceConfig.getErrorsTolerance()).thenReturn(ErrorsTolerance.ALL); final SourceRecord result = RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, - offsetManager); + s3OffsetManagerEntry); assertThat(result).isNull(); } @Test void testCreateSourceRecords() { final S3SourceRecord mockRecord = mock(S3SourceRecord.class); - when(mockRecord.getSourceRecord(any(OffsetManager.class))).thenReturn(mock(SourceRecord.class)); + when(mockRecord.getSourceRecord(any(S3OffsetManagerEntry.class))).thenReturn(mock(SourceRecord.class)); final SourceRecord sourceRecords = RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, - offsetManager); + s3OffsetManagerEntry); assertThat(sourceRecords).isNotNull(); } @@ -108,13 +109,12 @@ void testCreateSourceRecords() { @Test void errorToleranceOnNONE() { final S3SourceRecord mockRecord = mock(S3SourceRecord.class); - when(mockRecord.getSourceRecord(any(OffsetManager.class))).thenThrow(new DataException("generic issue")); + when(mockRecord.getSourceRecord(any(S3OffsetManagerEntry.class))).thenThrow(new DataException("generic issue")); when(s3SourceConfig.getErrorsTolerance()).thenReturn(ErrorsTolerance.NONE); - assertThatThrownBy( - () -> RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, offsetManager)) - .isInstanceOf(ConnectException.class) + assertThatThrownBy(() -> RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, + s3OffsetManagerEntry)).isInstanceOf(ConnectException.class) .hasMessage("Data Exception caught during S3 record to source record transformation"); } @@ -122,11 +122,11 @@ void errorToleranceOnNONE() { @Test void errorToleranceOnALL() { final S3SourceRecord mockRecord = mock(S3SourceRecord.class); - when(mockRecord.getSourceRecord(any(OffsetManager.class))).thenThrow(new DataException("generic issue")); + when(mockRecord.getSourceRecord(any(S3OffsetManagerEntry.class))).thenThrow(new DataException("generic issue")); when(s3SourceConfig.getErrorsTolerance()).thenReturn(ErrorsTolerance.ALL); - assertThat(RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, offsetManager)) + assertThat(RecordProcessor.createSourceRecord(mockRecord, s3SourceConfig, sourceClient, s3OffsetManagerEntry)) .isNull(); } diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java index af9b679fa..5f24f75fd 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java @@ -18,7 +18,6 @@ import static io.aiven.kafka.connect.s3.source.utils.SourceRecordIterator.BYTES_TRANSFORMATION_NUM_OF_RECS; import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.ArgumentMatchers.anyMap; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyInt; import static org.mockito.Mockito.anyLong; @@ -35,10 +34,12 @@ import java.util.Arrays; import java.util.Collections; import java.util.Iterator; +import java.util.Optional; import java.util.stream.Stream; import org.apache.kafka.connect.data.SchemaAndValue; +import io.aiven.kafka.connect.common.OffsetManager; import io.aiven.kafka.connect.common.source.input.AvroTransformer; import io.aiven.kafka.connect.common.source.input.ByteArrayTransformer; import io.aiven.kafka.connect.common.source.input.InputFormat; @@ -53,7 +54,8 @@ final class SourceRecordIteratorTest { private S3SourceConfig mockConfig; - private OffsetManager mockOffsetManager; + private S3OffsetManagerEntry mockS3OffsetManagerEntry; + private OffsetManager mockOffsetManager; private Transformer mockTransformer; private AWSV2SourceClient mockSourceApiClient; @@ -62,8 +64,10 @@ final class SourceRecordIteratorTest { public void setUp() { mockConfig = mock(S3SourceConfig.class); mockOffsetManager = mock(OffsetManager.class); + mockS3OffsetManagerEntry = mock(S3OffsetManagerEntry.class); mockTransformer = mock(Transformer.class); mockSourceApiClient = mock(AWSV2SourceClient.class); + when(mockConfig.getAwsS3BucketName()).thenReturn("bucket_name"); } @Test @@ -77,7 +81,7 @@ void testIteratorProcessesS3Objects() throws Exception { mockTransformer = TransformerFactory.getTransformer(InputFormat.BYTES); - when(mockOffsetManager.getOffsets()).thenReturn(Collections.emptyMap()); + when(mockS3OffsetManagerEntry.getProperties()).thenReturn(Collections.emptyMap()); when(mockSourceApiClient.getS3ObjectIterator(any())).thenReturn(Collections.emptyIterator()); Iterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, @@ -114,12 +118,11 @@ void testIteratorProcessesS3ObjectsForByteArrayTransformer() throws Exception { when(mockTransformer.getRecords(any(), anyString(), anyInt(), any(), anyLong())) .thenReturn(Stream.of(SchemaAndValue.NULL)); - when(mockOffsetManager.getOffsets()).thenReturn(Collections.emptyMap()); + when(mockOffsetManager.getEntry(any(), any())).thenReturn(Optional.of(mockS3OffsetManagerEntry)); when(mockSourceApiClient.getListOfObjectKeys(any())) .thenReturn(Collections.singletonList(key).listIterator()); - when(mockOffsetManager.recordsProcessedForObjectKey(anyMap(), anyString())) - .thenReturn(BYTES_TRANSFORMATION_NUM_OF_RECS); + when(mockS3OffsetManagerEntry.getRecordCount()).thenReturn(BYTES_TRANSFORMATION_NUM_OF_RECS); // should skip if any records were produced by source record iterator. final Iterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, @@ -137,8 +140,7 @@ void testIteratorProcessesS3ObjectsForByteArrayTransformer() throws Exception { when(mockSourceApiClient.getListOfObjectKeys(any())) .thenReturn(Collections.singletonList(key).listIterator()); - when(mockOffsetManager.recordsProcessedForObjectKey(anyMap(), anyString())) - .thenReturn(BYTES_TRANSFORMATION_NUM_OF_RECS); + when(mockS3OffsetManagerEntry.getRecordCount()).thenReturn(BYTES_TRANSFORMATION_NUM_OF_RECS); when(mockTransformer.getKeyData(anyString(), anyString(), any())).thenReturn(SchemaAndValue.NULL); when(mockTransformer.getRecords(any(), anyString(), anyInt(), any(), anyLong())) From c69bb653fcebe71b17710b57b24aec28b2f53da6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Thu, 9 Jan 2025 10:03:36 +0000 Subject: [PATCH 31/44] merged files for KCON-57 --- .../kafka/connect/common/OffsetManager.java | 260 ++++++++++++++++++ .../connect/common/OffsetManagerTest.java | 224 +++++++++++++++ .../s3/source/utils/S3OffsetManagerEntry.java | 227 +++++++++++++++ .../utils/S3OffsetManagerEntryTest.java | 148 ++++++++++ 4 files changed, 859 insertions(+) create mode 100644 commons/src/main/java/io/aiven/kafka/connect/common/OffsetManager.java create mode 100644 commons/src/test/java/io/aiven/kafka/connect/common/OffsetManagerTest.java create mode 100644 s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntry.java create mode 100644 s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/OffsetManager.java b/commons/src/main/java/io/aiven/kafka/connect/common/OffsetManager.java new file mode 100644 index 000000000..ef0078e4a --- /dev/null +++ b/commons/src/main/java/io/aiven/kafka/connect/common/OffsetManager.java @@ -0,0 +1,260 @@ +/* + * Copyright 2024 Aiven Oy + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.aiven.kafka.connect.common; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.function.Function; + +import org.apache.kafka.connect.source.SourceTaskContext; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class OffsetManager> { + /** The loger to write to */ + private static final Logger LOGGER = LoggerFactory.getLogger(OffsetManager.class); + + /** + * The local manager data. + */ + private final ConcurrentMap, Map> offsets; + + /** + * The context in which this is running. + */ + private final SourceTaskContext context; + + /** + * Constructor + * + * @param context + * the context for this instance to use. + */ + public OffsetManager(final SourceTaskContext context) { + this(context, new ConcurrentHashMap<>()); + } + + /** + * Package private for testing. + * + * @param context + * the context for this instance to use. + * @param offsets + * the offsets + */ + protected OffsetManager(final SourceTaskContext context, + final ConcurrentMap, Map> offsets) { + this.context = context; + this.offsets = offsets; + } + + /** + * Get an entry from the offset manager. This method will return the local copy if it has been created otherwise + * will get the data from Kafka. If there is not a local copy and not one from Kafka then an empty Optional is + * returned + * + * @param key + * the key for the entry. + * @param creator + * a function to create the connector defined offset entry from a Map of string to object. + * @return the entry. + */ + public Optional getEntry(final OffsetManagerKey key, final Function, E> creator) { + LOGGER.info("getEntry: {}", key.getPartitionMap()); + final Map data = offsets.compute(key.getPartitionMap(), (k, v) -> { + if (v == null) { + final Map kafkaData = context.offsetStorageReader().offset(key.getPartitionMap()); + LOGGER.info("Context stored offset map {}", kafkaData); + return kafkaData == null || kafkaData.isEmpty() ? null : kafkaData; + } else { + LOGGER.info("Previously stored offset map {}", v); + return v; + } + }); + return data == null ? Optional.empty() : Optional.of(creator.apply(data)); + } + + /** + * Copies the entry into the offset manager data. + * + * @param entry + * the entry to update. + */ + public void updateCurrentOffsets(final E entry) { + LOGGER.debug("Updating current offsets: {}", entry.getManagerKey().getPartitionMap()); + offsets.compute(entry.getManagerKey().getPartitionMap(), (k, v) -> { + if (v == null) { + return new HashMap<>(entry.getProperties()); + } else { + v.putAll(entry.getProperties()); + return v; + } + }); + } + + /** + * Removes the specified entry from the in memory table. Does not impact the records stored in the + * {@link SourceTaskContext}. + * + * @param key + * the key for the entry to remove. + */ + public void remove(final OffsetManagerKey key) { + LOGGER.info("Removing: {}", key.getPartitionMap()); + offsets.remove(key.getPartitionMap()); + } + + /** + * The definition of an entry in the OffsetManager. + */ + public interface OffsetManagerEntry> extends Comparable { + + /** + * Creates a new OffsetManagerEntry by wrapping the properties with the current implementation. This method may + * throw a RuntimeException if requried properties are not defined in the map. + * + * @param properties + * the properties to wrap. May be {@code null}. + * @return an OffsetManagerProperty + */ + T fromProperties(Map properties); + + /** + * Extracts the data from the entry in the correct format to return to Kafka. + * + * @return the properties in a format to return to Kafka. + */ + Map getProperties(); + + /** + * Gets the value of the named property. The value returned from a {@code null} key is implementation dependant. + * + * @param key + * the property to retrieve. + * @return the value associated with the property or @{code null} if not set. + * @throws NullPointerException + * if a {@code null} key is not supported. + */ + Object getProperty(String key); + + /** + * Sets a key/value pair. Will overwrite any existing value. Implementations of OffsetManagerEntry may declare + * specific keys as restricted. These are generally keys that are managed internally by the OffsetManagerEntry + * and may not be set except through provided setter methods or the constructor. + * + * @param key + * the key to set. + * @param value + * the value to set. + * @throws IllegalArgumentException + * if the key is restricted. + */ + void setProperty(String key, Object value); + + /** + * Gets the value of the named property as an {@code int}. + * + * @param key + * the property to retrieve. + * @return the value associated with the property or @{code null} if not set. + * @throws NullPointerException + * if a {@code null} key is not supported. + */ + default int getInt(final String key) { + return ((Number) getProperty(key)).intValue(); + } + + /** + * Gets the value of the named property as a {@code long} + * + * @param key + * the property to retrieve. + * @return the value associated with the property or @{code null} if not set. + * @throws NullPointerException + * if a {@code null} key is not supported. + */ + default long getLong(final String key) { + return ((Number) getProperty(key)).longValue(); + } + + /** + * Gets the value of the named property as a String. + * + * @param key + * the property to retrieve. + * @return the value associated with the property or @{code null} if not set. + * @throws NullPointerException + * if a {@code null} key is not supported. + */ + default String getString(final String key) { + return getProperty(key).toString(); + } + + /** + * ManagerKey getManagerKey + * + * @return The offset manager key for this entry. + */ + OffsetManagerKey getManagerKey(); + + /** + * Gets the Kafka topic for this entry. + * + * @return The Kafka topic for this entry. + */ + String getTopic(); + + /** + * Gets the Kafka partition for this entry. + * + * @return The Kafka partition for this entry. + */ + int getPartition(); + + /** + * Gets the number of records to skip to get to this record. This is the same as the zero-based index of this + * record if all records were in an array. + * + * @return The number of records to skip to get to this record. + */ + default long skipRecords() { + return 0; + } + + /** + * Increments the record count. + */ + void incrementRecordCount(); + } + + /** + * The OffsetManager Key. Must override hashCode() and equals(). + */ + @FunctionalInterface + public interface OffsetManagerKey { + /** + * gets the partition map used by Kafka to identify this Offset entry. + * + * @return The partition map used by Kafka to identify this Offset entry. + */ + Map getPartitionMap(); + } +} diff --git a/commons/src/test/java/io/aiven/kafka/connect/common/OffsetManagerTest.java b/commons/src/test/java/io/aiven/kafka/connect/common/OffsetManagerTest.java new file mode 100644 index 000000000..66ac323a5 --- /dev/null +++ b/commons/src/test/java/io/aiven/kafka/connect/common/OffsetManagerTest.java @@ -0,0 +1,224 @@ +/* + * Copyright 2024 Aiven Oy + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.aiven.kafka.connect.common; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.kafka.connect.source.SourceTaskContext; +import org.apache.kafka.connect.storage.OffsetStorageReader; + +import com.google.common.base.Objects; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +final class OffsetManagerTest { + + private SourceTaskContext sourceTaskContext; + + private OffsetStorageReader offsetStorageReader; + + private OffsetManager offsetManager; + @BeforeEach + void setup() { + offsetStorageReader = mock(OffsetStorageReader.class); + sourceTaskContext = mock(SourceTaskContext.class); + when(sourceTaskContext.offsetStorageReader()).thenReturn(offsetStorageReader); + offsetManager = new OffsetManager<>(sourceTaskContext); + } + + @Test + void testNewEntryWithDataFromContext() { + final Map partitionKey = new HashMap<>(); + partitionKey.put("segment1", "topic1"); + partitionKey.put("segment2", "a value"); + partitionKey.put("segment3", "something else"); + final Map offsetValue = new HashMap<>(partitionKey); + offsetValue.put("object_key_file", 5L); + when(offsetStorageReader.offset(partitionKey)).thenReturn(offsetValue); + + final Optional result = offsetManager.getEntry(() -> partitionKey, + TestingOffsetManagerEntry::new); + assertThat(result).isPresent(); + assertThat(result.get().data).isEqualTo(offsetValue); + } + + @Test + void testNewEntryWithoutDataFromContext() { + final Map partitionKey = new HashMap<>(); + partitionKey.put("segment1", "topic1"); + partitionKey.put("segment2", "a value"); + partitionKey.put("segment3", "something else"); + when(offsetStorageReader.offset(partitionKey)).thenReturn(new HashMap<>()); + + final Optional result = offsetManager.getEntry(() -> partitionKey, + TestingOffsetManagerEntry::new); + assertThat(result).isNotPresent(); + } + + @Test + void testUpdateCurrentEntry() { + final TestingOffsetManagerEntry offsetEntry = new TestingOffsetManagerEntry("bucket", "topic1", "thing"); + + final ConcurrentHashMap, Map> offsets = new ConcurrentHashMap<>(); + offsets.put(offsetEntry.getManagerKey().getPartitionMap(), offsetEntry.getProperties()); + + offsetManager = new OffsetManager<>(sourceTaskContext, offsets); + offsetEntry.setProperty("MyProperty", "WOW"); + + offsetManager.updateCurrentOffsets(offsetEntry); + + final Optional result = offsetManager.getEntry(offsetEntry.getManagerKey(), + TestingOffsetManagerEntry::new); + assertThat(result).isPresent(); + assertThat(result.get().getProperty("MyProperty")).isEqualTo("WOW"); + assertThat(result.get().getProperties()).isEqualTo(offsetEntry.getProperties()); + } + + @Test + void testUpdateNonExistentEntry() { + final TestingOffsetManagerEntry offsetEntry = new TestingOffsetManagerEntry("bucket", "topic1", "0"); + offsetEntry.setProperty("Random-property", "random value"); + offsetManager.updateCurrentOffsets(offsetEntry); + + final Optional result = offsetManager.getEntry(offsetEntry.getManagerKey(), + offsetEntry::fromProperties); + assertThat(result).isPresent(); + assertThat(result.get().getProperties()).isEqualTo(offsetEntry.getProperties()); + } + + @Test + void updateCurrentOffsetsDataNotLost() { + final TestingOffsetManagerEntry offsetEntry = new TestingOffsetManagerEntry("bucket", "topic1", "0"); + offsetEntry.setProperty("test", "WOW"); + offsetManager.updateCurrentOffsets(offsetEntry); + + final TestingOffsetManagerEntry offsetEntry2 = new TestingOffsetManagerEntry("bucket", "topic1", "0"); + offsetEntry2.setProperty("test2", "a thing"); + offsetManager.updateCurrentOffsets(offsetEntry2); + + final Optional result = offsetManager.getEntry(offsetEntry.getManagerKey(), + offsetEntry::fromProperties); + assertThat(result).isPresent(); + assertThat(result.get().getProperty("test")).isEqualTo("WOW"); + assertThat(result.get().getProperty("test2")).isEqualTo("a thing"); + } + + @SuppressWarnings("PMD.TestClassWithoutTestCases") // TODO figure out why this fails. + public static class TestingOffsetManagerEntry // NOPMD the above suppress warnings does not work. + implements + OffsetManager.OffsetManagerEntry { + public Map data; + + public int recordCount; + + public TestingOffsetManagerEntry(final String one, final String two, final String three) { + this(); + data.put("segment1", one); + data.put("segment2", two); + data.put("segment3", three); + } + + public TestingOffsetManagerEntry() { + data = new HashMap<>(); + data.put("segment1", "The First Segment"); + data.put("segment2", "The Second Segment"); + data.put("segment3", "The Third Segment"); + } + + public TestingOffsetManagerEntry(final Map properties) { + this(); + data.putAll(properties); + } + + @Override + public TestingOffsetManagerEntry fromProperties(final Map properties) { + return new TestingOffsetManagerEntry(properties); + } + + @Override + public Map getProperties() { + return data; + } + + @Override + public Object getProperty(final String key) { + return data.get(key); + } + + @Override + public void setProperty(final String key, final Object value) { + data.put(key, value); + } + + @Override + public OffsetManager.OffsetManagerKey getManagerKey() { + return () -> Map.of("segment1", data.get("segment1"), "segment2", data.get("segment2"), "segment3", + data.get("segment3")); + } + + @Override + public String getTopic() { + return getProperty("topic").toString(); + } + + @Override + public int getPartition() { + final Object value = getProperty("partition"); + return value instanceof Integer ? (Integer) value : 0; + } + + @Override + public void incrementRecordCount() { + recordCount++; + } + + @Override + public boolean equals(final Object other) { + if (other instanceof TestingOffsetManagerEntry) { + return this.compareTo((TestingOffsetManagerEntry) other) == 0; + } + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode(getProperty("segment1"), getProperty("segment2"), getProperty("segment3")); + } + + @Override + public int compareTo(final TestingOffsetManagerEntry other) { + if (other == this) { // NOPMD + return 0; + } + int result = ((String) getProperty("segment1")).compareTo((String) other.getProperty("segment1")); + if (result == 0) { + result = ((String) getProperty("segment2")).compareTo((String) other.getProperty("segment2")); + if (result == 0) { + result = ((String) getProperty("segment3")).compareTo((String) other.getProperty("segment3")); + } + } + return result; + } + } +} diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntry.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntry.java new file mode 100644 index 000000000..39a8af644 --- /dev/null +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntry.java @@ -0,0 +1,227 @@ +/* + * Copyright 2024 Aiven Oy + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.aiven.kafka.connect.s3.source.utils; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import io.aiven.kafka.connect.common.OffsetManager; + +import com.google.common.base.Objects; + +public final class S3OffsetManagerEntry implements OffsetManager.OffsetManagerEntry { + + // package private statics for testing. + // TODO make this package private after values in S3SourceTask are no longer needed + public static final String BUCKET = "bucket"; + public static final String OBJECT_KEY = "objectKey"; + public static final String TOPIC = "topic"; + public static final String PARTITION = "partition"; + static final String RECORD_COUNT = "recordCount"; + + /** + * THe list of Keys that may not be set via {@link #setProperty(String, Object)}. + */ + static final List RESTRICTED_KEYS = List.of(BUCKET, OBJECT_KEY, TOPIC, PARTITION, RECORD_COUNT); + /** The data map that stores all the values */ + private final Map data; + /** THe record count for the data map. Extracted here because it is used/updated frequently during processing */ + private long recordCount; + + /** + * Construct the S3OffsetManagerEntry. + * + * @param bucket + * the bucket we are using. + * @param s3ObjectKey + * the S3Object key. + * @param topic + * The topic we are using. + * @param partition + * the partition we are using. + */ + public S3OffsetManagerEntry(final String bucket, final String s3ObjectKey, final String topic, + final Integer partition) { + data = new HashMap<>(); + data.put(BUCKET, bucket); + data.put(OBJECT_KEY, s3ObjectKey); + data.put(TOPIC, topic); + data.put(PARTITION, partition); + } + + /** + * Wraps an existing property map as an S3OffsetManagerEntry. Creates a copy of the map for its internal use. + * + * @param properties + * the map of properties to wrap. + * @return an S3OffsetManagerEntry. + * @throws IllegalArgumentException + * if all the required fields are not present. + */ + public static S3OffsetManagerEntry wrap(final Map properties) { + if (properties == null) { + return null; + } + final Map ourProperties = new HashMap<>(properties); + long recordCount = 0; + final Object recordCountProperty = ourProperties.computeIfAbsent(RECORD_COUNT, s -> 0L); + if (recordCountProperty instanceof Number) { + recordCount = ((Number) recordCountProperty).longValue(); + } + final S3OffsetManagerEntry result = new S3OffsetManagerEntry(ourProperties); + result.recordCount = recordCount; + return result; + } + + /** + * Constructs an OffsetManagerEntry from an existing map. Used to reconstitute previously serialized + * S3OffsetManagerEntries. used by {@link #fromProperties(Map)} + * + * @param properties + * the property map. + */ + private S3OffsetManagerEntry(final Map properties) { + data = new HashMap<>(properties); + for (final String field : RESTRICTED_KEYS) { + if (data.get(field) == null) { + throw new IllegalArgumentException("Missing '" + field + "' property"); + } + } + } + + /** + * Creates an S3OffsetManagerEntry. Will return {@code null} if properties is {@code null}. + * + * @param properties + * the properties to wrap. May be {@code null}. + * @return an S3OffsetManagerEntry. + * @throws IllegalArgumentException + * if one of the {@link #RESTRICTED_KEYS} is missing. + */ + @Override + public S3OffsetManagerEntry fromProperties(final Map properties) { + return wrap(properties); + } + + @Override + public Object getProperty(final String key) { + if (RECORD_COUNT.equals(key)) { + return recordCount; + } + return data.get(key); + } + + @Override + public void setProperty(final String property, final Object value) { + if (RESTRICTED_KEYS.contains(property)) { + throw new IllegalArgumentException( + String.format("'%s' is a restricted key and may not be set using setProperty()", property)); + } + data.put(property, value); + } + + @Override + public void incrementRecordCount() { + recordCount++; + } + + /** + * Gets the umber of records extracted from data returned from S3. + * + * @return the umber of records extracted from data returned from S3. + */ + public long getRecordCount() { + return recordCount; + } + + /** + * Gets the S3Object key for the current object. + * + * @return the S3ObjectKey. + */ + public String getKey() { + return (String) data.get(OBJECT_KEY); + } + + @Override + public int getPartition() { + return getInt(PARTITION); + } + + @Override + public String getTopic() { + return getString(TOPIC); + } + + /** + * Gets the S3 bucket for the current object. + * + * @return the S3 Bucket for the current object. + */ + public String getBucket() { + return getString(BUCKET); + } + /** + * Creates a new offset map. No defensive copy is necessary. + * + * @return a new map of properties and values. + */ + @Override + public Map getProperties() { + final Map result = new HashMap<>(data); + result.put(RECORD_COUNT, recordCount); + return result; + } + /** + * Returns the OffsetManagerKey for this Entry. + * + * @return the OffsetManagerKey for this Entry. + */ + @Override + public OffsetManager.OffsetManagerKey getManagerKey() { + return () -> Map.of(BUCKET, data.get(BUCKET), OBJECT_KEY, data.get(OBJECT_KEY)); + } + + @Override + public boolean equals(final Object other) { + if (other instanceof S3OffsetManagerEntry) { + return compareTo((S3OffsetManagerEntry) other) == 0; + } + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode(getBucket(), getTopic(), getPartition()); + } + + @Override + public int compareTo(final S3OffsetManagerEntry other) { + if (this == other) { // NOPMD comparing instance + return 0; + } + int result = ((String) getProperty(BUCKET)).compareTo((String) other.getProperty(BUCKET)); + if (result == 0) { + result = getKey().compareTo(other.getKey()); + if (result == 0) { + result = Long.compare(getRecordCount(), other.getRecordCount()); + } + } + return result; + } +} diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java new file mode 100644 index 000000000..0a4c31a1c --- /dev/null +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java @@ -0,0 +1,148 @@ +/* + * Copyright 2024 Aiven Oy + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.aiven.kafka.connect.s3.source.utils; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import org.apache.kafka.connect.source.SourceTaskContext; +import org.apache.kafka.connect.storage.OffsetStorageReader; + +import io.aiven.kafka.connect.common.OffsetManager; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +final class S3OffsetManagerEntryTest { + + static final String TEST_BUCKET = "test-bucket"; + + static final String TOPIC = "TOPIC1"; + + static final int PARTITION = 1; + + static final String OBJECT_KEY = "object_key"; + + private SourceTaskContext sourceTaskContext; + + private OffsetManager offsetManager; + + private OffsetStorageReader offsetStorageReader; + + @BeforeEach + public void setUp() { + offsetStorageReader = mock(OffsetStorageReader.class); + sourceTaskContext = mock(SourceTaskContext.class); + when(sourceTaskContext.offsetStorageReader()).thenReturn(offsetStorageReader); + offsetManager = new OffsetManager<>(sourceTaskContext); + } + + private Map createPartitionMap() { + final Map partitionKey = new HashMap<>(); + partitionKey.put(S3OffsetManagerEntry.TOPIC, TOPIC); + partitionKey.put(S3OffsetManagerEntry.PARTITION, PARTITION); + partitionKey.put(S3OffsetManagerEntry.BUCKET, TEST_BUCKET); + partitionKey.put(S3OffsetManagerEntry.OBJECT_KEY, OBJECT_KEY); + return partitionKey; + } + + public static S3OffsetManagerEntry newEntry() { + return new S3OffsetManagerEntry(TEST_BUCKET, OBJECT_KEY, TOPIC, PARTITION); + } + + @Test + void testGetEntry() { + final Map storedData = new HashMap<>(); + storedData.putAll(createPartitionMap()); + storedData.put("random_entry", 5L); + + when(offsetStorageReader.offset(any())).thenReturn(storedData); + + final S3OffsetManagerEntry keyEntry = newEntry(); + final Optional entry = offsetManager.getEntry(keyEntry.getManagerKey(), + keyEntry::fromProperties); + assertThat(entry).isPresent(); + assertThat(entry.get().getPartition()).isEqualTo(PARTITION); + assertThat(entry.get().getRecordCount()).isEqualTo(0); + assertThat(entry.get().getTopic()).isEqualTo(TOPIC); + assertThat(entry.get().getBucket()).isEqualTo(TEST_BUCKET); + assertThat(entry.get().getProperty("random_entry")).isEqualTo(5L); + verify(sourceTaskContext, times(1)).offsetStorageReader(); + + // verify second read reads from local data + + final Optional entry2 = offsetManager.getEntry(entry.get().getManagerKey(), + entry.get()::fromProperties); + assertThat(entry2).isPresent(); + assertThat(entry2.get().getPartition()).isEqualTo(PARTITION); + assertThat(entry2.get().getRecordCount()).isEqualTo(0); + assertThat(entry2.get().getTopic()).isEqualTo(TOPIC); + assertThat(entry2.get().getBucket()).isEqualTo(TEST_BUCKET); + assertThat(entry2.get().getProperty("random_entry")).isEqualTo(5L); + verify(sourceTaskContext, times(1)).offsetStorageReader(); + } + + @Test + void testUpdate() { + final S3OffsetManagerEntry entry = new S3OffsetManagerEntry(TEST_BUCKET, OBJECT_KEY, TOPIC, PARTITION); + assertThat(entry.getRecordCount()).isEqualTo(0L); + assertThat(entry.getProperty("random_entry")).isNull(); + + offsetManager.updateCurrentOffsets(entry); + + entry.setProperty("random_entry", 5L); + entry.incrementRecordCount(); + assertThat(entry.getRecordCount()).isEqualTo(1L); + + offsetManager.updateCurrentOffsets(entry); + + final Optional entry2 = offsetManager.getEntry(entry.getManagerKey(), + entry::fromProperties); + assertThat(entry2).isPresent(); + assertThat(entry2.get().getPartition()).isEqualTo(PARTITION); + assertThat(entry2.get().getRecordCount()).isEqualTo(1L); + assertThat(entry2.get().getTopic()).isEqualTo(TOPIC); + assertThat(entry2.get().getBucket()).isEqualTo(TEST_BUCKET); + assertThat(entry2.get().getProperty("random_entry")).isEqualTo(5L); + verify(sourceTaskContext, times(0)).offsetStorageReader(); + } + + @Test + void testFromProperties() { + final S3OffsetManagerEntry entry = new S3OffsetManagerEntry(TEST_BUCKET, OBJECT_KEY, TOPIC, PARTITION); + assertThat(entry.getRecordCount()).isEqualTo(0L); + assertThat(entry.getProperty("random_entry")).isNull(); + + entry.setProperty("random_entry", 5L); + entry.incrementRecordCount(); + assertThat(entry.getRecordCount()).isEqualTo(1L); + assertThat(entry.getProperty("random_entry")).isEqualTo(5L); + + final S3OffsetManagerEntry other = entry.fromProperties(entry.getProperties()); + assertThat(other.getRecordCount()).isEqualTo(1L); + assertThat(other.getProperty("random_entry")).isEqualTo(5L); + + } +} From 2637f647e4b9012debea96c0b6058d2f4aef1c03 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Thu, 9 Jan 2025 13:31:32 +0000 Subject: [PATCH 32/44] fixed merge issues --- .../common/{ => source}/OffsetManager.java | 2 +- .../{ => source}/OffsetManagerTest.java | 2 +- .../connect/s3/source/AwsIntegrationTest.java | 2 +- .../connect/s3/source/IntegrationBase.java | 4 +- .../kafka/connect/s3/source/S3SourceTask.java | 2 +- .../connect/s3/source/utils/ConnectUtils.java | 39 ---------- .../s3/source/utils/S3OffsetManagerEntry.java | 71 ++++++++----------- .../s3/source/utils/SourceRecordIterator.java | 43 +++++------ .../connect/s3/source/S3SourceTaskTest.java | 2 +- .../utils/S3OffsetManagerEntryTest.java | 2 +- .../utils/SourceRecordIteratorTest.java | 30 +++++--- 11 files changed, 81 insertions(+), 118 deletions(-) rename commons/src/main/java/io/aiven/kafka/connect/common/{ => source}/OffsetManager.java (99%) rename commons/src/test/java/io/aiven/kafka/connect/common/{ => source}/OffsetManagerTest.java (99%) delete mode 100644 s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/ConnectUtils.java diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/OffsetManager.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java similarity index 99% rename from commons/src/main/java/io/aiven/kafka/connect/common/OffsetManager.java rename to commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java index ef0078e4a..0f607c801 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/OffsetManager.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package io.aiven.kafka.connect.common; +package io.aiven.kafka.connect.common.source; import java.util.HashMap; import java.util.Map; diff --git a/commons/src/test/java/io/aiven/kafka/connect/common/OffsetManagerTest.java b/commons/src/test/java/io/aiven/kafka/connect/common/source/OffsetManagerTest.java similarity index 99% rename from commons/src/test/java/io/aiven/kafka/connect/common/OffsetManagerTest.java rename to commons/src/test/java/io/aiven/kafka/connect/common/source/OffsetManagerTest.java index 66ac323a5..169750094 100644 --- a/commons/src/test/java/io/aiven/kafka/connect/common/OffsetManagerTest.java +++ b/commons/src/test/java/io/aiven/kafka/connect/common/source/OffsetManagerTest.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package io.aiven.kafka.connect.common; +package io.aiven.kafka.connect.common.source; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.mock; diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java index fc6e1d40b..77e6257e7 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java @@ -45,7 +45,7 @@ import org.apache.kafka.connect.source.SourceTaskContext; import org.apache.kafka.connect.storage.OffsetStorageReader; -import io.aiven.kafka.connect.common.OffsetManager; +import io.aiven.kafka.connect.common.source.OffsetManager; import io.aiven.kafka.connect.common.source.input.InputFormat; import io.aiven.kafka.connect.common.source.input.TransformerFactory; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java index 04b015f4e..073431541 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java @@ -51,6 +51,8 @@ import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.connect.json.JsonDeserializer; +import io.aiven.kafka.connect.common.source.OffsetManager; + import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; @@ -131,7 +133,7 @@ default void writeToS3WithKey(final String objectKey, final byte[] testDataBytes * @param partitionId * the partition id. * @return the key prefixed by {@link io.aiven.kafka.connect.s3.source.utils.S3OffsetManagerEntry#OBJECT_KEY} and - * {@link io.aiven.kafka.connect.common.OffsetManager} + * {@link OffsetManager} */ default String writeToS3(final String topicName, final byte[] testDataBytes, final String partitionId) { final String objectKey = org.apache.commons.lang3.StringUtils.defaultIfBlank(getS3Prefix(), "") + topicName diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index 8a4f9f4cf..269eb36ca 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -24,9 +24,9 @@ import org.apache.kafka.connect.source.SourceRecord; -import io.aiven.kafka.connect.common.OffsetManager; import io.aiven.kafka.connect.common.config.SourceCommonConfig; import io.aiven.kafka.connect.common.source.AbstractSourceTask; +import io.aiven.kafka.connect.common.source.OffsetManager; import io.aiven.kafka.connect.common.source.input.Transformer; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; import io.aiven.kafka.connect.s3.source.utils.AWSV2SourceClient; diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/ConnectUtils.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/ConnectUtils.java deleted file mode 100644 index 6c60bb8ed..000000000 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/ConnectUtils.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Copyright 2024 Aiven Oy - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.aiven.kafka.connect.s3.source.utils; - -import static io.aiven.kafka.connect.s3.source.S3SourceTask.BUCKET; -import static io.aiven.kafka.connect.s3.source.S3SourceTask.PARTITION; -import static io.aiven.kafka.connect.s3.source.S3SourceTask.TOPIC; - -import java.util.HashMap; -import java.util.Map; - -final public class ConnectUtils { - - private ConnectUtils() { - // hidden - } - public static Map getPartitionMap(final String topicName, final Integer defaultPartitionId, - final String bucketName) { - final Map partitionMap = new HashMap<>(); - partitionMap.put(BUCKET, bucketName); - partitionMap.put(TOPIC, topicName); - partitionMap.put(PARTITION, defaultPartitionId); - return partitionMap; - } -} diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntry.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntry.java index 39a8af644..dcc3d4e3d 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntry.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntry.java @@ -20,7 +20,7 @@ import java.util.List; import java.util.Map; -import io.aiven.kafka.connect.common.OffsetManager; +import io.aiven.kafka.connect.common.source.OffsetManager; import com.google.common.base.Objects; @@ -32,17 +32,22 @@ public final class S3OffsetManagerEntry implements OffsetManager.OffsetManagerEn public static final String OBJECT_KEY = "objectKey"; public static final String TOPIC = "topic"; public static final String PARTITION = "partition"; - static final String RECORD_COUNT = "recordCount"; + public static final String RECORD_COUNT = "recordCount"; /** * THe list of Keys that may not be set via {@link #setProperty(String, Object)}. */ - static final List RESTRICTED_KEYS = List.of(BUCKET, OBJECT_KEY, TOPIC, PARTITION, RECORD_COUNT); + static final List RESTRICTED_KEYS = List.of(RECORD_COUNT); /** The data map that stores all the values */ private final Map data; /** THe record count for the data map. Extracted here because it is used/updated frequently during processing */ private long recordCount; + private final String bucket; + private final String objectKey; + private final String topic; + private final Integer partition; + /** * Construct the S3OffsetManagerEntry. * @@ -57,35 +62,11 @@ public final class S3OffsetManagerEntry implements OffsetManager.OffsetManagerEn */ public S3OffsetManagerEntry(final String bucket, final String s3ObjectKey, final String topic, final Integer partition) { + this.bucket = bucket; + this.objectKey = s3ObjectKey; + this.topic = topic; + this.partition = partition; data = new HashMap<>(); - data.put(BUCKET, bucket); - data.put(OBJECT_KEY, s3ObjectKey); - data.put(TOPIC, topic); - data.put(PARTITION, partition); - } - - /** - * Wraps an existing property map as an S3OffsetManagerEntry. Creates a copy of the map for its internal use. - * - * @param properties - * the map of properties to wrap. - * @return an S3OffsetManagerEntry. - * @throws IllegalArgumentException - * if all the required fields are not present. - */ - public static S3OffsetManagerEntry wrap(final Map properties) { - if (properties == null) { - return null; - } - final Map ourProperties = new HashMap<>(properties); - long recordCount = 0; - final Object recordCountProperty = ourProperties.computeIfAbsent(RECORD_COUNT, s -> 0L); - if (recordCountProperty instanceof Number) { - recordCount = ((Number) recordCountProperty).longValue(); - } - final S3OffsetManagerEntry result = new S3OffsetManagerEntry(ourProperties); - result.recordCount = recordCount; - return result; } /** @@ -95,12 +76,13 @@ public static S3OffsetManagerEntry wrap(final Map properties) { * @param properties * the property map. */ - private S3OffsetManagerEntry(final Map properties) { - data = new HashMap<>(properties); - for (final String field : RESTRICTED_KEYS) { - if (data.get(field) == null) { - throw new IllegalArgumentException("Missing '" + field + "' property"); - } + private S3OffsetManagerEntry(final String bucket, final String s3ObjectKey, final String topic, + final Integer partition, final Map properties) { + this(bucket, s3ObjectKey, topic, partition); + data.putAll(properties); + final Object recordCountProperty = data.computeIfAbsent(RECORD_COUNT, s -> 0L); + if (recordCountProperty instanceof Number) { + recordCount = ((Number) recordCountProperty).longValue(); } } @@ -115,7 +97,10 @@ private S3OffsetManagerEntry(final Map properties) { */ @Override public S3OffsetManagerEntry fromProperties(final Map properties) { - return wrap(properties); + if (properties == null) { + return null; + } + return new S3OffsetManagerEntry(bucket, objectKey, topic, partition, properties); } @Override @@ -155,17 +140,17 @@ public long getRecordCount() { * @return the S3ObjectKey. */ public String getKey() { - return (String) data.get(OBJECT_KEY); + return objectKey; } @Override public int getPartition() { - return getInt(PARTITION); + return partition; } @Override public String getTopic() { - return getString(TOPIC); + return topic; } /** @@ -174,7 +159,7 @@ public String getTopic() { * @return the S3 Bucket for the current object. */ public String getBucket() { - return getString(BUCKET); + return bucket; } /** * Creates a new offset map. No defensive copy is necessary. @@ -194,7 +179,7 @@ public Map getProperties() { */ @Override public OffsetManager.OffsetManagerKey getManagerKey() { - return () -> Map.of(BUCKET, data.get(BUCKET), OBJECT_KEY, data.get(OBJECT_KEY)); + return () -> Map.of(BUCKET, bucket, OBJECT_KEY, objectKey); } @Override diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java index 97c229b21..ede1f9dd4 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java @@ -26,7 +26,7 @@ import org.apache.kafka.connect.data.SchemaAndValue; -import io.aiven.kafka.connect.common.OffsetManager; +import io.aiven.kafka.connect.common.source.OffsetManager; import io.aiven.kafka.connect.common.source.input.ByteArrayTransformer; import io.aiven.kafka.connect.common.source.input.Transformer; import io.aiven.kafka.connect.s3.source.config.S3SourceConfig; @@ -51,22 +51,19 @@ public final class SourceRecordIterator implements Iterator { /** The offset manager Entry we are working with */ private S3OffsetManagerEntry offsetManagerEntry; + /** The configuration for this S3 source */ private final S3SourceConfig s3SourceConfig; - + /** The transformer for the data conversions */ private final Transformer transformer; - // Once we decouple the S3Object from the Source Iterator we can change this to be the SourceApiClient - // At which point it will work for al our integrations. + /** The AWS client that provides the S3Objects */ private final AWSV2SourceClient sourceClient; - - private String topic; - private int partitionId; + /** The S3 bucket we are processing */ private final String bucketName; + /** The inner iterator to provides S3Object that have been filtered potentially had data extracted */ private final Iterator inner; - + /** The outer iterator that provides S3SourceRecords */ private Iterator outer; - private final Predicate fileNamePredicate; - public SourceRecordIterator(final S3SourceConfig s3SourceConfig, final OffsetManager offsetManager, final Transformer transformer, final AWSV2SourceClient sourceClient) { @@ -74,18 +71,21 @@ public SourceRecordIterator(final S3SourceConfig s3SourceConfig, this.s3SourceConfig = s3SourceConfig; this.offsetManager = offsetManager; this.bucketName = s3SourceConfig.getAwsS3BucketName(); - this.fileNamePredicate = buildPredicate(); - this.transformer = transformer; this.sourceClient = sourceClient; + final Predicate fileNamePredicate = buildFileNamePredicate(); - // call filters out bad file names and extracts topic/partition - inner = IteratorUtils.filteredIterator(sourceClient.getS3ObjectIterator(null), - s3Object -> this.fileNamePredicate.test(s3Object)); + // call filters out bad file names and creates the offsetManagerEntry. + inner = IteratorUtils.filteredIterator(sourceClient.getS3ObjectIterator(null), fileNamePredicate::test); outer = Collections.emptyIterator(); } - private Predicate buildPredicate() { + /** + * creates the file name predicate that matches file names and extracts the offsetManagerEntry data. + * + * @return a predicate to filter S3Objects. + */ + private Predicate buildFileNamePredicate() { return s3Object -> { final Matcher fileMatcher = FILE_DEFAULT_PATTERN.matcher(s3Object.key()); @@ -93,8 +93,6 @@ private Predicate buildPredicate() { if (fileMatcher.find()) { // TODO move this from the SourceRecordIterator so that we can decouple it from S3 and make it API // agnostic - topic = fileMatcher.group(PATTERN_TOPIC_KEY); - partitionId = Integer.parseInt(fileMatcher.group(PATTERN_PARTITION_KEY)); final S3OffsetManagerEntry keyEntry = new S3OffsetManagerEntry(bucketName, s3Object.key(), fileMatcher.group(PATTERN_TOPIC_KEY), Integer.parseInt(fileMatcher.group(PATTERN_PARTITION_KEY))); @@ -140,10 +138,12 @@ private Stream convert(final S3Object s3Object) { return Stream.empty(); } - final SchemaAndValue keyData = transformer.getKeyData(s3Object.key(), topic, s3SourceConfig); + final SchemaAndValue keyData = transformer.getKeyData(s3Object.key(), offsetManagerEntry.getTopic(), + s3SourceConfig); return transformer - .getRecords(sourceClient.getObject(s3Object.key()), topic, partitionId, s3SourceConfig, recordCount) + .getRecords(sourceClient.getObject(s3Object.key()), offsetManagerEntry.getTopic(), + offsetManagerEntry.getPartition(), s3SourceConfig, recordCount) .map(new Mapper(offsetManagerEntry, keyData)); } @@ -161,6 +161,9 @@ static class Mapper implements Function { private final SchemaAndValue keyData; public Mapper(final S3OffsetManagerEntry entry, final SchemaAndValue keyData) { + // TODO this is the point where the global S3OffsetManagerEntry becomes local and we can do a lookahead type + // operation within the Transformer + // to see if there are more records. this.entry = entry; this.keyData = keyData; } diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java index 0c51665c2..ddfccf113 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java @@ -352,7 +352,7 @@ public S3SourceRecord next() { stopWatch.start(); pollResult = s3SourceTask.poll(); stopWatch.stop(); - if (results.size() == 3) { + if (results.size() == lst.size()) { assertThat(pollResult).isNull(); } else { results.addAll(pollResult); diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java index 0a4c31a1c..9fdc5671f 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java @@ -30,7 +30,7 @@ import org.apache.kafka.connect.source.SourceTaskContext; import org.apache.kafka.connect.storage.OffsetStorageReader; -import io.aiven.kafka.connect.common.OffsetManager; +import io.aiven.kafka.connect.common.source.OffsetManager; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java index 5f24f75fd..6848c92e6 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java @@ -16,6 +16,7 @@ package io.aiven.kafka.connect.s3.source.utils; +import static io.aiven.kafka.connect.s3.source.utils.S3OffsetManagerEntry.RECORD_COUNT; import static io.aiven.kafka.connect.s3.source.utils.SourceRecordIterator.BYTES_TRANSFORMATION_NUM_OF_RECS; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.any; @@ -34,13 +35,15 @@ import java.util.Arrays; import java.util.Collections; import java.util.Iterator; +import java.util.Map; import java.util.Optional; import java.util.stream.Stream; import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.source.SourceTaskContext; +import org.apache.kafka.connect.storage.OffsetStorageReader; -import io.aiven.kafka.connect.common.OffsetManager; -import io.aiven.kafka.connect.common.source.input.AvroTransformer; +import io.aiven.kafka.connect.common.source.OffsetManager; import io.aiven.kafka.connect.common.source.input.ByteArrayTransformer; import io.aiven.kafka.connect.common.source.input.InputFormat; import io.aiven.kafka.connect.common.source.input.Transformer; @@ -134,25 +137,34 @@ void testIteratorProcessesS3ObjectsForByteArrayTransformer() throws Exception { // With AvroTransformer try (InputStream inputStream = new ByteArrayInputStream("Hello World".getBytes(StandardCharsets.UTF_8))) { + when(mockSourceApiClient.getObject(key)).thenReturn(() -> inputStream); when(mockSourceApiClient.getS3ObjectIterator(any())).thenReturn(Arrays.asList(s3Object).iterator()); - mockTransformer = mock(AvroTransformer.class); when(mockSourceApiClient.getListOfObjectKeys(any())) .thenReturn(Collections.singletonList(key).listIterator()); - when(mockS3OffsetManagerEntry.getRecordCount()).thenReturn(BYTES_TRANSFORMATION_NUM_OF_RECS); + final OffsetStorageReader offsetStorageReader = mock(OffsetStorageReader.class); + when(offsetStorageReader.offset(any(Map.class))).thenReturn(Map.of(RECORD_COUNT, 1)); + + final SourceTaskContext context = mock(SourceTaskContext.class); + when(context.offsetStorageReader()).thenReturn(offsetStorageReader); - when(mockTransformer.getKeyData(anyString(), anyString(), any())).thenReturn(SchemaAndValue.NULL); + mockOffsetManager = new OffsetManager(context); + + mockTransformer = mock(Transformer.class); + final SchemaAndValue schemaKey = new SchemaAndValue(null, "KEY"); + final SchemaAndValue schemaValue = new SchemaAndValue(null, "VALUE"); + when(mockTransformer.getKeyData(anyString(), anyString(), any())).thenReturn(schemaKey); when(mockTransformer.getRecords(any(), anyString(), anyInt(), any(), anyLong())) - .thenReturn(Arrays.asList(SchemaAndValue.NULL).stream()); + .thenReturn(Arrays.asList(schemaValue).stream()); final Iterator iterator = new SourceRecordIterator(mockConfig, mockOffsetManager, mockTransformer, mockSourceApiClient); assertThat(iterator.hasNext()).isTrue(); - iterator.next(); - + final S3SourceRecord record = iterator.next(); + assertThat(record.getValue().value()).isEqualTo("VALUE"); + assertThat(record.getOffsetManagerEntry().getRecordCount()).isEqualTo(2); verify(mockTransformer, times(1)).getRecords(any(), anyString(), anyInt(), any(), anyLong()); } } - } From 551058614951c8de302137330efb52d869ad96bc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Thu, 9 Jan 2025 15:13:21 +0000 Subject: [PATCH 33/44] Added handling of processed Offsets --- .../kafka/connect/common/source/OffsetManager.java | 13 +++++++++++++ .../aiven/kafka/connect/s3/source/S3SourceTask.java | 1 + 2 files changed, 14 insertions(+) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java index 0f607c801..60916c27c 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java @@ -23,6 +23,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.function.Function; +import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceTaskContext; import org.slf4j.Logger; @@ -122,6 +123,18 @@ public void remove(final OffsetManagerKey key) { offsets.remove(key.getPartitionMap()); } + /** + * Removes the specified entry from the in memory table. Does not impact the records stored in the + * {@link SourceTaskContext}. + * + * @param sourceRecord + * the SourceRecord that contains the key to be removed. + */ + public void remove(final SourceRecord sourceRecord) { + LOGGER.info("Removing: {}", sourceRecord.sourcePartition()); + offsets.remove(sourceRecord.sourcePartition()); + } + /** * The definition of an entry in the OffsetManager. */ diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index 269eb36ca..dc1c9a191 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -140,6 +140,7 @@ public void commitRecord(final SourceRecord record) { if (LOGGER.isDebugEnabled()) { LOGGER.debug("Committed individual record {} committed", (Map) record.sourceOffset()); } + offsetManager.remove(record); } /** From 3e948ba07ed690674e974a77465314541ec73083 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Fri, 10 Jan 2025 14:09:13 +0000 Subject: [PATCH 34/44] updates as per review --- .../kafka/connect/common/source/OffsetManager.java | 10 +++++----- .../aiven/kafka/connect/s3/source/IntegrationBase.java | 8 ++++++-- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java index 60916c27c..180b79206 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java @@ -79,14 +79,14 @@ protected OffsetManager(final SourceTaskContext context, * @return the entry. */ public Optional getEntry(final OffsetManagerKey key, final Function, E> creator) { - LOGGER.info("getEntry: {}", key.getPartitionMap()); + LOGGER.debug("getEntry: {}", key.getPartitionMap()); final Map data = offsets.compute(key.getPartitionMap(), (k, v) -> { if (v == null) { final Map kafkaData = context.offsetStorageReader().offset(key.getPartitionMap()); - LOGGER.info("Context stored offset map {}", kafkaData); + LOGGER.debug("Context stored offset map {}", kafkaData); return kafkaData == null || kafkaData.isEmpty() ? null : kafkaData; } else { - LOGGER.info("Previously stored offset map {}", v); + LOGGER.debug("Previously stored offset map {}", v); return v; } }); @@ -119,7 +119,7 @@ public void updateCurrentOffsets(final E entry) { * the key for the entry to remove. */ public void remove(final OffsetManagerKey key) { - LOGGER.info("Removing: {}", key.getPartitionMap()); + LOGGER.debug("Removing: {}", key.getPartitionMap()); offsets.remove(key.getPartitionMap()); } @@ -131,7 +131,7 @@ public void remove(final OffsetManagerKey key) { * the SourceRecord that contains the key to be removed. */ public void remove(final SourceRecord sourceRecord) { - LOGGER.info("Removing: {}", sourceRecord.sourcePartition()); + LOGGER.debug("Removing: {}", sourceRecord.sourcePartition()); offsets.remove(sourceRecord.sourcePartition()); } diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java index 073431541..4844b40cc 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java @@ -263,10 +263,14 @@ static Map consumeOffsetMessages(KafkaConsumer c final Map messages = new HashMap<>(); final ConsumerRecords records = consumer.poll(Duration.ofSeconds(1)); for (final ConsumerRecord record : records) { - Map offsetRec = OBJECT_MAPPER.readValue(record.value(), new TypeReference<>() { // NOPMD + final Map offsetRec = OBJECT_MAPPER.readValue(record.value(), new TypeReference<>() { // NOPMD }); - messages.put((String) offsetRec.get(OBJECT_KEY), offsetRec.get(RECORD_COUNT)); + final List key = OBJECT_MAPPER.readValue(record.key(), new TypeReference<>() { // NOPMD + }); + final Map keyDetails = (Map) key.get(1); + messages.put((String) keyDetails.get(OBJECT_KEY), offsetRec.get(RECORD_COUNT)); } + return messages; } From 81d46a483bd6a8a6b1fb792cc9859dff96faba6c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Fri, 10 Jan 2025 16:40:58 +0000 Subject: [PATCH 35/44] removed unused methods --- .../common/source/AbstractSourceTask.java | 2 +- .../connect/common/source/OffsetManager.java | 10 ---- .../common/source/AbstractSourceTaskTest.java | 2 +- .../common/source/OffsetManagerTest.java | 54 +------------------ .../connect/s3/source/IntegrationBase.java | 1 + .../kafka/connect/s3/source/S3SourceTask.java | 1 - .../utils/S3OffsetManagerEntryTest.java | 26 +-------- 7 files changed, 6 insertions(+), 90 deletions(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java index f55257f46..c8baa95b1 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/AbstractSourceTask.java @@ -163,7 +163,7 @@ private boolean tryAdd(final List results, final Iterator offsetManager; + @BeforeEach void setup() { offsetStorageReader = mock(OffsetStorageReader.class); - sourceTaskContext = mock(SourceTaskContext.class); + final SourceTaskContext sourceTaskContext = mock(SourceTaskContext.class); when(sourceTaskContext.offsetStorageReader()).thenReturn(offsetStorageReader); offsetManager = new OffsetManager<>(sourceTaskContext); } @@ -76,54 +74,6 @@ void testNewEntryWithoutDataFromContext() { assertThat(result).isNotPresent(); } - @Test - void testUpdateCurrentEntry() { - final TestingOffsetManagerEntry offsetEntry = new TestingOffsetManagerEntry("bucket", "topic1", "thing"); - - final ConcurrentHashMap, Map> offsets = new ConcurrentHashMap<>(); - offsets.put(offsetEntry.getManagerKey().getPartitionMap(), offsetEntry.getProperties()); - - offsetManager = new OffsetManager<>(sourceTaskContext, offsets); - offsetEntry.setProperty("MyProperty", "WOW"); - - offsetManager.updateCurrentOffsets(offsetEntry); - - final Optional result = offsetManager.getEntry(offsetEntry.getManagerKey(), - TestingOffsetManagerEntry::new); - assertThat(result).isPresent(); - assertThat(result.get().getProperty("MyProperty")).isEqualTo("WOW"); - assertThat(result.get().getProperties()).isEqualTo(offsetEntry.getProperties()); - } - - @Test - void testUpdateNonExistentEntry() { - final TestingOffsetManagerEntry offsetEntry = new TestingOffsetManagerEntry("bucket", "topic1", "0"); - offsetEntry.setProperty("Random-property", "random value"); - offsetManager.updateCurrentOffsets(offsetEntry); - - final Optional result = offsetManager.getEntry(offsetEntry.getManagerKey(), - offsetEntry::fromProperties); - assertThat(result).isPresent(); - assertThat(result.get().getProperties()).isEqualTo(offsetEntry.getProperties()); - } - - @Test - void updateCurrentOffsetsDataNotLost() { - final TestingOffsetManagerEntry offsetEntry = new TestingOffsetManagerEntry("bucket", "topic1", "0"); - offsetEntry.setProperty("test", "WOW"); - offsetManager.updateCurrentOffsets(offsetEntry); - - final TestingOffsetManagerEntry offsetEntry2 = new TestingOffsetManagerEntry("bucket", "topic1", "0"); - offsetEntry2.setProperty("test2", "a thing"); - offsetManager.updateCurrentOffsets(offsetEntry2); - - final Optional result = offsetManager.getEntry(offsetEntry.getManagerKey(), - offsetEntry::fromProperties); - assertThat(result).isPresent(); - assertThat(result.get().getProperty("test")).isEqualTo("WOW"); - assertThat(result.get().getProperty("test2")).isEqualTo("a thing"); - } - @SuppressWarnings("PMD.TestClassWithoutTestCases") // TODO figure out why this fails. public static class TestingOffsetManagerEntry // NOPMD the above suppress warnings does not work. implements diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java index 4844b40cc..d391b9eb7 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java @@ -267,6 +267,7 @@ static Map consumeOffsetMessages(KafkaConsumer c }); final List key = OBJECT_MAPPER.readValue(record.key(), new TypeReference<>() { // NOPMD }); + // key.get(0) will return the name of the connector the commit is from. final Map keyDetails = (Map) key.get(1); messages.put((String) keyDetails.get(OBJECT_KEY), offsetRec.get(RECORD_COUNT)); } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index dc1c9a191..fb0ab4c45 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -111,7 +111,6 @@ public boolean hasNext() { public SourceRecord next() { final S3SourceRecord s3SourceRecord = s3SourceRecordIterator.next(); final S3OffsetManagerEntry entry = s3SourceRecord.getOffsetManagerEntry(); - offsetManager.updateCurrentOffsets(entry); return RecordProcessor.createSourceRecord(s3SourceRecord, s3SourceConfig, awsv2SourceClient, entry); } }; diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java index 9fdc5671f..94a51af7e 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java @@ -57,6 +57,7 @@ public void setUp() { sourceTaskContext = mock(SourceTaskContext.class); when(sourceTaskContext.offsetStorageReader()).thenReturn(offsetStorageReader); offsetManager = new OffsetManager<>(sourceTaskContext); + } private Map createPartitionMap() { @@ -104,31 +105,6 @@ void testGetEntry() { verify(sourceTaskContext, times(1)).offsetStorageReader(); } - @Test - void testUpdate() { - final S3OffsetManagerEntry entry = new S3OffsetManagerEntry(TEST_BUCKET, OBJECT_KEY, TOPIC, PARTITION); - assertThat(entry.getRecordCount()).isEqualTo(0L); - assertThat(entry.getProperty("random_entry")).isNull(); - - offsetManager.updateCurrentOffsets(entry); - - entry.setProperty("random_entry", 5L); - entry.incrementRecordCount(); - assertThat(entry.getRecordCount()).isEqualTo(1L); - - offsetManager.updateCurrentOffsets(entry); - - final Optional entry2 = offsetManager.getEntry(entry.getManagerKey(), - entry::fromProperties); - assertThat(entry2).isPresent(); - assertThat(entry2.get().getPartition()).isEqualTo(PARTITION); - assertThat(entry2.get().getRecordCount()).isEqualTo(1L); - assertThat(entry2.get().getTopic()).isEqualTo(TOPIC); - assertThat(entry2.get().getBucket()).isEqualTo(TEST_BUCKET); - assertThat(entry2.get().getProperty("random_entry")).isEqualTo(5L); - verify(sourceTaskContext, times(0)).offsetStorageReader(); - } - @Test void testFromProperties() { final S3OffsetManagerEntry entry = new S3OffsetManagerEntry(TEST_BUCKET, OBJECT_KEY, TOPIC, PARTITION); From 96d15232844bf9e66724e68560dfba5ae3b0602f Mon Sep 17 00:00:00 2001 From: Claude Warren Date: Tue, 14 Jan 2025 10:46:50 +0000 Subject: [PATCH 36/44] Update commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java Co-authored-by: Murali Basani --- .../io/aiven/kafka/connect/common/source/OffsetManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java index 6416e698b..71016347e 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java @@ -30,7 +30,7 @@ import org.slf4j.LoggerFactory; public class OffsetManager> { - /** The loger to write to */ + /** The logger to write to */ private static final Logger LOGGER = LoggerFactory.getLogger(OffsetManager.class); /** From e429d8aa0635b872d2b0c10fb7ea84488a04dcfe Mon Sep 17 00:00:00 2001 From: Claude Warren Date: Tue, 14 Jan 2025 10:48:09 +0000 Subject: [PATCH 37/44] Update commons/src/test/java/io/aiven/kafka/connect/common/source/OffsetManagerTest.java Co-authored-by: Murali Basani --- .../io/aiven/kafka/connect/common/source/OffsetManagerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/commons/src/test/java/io/aiven/kafka/connect/common/source/OffsetManagerTest.java b/commons/src/test/java/io/aiven/kafka/connect/common/source/OffsetManagerTest.java index fa1e3c37e..c05ac73e4 100644 --- a/commons/src/test/java/io/aiven/kafka/connect/common/source/OffsetManagerTest.java +++ b/commons/src/test/java/io/aiven/kafka/connect/common/source/OffsetManagerTest.java @@ -1,5 +1,5 @@ /* - * Copyright 2024 Aiven Oy + * Copyright 2025 Aiven Oy * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. From 1d4b5e439dbd9c986aff322437db491475d2b44f Mon Sep 17 00:00:00 2001 From: Claude Warren Date: Tue, 14 Jan 2025 10:48:44 +0000 Subject: [PATCH 38/44] Update s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntry.java Co-authored-by: Murali Basani --- .../kafka/connect/s3/source/utils/S3OffsetManagerEntry.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntry.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntry.java index dcc3d4e3d..a9868acfc 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntry.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntry.java @@ -1,5 +1,5 @@ /* - * Copyright 2024 Aiven Oy + * Copyright 2025 Aiven Oy * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. From e4187f5177e04d7779a37327bf226892efcf3e86 Mon Sep 17 00:00:00 2001 From: Claude Warren Date: Tue, 14 Jan 2025 10:49:47 +0000 Subject: [PATCH 39/44] Update s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java Co-authored-by: Murali Basani --- .../kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java index 94a51af7e..8c4920198 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java @@ -1,5 +1,5 @@ /* - * Copyright 2024 Aiven Oy + * Copyright 2025 Aiven Oy * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. From b8a64d3de555129fab8013005888e4c9f2cd6ddf Mon Sep 17 00:00:00 2001 From: Claude Warren Date: Tue, 14 Jan 2025 10:50:58 +0000 Subject: [PATCH 40/44] Update commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java Co-authored-by: Murali Basani --- .../io/aiven/kafka/connect/common/source/OffsetManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java index 71016347e..a24614642 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java @@ -1,5 +1,5 @@ /* - * Copyright 2024 Aiven Oy + * Copyright 2025 Aiven Oy * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. From cd1f25f70c58fedab8a937b439e7c587c5f5d9aa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Tue, 14 Jan 2025 11:15:50 +0000 Subject: [PATCH 41/44] fixed topicName and bucketName --- .../connect/s3/source/AwsIntegrationTest.java | 38 +++++++++---------- .../connect/s3/source/IntegrationBase.java | 14 +++---- .../connect/s3/source/IntegrationTest.java | 10 ++--- .../s3/source/utils/SourceRecordIterator.java | 6 +-- .../connect/s3/source/S3SourceTaskTest.java | 6 +-- 5 files changed, 37 insertions(+), 37 deletions(-) diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java index 77e6257e7..df5b79a67 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/AwsIntegrationTest.java @@ -134,8 +134,8 @@ private Map getConfig(final String topics, final int maxTasks) { */ @Test void sourceRecordIteratorBytesTest(final TestInfo testInfo) { - final var topicName = IntegrationBase.topicName(testInfo); - final Map configData = getConfig(topicName, 1); + final var topic = IntegrationBase.getTopic(testInfo); + final Map configData = getConfig(topic, 1); configData.put(INPUT_FORMAT_KEY, InputFormat.BYTES.getValue()); @@ -145,14 +145,14 @@ void sourceRecordIteratorBytesTest(final TestInfo testInfo) { final List offsetKeys = new ArrayList<>(); final List expectedKeys = new ArrayList<>(); // write 2 objects to s3 - expectedKeys.add(writeToS3(topicName, testData1.getBytes(StandardCharsets.UTF_8), "00000")); - expectedKeys.add(writeToS3(topicName, testData2.getBytes(StandardCharsets.UTF_8), "00000")); - expectedKeys.add(writeToS3(topicName, testData1.getBytes(StandardCharsets.UTF_8), "00001")); - expectedKeys.add(writeToS3(topicName, testData2.getBytes(StandardCharsets.UTF_8), "00001")); + expectedKeys.add(writeToS3(topic, testData1.getBytes(StandardCharsets.UTF_8), "00000")); + expectedKeys.add(writeToS3(topic, testData2.getBytes(StandardCharsets.UTF_8), "00000")); + expectedKeys.add(writeToS3(topic, testData1.getBytes(StandardCharsets.UTF_8), "00001")); + expectedKeys.add(writeToS3(topic, testData2.getBytes(StandardCharsets.UTF_8), "00001")); // we don't expext the empty one. offsetKeys.addAll(expectedKeys); - offsetKeys.add(writeToS3(topicName, new byte[0], "00003")); + offsetKeys.add(writeToS3(topic, new byte[0], "00003")); assertThat(testBucketAccessor.listObjects()).hasSize(5); @@ -181,9 +181,9 @@ void sourceRecordIteratorBytesTest(final TestInfo testInfo) { @Test void sourceRecordIteratorAvroTest(final TestInfo testInfo) throws IOException { - final var topicName = IntegrationBase.topicName(testInfo); + final var topic = IntegrationBase.getTopic(testInfo); - final Map configData = getConfig(topicName, 1); + final Map configData = getConfig(topic, 1); configData.put(INPUT_FORMAT_KEY, InputFormat.AVRO.getValue()); configData.put(VALUE_CONVERTER_KEY, "io.confluent.connect.avro.AvroConverter"); @@ -210,12 +210,12 @@ void sourceRecordIteratorAvroTest(final TestInfo testInfo) throws IOException { final Set offsetKeys = new HashSet<>(); - offsetKeys.add(writeToS3(topicName, outputStream1, "00001")); - offsetKeys.add(writeToS3(topicName, outputStream2, "00001")); + offsetKeys.add(writeToS3(topic, outputStream1, "00001")); + offsetKeys.add(writeToS3(topic, outputStream2, "00001")); - offsetKeys.add(writeToS3(topicName, outputStream3, "00002")); - offsetKeys.add(writeToS3(topicName, outputStream4, "00002")); - offsetKeys.add(writeToS3(topicName, outputStream5, "00002")); + offsetKeys.add(writeToS3(topic, outputStream3, "00002")); + offsetKeys.add(writeToS3(topic, outputStream4, "00002")); + offsetKeys.add(writeToS3(topic, outputStream5, "00002")); assertThat(testBucketAccessor.listObjects()).hasSize(5); @@ -260,8 +260,8 @@ void sourceRecordIteratorAvroTest(final TestInfo testInfo) throws IOException { @Test void verifyIteratorRehydration(final TestInfo testInfo) { // create 2 files. - final var topicName = IntegrationBase.topicName(testInfo); - final Map configData = getConfig(topicName, 1); + final var topic = IntegrationBase.getTopic(testInfo); + final Map configData = getConfig(topic, 1); configData.put(INPUT_FORMAT_KEY, InputFormat.BYTES.getValue()); @@ -274,8 +274,8 @@ void verifyIteratorRehydration(final TestInfo testInfo) { final List actualKeys = new ArrayList<>(); // write 2 objects to s3 - expectedKeys.add(writeToS3(topicName, testData1.getBytes(StandardCharsets.UTF_8), "00000")); - expectedKeys.add(writeToS3(topicName, testData2.getBytes(StandardCharsets.UTF_8), "00000")); + expectedKeys.add(writeToS3(topic, testData1.getBytes(StandardCharsets.UTF_8), "00000")); + expectedKeys.add(writeToS3(topic, testData2.getBytes(StandardCharsets.UTF_8), "00000")); assertThat(testBucketAccessor.listObjects()).hasSize(2); @@ -293,7 +293,7 @@ void verifyIteratorRehydration(final TestInfo testInfo) { assertThat(actualKeys).containsAll(expectedKeys); // write 3rd object to s3 - expectedKeys.add(writeToS3(topicName, testData3.getBytes(StandardCharsets.UTF_8), "00000")); + expectedKeys.add(writeToS3(topic, testData3.getBytes(StandardCharsets.UTF_8), "00000")); assertThat(testBucketAccessor.listObjects()).hasSize(3); assertThat(iter).hasNext(); diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java index d391b9eb7..140bf3ae5 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java @@ -124,9 +124,9 @@ default void writeToS3WithKey(final String objectKey, final byte[] testDataBytes } /** - * Writes to S3 using a key of the form {@code [prefix]topicName-partitionId-systemTime.txt}. + * Writes to S3 using a key of the form {@code [prefix]topic-partitionId-systemTime.txt}. * - * @param topicName + * @param topic * the topic name to use * @param testDataBytes * the data. @@ -135,8 +135,8 @@ default void writeToS3WithKey(final String objectKey, final byte[] testDataBytes * @return the key prefixed by {@link io.aiven.kafka.connect.s3.source.utils.S3OffsetManagerEntry#OBJECT_KEY} and * {@link OffsetManager} */ - default String writeToS3(final String topicName, final byte[] testDataBytes, final String partitionId) { - final String objectKey = org.apache.commons.lang3.StringUtils.defaultIfBlank(getS3Prefix(), "") + topicName + default String writeToS3(final String topic, final byte[] testDataBytes, final String partitionId) { + final String objectKey = org.apache.commons.lang3.StringUtils.defaultIfBlank(getS3Prefix(), "") + topic + "-" + partitionId + "-" + System.currentTimeMillis() + ".txt"; writeToS3WithKey(objectKey, testDataBytes); return objectKey; @@ -163,13 +163,13 @@ static Path getPluginDir() throws IOException { return Files.createDirectories(testDir.resolve(PLUGINS_S3_SOURCE_CONNECTOR_FOR_APACHE_KAFKA)); } - static String topicName(final TestInfo testInfo) { + static String getTopic(final TestInfo testInfo) { return testInfo.getTestMethod().get().getName(); } - static void createTopics(final AdminClient adminClient, final List topicNames) + static void createTopics(final AdminClient adminClient, final List topics) throws ExecutionException, InterruptedException { - final var newTopics = topicNames.stream().map(s -> new NewTopic(s, 4, (short) 1)).collect(Collectors.toList()); + final var newTopics = topics.stream().map(s -> new NewTopic(s, 4, (short) 1)).collect(Collectors.toList()); adminClient.createTopics(newTopics).all().get(); } diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java index 083d8627e..e591ed154 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationTest.java @@ -130,7 +130,7 @@ void setUp(final TestInfo testInfo) throws Exception { connectRunner.startConnectCluster(CONNECTOR_NAME, localListenerPort, containerListenerPort); adminClient = newAdminClient(connectRunner.getBootstrapServers()); - final String topicName = IntegrationBase.topicName(testInfo); + final String topicName = IntegrationBase.getTopic(testInfo); final var topics = List.of(topicName); IntegrationBase.createTopics(adminClient, topics); @@ -153,7 +153,7 @@ void tearDown() { @Test void bytesTest(final TestInfo testInfo) { - final var topicName = IntegrationBase.topicName(testInfo); + final var topicName = IntegrationBase.getTopic(testInfo); final Map connectorConfig = getConfig(CONNECTOR_NAME, topicName, 1); connectorConfig.put(INPUT_FORMAT_KEY, InputFormat.BYTES.getValue()); @@ -189,7 +189,7 @@ void bytesTest(final TestInfo testInfo) { @Test void avroTest(final TestInfo testInfo) throws IOException { - final var topicName = IntegrationBase.topicName(testInfo); + final var topicName = IntegrationBase.getTopic(testInfo); final Map connectorConfig = getAvroConfig(topicName, InputFormat.AVRO); connectRunner.configureConnector(CONNECTOR_NAME, connectorConfig); @@ -246,7 +246,7 @@ void avroTest(final TestInfo testInfo) throws IOException { @Test void parquetTest(final TestInfo testInfo) throws IOException { - final var topicName = IntegrationBase.topicName(testInfo); + final var topicName = IntegrationBase.getTopic(testInfo); final String partition = "00000"; final String fileName = org.apache.commons.lang3.StringUtils.defaultIfBlank(getS3Prefix(), "") + topicName + "-" @@ -287,7 +287,7 @@ private Map getAvroConfig(final String topicName, final InputFor @Test void jsonTest(final TestInfo testInfo) { - final var topicName = IntegrationBase.topicName(testInfo); + final var topicName = IntegrationBase.getTopic(testInfo); final Map connectorConfig = getConfig(CONNECTOR_NAME, topicName, 1); connectorConfig.put(INPUT_FORMAT_KEY, InputFormat.JSONL.getValue()); connectorConfig.put(VALUE_CONVERTER_KEY, "org.apache.kafka.connect.json.JsonConverter"); diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java index ede1f9dd4..ed6e4972e 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java @@ -58,7 +58,7 @@ public final class SourceRecordIterator implements Iterator { /** The AWS client that provides the S3Objects */ private final AWSV2SourceClient sourceClient; /** The S3 bucket we are processing */ - private final String bucketName; + private final String bucket; /** The inner iterator to provides S3Object that have been filtered potentially had data extracted */ private final Iterator inner; /** The outer iterator that provides S3SourceRecords */ @@ -70,7 +70,7 @@ public SourceRecordIterator(final S3SourceConfig s3SourceConfig, super(); this.s3SourceConfig = s3SourceConfig; this.offsetManager = offsetManager; - this.bucketName = s3SourceConfig.getAwsS3BucketName(); + this.bucket = s3SourceConfig.getAwsS3BucketName(); this.transformer = transformer; this.sourceClient = sourceClient; final Predicate fileNamePredicate = buildFileNamePredicate(); @@ -93,7 +93,7 @@ private Predicate buildFileNamePredicate() { if (fileMatcher.find()) { // TODO move this from the SourceRecordIterator so that we can decouple it from S3 and make it API // agnostic - final S3OffsetManagerEntry keyEntry = new S3OffsetManagerEntry(bucketName, s3Object.key(), + final S3OffsetManagerEntry keyEntry = new S3OffsetManagerEntry(bucket, s3Object.key(), fileMatcher.group(PATTERN_TOPIC_KEY), Integer.parseInt(fileMatcher.group(PATTERN_PARTITION_KEY))); offsetManagerEntry = offsetManager.getEntry(keyEntry.getManagerKey(), keyEntry::fromProperties) diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java index ddfccf113..bc0ff44a5 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java @@ -153,9 +153,9 @@ void testStop() { assertThat(s3SourceTask.isRunning()).isFalse(); } - private static S3SourceRecord createS3SourceRecord(final String topicName, final Integer defaultPartitionId, - final String bucketName, final String objectKey, final byte[] key, final byte[] value) { - return new S3SourceRecord(new S3OffsetManagerEntry(bucketName, objectKey, topicName, defaultPartitionId), + private static S3SourceRecord createS3SourceRecord(final String topic, final Integer defaultPartitionId, + final String bucket, final String objectKey, final byte[] key, final byte[] value) { + return new S3SourceRecord(new S3OffsetManagerEntry(bucket, objectKey, topic, defaultPartitionId), new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, key), new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, value)); } From cd2650a12ba8ce4238c4e956a051bf069fce5451 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Tue, 14 Jan 2025 11:25:49 +0000 Subject: [PATCH 42/44] Changes as per review --- .../io/aiven/kafka/connect/common/source/OffsetManager.java | 4 ++-- .../java/io/aiven/kafka/connect/s3/source/S3SourceTask.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java b/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java index a24614642..76ff95a95 100644 --- a/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java +++ b/commons/src/main/java/io/aiven/kafka/connect/common/source/OffsetManager.java @@ -118,7 +118,7 @@ public void updateCurrentOffsets(final E entry) { * @param key * the key for the entry to remove. */ - public void remove(final OffsetManagerKey key) { + public void removeEntry(final OffsetManagerKey key) { LOGGER.debug("Removing: {}", key.getPartitionMap()); offsets.remove(key.getPartitionMap()); } @@ -130,7 +130,7 @@ public void remove(final OffsetManagerKey key) { * @param sourceRecord * the SourceRecord that contains the key to be removed. */ - public void remove(final SourceRecord sourceRecord) { + public void removeEntry(final SourceRecord sourceRecord) { LOGGER.debug("Removing: {}", sourceRecord.sourcePartition()); offsets.remove(sourceRecord.sourcePartition()); } diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java index fb0ab4c45..b307c13cb 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/S3SourceTask.java @@ -139,7 +139,7 @@ public void commitRecord(final SourceRecord record) { if (LOGGER.isDebugEnabled()) { LOGGER.debug("Committed individual record {} committed", (Map) record.sourceOffset()); } - offsetManager.remove(record); + offsetManager.removeEntry(record); } /** From b444b446b3b74db6ec2bd8c8612ed9021e13f7f7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Tue, 14 Jan 2025 11:32:57 +0000 Subject: [PATCH 43/44] fixed spotless errors --- .../io/aiven/kafka/connect/s3/source/IntegrationBase.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java index 140bf3ae5..879fc39cc 100644 --- a/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java +++ b/s3-source-connector/src/integration-test/java/io/aiven/kafka/connect/s3/source/IntegrationBase.java @@ -136,8 +136,8 @@ default void writeToS3WithKey(final String objectKey, final byte[] testDataBytes * {@link OffsetManager} */ default String writeToS3(final String topic, final byte[] testDataBytes, final String partitionId) { - final String objectKey = org.apache.commons.lang3.StringUtils.defaultIfBlank(getS3Prefix(), "") + topic - + "-" + partitionId + "-" + System.currentTimeMillis() + ".txt"; + final String objectKey = org.apache.commons.lang3.StringUtils.defaultIfBlank(getS3Prefix(), "") + topic + "-" + + partitionId + "-" + System.currentTimeMillis() + ".txt"; writeToS3WithKey(objectKey, testDataBytes); return objectKey; From 394f2d17a6a1e1cb75117cd7299da993a5e2fe94 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C2=A8Claude?= <¨claude.warren@aiven.io¨> Date: Thu, 16 Jan 2025 10:11:12 +0000 Subject: [PATCH 44/44] modified to solve data change errors --- .../s3/source/utils/AWSV2SourceClient.java | 2 +- .../s3/source/utils/S3SourceRecord.java | 60 ++- .../s3/source/utils/SourceRecordIterator.java | 97 +++-- .../connect/s3/source/S3SourceTaskTest.java | 10 +- .../utils/S3OffsetManagerEntryTest.java | 1 - .../utils/SourceRecordIteratorTest.java | 385 ++++++++---------- 6 files changed, 281 insertions(+), 274 deletions(-) diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java index 19b726dc4..ba836c051 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/AWSV2SourceClient.java @@ -77,7 +77,7 @@ public AWSV2SourceClient(final S3SourceConfig s3SourceConfig) { * the beginning key, or {@code null} to start at the beginning. * @return a Stream of S3Objects for the current state of the S3 storage. */ - private Stream getS3ObjectStream(final String startToken) { + Stream getS3ObjectStream(final String startToken) { final ListObjectsV2Request request = ListObjectsV2Request.builder() .bucket(bucketName) .maxKeys(s3SourceConfig.getS3ConfigFragment().getFetchPageSize() * PAGE_SIZE_FACTOR) diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java index 4c1df43cf..e273c1faa 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/S3SourceRecord.java @@ -18,23 +18,69 @@ import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.source.SourceRecord; +import software.amazon.awssdk.services.s3.model.S3Object; public class S3SourceRecord { - private final SchemaAndValue keyData; - private final SchemaAndValue valueData; + + private SchemaAndValue keyData; + private SchemaAndValue valueData; /** The S3OffsetManagerEntry for this source record */ - private final S3OffsetManagerEntry offsetManagerEntry; + private S3OffsetManagerEntry offsetManagerEntry; + + + private final S3Object s3Object; + + + public S3SourceRecord(S3Object s3Object) { + this.s3Object = s3Object; + } - public S3SourceRecord(final S3OffsetManagerEntry offsetManagerEntry, final SchemaAndValue keyData, - final SchemaAndValue valueData) { - this.offsetManagerEntry = offsetManagerEntry.fromProperties(offsetManagerEntry.getProperties()); + public S3SourceRecord clone() { + S3SourceRecord result = new S3SourceRecord(s3Object); + result.setOffsetManagerEntry(offsetManagerEntry.fromProperties(offsetManagerEntry.getProperties())); + result.keyData = keyData; + result.valueData = valueData; + return result; + } + + public void setOffsetManagerEntry(S3OffsetManagerEntry offsetManagerEntry) { + this.offsetManagerEntry = offsetManagerEntry; + } + + public long getRecordCount() { + return offsetManagerEntry == null ? 0 : offsetManagerEntry.getRecordCount(); + } + + public void setKeyData(SchemaAndValue keyData) { this.keyData = keyData; + } + + public void incrementRecordCount() { + this.offsetManagerEntry.incrementRecordCount(); + } + + public void setValueData(SchemaAndValue valueData) { this.valueData = valueData; } + public String getTopic() { + return offsetManagerEntry.getTopic(); + } + + public int getPartition() { + return offsetManagerEntry.getPartition(); + } + +// public S3SourceRecord(final S3OffsetManagerEntry offsetManagerEntry, final SchemaAndValue keyData, +// final SchemaAndValue valueData) { +// this.offsetManagerEntry = offsetManagerEntry.fromProperties(offsetManagerEntry.getProperties()); +// this.keyData = keyData; +// this.valueData = valueData; +// } + public String getObjectKey() { - return offsetManagerEntry.getKey(); + return s3Object.key(); } public SchemaAndValue getKey() { diff --git a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java index 8017b3f06..6437a12b2 100644 --- a/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java +++ b/s3-source-connector/src/main/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIterator.java @@ -25,6 +25,7 @@ import java.util.regex.Pattern; import java.util.stream.Stream; +import org.apache.commons.collections4.IteratorUtils; import org.apache.kafka.connect.data.SchemaAndValue; import io.aiven.kafka.connect.common.source.OffsetManager; @@ -45,8 +46,6 @@ public final class SourceRecordIterator implements Iterator { /** The OffsetManager that we are using */ private final OffsetManager offsetManager; - /** The offset manager Entry we are working with */ - private S3OffsetManagerEntry offsetManagerEntry; /** The configuration for this S3 source */ private final S3SourceConfig s3SourceConfig; @@ -62,12 +61,14 @@ public final class SourceRecordIterator implements Iterator { private final int taskId; /** The inner iterator to provides S3Object that have been filtered potentially had data extracted */ - private final Iterator inner; + private final Iterator inner; /** The outer iterator that provides S3SourceRecords */ private Iterator outer; final FileMatching fileMatching; + final Predicate> taskAssignment; + public SourceRecordIterator(final S3SourceConfig s3SourceConfig, final OffsetManager offsetManager, final Transformer transformer, final AWSV2SourceClient sourceClient, final DistributionStrategy distributionStrategy, final String filePattern, final int taskId) { @@ -81,12 +82,14 @@ public SourceRecordIterator(final S3SourceConfig s3SourceConfig, final OffsetMan this.taskId = taskId; fileMatching = new FileMatching(filePattern); - // Initialize predicates - sourceClient.addPredicate(fileMatching); - sourceClient.addPredicate(s3Object -> distributionStrategy.isPartOfTask(taskId, s3Object.key(), fileMatching.pattern)); + taskAssignment = new TaskAssignment(distributionStrategy, fileMatching.pattern); + + Stream s3SourceRecordStream = sourceClient.getS3ObjectStream(null) + .map(fileMatching) + .filter(taskAssignment) + .map(Optional::get); - // call filters out bad file names and extracts topic/partition - inner = sourceClient.getS3ObjectIterator(null); + inner = s3SourceRecordStream.iterator(); outer = Collections.emptyIterator(); } @@ -112,26 +115,19 @@ public void remove() { /** * Converts the S3Object into stream of S3SourceRecords. * - * @param s3Object - * the S3Object to read data from. + * @param s3SourceRecord + * the SourceRecord that drives the creation of source records with values. * @return a stream of S3SourceRecords created from the input stream of the S3Object. */ - private Stream convert(final S3Object s3Object) { - - final long recordCount = offsetManagerEntry.getRecordCount(); - - // Optimizing without reading stream again. - if (transformer instanceof ByteArrayTransformer && recordCount > 0) { - return Stream.empty(); - } + private Stream convert(final S3SourceRecord s3SourceRecord) { - final SchemaAndValue keyData = transformer.getKeyData(s3Object.key(), offsetManagerEntry.getTopic(), - s3SourceConfig); + s3SourceRecord.setKeyData(transformer.getKeyData(s3SourceRecord.getObjectKey(), s3SourceRecord.getTopic(), + s3SourceConfig)); return transformer - .getRecords(sourceClient.getObject(s3Object.key()), offsetManagerEntry.getTopic(), - offsetManagerEntry.getPartition(), s3SourceConfig, recordCount) - .map(new Mapper(offsetManagerEntry, keyData)); + .getRecords(sourceClient.getObject(s3SourceRecord.getObjectKey()), s3SourceRecord.getTopic(), + s3SourceRecord.getPartition(), s3SourceConfig, s3SourceRecord.getRecordCount()) + .map(new Mapper(s3SourceRecord)); } /** @@ -139,30 +135,49 @@ private Stream convert(final S3Object s3Object) { */ static class Mapper implements Function { /** - * The partition map + * The S3SourceRecord that produceces the values. */ - private final S3OffsetManagerEntry entry; - /** - * The schema and value for the key - */ - private final SchemaAndValue keyData; + private final S3SourceRecord sourceRecord; - public Mapper(final S3OffsetManagerEntry entry, final SchemaAndValue keyData) { + public Mapper(final S3SourceRecord sourceRecord) { // TODO this is the point where the global S3OffsetManagerEntry becomes local and we can do a lookahead type // operation within the Transformer // to see if there are more records. - this.entry = entry; - this.keyData = keyData; + this.sourceRecord = sourceRecord.clone(); } @Override public S3SourceRecord apply(final SchemaAndValue valueData) { - entry.incrementRecordCount(); - return new S3SourceRecord(entry, keyData, valueData); + sourceRecord.incrementRecordCount(); + S3SourceRecord result = sourceRecord.clone(); + result.setValueData(valueData); + return result; } } - class FileMatching implements Predicate { + class TaskAssignment implements Predicate> { + final DistributionStrategy distributionStrategy; + final Pattern pattern; + + TaskAssignment(DistributionStrategy distributionStrategy, Pattern pattern) { + this.distributionStrategy = distributionStrategy; + this.pattern = pattern; + } + + + @Override + public boolean test(Optional s3SourceRecord) { + if (s3SourceRecord.isPresent()) { + S3SourceRecord record = s3SourceRecord.get(); + if (distributionStrategy.isPartOfTask(taskId, record.getObjectKey(), pattern)) { + return true; + } + } + return false; + } + } + + class FileMatching implements Function> { Pattern pattern; FileMatching(String filePattern) { @@ -170,14 +185,18 @@ class FileMatching implements Predicate { } @Override - public boolean test(S3Object s3Object) { + public Optional apply(S3Object s3Object) { Optional topic = FilePatternUtils.getTopic(pattern, s3Object.key()); OptionalInt partition = FilePatternUtils.getPartitionId(pattern, s3Object.key()); if (topic.isPresent() && partition.isPresent()) { - offsetManagerEntry = new S3OffsetManagerEntry(bucket, s3Object.key(), topic.get(), partition.getAsInt()); - return true; + S3SourceRecord s3SourceRecord = new S3SourceRecord(s3Object); + S3OffsetManagerEntry offsetManagerEntry = new S3OffsetManagerEntry(bucket, s3Object.key(), topic.get(), partition.getAsInt()); + offsetManagerEntry = offsetManager.getEntry(offsetManagerEntry.getManagerKey(), offsetManagerEntry::fromProperties).orElse(offsetManagerEntry); + s3SourceRecord.setOffsetManagerEntry(offsetManagerEntry); + return Optional.of(s3SourceRecord); } - return false; + return Optional.empty(); } } + } diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java index d05de58d0..a58455633 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/S3SourceTaskTest.java @@ -65,6 +65,7 @@ import software.amazon.awssdk.core.retry.RetryMode; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.S3Configuration; +import software.amazon.awssdk.services.s3.model.S3Object; final class S3SourceTaskTest { @@ -155,9 +156,12 @@ void testStop() { private static S3SourceRecord createS3SourceRecord(final String topic, final Integer defaultPartitionId, final String bucket, final String objectKey, final byte[] key, final byte[] value) { - return new S3SourceRecord(new S3OffsetManagerEntry(bucket, objectKey, topic, defaultPartitionId), - new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, key), - new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, value)); + + S3SourceRecord result = new S3SourceRecord(S3Object.builder().key(objectKey).size((long)value.length).build()); + result.setOffsetManagerEntry(new S3OffsetManagerEntry(bucket, objectKey, topic, defaultPartitionId)); + result.setKeyData(new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, key)); + result.setValueData(new SchemaAndValue(Schema.OPTIONAL_BYTES_SCHEMA, value)); + return result; } private void startSourceTask(final S3SourceTask s3SourceTask) { diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java index 8c4920198..acfdc5cc6 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/S3OffsetManagerEntryTest.java @@ -57,7 +57,6 @@ public void setUp() { sourceTaskContext = mock(SourceTaskContext.class); when(sourceTaskContext.offsetStorageReader()).thenReturn(offsetStorageReader); offsetManager = new OffsetManager<>(sourceTaskContext); - } private Map createPartitionMap() { diff --git a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java index 49fda463f..a6572f0f5 100644 --- a/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java +++ b/s3-source-connector/src/test/java/io/aiven/kafka/connect/s3/source/utils/SourceRecordIteratorTest.java @@ -65,6 +65,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; +import org.junit.jupiter.params.provider.ValueSource; import software.amazon.awssdk.core.ResponseBytes; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.GetObjectRequest; @@ -74,247 +75,185 @@ final class SourceRecordIteratorTest { - private S3OffsetManagerEntry mockS3OffsetManagerEntry; - private OffsetManager mockOffsetManager; - private Transformer mockTransformer; + private AWSV2SourceClient mockSourceApiClient; - private S3Client mockS3Client; + private OffsetStorageReader offsetStorageReader; - private AWSV2SourceClient sourceApiClient; + private OffsetManager offsetManager; + + private S3SourceConfig mockS3SourceConfig; + +// public SourceRecordIterator(final S3SourceConfig s3SourceConfig, final OffsetManager offsetManager, +// final Transformer transformer, final AWSV2SourceClient sourceClient, +// final DistributionStrategy distributionStrategy, final String filePattern, final int taskId) @BeforeEach - public void setUp() { - mockOffsetManager = mock(OffsetManager.class); - mockS3OffsetManagerEntry = mock(S3OffsetManagerEntry.class); - mockTransformer = mock(Transformer.class); - mockS3Client = mock(S3Client.class); + void setup() { + final SourceTaskContext mockTaskContext = mock(SourceTaskContext.class); + offsetStorageReader = mock(OffsetStorageReader.class); + when(mockTaskContext.offsetStorageReader()).thenReturn(offsetStorageReader); + offsetManager = new OffsetManager(mockTaskContext); + mockSourceApiClient = mock(AWSV2SourceClient.class); + mockS3SourceConfig = mock(S3SourceConfig.class); + when(mockS3SourceConfig.getAwsS3BucketName()).thenReturn("bucket-name"); } - private S3SourceConfig getConfig(Map data) { - Map defaults = new HashMap<>(); - defaults.put(AWS_S3_BUCKET_NAME_CONFIG, "bucket-name"); - defaults.putAll(data); - return new S3SourceConfig(defaults); - } +// private S3SourceConfig getConfig(Map data) { +// Map defaults = new HashMap<>(); +// defaults.put(AWS_S3_BUCKET_NAME_CONFIG, "bucket-name"); +// defaults.putAll(data); +// return new S3SourceConfig(defaults); +// } @Test void testIteratorProcessesS3Objects() throws Exception { final String key = "topic-00001-abc123.txt"; final String filePattern = "{{topic}}-{{partition}}"; + final Transformer transformer = TransformerFactory.getTransformer(InputFormat.BYTES); + when(offsetStorageReader.offset(any(Map.class))).thenReturn(new HashMap<>()); + when(mockSourceApiClient.getS3ObjectStream(any())).thenReturn(Stream.empty()); + Iterator iterator = new SourceRecordIterator(mockS3SourceConfig, offsetManager, transformer, + mockSourceApiClient, new HashDistributionStrategy(1), + filePattern, 0); - S3SourceConfig config = getConfig(Collections.emptyMap()); - - S3ClientBuilder builder = new S3ClientBuilder(); - - sourceApiClient = new AWSV2SourceClient(builder.build(), config ); - - - mockTransformer = TransformerFactory.getTransformer(InputFormat.BYTES); - - when(mockS3OffsetManagerEntry.getProperties()).thenReturn(Collections.emptyMap()); - - Iterator iterator = new SourceRecordIterator(config, mockOffsetManager, mockTransformer, - sourceApiClient, new HashDistributionStrategy(1), - filePattern, 0); - - assertThat(iterator).isExhausted(); - - builder.reset().addObject(key, "Hello World").endOfBlock(); - - sourceApiClient = new AWSV2SourceClient(builder.build(), config ); - - Iterator s3ObjectIterator = new SourceRecordIterator(config, mockOffsetManager, mockTransformer, sourceApiClient, - new HashDistributionStrategy(1), filePattern, 0); - - assertThat(iterator).hasNext(); - assertThat(iterator.next()).isNotNull(); - assertThat(iterator).isExhausted(); - - } - - @Test - void testIteratorProcessesS3ObjectsForByteArrayTransformer() throws Exception { - final String key = "topic-00001-abc123.txt"; - final String filePattern = "{{topic}}-{{partition}}-{{start_offset}}"; - final S3Object s3Object = S3Object.builder().key(key).build(); - - S3SourceConfig config = getConfig(Collections.emptyMap()); - // With ByteArrayTransformer - try (InputStream inputStream = new ByteArrayInputStream("Hello World".getBytes(StandardCharsets.UTF_8))) { - when(sourceApiClient.getObject(key)).thenReturn(() -> inputStream); - - - when(sourceApiClient.getS3ObjectIterator(any())).thenReturn(Arrays.asList(s3Object).iterator()); - - mockTransformer = mock(ByteArrayTransformer.class); - when(mockTransformer.getRecords(any(), anyString(), anyInt(), any(), anyLong())) - .thenReturn(Stream.of(SchemaAndValue.NULL)); - - when(mockOffsetManager.getEntry(any(), any())).thenReturn(Optional.of(mockS3OffsetManagerEntry)); - - when(sourceApiClient.getListOfObjectKeys(any())) - .thenReturn(Collections.singletonList(key).listIterator()); - when(mockS3OffsetManagerEntry.getRecordCount()).thenReturn(BYTES_TRANSFORMATION_NUM_OF_RECS); - - // should skip if any records were produced by source record iterator. - final Iterator iterator = new SourceRecordIterator(config, mockOffsetManager, - mockTransformer, sourceApiClient, new HashDistributionStrategy(1), filePattern, 0); - - assertThat(iterator).isExhausted(); - verify(sourceApiClient, never()).getObject(any()); - verify(mockTransformer, never()).getRecords(any(), anyString(), anyInt(), any(), anyLong()); - } + assertThat(iterator).isExhausted(); - // With AvroTransformer - try (InputStream inputStream = new ByteArrayInputStream("Hello World".getBytes(StandardCharsets.UTF_8))) { - when(sourceApiClient.getObject(key)).thenReturn(() -> inputStream); + S3Object s3Object = S3Object.builder().key(key).size(1L).build(); + when(mockSourceApiClient.getS3ObjectStream(any())).thenReturn(Collections.singletonList(s3Object).stream()); + when(mockSourceApiClient.getObject(key)).thenReturn(() -> new ByteArrayInputStream("Hello World".getBytes(StandardCharsets.UTF_8))); - when(sourceApiClient.getS3ObjectIterator(any())).thenReturn(Arrays.asList(s3Object).iterator()); - when(sourceApiClient.getListOfObjectKeys(any())) - .thenReturn(Collections.singletonList(key).listIterator()); + iterator = new SourceRecordIterator(mockS3SourceConfig, offsetManager, transformer, mockSourceApiClient, + new HashDistributionStrategy(1), filePattern, 0); - final OffsetStorageReader offsetStorageReader = mock(OffsetStorageReader.class); - when(offsetStorageReader.offset(any(Map.class))).thenReturn(Map.of(RECORD_COUNT, 1)); + assertThat(iterator).hasNext(); + assertThat(iterator.next()).isNotNull(); + assertThat(iterator).isExhausted(); - final SourceTaskContext context = mock(SourceTaskContext.class); - when(context.offsetStorageReader()).thenReturn(offsetStorageReader); - - mockOffsetManager = new OffsetManager(context); - - mockTransformer = mock(Transformer.class); - final SchemaAndValue schemaKey = new SchemaAndValue(null, "KEY"); - final SchemaAndValue schemaValue = new SchemaAndValue(null, "VALUE"); - when(mockTransformer.getKeyData(anyString(), anyString(), any())).thenReturn(schemaKey); - when(mockTransformer.getRecords(any(), anyString(), anyInt(), any(), anyLong())) - .thenReturn(Arrays.asList(schemaValue).stream()); - - final Iterator iterator = new SourceRecordIterator(config, mockOffsetManager, - mockTransformer, sourceApiClient, new HashDistributionStrategy(1), filePattern, 0); - assertThat(iterator.hasNext()).isTrue(); - final S3SourceRecord record = iterator.next(); - assertThat(record.getValue().value()).isEqualTo("VALUE"); - assertThat(record.getOffsetManagerEntry().getRecordCount()).isEqualTo(2); - verify(mockTransformer, times(1)).getRecords(any(), anyString(), anyInt(), any(), anyLong()); - } - } - @ParameterizedTest - @CsvSource({ "4, 2, key1", "4, 3, key2", "4, 0, key3", "4, 1, key4" }) - void testFetchObjectSummariesWithOneNonZeroByteObjectWithTaskIdAssigned(final int maxTasks, final int taskId, - final String objectKey) { - - final String filePattern = "{{topic}}-{{partition}}-{{start_offset}}"; - mockTransformer = TransformerFactory.getTransformer(InputFormat.BYTES); - S3SourceConfig config = getConfig(Collections.emptyMap()); - - final S3Object obj = S3Object.builder().key(objectKey).build(); - - final ByteArrayInputStream bais = new ByteArrayInputStream("Hello World".getBytes(StandardCharsets.UTF_8)); - final DistributionStrategy distributionStrategy = new HashDistributionStrategy(maxTasks) ; - - when(sourceApiClient.getS3ObjectIterator(any())).thenReturn(Arrays.asList(obj).iterator()); - when(sourceApiClient.getObject(any())).thenReturn(() -> bais); - final SourceRecordIterator iterator = new SourceRecordIterator(config, mockOffsetManager, mockTransformer, - sourceApiClient, distributionStrategy, filePattern, taskId); - SourceRecordIterator.FileMatching fileMatching = iterator.new FileMatching(filePattern); - Predicate s3ObjectPredicate = fileMatching; - s3ObjectPredicate = s3ObjectPredicate.and(s3Object -> distributionStrategy.isPartOfTask(taskId, s3Object.key(), fileMatching.pattern)); - // Assert - assertThat(s3ObjectPredicate).accepts(obj); } - @ParameterizedTest - @CsvSource({ "4, 1, topic1-2-0", "4, 3, key1", "4, 0, key1", "4, 1, key2", "4, 2, key2", "4, 0, key2", "4, 1, key3", - "4, 2, key3", "4, 3, key3", "4, 0, key4", "4, 2, key4", "4, 3, key4" }) - void testFetchObjectSummariesWithOneNonZeroByteObjectWithTaskIdUnassigned(final int maxTasks, final int taskId, - final String objectKey) { - final String filePattern = "{{topic}}-{{partition}}-{{start_offset}}"; - mockTransformer = TransformerFactory.getTransformer(InputFormat.BYTES); - S3SourceConfig config = getConfig(Collections.emptyMap()); - - final S3Object obj = S3Object.builder().key(objectKey).build(); - - final ByteArrayInputStream bais = new ByteArrayInputStream("Hello World".getBytes(StandardCharsets.UTF_8)); - final DistributionStrategy distributionStrategy = new HashDistributionStrategy(maxTasks) ; - - when(sourceApiClient.getS3ObjectIterator(any())).thenReturn(Arrays.asList(obj).iterator()); - when(sourceApiClient.getObject(any())).thenReturn(() -> bais); - final SourceRecordIterator iterator = new SourceRecordIterator(config, mockOffsetManager, mockTransformer, - sourceApiClient,distributionStrategy, filePattern, taskId); - SourceRecordIterator.FileMatching fileMatching = iterator.new FileMatching(filePattern); - Predicate s3ObjectPredicate = fileMatching; - s3ObjectPredicate = s3ObjectPredicate.and(s3Object -> distributionStrategy.isPartOfTask(taskId, s3Object.key(), fileMatching.pattern)); - // Assert - assertThat(s3ObjectPredicate.test(obj)).as("Predicate should accept the objectKey: " + objectKey).isFalse(); - } - - @Test - public void x() { - S3ClientBuilder builder = new S3ClientBuilder(); - builder.addObject("Key", "value"); - S3Client client = builder.build(); - ListObjectsV2Response response = client.listObjectsV2(ListObjectsV2Request.builder().build()); - assertThat(response.contents()).isNotEmpty(); - - sourceApiClient = new AWSV2SourceClient(builder.build(), getConfig(Collections.emptyMap())); - Iterator iter = sourceApiClient.getS3ObjectIterator(null); - assertThat(iter.hasNext()); - } - - - class S3ClientBuilder { - Queue, Map>> blocks = new LinkedList<>(); - List objects = new ArrayList<>(); - Map data = new HashMap<>(); - - public S3ClientBuilder addObject(String key, byte[] data) { - objects.add(S3Object.builder().key(key).size((long)data.length).build()); - this.data.put(key, data); - return this; - } - - public S3ClientBuilder endOfBlock() { - blocks.add(Pair.of(objects, data)); - return reset(); - } - - public S3ClientBuilder reset() { - objects = new ArrayList<>(); - data = new HashMap<>(); - return this; - } - - public S3ClientBuilder addObject(String key, String data) { - return addObject(key, data.getBytes(StandardCharsets.UTF_8)); - } - - private ResponseBytes getResponse(String key) { - return ResponseBytes.fromByteArray(new byte[0],data.get(key)); - } - - private ListObjectsV2Response dequeueData() { - if (blocks.isEmpty()) { - objects = Collections.emptyList(); - data = Collections.emptyMap(); - } else { - Pair, Map> pair = blocks.remove(); - objects = pair.getLeft(); - data = pair.getRight(); - } - return ListObjectsV2Response.builder().contents(objects).isTruncated(false).build(); - } - - public S3Client build() { - if (!objects.isEmpty()) { - endOfBlock(); - } - S3Client result = mock(S3Client.class); - when(result.listObjectsV2(any(ListObjectsV2Request.class))).thenAnswer(env -> dequeueData()); - when(result.listObjectsV2(any(Consumer.class))).thenAnswer(env -> dequeueData()); - when(result.getObjectAsBytes(any(GetObjectRequest.class))).thenAnswer(env -> getResponse(env.getArgument(0, GetObjectRequest.class).key())); - return result; - } - - } +// @Test +// void testIteratorProcessesS3ObjectsForByteArrayTransformer() throws Exception { +// final String key = "topic-00001-abc123.txt"; +// final String filePattern = "{{topic}}-{{partition}}"; +// final S3ClientBuilder builder = new S3ClientBuilder(); +// final S3SourceConfig config = getConfig(Collections.emptyMap()); +// +// // With ByteArrayTransformer +// Transformer transformer = TransformerFactory.getTransformer(InputFormat.BYTES); +// +// builder.addObject(key, "Hello World"); +// +// sourceApiClient = new AWSV2SourceClient(builder.build(), config); +// +// when(offsetStorageReader.offset(any(Map.class))).thenReturn(Map.of(RECORD_COUNT, 1)); +// +// +// // should skip if any records were produced by source record iterator. +// Iterator iterator = new SourceRecordIterator(config, offsetManager, +// transformer, sourceApiClient, new HashDistributionStrategy(1), filePattern, 0); +// +// assertThat(iterator).isExhausted(); +// +// +// // With JsonTransformer +// StringBuilder jsonContent = new StringBuilder(); +// for (int i = 0; i < 5; i++) { +// jsonContent.append(String.format("{\"message\": \"Hello World\", \"id\":\"%s\"}%n", i)); +// } +// +// builder.reset().addObject(key, jsonContent.toString()); +// sourceApiClient = new AWSV2SourceClient(builder.build(), config); +// +// transformer = TransformerFactory.getTransformer(InputFormat.JSONL); +// +// iterator = new SourceRecordIterator(config, offsetManager, +// transformer, sourceApiClient, new HashDistributionStrategy(1), filePattern, 0); +// assertThat(iterator).hasNext(); +// final S3SourceRecord record = iterator.next(); +// assertThat((Map) record.getValue().value()).containsEntry("id", "1"); +// } +// +// @ParameterizedTest +// @ValueSource( ints={0, 1, 2, 3}) +// void testRetrieveTaskBasedData(int taskId) { +// Map expectedTask = Map.of(3, "key-1", 0, "key-2", 1, "key-3", 2, "key-4"); +// +// int maxTasks = 4; +// final String filePattern = "{{topic}}-{{partition}}"; +// final S3SourceConfig config = getConfig(Collections.emptyMap()); +// Transformer transformer = TransformerFactory.getTransformer(InputFormat.BYTES); +// +// final S3ClientBuilder builder = new S3ClientBuilder(); +// for (String key : expectedTask.values()) { +// builder.addObject(key, "Hello World - "+key); +// } +// sourceApiClient = new AWSV2SourceClient(builder.build(), config); +// +// final DistributionStrategy distributionStrategy = new HashDistributionStrategy(maxTasks) ; +// +// final SourceRecordIterator iterator = new SourceRecordIterator(config, offsetManager, transformer, +// sourceApiClient, distributionStrategy, filePattern, taskId); +// +// assertThat(iterator.hasNext()); +// S3SourceRecord record = iterator.next(); +// assertThat(record.getObjectKey()).isEqualTo(expectedTask.get(taskId)); +// assertThat(iterator).isExhausted(); +// } +// +// class S3ClientBuilder { +// Queue, Map>> blocks = new LinkedList<>(); +// List objects = new ArrayList<>(); +// Map data = new HashMap<>(); +// +// public S3ClientBuilder addObject(String key, byte[] data) { +// objects.add(S3Object.builder().key(key).size((long)data.length).build()); +// this.data.put(key, data); +// return this; +// } +// +// public S3ClientBuilder endOfBlock() { +// blocks.add(Pair.of(objects, data)); +// return reset(); +// } +// +// public S3ClientBuilder reset() { +// objects = new ArrayList<>(); +// data = new HashMap<>(); +// return this; +// } +// +// public S3ClientBuilder addObject(String key, String data) { +// return addObject(key, data.getBytes(StandardCharsets.UTF_8)); +// } +// +// private ResponseBytes getResponse(String key) { +// return ResponseBytes.fromByteArray(new byte[0],data.get(key)); +// } +// +// private ListObjectsV2Response dequeueData() { +// if (blocks.isEmpty()) { +// objects = Collections.emptyList(); +// data = Collections.emptyMap(); +// } else { +// Pair, Map> pair = blocks.remove(); +// objects = pair.getLeft(); +// data = pair.getRight(); +// } +// return ListObjectsV2Response.builder().contents(objects).isTruncated(false).build(); +// } +// +// public S3Client build() { +// if (!objects.isEmpty()) { +// endOfBlock(); +// } +// S3Client result = mock(S3Client.class); +// when(result.listObjectsV2(any(ListObjectsV2Request.class))).thenAnswer(env -> dequeueData()); +// when(result.listObjectsV2(any(Consumer.class))).thenAnswer(env -> dequeueData()); +// when(result.getObjectAsBytes(any(GetObjectRequest.class))).thenAnswer(env -> getResponse(env.getArgument(0, GetObjectRequest.class).key())); +// return result; +// } +// +// } }