From 85f25ed48cf526b8da4ea19ce3779334e4fb2201 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Tue, 31 Oct 2023 13:12:38 -0700 Subject: [PATCH 01/24] * do stuff --- .../kinesis/KinesisRecordSupplier.java | 5 +- .../MemoryBoundLinkedBlockingQueue.java | 58 +++++++++++++++++++ 2 files changed, 61 insertions(+), 2 deletions(-) create mode 100644 extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/MemoryBoundLinkedBlockingQueue.java diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index f0645f8f82cc..513b6eb99783 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -432,7 +432,8 @@ public KinesisRecordSupplier( int recordBufferFullWait, int maxRecordsPerPoll, boolean useEarliestSequenceNumber, - boolean useListShards + boolean useListShards, + long queueMaxByteSize ) { Preconditions.checkNotNull(amazonKinesis); @@ -488,7 +489,7 @@ public KinesisRecordSupplier( ); } - records = new LinkedBlockingQueue<>(recordBufferSize); + records = new MemoryBoundLinkedBlockingQueue<>(new LinkedBlockingQueue<>(), queueMaxByteSize); } public static AmazonKinesis getAmazonKinesisClient( diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/MemoryBoundLinkedBlockingQueue.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/MemoryBoundLinkedBlockingQueue.java new file mode 100644 index 000000000000..925c4e7991be --- /dev/null +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/MemoryBoundLinkedBlockingQueue.java @@ -0,0 +1,58 @@ +package org.apache.druid.indexing.kinesis; + +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +public class MemoryBoundLinkedBlockingQueue extends LinkedBlockingQueue { + private final LinkedBlockingQueue queue; + private final long byteLimit; + private long currentSizeBytes; + + public MemoryBoundLinkedBlockingQueue(LinkedBlockingQueue queue, long byteLimit) { + this.queue = queue; + this.byteLimit = byteLimit; + this.currentSizeBytes = 0L; + } + + /** + * Inserts the specified element into this queue, waiting up to the + * specified wait time if necessary for space to become available. + * + * @param e the element to add + * @param timeout how long to wait before giving up, in units of + * {@code unit} + * @param unit a {@code TimeUnit} determining how to interpret the + * {@code timeout} parameter + * + * @return {@code true} if successful, or {@code false} if + * the specified waiting time elapses before space is available + * @throws InterruptedException if interrupted while waiting + * @throws ClassCastException if the class of the specified element + * prevents it from being added to this queue + * @throws NullPointerException if the specified element is null + * @throws IllegalArgumentException if some property of the specified + * element prevents it from being added to this queue + */ + public boolean offer(E e, long sizeBytes, long timeout, TimeUnit unit) + throws InterruptedException { + synchronized (this) { + if (currentSizeBytes >= byteLimit) { + return false; + } + boolean successful = queue.offer(e, timeout, unit); + if (successful) { + currentSizeBytes += sizeBytes; + } + return successful; + } + } + + @Override + public int remainingCapacity() { + return (int) (byteLimit - currentSizeBytes); + } + + public long currentByteSize() { + return currentSizeBytes; + } +} From 013b9d7a9c095c709a89b27959b835e54668bb2b Mon Sep 17 00:00:00 2001 From: zachjsh Date: Thu, 9 Nov 2023 19:27:53 -0500 Subject: [PATCH 02/24] * move existing MemoryBoundLinkedBlockingQueue to druid core and extend as needed --- .../druid/emitter/kafka/KafkaEmitter.java | 6 +- .../kinesis/KinesisRecordSupplier.java | 67 ++++++++------- .../MemoryBoundLinkedBlockingQueue.java | 58 ------------- .../MemoryBoundLinkedBlockingQueue.java | 84 ++++++++++++++++++- 4 files changed, 119 insertions(+), 96 deletions(-) delete mode 100644 extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/MemoryBoundLinkedBlockingQueue.java rename {extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka => processing/src/main/java/org/apache/druid/java/util/common}/MemoryBoundLinkedBlockingQueue.java (53%) diff --git a/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitter.java b/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitter.java index dd8f3665f537..410af3660358 100644 --- a/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitter.java +++ b/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitter.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import org.apache.druid.emitter.kafka.KafkaEmitterConfig.EventType; -import org.apache.druid.emitter.kafka.MemoryBoundLinkedBlockingQueue.ObjectContainer; +import org.apache.druid.java.util.common.MemoryBoundLinkedBlockingQueue; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.common.logger.Logger; @@ -173,7 +173,7 @@ private void sendSegmentMetadataToKafka() private void sendToKafka(final String topic, MemoryBoundLinkedBlockingQueue recordQueue, Callback callback) { - ObjectContainer objectToSend; + MemoryBoundLinkedBlockingQueue.ObjectContainer objectToSend; try { while (true) { objectToSend = recordQueue.take(); @@ -199,7 +199,7 @@ public void emit(final Event event) String resultJson = jsonMapper.writeValueAsString(map); - ObjectContainer objectContainer = new ObjectContainer<>( + MemoryBoundLinkedBlockingQueue.ObjectContainer objectContainer = new MemoryBoundLinkedBlockingQueue.ObjectContainer<>( resultJson, StringUtils.toUtf8(resultJson).length ); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 513b6eb99783..a96c8a93711e 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -46,7 +46,6 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; -import com.google.common.collect.Queues; import org.apache.druid.common.aws.AWSClientUtil; import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.common.aws.AWSCredentialsUtils; @@ -58,6 +57,7 @@ import org.apache.druid.indexing.seekablestream.common.StreamException; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.MemoryBoundLinkedBlockingQueue; import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; @@ -78,12 +78,10 @@ import java.util.Set; import java.util.TreeSet; import java.util.UUID; -import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executors; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; @@ -212,7 +210,6 @@ private Runnable fetchRecords() // used for retrying on InterruptedException GetRecordsResult recordsResult = null; OrderedPartitionableRecord currRecord; - try { if (shardIterator == null) { @@ -228,7 +225,11 @@ private Runnable fetchRecords() recordsResult = null; - if (!records.offer(currRecord, recordBufferOfferTimeout, TimeUnit.MILLISECONDS)) { + if (!records.offer( + new MemoryBoundLinkedBlockingQueue.ObjectContainer<>(currRecord, 0), + recordBufferOfferTimeout, + TimeUnit.MILLISECONDS + )) { log.warn("Kinesis records are being processed slower than they are fetched. " + "OrderedPartitionableRecord buffer full, retrying in [%,dms].", recordBufferFullWait); @@ -245,28 +246,26 @@ private Runnable fetchRecords() // list will come back empty if there are no records for (Record kinesisRecord : recordsResult.getRecords()) { - final List data; + if (deaggregateHandle == null || getDataHandle == null) { + throw new ISE("deaggregateHandle or getDataHandle is null!"); + } - if (deaggregate) { - if (deaggregateHandle == null || getDataHandle == null) { - throw new ISE("deaggregateHandle or getDataHandle is null!"); - } - - data = new ArrayList<>(); + data = new ArrayList<>(); - final List userRecords = (List) deaggregateHandle.invokeExact( - Collections.singletonList(kinesisRecord) - ); + final List userRecords = (List) deaggregateHandle.invokeExact( + Collections.singletonList(kinesisRecord) + ); - for (Object userRecord : userRecords) { - data.add(new ByteEntity((ByteBuffer) getDataHandle.invoke(userRecord))); - } - } else { - data = Collections.singletonList(new ByteEntity(kinesisRecord.getData())); + int recordSize = 0; + for (Object userRecord : userRecords) { + ByteEntity byteEntity = new ByteEntity((ByteBuffer) getDataHandle.invoke(userRecord)); + recordSize += byteEntity.getBuffer().array().length; + data.add(byteEntity); } + currRecord = new OrderedPartitionableRecord<>( streamPartition.getStream(), streamPartition.getPartitionId(), @@ -277,10 +276,11 @@ private Runnable fetchRecords() if (log.isTraceEnabled()) { log.trace( - "Stream[%s] / partition[%s] / sequenceNum[%s] / bufferRemainingCapacity[%d]: %s", + "Stream[%s] / partition[%s] / sequenceNum[%s] / bufferByteCapacity[%d] / bufferRemainingByteCapacity[%d]: %s", currRecord.getStream(), currRecord.getPartitionId(), currRecord.getSequenceNumber(), + records.byteSize(), records.remainingCapacity(), currRecord.getData() .stream() @@ -294,7 +294,11 @@ private Runnable fetchRecords() // If the buffer was full and we weren't able to add the message, grab a new stream iterator starting // from this message and back off for a bit to let the buffer drain before retrying. - if (!records.offer(currRecord, recordBufferOfferTimeout, TimeUnit.MILLISECONDS)) { + if (!records.offer( + new MemoryBoundLinkedBlockingQueue.ObjectContainer<>(currRecord, recordSize), + recordBufferOfferTimeout, + TimeUnit.MILLISECONDS + )) { log.warn( "Kinesis records are being processed slower than they are fetched. " + "OrderedPartitionableRecord buffer full, storing iterator and retrying in [%,dms].", @@ -415,7 +419,7 @@ private long getPartitionTimeLag() private final ConcurrentMap, PartitionResource> partitionResources = new ConcurrentHashMap<>(); - private BlockingQueue> records; + private MemoryBoundLinkedBlockingQueue> records; private final boolean backgroundFetchEnabled; private volatile boolean closed = false; @@ -489,7 +493,7 @@ public KinesisRecordSupplier( ); } - records = new MemoryBoundLinkedBlockingQueue<>(new LinkedBlockingQueue<>(), queueMaxByteSize); + records = new MemoryBoundLinkedBlockingQueue<>(queueMaxByteSize); } public static AmazonKinesis getAmazonKinesisClient( @@ -638,10 +642,9 @@ public List> poll(long ti try { int expectedSize = Math.min(Math.max(records.size(), 1), maxRecordsPerPoll); - List> polledRecords = new ArrayList<>(expectedSize); + List>> polledRecords = new ArrayList<>(expectedSize); - Queues.drain( - records, + records.drain( polledRecords, expectedSize, timeout, @@ -649,10 +652,12 @@ public List> poll(long ti ); polledRecords = polledRecords.stream() - .filter(x -> partitionResources.containsKey(x.getStreamPartition())) + .filter(x -> partitionResources.containsKey(x.getData().getStreamPartition())) .collect(Collectors.toList()); - return polledRecords; + return polledRecords.stream() + .map(MemoryBoundLinkedBlockingQueue.ObjectContainer::getData) + .collect(Collectors.toList()); } catch (InterruptedException e) { log.warn(e, "Interrupted while polling"); @@ -1060,10 +1065,10 @@ private void filterBufferAndResetBackgroundFetch(Set> pa } // filter records in buffer and only retain ones whose partition was not seeked - BlockingQueue> newQ = new LinkedBlockingQueue<>(recordBufferSize); + MemoryBoundLinkedBlockingQueue> newQ = new MemoryBoundLinkedBlockingQueue<>(recordBufferSize); records.stream() - .filter(x -> !partitions.contains(x.getStreamPartition())) + .filter(x -> !partitions.contains(x.getData().getStreamPartition())) .forEachOrdered(newQ::offer); records = newQ; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/MemoryBoundLinkedBlockingQueue.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/MemoryBoundLinkedBlockingQueue.java deleted file mode 100644 index 925c4e7991be..000000000000 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/MemoryBoundLinkedBlockingQueue.java +++ /dev/null @@ -1,58 +0,0 @@ -package org.apache.druid.indexing.kinesis; - -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; - -public class MemoryBoundLinkedBlockingQueue extends LinkedBlockingQueue { - private final LinkedBlockingQueue queue; - private final long byteLimit; - private long currentSizeBytes; - - public MemoryBoundLinkedBlockingQueue(LinkedBlockingQueue queue, long byteLimit) { - this.queue = queue; - this.byteLimit = byteLimit; - this.currentSizeBytes = 0L; - } - - /** - * Inserts the specified element into this queue, waiting up to the - * specified wait time if necessary for space to become available. - * - * @param e the element to add - * @param timeout how long to wait before giving up, in units of - * {@code unit} - * @param unit a {@code TimeUnit} determining how to interpret the - * {@code timeout} parameter - * - * @return {@code true} if successful, or {@code false} if - * the specified waiting time elapses before space is available - * @throws InterruptedException if interrupted while waiting - * @throws ClassCastException if the class of the specified element - * prevents it from being added to this queue - * @throws NullPointerException if the specified element is null - * @throws IllegalArgumentException if some property of the specified - * element prevents it from being added to this queue - */ - public boolean offer(E e, long sizeBytes, long timeout, TimeUnit unit) - throws InterruptedException { - synchronized (this) { - if (currentSizeBytes >= byteLimit) { - return false; - } - boolean successful = queue.offer(e, timeout, unit); - if (successful) { - currentSizeBytes += sizeBytes; - } - return successful; - } - } - - @Override - public int remainingCapacity() { - return (int) (byteLimit - currentSizeBytes); - } - - public long currentByteSize() { - return currentSizeBytes; - } -} diff --git a/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/MemoryBoundLinkedBlockingQueue.java b/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java similarity index 53% rename from extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/MemoryBoundLinkedBlockingQueue.java rename to processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java index fb6cae8ee954..b0d6efe97935 100644 --- a/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/MemoryBoundLinkedBlockingQueue.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java @@ -17,10 +17,15 @@ * under the License. */ -package org.apache.druid.emitter.kafka; +package org.apache.druid.java.util.common; +import com.google.common.base.Preconditions; + +import java.util.Collection; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Stream; /** * Similar to LinkedBlockingQueue but can be bounded by the total byte size of the items present in the queue @@ -53,6 +58,25 @@ public boolean offer(ObjectContainer item) return false; } + public boolean offer(ObjectContainer item, long timeout, TimeUnit unit) throws InterruptedException + { + final long itemLength = item.getSize(); + + try { + if (currentMemory.addAndGet(itemLength) <= memoryBound) { + if (queue.offer(item, timeout, unit)) { + return true; + } + } + } + catch (InterruptedException e) { + currentMemory.addAndGet(-itemLength); + throw e; + } + currentMemory.addAndGet(-itemLength); + return false; + } + // blocks until at least one item is available to take public ObjectContainer take() throws InterruptedException { @@ -61,12 +85,63 @@ public ObjectContainer take() throws InterruptedException return ret; } + public Stream> stream() + { + return queue.stream(); + } + + // This code is taken from {om.google.common.collect.Queues#Drain}} + public int drain(Collection> buffer, int numElements, long timeout, TimeUnit unit) + throws InterruptedException + { + Preconditions.checkNotNull(buffer); + long deadline = System.nanoTime() + unit.toNanos(timeout); + int added = 0; + + while (added < numElements) { + added += queue.drainTo(buffer, numElements - added); + buffer.forEach( + i -> { + currentMemory.addAndGet(-((ObjectContainer) i).getSize()); + } + ); + if (added < numElements) { + ObjectContainer e = queue.poll(deadline - System.nanoTime(), TimeUnit.NANOSECONDS); + + if (e == null) { + break; + } + + currentMemory.addAndGet(-e.getSize()); + buffer.add(e); + ++added; + } + } + + return added; + } + + public int size() + { + return queue.size(); + } + + public long byteSize() + { + return currentMemory.get(); + } + + public long remainingCapacity() + { + return memoryBound - currentMemory.get(); + } + public static class ObjectContainer { - private T data; - private long size; + private final T data; + private final long size; - ObjectContainer(T data, long size) + public ObjectContainer(T data, long size) { this.data = data; this.size = size; @@ -83,3 +158,4 @@ public long getSize() } } } + From 39545aaded77b238c65de3c7f0f25435ae297db5 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Mon, 13 Nov 2023 20:09:12 -0500 Subject: [PATCH 03/24] * fix configs, tests --- .../indexing/kinesis/KinesisIndexTask.java | 41 +++-- .../kinesis/KinesisIndexTaskIOConfig.java | 56 +----- .../kinesis/KinesisIndexTaskTuningConfig.java | 34 ++-- .../kinesis/KinesisRecordSupplier.java | 57 +++--- .../indexing/kinesis/KinesisSamplerSpec.java | 6 +- .../kinesis/supervisor/KinesisSupervisor.java | 10 +- .../supervisor/KinesisSupervisorIOConfig.java | 25 +-- .../indexing/kinesis/KinesisIOConfigTest.java | 13 +- .../kinesis/KinesisIndexTaskSerdeTest.java | 4 +- .../kinesis/KinesisIndexTaskTest.java | 31 ++-- .../KinesisIndexTaskTuningConfigTest.java | 6 +- .../kinesis/KinesisRecordSupplierTest.java | 169 ++++-------------- .../kinesis/KinesisSamplerSpecTest.java | 12 +- .../KinesisSupervisorIOConfigTest.java | 4 - .../supervisor/KinesisSupervisorTest.java | 36 +--- 15 files changed, 135 insertions(+), 369 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 1aa6d5b2e3be..46ccf5aee96a 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -105,21 +105,18 @@ protected KinesisRecordSupplier newTaskRecordSupplier(final TaskToolbox toolbox) { KinesisIndexTaskIOConfig ioConfig = ((KinesisIndexTaskIOConfig) super.ioConfig); KinesisIndexTaskTuningConfig tuningConfig = ((KinesisIndexTaskTuningConfig) super.tuningConfig); - final int fetchThreads = computeFetchThreads(runtimeInfo, tuningConfig.getFetchThreads()); - final int recordsPerFetch = ioConfig.getRecordsPerFetchOrDefault(runtimeInfo.getMaxHeapSizeBytes(), fetchThreads); - final int recordBufferSize = - tuningConfig.getRecordBufferSizeOrDefault(runtimeInfo.getMaxHeapSizeBytes(), ioConfig.isDeaggregate()); - final int maxRecordsPerPoll = tuningConfig.getMaxRecordsPerPollOrDefault(ioConfig.isDeaggregate()); + final int recordBufferSizeBytes = + tuningConfig.getRecordBufferSizeBytesOrDefault(runtimeInfo.getMaxHeapSizeBytes()); + final int fetchThreads = computeFetchThreads(runtimeInfo, recordBufferSizeBytes, tuningConfig.getFetchThreads()); + final int maxRecordsPerPoll = tuningConfig.getMaxRecordsPerPollOrDefault(); log.info( - "Starting record supplier with fetchThreads [%d], fetchDelayMillis [%d], recordsPerFetch [%d], " - + "recordBufferSize [%d], maxRecordsPerPoll [%d], deaggregate [%s].", + "Starting record supplier with fetchThreads [%d], fetchDelayMillis [%d], " + + "recordBufferSizeBytes [%d], maxRecordsPerPoll [%d]", fetchThreads, ioConfig.getFetchDelayMillis(), - recordsPerFetch, - recordBufferSize, - maxRecordsPerPoll, - ioConfig.isDeaggregate() + recordBufferSizeBytes, + maxRecordsPerPoll ); return new KinesisRecordSupplier( @@ -129,11 +126,9 @@ protected KinesisRecordSupplier newTaskRecordSupplier(final TaskToolbox toolbox) ioConfig.getAwsAssumedRoleArn(), ioConfig.getAwsExternalId() ), - recordsPerFetch, ioConfig.getFetchDelayMillis(), fetchThreads, - ioConfig.isDeaggregate(), - recordBufferSize, + recordBufferSizeBytes, tuningConfig.getRecordBufferOfferTimeout(), tuningConfig.getRecordBufferFullWait(), maxRecordsPerPoll, @@ -179,15 +174,29 @@ AWSCredentialsConfig getAwsCredentialsConfig() } @VisibleForTesting - static int computeFetchThreads(final RuntimeInfo runtimeInfo, final Integer configuredFetchThreads) + static int computeFetchThreads( + final RuntimeInfo runtimeInfo, + final long recordBufferSizeBytes, + final Integer configuredFetchThreads + ) { - final int fetchThreads; + int fetchThreads; if (configuredFetchThreads != null) { fetchThreads = configuredFetchThreads; } else { fetchThreads = runtimeInfo.getAvailableProcessors() * 2; } + // assume that each fetchThread return 10MB (assummed size of aggregated record = 1MB, and + // records per fetch is 10000 max), and cap fetchThreads at this amount. Don't fail if specified + // to be greater than this as to not cause failure for older configurations, but log warning + // if fetchThreads lowered because of this. + int maxFetchThreads = Math.max(1, (int) (recordBufferSizeBytes / 10_000_000L)); + if (fetchThreads > maxFetchThreads) { + log.warn("fetchThreads [%d] being lowered to [%d]", fetchThreads, maxFetchThreads); + fetchThreads = maxFetchThreads; + } + Preconditions.checkArgument( fetchThreads > 0, "Must have at least one background fetch thread for the record supplier" diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java index 0572c3170062..881d68ba8968 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import com.google.common.primitives.Ints; import org.apache.druid.data.input.InputFormat; import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; @@ -41,21 +40,19 @@ public class KinesisIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig kclUserRecordclass = Class.forName("com.amazonaws.services.kinesis.clientlibrary.types.UserRecord"); + MethodHandles.Lookup lookup = MethodHandles.publicLookup(); - Method deaggregateMethod = kclUserRecordclass.getMethod("deaggregate", List.class); - Method getDataMethod = kclUserRecordclass.getMethod("getData"); + Method deaggregateMethod = kclUserRecordclass.getMethod("deaggregate", List.class); + Method getDataMethod = kclUserRecordclass.getMethod("getData"); - deaggregateHandle = lookup.unreflect(deaggregateMethod); - getDataHandle = lookup.unreflect(getDataMethod); - } - catch (ClassNotFoundException e) { - throw new ISE(e, "cannot find class[com.amazonaws.services.kinesis.clientlibrary.types.UserRecord], " - + "note that when using deaggregate=true, you must provide the Kinesis Client Library jar in the classpath"); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } else { - deaggregateHandle = null; - getDataHandle = null; + deaggregateHandle = lookup.unreflect(deaggregateMethod); + getDataHandle = lookup.unreflect(getDataMethod); + } + catch (ClassNotFoundException e) { + throw new ISE(e, "cannot find class[com.amazonaws.services.kinesis.clientlibrary.types.UserRecord], " + + "note that when using deaggregate=true, you must provide the Kinesis Client Library jar in the classpath"); + } + catch (Exception e) { + throw new RuntimeException(e); } if (backgroundFetchEnabled) { @@ -493,7 +479,7 @@ public KinesisRecordSupplier( ); } - records = new MemoryBoundLinkedBlockingQueue<>(queueMaxByteSize); + records = new MemoryBoundLinkedBlockingQueue<>(recordBufferSizeBytes); } public static AmazonKinesis getAmazonKinesisClient( @@ -1065,7 +1051,8 @@ private void filterBufferAndResetBackgroundFetch(Set> pa } // filter records in buffer and only retain ones whose partition was not seeked - MemoryBoundLinkedBlockingQueue> newQ = new MemoryBoundLinkedBlockingQueue<>(recordBufferSize); + MemoryBoundLinkedBlockingQueue> newQ = + new MemoryBoundLinkedBlockingQueue<>(recordBufferSizeBytes); records.stream() .filter(x -> !partitions.contains(x.getData().getStreamPartition())) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpec.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpec.java index 29e909eb0bb7..08d9474479b4 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpec.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpec.java @@ -73,14 +73,12 @@ protected KinesisRecordSupplier createRecordSupplier() ioConfig.getAwsAssumedRoleArn(), ioConfig.getAwsExternalId() ), - ioConfig.getRecordsPerFetch() != null ? ioConfig.getRecordsPerFetch() : DEFAULT_RECORDS_PER_FETCH, ioConfig.getFetchDelayMillis(), 1, - ioConfig.isDeaggregate(), - tuningConfig.getRecordBufferSizeOrDefault(Runtime.getRuntime().maxMemory(), ioConfig.isDeaggregate()), + tuningConfig.getRecordBufferSizeBytesOrDefault(Runtime.getRuntime().maxMemory()), tuningConfig.getRecordBufferOfferTimeout(), tuningConfig.getRecordBufferFullWait(), - tuningConfig.getMaxRecordsPerPollOrDefault(ioConfig.isDeaggregate()), + tuningConfig.getMaxRecordsPerPollOrDefault(), ioConfig.isUseEarliestSequenceNumber(), tuningConfig.isUseListShards() ); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index accd8316f66f..5aae6af06fb2 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -143,11 +143,9 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( maximumMessageTime, ioConfig.getInputFormat(), ioConfig.getEndpoint(), - ioConfig.getRecordsPerFetch(), ioConfig.getFetchDelayMillis(), ioConfig.getAwsAssumedRoleArn(), - ioConfig.getAwsExternalId(), - ioConfig.isDeaggregate() + ioConfig.getAwsExternalId() ); } @@ -197,14 +195,12 @@ protected RecordSupplier setupRecordSupplier() throw ioConfig.getAwsAssumedRoleArn(), ioConfig.getAwsExternalId() ), - 0, // no records-per-fetch, it is not used ioConfig.getFetchDelayMillis(), 0, // skip starting background fetch, it is not used - ioConfig.isDeaggregate(), - taskTuningConfig.getRecordBufferSizeOrDefault(Runtime.getRuntime().maxMemory(), ioConfig.isDeaggregate()), + taskTuningConfig.getRecordBufferSizeBytesOrDefault(Runtime.getRuntime().maxMemory()), taskTuningConfig.getRecordBufferOfferTimeout(), taskTuningConfig.getRecordBufferFullWait(), - taskTuningConfig.getMaxRecordsPerPollOrDefault(ioConfig.isDeaggregate()), + taskTuningConfig.getMaxRecordsPerPollOrDefault(), ioConfig.isUseEarliestSequenceNumber(), spec.getSpec().getTuningConfig().isUseListShards() ); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java index a568aea263ca..daaceed3a1e4 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java @@ -48,12 +48,10 @@ public class KinesisSupervisorIOConfig extends SeekableStreamSupervisorIOConfig // throw ProvisionedThroughputExceededException. Note that GetRecords won't return any data when it throws an // exception. For this reason, we recommend that you wait one second between calls to GetRecords; however, it's // possible that the application will get exceptions for longer than 1 second. - private final Integer recordsPerFetch; private final int fetchDelayMillis; private final String awsAssumedRoleArn; private final String awsExternalId; - private final boolean deaggregate; @JsonCreator public KinesisSupervisorIOConfig( @@ -71,12 +69,10 @@ public KinesisSupervisorIOConfig( @JsonProperty("lateMessageRejectionPeriod") Period lateMessageRejectionPeriod, @JsonProperty("earlyMessageRejectionPeriod") Period earlyMessageRejectionPeriod, @JsonProperty("lateMessageRejectionStartDateTime") DateTime lateMessageRejectionStartDateTime, - @JsonProperty("recordsPerFetch") Integer recordsPerFetch, @JsonProperty("fetchDelayMillis") Integer fetchDelayMillis, @JsonProperty("awsAssumedRoleArn") String awsAssumedRoleArn, @JsonProperty("awsExternalId") String awsExternalId, - @Nullable @JsonProperty("autoScalerConfig") AutoScalerConfig autoScalerConfig, - @JsonProperty("deaggregate") boolean deaggregate + @Nullable @JsonProperty("autoScalerConfig") AutoScalerConfig autoScalerConfig ) { super( @@ -100,13 +96,11 @@ public KinesisSupervisorIOConfig( this.endpoint = endpoint != null ? endpoint : (region != null ? region.getEndpoint() : KinesisRegion.US_EAST_1.getEndpoint()); - this.recordsPerFetch = recordsPerFetch; this.fetchDelayMillis = fetchDelayMillis != null ? fetchDelayMillis : KinesisIndexTaskIOConfig.DEFAULT_FETCH_DELAY_MILLIS; this.awsAssumedRoleArn = awsAssumedRoleArn; this.awsExternalId = awsExternalId; - this.deaggregate = deaggregate; } @JsonProperty @@ -115,14 +109,6 @@ public String getEndpoint() return endpoint; } - @Nullable - @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) - public Integer getRecordsPerFetch() - { - return recordsPerFetch; - } - @JsonProperty @JsonInclude(JsonInclude.Include.NON_DEFAULT) public int getFetchDelayMillis() @@ -144,13 +130,6 @@ public String getAwsExternalId() return awsExternalId; } - @JsonProperty - @JsonInclude(JsonInclude.Include.NON_DEFAULT) - public boolean isDeaggregate() - { - return deaggregate; - } - @Override public String toString() { @@ -168,11 +147,9 @@ public String toString() ", lateMessageRejectionPeriod=" + getLateMessageRejectionPeriod() + ", earlyMessageRejectionPeriod=" + getEarlyMessageRejectionPeriod() + ", lateMessageRejectionStartDateTime=" + getLateMessageRejectionStartDateTime() + - ", recordsPerFetch=" + recordsPerFetch + ", fetchDelayMillis=" + fetchDelayMillis + ", awsAssumedRoleArn='" + awsAssumedRoleArn + '\'' + ", awsExternalId='" + awsExternalId + '\'' + - ", deaggregate=" + deaggregate + '}'; } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java index 0d6af4343aae..bf262144b35d 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java @@ -93,13 +93,10 @@ public void testSerdeWithDefaults() throws Exception Assert.assertTrue(config.isUseTransaction()); Assert.assertFalse("minimumMessageTime", config.getMinimumMessageTime().isPresent()); Assert.assertEquals(config.getEndpoint(), "kinesis.us-east-1.amazonaws.com"); - Assert.assertNull(config.getRecordsPerFetchConfigured()); - Assert.assertEquals(config.getRecordsPerFetchOrDefault(1_000_000_000, 4), 1250); Assert.assertEquals(config.getFetchDelayMillis(), 0); Assert.assertEquals(Collections.emptySet(), config.getStartSequenceNumbers().getExclusivePartitions()); Assert.assertNull(config.getAwsAssumedRoleArn()); Assert.assertNull(config.getAwsExternalId()); - Assert.assertFalse(config.isDeaggregate()); } @Test @@ -150,11 +147,9 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(DateTimes.of("2016-05-31T14:00Z"), config.getMaximumMessageTime().get()); Assert.assertEquals(config.getEndpoint(), "kinesis.us-east-2.amazonaws.com"); Assert.assertEquals(config.getStartSequenceNumbers().getExclusivePartitions(), ImmutableSet.of("0")); - Assert.assertEquals(1000, (int) config.getRecordsPerFetchConfigured()); Assert.assertEquals(1000, config.getFetchDelayMillis()); Assert.assertEquals("role", config.getAwsAssumedRoleArn()); Assert.assertEquals("awsexternalid", config.getAwsExternalId()); - Assert.assertTrue(config.isDeaggregate()); } @Test @@ -272,11 +267,9 @@ public void testDeserializeToOldIoConfig() throws IOException DateTimes.nowUtc(), null, "endpoint", - 1000, 2000, "awsAssumedRoleArn", - "awsExternalId", - true + "awsExternalId" ); final byte[] json = mapper.writeValueAsBytes(currentConfig); @@ -302,11 +295,9 @@ public void testDeserializeToOldIoConfig() throws IOException Assert.assertEquals(currentConfig.getMinimumMessageTime(), oldConfig.getMinimumMessageTime()); Assert.assertEquals(currentConfig.getMaximumMessageTime(), oldConfig.getMaximumMessageTime()); Assert.assertEquals(currentConfig.getEndpoint(), oldConfig.getEndpoint()); - Assert.assertEquals((int) currentConfig.getRecordsPerFetchConfigured(), oldConfig.getRecordsPerFetch()); Assert.assertEquals(currentConfig.getFetchDelayMillis(), oldConfig.getFetchDelayMillis()); Assert.assertEquals(currentConfig.getAwsAssumedRoleArn(), oldConfig.getAwsAssumedRoleArn()); Assert.assertEquals(currentConfig.getAwsExternalId(), oldConfig.getAwsExternalId()); - Assert.assertEquals(currentConfig.isDeaggregate(), oldConfig.isDeaggregate()); } @Test @@ -349,11 +340,9 @@ public void testDeserializeFromOldIoConfig() throws IOException Assert.assertEquals(oldConfig.getMinimumMessageTime(), currentConfig.getMinimumMessageTime()); Assert.assertEquals(oldConfig.getMaximumMessageTime(), currentConfig.getMaximumMessageTime()); Assert.assertEquals(oldConfig.getEndpoint(), currentConfig.getEndpoint()); - Assert.assertEquals(oldConfig.getRecordsPerFetch(), (int) currentConfig.getRecordsPerFetchConfigured()); Assert.assertEquals(oldConfig.getFetchDelayMillis(), currentConfig.getFetchDelayMillis()); Assert.assertEquals(oldConfig.getAwsAssumedRoleArn(), currentConfig.getAwsAssumedRoleArn()); Assert.assertEquals(oldConfig.getAwsExternalId(), currentConfig.getAwsExternalId()); - Assert.assertEquals(oldConfig.isDeaggregate(), currentConfig.isDeaggregate()); } private static class OldKinesisIndexTaskIoConfig implements IOConfig diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java index 2cba3f54187f..09b70500556b 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java @@ -90,9 +90,7 @@ public class KinesisIndexTaskSerdeTest "endpoint", null, null, - null, - null, - false + null ); private static final String ACCESS_KEY = "test-access-key"; private static final String SECRET_KEY = "test-secret-key"; diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 69516979f3e1..356dfa063eda 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -779,9 +779,7 @@ public void testRunWithMinimumMessageTime() throws Exception "awsEndpoint", null, null, - null, - null, - false + null ) ); @@ -843,9 +841,7 @@ public void testRunWithMaximumMessageTime() throws Exception "awsEndpoint", null, null, - null, - null, - false + null ) ); @@ -1935,9 +1931,7 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception "awsEndpoint", null, null, - null, - null, - false + null ), context ); @@ -2099,9 +2093,7 @@ public void testSequencesFromContext() throws IOException "awsEndpoint", null, null, - null, - null, - false + null ), context ); @@ -2252,15 +2244,18 @@ public void testComputeFetchThreads() final DruidProcessingConfigTest.MockRuntimeInfo runtimeInfo = new DruidProcessingConfigTest.MockRuntimeInfo(3, 1000, 2000); - Assert.assertEquals(6, KinesisIndexTask.computeFetchThreads(runtimeInfo, null)); - Assert.assertEquals(2, KinesisIndexTask.computeFetchThreads(runtimeInfo, 2)); + Assert.assertEquals(6, KinesisIndexTask.computeFetchThreads(runtimeInfo, 100_000_000, null)); + Assert.assertEquals(2, KinesisIndexTask.computeFetchThreads(runtimeInfo, 100_000_000, 2)); + + Assert.assertEquals(5, KinesisIndexTask.computeFetchThreads(runtimeInfo, 50_000_000, null)); + Assert.assertEquals(5, KinesisIndexTask.computeFetchThreads(runtimeInfo, 50_000_000, 6)); Assert.assertThrows( IllegalArgumentException.class, - () -> KinesisIndexTask.computeFetchThreads(runtimeInfo, 0) + () -> KinesisIndexTask.computeFetchThreads(runtimeInfo, 100_000_000, 0) ); Assert.assertThrows( IllegalArgumentException.class, - () -> KinesisIndexTask.computeFetchThreads(runtimeInfo, -1) + () -> KinesisIndexTask.computeFetchThreads(runtimeInfo, 100_000_000, -1) ); } @@ -2297,9 +2292,7 @@ private KinesisIndexTask createTask( "awsEndpoint", null, null, - null, - null, - false + null ), null ); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index cd99521c18e8..7f2318bb21c9 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -78,7 +78,9 @@ public void testSerdeWithDefaults() throws Exception Assert.assertFalse(config.isReportParseExceptions()); Assert.assertEquals(Duration.ofMinutes(15).toMillis(), config.getHandoffConditionTimeout()); Assert.assertNull(config.getRecordBufferSizeConfigured()); - Assert.assertEquals(10000, config.getRecordBufferSizeOrDefault(1_000_000_000, false)); + Assert.assertEquals(100_000_000, config.getRecordBufferSizeBytesOrDefault(2_000_000_000)); + Assert.assertEquals(100_000_000, config.getRecordBufferSizeBytesOrDefault(1_000_000_000)); + Assert.assertEquals(10_000_000, config.getRecordBufferSizeBytesOrDefault(100_000_000)); Assert.assertEquals(5000, config.getRecordBufferOfferTimeout()); Assert.assertEquals(5000, config.getRecordBufferFullWait()); Assert.assertNull(config.getFetchThreads()); @@ -126,7 +128,7 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertTrue(config.isReportParseExceptions()); Assert.assertEquals(100, config.getHandoffConditionTimeout()); Assert.assertEquals(1000, (int) config.getRecordBufferSizeConfigured()); - Assert.assertEquals(1000, config.getRecordBufferSizeOrDefault(1_000_000_000, false)); + Assert.assertEquals(1000, config.getRecordBufferSizeBytesOrDefault(1_000_000_000)); Assert.assertEquals(500, config.getRecordBufferOfferTimeout()); Assert.assertEquals(500, config.getRecordBufferFullWait()); Assert.assertEquals(2, (int) config.getFetchThreads()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java index af0755fcd4bb..5d1b77f38584 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java @@ -147,8 +147,6 @@ private static ByteBuffer jb(String timestamp, String dim1, String dim2, String throw new RuntimeException(e); } } - - private static int recordsPerFetch; private static AmazonKinesis kinesis; private static ListShardsResult listShardsResult0; private static ListShardsResult listShardsResult1; @@ -180,7 +178,6 @@ public void setupTest() getRecordsResult1 = createMock(GetRecordsResult.class); shard0 = createMock(Shard.class); shard1 = createMock(Shard.class); - recordsPerFetch = 1; } @After @@ -219,10 +216,8 @@ public void testSupplierSetup_withoutListShards() recordSupplier = new KinesisRecordSupplier( kinesis, - recordsPerFetch, 0, 2, - false, 100, 5000, 5000, @@ -278,10 +273,8 @@ public void testSupplierSetup_withListShards() recordSupplier = new KinesisRecordSupplier( kinesis, - recordsPerFetch, 0, 2, - false, 100, 5000, 5000, @@ -312,7 +305,12 @@ public void testSupplierSetup_withListShards() Assert.assertEquals(expectedRequest1, capturedRequest1.getValue()); } - private static GetRecordsRequest generateGetRecordsReq(String shardIterator, int limit) + private static GetRecordsRequest generateGetRecordsReq(String shardIterator) + { + return new GetRecordsRequest().withShardIterator(shardIterator); + } + + private static GetRecordsRequest generateGetRecordsWithLimitReq(String shardIterator, int limit) { return new GetRecordsRequest().withShardIterator(shardIterator).withLimit(limit); } @@ -326,87 +324,9 @@ private static List> clea .collect(Collectors.toList()); } - @Test - public void testPoll() throws InterruptedException - { - recordsPerFetch = 100; - - EasyMock.expect(kinesis.getShardIterator( - EasyMock.anyObject(), - EasyMock.eq(SHARD_ID0), - EasyMock.anyString(), - EasyMock.anyString() - )).andReturn( - getShardIteratorResult0).anyTimes(); - - EasyMock.expect(kinesis.getShardIterator( - EasyMock.anyObject(), - EasyMock.eq(SHARD_ID1), - EasyMock.anyString(), - EasyMock.anyString() - )).andReturn( - getShardIteratorResult1).anyTimes(); - - EasyMock.expect(getShardIteratorResult0.getShardIterator()).andReturn(SHARD0_ITERATOR).anyTimes(); - EasyMock.expect(getShardIteratorResult1.getShardIterator()).andReturn(SHARD1_ITERATOR).anyTimes(); - EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD0_ITERATOR, recordsPerFetch))) - .andReturn(getRecordsResult0) - .anyTimes(); - EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD1_ITERATOR, recordsPerFetch))) - .andReturn(getRecordsResult1) - .anyTimes(); - EasyMock.expect(getRecordsResult0.getRecords()).andReturn(SHARD0_RECORDS).once(); - EasyMock.expect(getRecordsResult1.getRecords()).andReturn(SHARD1_RECORDS).once(); - EasyMock.expect(getRecordsResult0.getNextShardIterator()).andReturn(null).anyTimes(); - EasyMock.expect(getRecordsResult1.getNextShardIterator()).andReturn(null).anyTimes(); - EasyMock.expect(getRecordsResult0.getMillisBehindLatest()).andReturn(SHARD0_LAG_MILLIS).once(); - EasyMock.expect(getRecordsResult1.getMillisBehindLatest()).andReturn(SHARD1_LAG_MILLIS).once(); - - replayAll(); - - Set> partitions = ImmutableSet.of( - StreamPartition.of(STREAM, SHARD_ID0), - StreamPartition.of(STREAM, SHARD_ID1) - ); - - - recordSupplier = new KinesisRecordSupplier( - kinesis, - recordsPerFetch, - 0, - 2, - false, - 100, - 5000, - 5000, - 100, - true, - false - ); - - recordSupplier.assign(partitions); - recordSupplier.seekToEarliest(partitions); - recordSupplier.start(); - - while (recordSupplier.bufferSize() < 12) { - Thread.sleep(100); - } - - List> polledRecords = cleanRecords(recordSupplier.poll( - POLL_TIMEOUT_MILLIS)); - - verifyAll(); - - Assert.assertEquals(partitions, recordSupplier.getAssignment()); - Assert.assertTrue(polledRecords.containsAll(ALL_RECORDS)); - Assert.assertEquals(SHARDS_LAG_MILLIS, recordSupplier.getPartitionResourcesTimeLag()); - } - @Test public void testPollWithKinesisInternalFailure() throws InterruptedException { - recordsPerFetch = 100; - EasyMock.expect(kinesis.getShardIterator( EasyMock.anyObject(), EasyMock.eq(SHARD_ID0), @@ -425,10 +345,10 @@ public void testPollWithKinesisInternalFailure() throws InterruptedException EasyMock.expect(getShardIteratorResult0.getShardIterator()).andReturn(SHARD0_ITERATOR).anyTimes(); EasyMock.expect(getShardIteratorResult1.getShardIterator()).andReturn(SHARD1_ITERATOR).anyTimes(); - EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD0_ITERATOR, recordsPerFetch))) + EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD0_ITERATOR))) .andReturn(getRecordsResult0) .anyTimes(); - EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD1_ITERATOR, recordsPerFetch))) + EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD1_ITERATOR))) .andReturn(getRecordsResult1) .anyTimes(); AmazonServiceException getException = new AmazonServiceException("InternalFailure"); @@ -460,11 +380,9 @@ public void testPollWithKinesisInternalFailure() throws InterruptedException recordSupplier = new KinesisRecordSupplier( kinesis, - recordsPerFetch, 0, 2, - false, - 100, + 10_000, 5000, 5000, 100, @@ -493,8 +411,6 @@ public void testPollWithKinesisInternalFailure() throws InterruptedException @Test public void testPollWithKinesisNonRetryableFailure() throws InterruptedException { - recordsPerFetch = 100; - EasyMock.expect(kinesis.getShardIterator( EasyMock.anyObject(), EasyMock.eq(SHARD_ID0), @@ -508,7 +424,7 @@ public void testPollWithKinesisNonRetryableFailure() throws InterruptedException getException.setStatusCode(400); getException.setServiceName("AmazonKinesis"); EasyMock.expect(getShardIteratorResult0.getShardIterator()).andReturn(SHARD0_ITERATOR).anyTimes(); - EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD0_ITERATOR, recordsPerFetch))) + EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD0_ITERATOR))) .andThrow(getException) .once(); @@ -521,10 +437,8 @@ public void testPollWithKinesisNonRetryableFailure() throws InterruptedException recordSupplier = new KinesisRecordSupplier( kinesis, - recordsPerFetch, 0, 1, - false, 100, 5000, 5000, @@ -556,8 +470,6 @@ public void testPollWithKinesisNonRetryableFailure() throws InterruptedException public void testSeek() throws InterruptedException { - recordsPerFetch = 100; - EasyMock.expect(kinesis.getShardIterator( EasyMock.anyObject(), EasyMock.eq(SHARD_ID0), @@ -576,10 +488,10 @@ public void testSeek() EasyMock.expect(getShardIteratorResult0.getShardIterator()).andReturn(SHARD0_ITERATOR).anyTimes(); EasyMock.expect(getShardIteratorResult1.getShardIterator()).andReturn(SHARD1_ITERATOR).anyTimes(); - EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD0_ITERATOR, recordsPerFetch))) + EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD0_ITERATOR))) .andReturn(getRecordsResult0) .anyTimes(); - EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD1_ITERATOR, recordsPerFetch))) + EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD1_ITERATOR))) .andReturn(getRecordsResult1) .anyTimes(); EasyMock.expect(getRecordsResult0.getRecords()).andReturn(SHARD0_RECORDS.subList(1, SHARD0_RECORDS.size())).once(); @@ -600,11 +512,9 @@ public void testSeek() recordSupplier = new KinesisRecordSupplier( kinesis, - recordsPerFetch, 0, 2, - false, - 100, + 10_000, 5000, 5000, 100, @@ -636,8 +546,6 @@ public void testSeek() public void testSeekToLatest() throws InterruptedException { - recordsPerFetch = 100; - EasyMock.expect(kinesis.getShardIterator( EasyMock.anyObject(), EasyMock.eq(SHARD_ID0), @@ -668,10 +576,8 @@ public void testSeekToLatest() recordSupplier = new KinesisRecordSupplier( kinesis, - recordsPerFetch, 0, 2, - false, 100, 5000, 5000, @@ -703,10 +609,8 @@ public void testSeekUnassigned() throws InterruptedException recordSupplier = new KinesisRecordSupplier( kinesis, - 1, 0, 2, - false, 100, 5000, 5000, @@ -725,7 +629,6 @@ public void testPollAfterSeek() throws InterruptedException { // tests that after doing a seek, the now invalid records in buffer is cleaned up properly - recordsPerFetch = 100; EasyMock.expect(kinesis.getShardIterator( EasyMock.anyObject(), @@ -745,10 +648,10 @@ public void testPollAfterSeek() EasyMock.expect(getShardIteratorResult1.getShardIterator()).andReturn(SHARD1_ITERATOR).once(); EasyMock.expect(getShardIteratorResult0.getShardIterator()).andReturn(SHARD0_ITERATOR).once(); - EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD1_ITERATOR, recordsPerFetch))) + EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD1_ITERATOR))) .andReturn(getRecordsResult1) .once(); - EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD0_ITERATOR, recordsPerFetch))) + EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD0_ITERATOR))) .andReturn(getRecordsResult0) .once(); EasyMock.expect(getRecordsResult1.getRecords()).andReturn(SHARD1_RECORDS.subList(5, SHARD1_RECORDS.size())).once(); @@ -766,11 +669,9 @@ public void testPollAfterSeek() recordSupplier = new KinesisRecordSupplier( kinesis, - recordsPerFetch, 0, 2, - false, - 100, + 10_000, 5000, 5000, 1, @@ -816,8 +717,6 @@ public void testPollAfterSeek() @Test public void testPollDeaggregate() throws InterruptedException { - recordsPerFetch = 100; - EasyMock.expect(kinesis.getShardIterator( EasyMock.anyObject(), EasyMock.eq(SHARD_ID0), @@ -836,10 +735,10 @@ public void testPollDeaggregate() throws InterruptedException EasyMock.expect(getShardIteratorResult0.getShardIterator()).andReturn(SHARD0_ITERATOR).anyTimes(); EasyMock.expect(getShardIteratorResult1.getShardIterator()).andReturn(SHARD1_ITERATOR).anyTimes(); - EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD0_ITERATOR, recordsPerFetch))) + EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD0_ITERATOR))) .andReturn(getRecordsResult0) .anyTimes(); - EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD1_ITERATOR, recordsPerFetch))) + EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD1_ITERATOR))) .andReturn(getRecordsResult1) .anyTimes(); EasyMock.expect(getRecordsResult0.getRecords()).andReturn(SHARD0_RECORDS).once(); @@ -859,11 +758,9 @@ public void testPollDeaggregate() throws InterruptedException recordSupplier = new KinesisRecordSupplier( kinesis, - recordsPerFetch, 0, 2, - true, - 100, + 10_000, 5000, 5000, 100, @@ -922,7 +819,7 @@ public void getLatestSequenceNumberWhenKinesisRetryableException() EasyMock.expect(getShardIteratorResult0.getShardIterator()).andReturn(SHARD0_ITERATOR).once(); AmazonClientException ex = new AmazonClientException(new IOException()); - EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD0_ITERATOR, 1000))) + EasyMock.expect(kinesis.getRecords(generateGetRecordsWithLimitReq(SHARD0_ITERATOR, 1000))) .andThrow(ex) .andReturn(getRecordsResult0) .once(); @@ -935,11 +832,9 @@ public void getLatestSequenceNumberWhenKinesisRetryableException() recordSupplier = new KinesisRecordSupplier( kinesis, - recordsPerFetch, 0, 2, - true, - 100, + 10_000, 5000, 5000, 100, @@ -961,7 +856,7 @@ private KinesisRecordSupplier getSequenceNumberWhenNoRecordsHelperForOpenShard() EasyMock.expect(getShardIteratorResult0.getShardIterator()).andReturn(SHARD0_ITERATOR).times(1); - EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD0_ITERATOR, 1000))) + EasyMock.expect(kinesis.getRecords(generateGetRecordsWithLimitReq(SHARD0_ITERATOR, 1000))) .andReturn(getRecordsResult0) .times(1); @@ -972,11 +867,9 @@ private KinesisRecordSupplier getSequenceNumberWhenNoRecordsHelperForOpenShard() recordSupplier = new KinesisRecordSupplier( kinesis, - recordsPerFetch, 0, 2, - true, - 100, + 10_000, 5000, 5000, 100, @@ -1033,12 +926,18 @@ public void getPartitionTimeLag() throws InterruptedException EasyMock.expect(getShardIteratorResult0.getShardIterator()).andReturn(SHARD0_ITERATOR).anyTimes(); EasyMock.expect(getShardIteratorResult1.getShardIterator()).andReturn(SHARD1_ITERATOR).anyTimes(); - EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD0_ITERATOR, recordsPerFetch))) + EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD0_ITERATOR))) .andReturn(getRecordsResult0) .anyTimes(); - EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD1_ITERATOR, recordsPerFetch))) + EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD1_ITERATOR))) .andReturn(getRecordsResult1) .anyTimes(); + EasyMock.expect(kinesis.getRecords(generateGetRecordsWithLimitReq(SHARD0_ITERATOR, 1))) + .andReturn(getRecordsResult0) + .anyTimes(); + EasyMock.expect(kinesis.getRecords(generateGetRecordsWithLimitReq(SHARD1_ITERATOR, 1))) + .andReturn(getRecordsResult1) + .anyTimes(); EasyMock.expect(getRecordsResult0.getRecords()).andReturn(SHARD0_RECORDS).times(2); EasyMock.expect(getRecordsResult1.getRecords()).andReturn(SHARD1_RECORDS_EMPTY).times(2); EasyMock.expect(getRecordsResult0.getNextShardIterator()).andReturn(null).anyTimes(); @@ -1055,11 +954,9 @@ public void getPartitionTimeLag() throws InterruptedException recordSupplier = new KinesisRecordSupplier( kinesis, - recordsPerFetch, 0, 2, - true, - 100, + 10_000, 5000, 5000, 100, @@ -1111,10 +1008,8 @@ public void testIsOffsetAvailable() { AmazonKinesis mockKinesis = EasyMock.mock(AmazonKinesis.class); KinesisRecordSupplier target = new KinesisRecordSupplier(mockKinesis, - recordsPerFetch, 0, 2, - false, 100, 5000, 5000, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java index b0ba730a3502..f95c4c6f3841 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java @@ -146,9 +146,7 @@ public void testSample() throws InterruptedException null, null, null, - null, - null, - false + null ), null, null, @@ -224,9 +222,7 @@ public void testSampleWithInputRowParser() throws IOException, InterruptedExcept null, null, null, - null, - null, - false + null ), null, null, @@ -277,9 +273,7 @@ public void testGetInputSourceResources() null, null, null, - null, - null, - false + null ), null, null, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java index fabde1852aad..d06ea43523c7 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java @@ -70,11 +70,9 @@ public void testSerdeWithDefaults() throws Exception Assert.assertFalse("lateMessageRejectionPeriod", config.getLateMessageRejectionPeriod().isPresent()); Assert.assertFalse("earlyMessageRejectionPeriod", config.getEarlyMessageRejectionPeriod().isPresent()); Assert.assertFalse("lateMessageRejectionStartDateTime", config.getLateMessageRejectionStartDateTime().isPresent()); - Assert.assertNull(config.getRecordsPerFetch()); Assert.assertEquals(0, config.getFetchDelayMillis()); Assert.assertNull(config.getAwsAssumedRoleArn()); Assert.assertNull(config.getAwsExternalId()); - Assert.assertFalse(config.isDeaggregate()); } @Test @@ -117,11 +115,9 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(Duration.standardMinutes(45), config.getCompletionTimeout()); Assert.assertEquals(Duration.standardHours(1), config.getLateMessageRejectionPeriod().get()); Assert.assertEquals(Duration.standardHours(1), config.getEarlyMessageRejectionPeriod().get()); - Assert.assertEquals((Integer) 4000, config.getRecordsPerFetch()); Assert.assertEquals(1000, config.getFetchDelayMillis()); Assert.assertEquals("role", config.getAwsAssumedRoleArn()); Assert.assertEquals("awsexternalid", config.getAwsExternalId()); - Assert.assertTrue(config.isDeaggregate()); } @Test diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 39d943dbebef..67bc95b19c3c 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -462,12 +462,10 @@ public void testRecordSupplier() null, null, null, - 100, 1000, null, null, - null, - false + null ); KinesisIndexTaskClientFactory clientFactory = new KinesisIndexTaskClientFactory(null, OBJECT_MAPPER); KinesisSupervisor supervisor = new KinesisSupervisor( @@ -526,12 +524,10 @@ public void testKinesisIOConfigInitAndAutoscalerConfigCreation() null, null, null, - 100, 1000, null, null, - null, - false + null ); AutoScalerConfig autoscalerConfigNull = kinesisSupervisorIOConfigWithNullAutoScalerConfig.getAutoScalerConfig(); @@ -553,12 +549,10 @@ public void testKinesisIOConfigInitAndAutoscalerConfigCreation() null, null, null, - 100, 1000, null, null, - OBJECT_MAPPER.convertValue(new HashMap<>(), AutoScalerConfig.class), - false + OBJECT_MAPPER.convertValue(new HashMap<>(), AutoScalerConfig.class) ); AutoScalerConfig autoscalerConfig = kinesisSupervisorIOConfigWithEmptyAutoScalerConfig.getAutoScalerConfig(); @@ -4104,9 +4098,7 @@ public void testCorrectInputSources() null, null, null, - null, - null, - false + null ), null, null, @@ -5029,9 +5021,7 @@ private TestableKinesisSupervisor getTestableSupervisor( null, null, null, - null, - null, - false + null ); KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory( @@ -5167,12 +5157,10 @@ private TestableKinesisSupervisor getTestableSupervisor( lateMessageRejectionPeriod, earlyMessageRejectionPeriod, null, - recordsPerFetch, fetchDelayMillis, null, null, - autoScalerConfig, - false + autoScalerConfig ); KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory( @@ -5257,12 +5245,10 @@ private TestableKinesisSupervisor getTestableSupervisorCustomIsTaskCurrent( lateMessageRejectionPeriod, earlyMessageRejectionPeriod, null, - recordsPerFetch, fetchDelayMillis, null, null, - null, - false + null ); KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory( @@ -5346,12 +5332,10 @@ private KinesisSupervisor getSupervisor( lateMessageRejectionPeriod, earlyMessageRejectionPeriod, null, - recordsPerFetch, fetchDelayMillis, null, null, - null, - false + null ); KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory( @@ -5495,9 +5479,7 @@ private KinesisIndexTask createKinesisIndexTask( "awsEndpoint", null, null, - null, - null, - false + null ), Collections.emptyMap(), false, From ff7730238a466817f1202860a0c9f59ae561d84c Mon Sep 17 00:00:00 2001 From: zachjsh Date: Tue, 14 Nov 2023 03:24:17 -0500 Subject: [PATCH 04/24] * add tests --- .../MemoryBoundLinkedBlockingQueue.java | 9 +- .../MemoryBoundLinkedBlockingQueueTest.java | 242 ++++++++++++++++++ 2 files changed, 250 insertions(+), 1 deletion(-) create mode 100644 processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java diff --git a/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java b/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java index b0d6efe97935..322757259359 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java @@ -19,6 +19,7 @@ package org.apache.druid.java.util.common; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import java.util.Collection; @@ -38,10 +39,16 @@ public class MemoryBoundLinkedBlockingQueue private final LinkedBlockingQueue> queue; public MemoryBoundLinkedBlockingQueue(long memoryBound) + { + this(new LinkedBlockingQueue<>(), memoryBound); + } + + @VisibleForTesting + MemoryBoundLinkedBlockingQueue(LinkedBlockingQueue> queue, long memoryBound) { this.memoryBound = memoryBound; this.currentMemory = new AtomicLong(0L); - this.queue = new LinkedBlockingQueue<>(); + this.queue = queue; } // returns true/false depending on whether item was added or not diff --git a/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java b/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java new file mode 100644 index 000000000000..c72ca62562b5 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.druid.java.util.common; + +import com.google.common.collect.ImmutableList; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class MemoryBoundLinkedBlockingQueueTest +{ + @Test + public void test_offer_emptyQueueWithEnoughCapacity_true() + { + + long byteCapacity = 10L; + MemoryBoundLinkedBlockingQueue queue = setupQueue(byteCapacity, ImmutableList.of()); + byte[] item = "item".getBytes(StandardCharsets.UTF_8); + + boolean succeeds = queue.offer(new MemoryBoundLinkedBlockingQueue.ObjectContainer<>(item, item.length)); + + long expectedByteSize = item.length; + Assert.assertTrue(succeeds); + Assert.assertEquals(1, queue.size()); + Assert.assertEquals(expectedByteSize, queue.byteSize()); + Assert.assertEquals(byteCapacity - item.length, queue.remainingCapacity()); + } + + @Test + public void test_offer_nonEmptyQueueWithEnoughCapacity_true() + { + + long byteCapacity = 10L; + byte[] item1 = "item1".getBytes(StandardCharsets.UTF_8); + byte[] item2 = "item2".getBytes(StandardCharsets.UTF_8); + Collection> items = buildItemContainers( + ImmutableList.of(item1) + ); + MemoryBoundLinkedBlockingQueue queue = setupQueue(byteCapacity, items); + + boolean succeeds = queue.offer(new MemoryBoundLinkedBlockingQueue.ObjectContainer<>(item2, item2.length)); + + long expectedByteSize = item1.length + item2.length; + Assert.assertTrue(succeeds); + Assert.assertEquals(2, queue.size()); + Assert.assertEquals(expectedByteSize, queue.byteSize()); + Assert.assertEquals(byteCapacity - expectedByteSize, queue.remainingCapacity()); + } + + @Test + public void test_offer_queueWithoutEnoughCapacity_false() + { + + long byteCapacity = 7L; + byte[] item1 = "item1".getBytes(StandardCharsets.UTF_8); + byte[] item2 = "item2".getBytes(StandardCharsets.UTF_8); + Collection> items = buildItemContainers( + ImmutableList.of(item1) + ); + MemoryBoundLinkedBlockingQueue queue = setupQueue(byteCapacity, items); + + boolean succeeds = queue.offer(new MemoryBoundLinkedBlockingQueue.ObjectContainer<>(item2, item2.length)); + + long expectedByteSize = item1.length; + Assert.assertFalse(succeeds); + Assert.assertEquals(1, queue.size()); + Assert.assertEquals(expectedByteSize, queue.byteSize()); + Assert.assertEquals(byteCapacity - expectedByteSize, queue.remainingCapacity()); + } + + @Test + public void test_offerWithTimeLimit_interruptedExceptinThrown_throws() + { + long byteCapacity = 10L; + MemoryBoundLinkedBlockingQueue queue = setupQueue( + byteCapacity, + ImmutableList.of(), + new InterruptedExceptionThrowingQueue() + ); + byte[] item = "item".getBytes(StandardCharsets.UTF_8); + + Assert.assertThrows( + InterruptedException.class, + () -> queue.offer( + new MemoryBoundLinkedBlockingQueue.ObjectContainer<>(item, item.length), + 1L, + TimeUnit.MILLISECONDS + ) + ); + + Assert.assertEquals(0, queue.size()); + Assert.assertEquals(0L, queue.byteSize()); + Assert.assertEquals(byteCapacity, queue.remainingCapacity()); + } + + @Test + public void test_take_nonEmptyQueue_expected() throws InterruptedException + { + + long byteCapacity = 10L; + byte[] item1 = "item1".getBytes(StandardCharsets.UTF_8); + byte[] item2 = "item2".getBytes(StandardCharsets.UTF_8); + MemoryBoundLinkedBlockingQueue.ObjectContainer item1Container = + new MemoryBoundLinkedBlockingQueue.ObjectContainer<>(item1, item1.length); + MemoryBoundLinkedBlockingQueue.ObjectContainer item2Container = + new MemoryBoundLinkedBlockingQueue.ObjectContainer<>(item2, item2.length); + MemoryBoundLinkedBlockingQueue queue = setupQueue(byteCapacity, ImmutableList.of()); + Assert.assertTrue(queue.offer(item1Container)); + Assert.assertTrue(queue.offer(item2Container)); + + MemoryBoundLinkedBlockingQueue.ObjectContainer takenItem = queue.take(); + long expectedByteSize = item2.length; + + Assert.assertSame(item1Container, takenItem); + Assert.assertEquals(1, queue.size()); + Assert.assertEquals(expectedByteSize, queue.byteSize()); + Assert.assertEquals(byteCapacity - expectedByteSize, queue.remainingCapacity()); + } + + @Test + public void test_drain_queueWithOneItem_succeeds() throws InterruptedException + { + + long byteCapacity = 7L; + byte[] item1 = "item1".getBytes(StandardCharsets.UTF_8); + Collection> items = buildItemContainers( + ImmutableList.of(item1) + ); + MemoryBoundLinkedBlockingQueue queue = setupQueue(byteCapacity, items); + Collection> buffer = new ArrayList<>(); + + int numAdded = queue.drain(buffer, 1, 1, TimeUnit.MINUTES); + + Assert.assertTrue(numAdded == 1 && numAdded == buffer.size()); + Assert.assertEquals(0, queue.size()); + Assert.assertEquals(0L, queue.byteSize()); + Assert.assertEquals(byteCapacity, queue.remainingCapacity()); + } + + @Test + public void test_drain_queueWithMultipleItems_succeeds() throws InterruptedException + { + + long byteCapacity = 10L; + byte[] item1 = "item1".getBytes(StandardCharsets.UTF_8); + byte[] item2 = "item2".getBytes(StandardCharsets.UTF_8); + Collection> items = buildItemContainers( + ImmutableList.of(item1, item2) + ); + MemoryBoundLinkedBlockingQueue queue = setupQueue(byteCapacity, items, new NotAllDrainedQueue()); + Collection> buffer = new ArrayList<>(); + + int numAdded = queue.drain(buffer, 2, 1, TimeUnit.MINUTES); + + Assert.assertTrue(numAdded == 2 && numAdded == buffer.size()); + Assert.assertEquals(0, queue.size()); + Assert.assertEquals(0L, queue.byteSize()); + Assert.assertEquals(byteCapacity, queue.remainingCapacity()); + } + + private static MemoryBoundLinkedBlockingQueue setupQueue( + long byteCapacity, + Collection> items + ) + { + return setupQueue(byteCapacity, items, null); + } + + private static MemoryBoundLinkedBlockingQueue setupQueue( + long byteCapacity, + Collection> items, + @Nullable LinkedBlockingQueue> underlyingQueue + ) + { + Assert.assertTrue(getTotalSizeOfItems(items) <= byteCapacity); + MemoryBoundLinkedBlockingQueue queue = underlyingQueue != null ? + new MemoryBoundLinkedBlockingQueue<>(underlyingQueue, byteCapacity) : new MemoryBoundLinkedBlockingQueue<>(byteCapacity); + items.forEach(i -> Assert.assertTrue(queue.offer(i))); + return queue; + } + + private static Collection> buildItemContainers( + Collection items + ) + { + return items.stream() + .map(i -> new MemoryBoundLinkedBlockingQueue.ObjectContainer<>(i, i.length)) + .collect(Collectors.toList()); + } + + private static long getTotalSizeOfItems(Collection> items) + { + return items.stream().mapToLong(MemoryBoundLinkedBlockingQueue.ObjectContainer::getSize).sum(); + } + + static class InterruptedExceptionThrowingQueue + extends LinkedBlockingQueue> + { + @Override + public boolean offer(MemoryBoundLinkedBlockingQueue.ObjectContainer item, long timeout, TimeUnit unit) + throws InterruptedException + { + throw new InterruptedException("exception thrown"); + } + } + + static class NotAllDrainedQueue + extends LinkedBlockingQueue> + { + @Override + public int drainTo(Collection> c, int maxElements) + { + MemoryBoundLinkedBlockingQueue.ObjectContainer firstItem = this.poll(); + c.add(firstItem); + return 1; + } + } +} From 7b155f64161a03904d3a3dc98a5d3b9becae1292 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Tue, 14 Nov 2023 03:45:08 -0500 Subject: [PATCH 05/24] * fix serde --- .../indexing/kinesis/KinesisIndexTaskTuningConfig.java | 6 +++--- .../kinesis/supervisor/KinesisSupervisorTuningConfig.java | 4 ++-- .../indexing/kinesis/KinesisIndexTaskTuningConfigTest.java | 2 +- .../test/TestModifiedKinesisIndexTaskTuningConfig.java | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java index 4f6d96067f5a..34243ef6e729 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java @@ -144,7 +144,7 @@ private KinesisIndexTaskTuningConfig( @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, @JsonProperty("skipSequenceNumberAvailabilityCheck") Boolean skipSequenceNumberAvailabilityCheck, - @JsonProperty("recordBufferSize") Integer recordBufferSizeBytes, + @JsonProperty("recordBufferSizeBytes") Integer recordBufferSizeBytes, @JsonProperty("recordBufferOfferTimeout") Integer recordBufferOfferTimeout, @JsonProperty("recordBufferFullWait") Integer recordBufferFullWait, @JsonProperty("fetchThreads") Integer fetchThreads, @@ -186,7 +186,7 @@ private KinesisIndexTaskTuningConfig( } @Nullable - @JsonProperty("recordBufferSize") + @JsonProperty("recordBufferSizeBytes") @JsonInclude(JsonInclude.Include.NON_NULL) public Integer getRecordBufferSizeConfigured() { @@ -319,7 +319,7 @@ public String toString() ", handoffConditionTimeout=" + getHandoffConditionTimeout() + ", resetOffsetAutomatically=" + isResetOffsetAutomatically() + ", skipSequenceNumberAvailabilityCheck=" + isSkipSequenceNumberAvailabilityCheck() + - ", recordBufferSize=" + recordBufferSizeBytes + + ", recordBufferSizeBytes=" + recordBufferSizeBytes + ", recordBufferOfferTimeout=" + recordBufferOfferTimeout + ", recordBufferFullWait=" + recordBufferFullWait + ", fetchThreads=" + fetchThreads + diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java index 357bc9e57ca4..c9a321e3cc39 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -98,7 +98,7 @@ public KinesisSupervisorTuningConfig( @JsonProperty("chatRetries") Long chatRetries, @JsonProperty("httpTimeout") Period httpTimeout, @JsonProperty("shutdownTimeout") Period shutdownTimeout, - @JsonProperty("recordBufferSize") Integer recordBufferSize, + @JsonProperty("recordBufferSizeBytes") Integer recordBufferSizeBytes, @JsonProperty("recordBufferOfferTimeout") Integer recordBufferOfferTimeout, @JsonProperty("recordBufferFullWait") Integer recordBufferFullWait, @JsonProperty("fetchThreads") Integer fetchThreads, @@ -128,7 +128,7 @@ public KinesisSupervisorTuningConfig( handoffConditionTimeout, resetOffsetAutomatically, skipSequenceNumberAvailabilityCheck, - recordBufferSize, + recordBufferSizeBytes, recordBufferOfferTimeout, recordBufferFullWait, fetchThreads, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index 7f2318bb21c9..3f064450426f 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -100,7 +100,7 @@ public void testSerdeWithNonDefaults() throws Exception + " \"maxPendingPersists\": 100,\n" + " \"reportParseExceptions\": true,\n" + " \"handoffConditionTimeout\": 100,\n" - + " \"recordBufferSize\": 1000,\n" + + " \"recordBufferSizeBytes\": 1000,\n" + " \"recordBufferOfferTimeout\": 500,\n" + " \"recordBufferFullWait\": 500,\n" + " \"resetOffsetAutomatically\": false,\n" diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java index 70611266efe3..9933d61a7df1 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java @@ -51,7 +51,7 @@ public TestModifiedKinesisIndexTaskTuningConfig( @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, @JsonProperty("skipSequenceNumberAvailabilityCheck") Boolean skipSequenceNumberAvailabilityCheck, - @JsonProperty("recordBufferSize") Integer recordBufferSize, + @JsonProperty("recordBufferSizeBytes") Integer recordBufferSizeBytes, @JsonProperty("recordBufferOfferTimeout") Integer recordBufferOfferTimeout, @JsonProperty("recordBufferFullWait") Integer recordBufferFullWait, @JsonProperty("fetchThreads") Integer fetchThreads, @@ -80,7 +80,7 @@ public TestModifiedKinesisIndexTaskTuningConfig( handoffConditionTimeout, resetOffsetAutomatically, skipSequenceNumberAvailabilityCheck, - recordBufferSize, + recordBufferSizeBytes, recordBufferOfferTimeout, recordBufferFullWait, fetchThreads, From 74e0ad23ee8de541f59509cef71f1f89302ee329 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Tue, 14 Nov 2023 03:55:23 -0500 Subject: [PATCH 06/24] * minor fix to logging --- .../apache/druid/indexing/kinesis/KinesisRecordSupplier.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index f6be70e786f5..6ce59d531bd0 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -275,7 +275,7 @@ private Runnable fetchRecords() if (log.isTraceEnabled()) { log.trace( - "Stream[%s] / partition[%s] / sequenceNum[%s] / bufferByteCapacity[%d] / bufferRemainingByteCapacity[%d]: %s", + "Stream[%s] / partition[%s] / sequenceNum[%s] / bufferByteSize[%d] / bufferRemainingByteCapacity[%d]: %s", currRecord.getStream(), currRecord.getPartitionId(), currRecord.getSequenceNumber(), From 9547251103d9d4b902e388e820e64d9b256b0790 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Tue, 14 Nov 2023 04:00:41 -0500 Subject: [PATCH 07/24] * update comment --- .../apache/druid/indexing/kinesis/KinesisIndexTask.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 46ccf5aee96a..82b1e5c3b86d 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -187,10 +187,11 @@ static int computeFetchThreads( fetchThreads = runtimeInfo.getAvailableProcessors() * 2; } - // assume that each fetchThread return 10MB (assummed size of aggregated record = 1MB, and - // records per fetch is 10000 max), and cap fetchThreads at this amount. Don't fail if specified - // to be greater than this as to not cause failure for older configurations, but log warning - // if fetchThreads lowered because of this. + // Each fetchThread can return upto 10MB at a time + // (https://docs.aws.amazon.com/streams/latest/dev/service-sizes-and-limits.html), cap fetchThreads so that + // we don't exceed the max buffer byte size worth of records at a time. Don't fail if fetchThreads specified + // is greater than this as to not cause failure for older configurations, but log warning in this case, and lower + // fetchThreads implicitly. int maxFetchThreads = Math.max(1, (int) (recordBufferSizeBytes / 10_000_000L)); if (fetchThreads > maxFetchThreads) { log.warn("fetchThreads [%d] being lowered to [%d]", fetchThreads, maxFetchThreads); From d7f9c26edef9f277ca9675b9a1d9870c5466a6a5 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Wed, 15 Nov 2023 17:55:19 -0500 Subject: [PATCH 08/24] * remove references to removed config properties from documentation and web-console * fix fetchThreads calculation --- .../extensions-core/kinesis-ingestion.md | 26 +++++++---------- .../indexing/kinesis/KinesisIndexTask.java | 14 ++++++--- .../kinesis/KinesisIndexTaskTuningConfig.java | 4 +-- .../KinesisSupervisorTuningConfig.java | 4 +-- .../indexing/kinesis/KinesisIOConfigTest.java | 29 ++----------------- .../kinesis/KinesisIndexTaskTest.java | 16 +++++----- .../KinesisIndexTaskTuningConfigTest.java | 10 +++---- .../KinesisSupervisorIOConfigTest.java | 4 +-- .../supervisor/KinesisSupervisorTest.java | 9 ------ ...tModifiedKinesisIndexTaskTuningConfig.java | 2 +- .../ingestion-spec/ingestion-spec.tsx | 22 +++----------- 11 files changed, 47 insertions(+), 93 deletions(-) diff --git a/docs/development/extensions-core/kinesis-ingestion.md b/docs/development/extensions-core/kinesis-ingestion.md index 0c3066d2f79d..c3d8143f8b66 100644 --- a/docs/development/extensions-core/kinesis-ingestion.md +++ b/docs/development/extensions-core/kinesis-ingestion.md @@ -241,11 +241,9 @@ The following table outlines the configuration options for `ioConfig`: |`completionTimeout`|ISO 8601 period|The length of time to wait before Druid declares a publishing task has failed and terminates it. If this is set too low, your tasks may never publish. The publishing clock for a task begins roughly after `taskDuration` elapses.|No|PT6H| |`lateMessageRejectionPeriod`|ISO 8601 period|Configure tasks to reject messages with timestamps earlier than this period before the task is created. For example, if `lateMessageRejectionPeriod` is set to `PT1H` and the supervisor creates a task at `2016-01-01T12:00Z`, messages with timestamps earlier than `2016-01-01T11:00Z` are dropped. This may help prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments, such as a streaming and a nightly batch ingestion pipeline.|No|| |`earlyMessageRejectionPeriod`|ISO 8601 period|Configure tasks to reject messages with timestamps later than this period after the task reached its `taskDuration`. For example, if `earlyMessageRejectionPeriod` is set to `PT1H`, the `taskDuration` is set to `PT1H` and the supervisor creates a task at `2016-01-01T12:00Z`. Messages with timestamps later than `2016-01-01T14:00Z` are dropped. **Note:** Tasks sometimes run past their task duration, for example, in cases of supervisor failover. Setting `earlyMessageRejectionPeriod` too low may cause messages to be dropped unexpectedly whenever a task runs past its originally configured task duration.|No|| -|`recordsPerFetch`|Integer|The number of records to request per call to fetch records from Kinesis.|No| See [Determine fetch settings](#determine-fetch-settings) for defaults.| |`fetchDelayMillis`|Integer|Time in milliseconds to wait between subsequent calls to fetch records from Kinesis. See [Determine fetch settings](#determine-fetch-settings).|No|0| |`awsAssumedRoleArn`|String|The AWS assumed role to use for additional permissions.|No|| |`awsExternalId`|String|The AWS external ID to use for additional permissions.|No|| -|`deaggregate`|Boolean|Whether to use the deaggregate function of the Kinesis Client Library (KCL).|No|| |`autoScalerConfig`|Object|Defines autoscaling behavior for Kinesis ingest tasks. See [Task autoscaler properties](#task-autoscaler-properties) for more information.|No|null| ### Task autoscaler properties @@ -406,7 +404,7 @@ The following table outlines the configuration options for `tuningConfig`: |`chatRetries`|Integer|The number of times Druid retries HTTP requests to indexing tasks before considering tasks unresponsive.|No|8| |`httpTimeout`|ISO 8601 period|The period of time to wait for a HTTP response from an indexing task.|No|PT10S| |`shutdownTimeout`|ISO 8601 period|The period of time to wait for the supervisor to attempt a graceful shutdown of tasks before exiting.|No|PT80S| -|`recordBufferSize`|Integer|The size of the buffer (number of events) Druid uses between the Kinesis fetch threads and the main ingestion thread.|No|See [Determine fetch settings](#determine-fetch-settings) for defaults.| +|`recordBufferSizeBytes`|Integer| The size of the buffer (heap memory bytes) Druid uses between the Kinesis fetch threads and the main ingestion thread. |No|See [Determine fetch settings](#determine-fetch-settings) for defaults.| |`recordBufferOfferTimeout`|Integer|The number of milliseconds to wait for space to become available in the buffer before timing out.|No|5000| |`recordBufferFullWait`|Integer|The number of milliseconds to wait for the buffer to drain before Druid attempts to fetch records from Kinesis again.|No|5000| |`fetchThreads`|Integer|The size of the pool of threads fetching data from Kinesis. There is no benefit in having more threads than Kinesis shards.|No| `procs * 2`, where `procs` is the number of processors available to the task.| @@ -419,6 +417,7 @@ The following table outlines the configuration options for `tuningConfig`: |`offsetFetchPeriod`|ISO 8601 period|Determines how often the supervisor queries Kinesis and the indexing tasks to fetch current offsets and calculate lag. If the user-specified value is below the minimum value of PT5S, the supervisor ignores the value and uses the minimum value instead.|No|PT30S| |`useListShards`|Boolean|Indicates if `listShards` API of AWS Kinesis SDK can be used to prevent `LimitExceededException` during ingestion. You must set the necessary `IAM` permissions.|No|`false`| + ### IndexSpec The following table outlines the configuration options for `indexSpec`: @@ -656,25 +655,22 @@ For more detail, see [Segment size optimization](../../operations/segment-optimi Kinesis indexing tasks fetch records using `fetchThreads` threads. If `fetchThreads` is higher than the number of Kinesis shards, the excess threads are unused. -Each fetch thread fetches up to `recordsPerFetch` records at once from a Kinesis shard, with a delay between fetches +Each fetch thread fetches up to 10MB of records at once from a Kinesis shard, with a delay between fetches of `fetchDelayMillis`. -The records fetched by each thread are pushed into a shared queue of size `recordBufferSize`. +The records fetched by each thread are pushed into a shared queue of size `recordBufferSizeBytes`. The main runner thread for each task polls up to `maxRecordsPerPoll` records from the queue at once. -When using Kinesis Producer Library's aggregation feature, that is when [`deaggregate`](#deaggregation) is set, -each of these parameters refers to aggregated records rather than individual records. - The default values for these parameters are: - `fetchThreads`: Twice the number of processors available to the task. The number of processors available to the task is the total number of processors on the server, divided by `druid.worker.capacity` (the number of task slots on that -particular server). +particular server). This value is further limited so that the total data record data fetched at a given time does not +exceed 5% of the max heap configured, assuming that each thread fetches 10MB of recrods at once. If the value specified +for this configuration is higher than this limit, no failure occurs, but a warning is logged, and the value is +implicitly lowered to the max allowed by this constraint. - `fetchDelayMillis`: 0 (no delay between fetches). -- `recordsPerFetch`: 100 MB or an estimated 5% of available heap, whichever is smaller, divided by `fetchThreads`. -For estimation purposes, Druid uses a figure of 10 KB for regular records and 1 MB for [aggregated records](#deaggregation). -- `recordBufferSize`: 100 MB or an estimated 10% of available heap, whichever is smaller. -For estimation purposes, Druid uses a figure of 10 KB for regular records and 1 MB for [aggregated records](#deaggregation). -- `maxRecordsPerPoll`: 100 for regular records, 1 for [aggregated records](#deaggregation). +- `recordBufferSizeBytes`: 100 MB or an estimated 10% of available heap, whichever is smaller. +- `maxRecordsPerPoll`: 1. Kinesis places the following restrictions on calls to fetch records: @@ -697,8 +693,6 @@ Kinesis stream. The Kinesis indexing service supports de-aggregation of multiple rows packed into a single record by the Kinesis Producer Library's aggregate method for more efficient data transfer. -To enable this feature, set `deaggregate` to true in your `ioConfig` when submitting a supervisor spec. - ## Resharding When changing the shard count for a Kinesis stream, there is a window of time around the resharding operation with early shutdown of Kinesis ingestion tasks and possible task failures. diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 82b1e5c3b86d..219999be4ad4 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -107,7 +107,7 @@ protected KinesisRecordSupplier newTaskRecordSupplier(final TaskToolbox toolbox) KinesisIndexTaskTuningConfig tuningConfig = ((KinesisIndexTaskTuningConfig) super.tuningConfig); final int recordBufferSizeBytes = tuningConfig.getRecordBufferSizeBytesOrDefault(runtimeInfo.getMaxHeapSizeBytes()); - final int fetchThreads = computeFetchThreads(runtimeInfo, recordBufferSizeBytes, tuningConfig.getFetchThreads()); + final int fetchThreads = computeFetchThreads(runtimeInfo, tuningConfig.getFetchThreads()); final int maxRecordsPerPoll = tuningConfig.getMaxRecordsPerPollOrDefault(); log.info( @@ -176,7 +176,6 @@ AWSCredentialsConfig getAwsCredentialsConfig() @VisibleForTesting static int computeFetchThreads( final RuntimeInfo runtimeInfo, - final long recordBufferSizeBytes, final Integer configuredFetchThreads ) { @@ -189,10 +188,17 @@ static int computeFetchThreads( // Each fetchThread can return upto 10MB at a time // (https://docs.aws.amazon.com/streams/latest/dev/service-sizes-and-limits.html), cap fetchThreads so that - // we don't exceed the max buffer byte size worth of records at a time. Don't fail if fetchThreads specified + // we don't exceed more than the least of 100MB or 5% of heap at a time. Don't fail if fetchThreads specified // is greater than this as to not cause failure for older configurations, but log warning in this case, and lower // fetchThreads implicitly. - int maxFetchThreads = Math.max(1, (int) (recordBufferSizeBytes / 10_000_000L)); + final long memoryToUse = Math.min( + KinesisIndexTaskIOConfig.MAX_RECORD_FETCH_MEMORY, + (long) (runtimeInfo.getMaxHeapSizeBytes() * KinesisIndexTaskIOConfig.RECORD_FETCH_MEMORY_MAX_HEAP_FRACTION) + ); + int maxFetchThreads = Math.max( + 1, + (int) (memoryToUse / 10_000_000L) + ); if (fetchThreads > maxFetchThreads) { log.warn("fetchThreads [%d] being lowered to [%d]", fetchThreads, maxFetchThreads); fetchThreads = maxFetchThreads; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java index 34243ef6e729..a703628303b5 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java @@ -188,7 +188,7 @@ private KinesisIndexTaskTuningConfig( @Nullable @JsonProperty("recordBufferSizeBytes") @JsonInclude(JsonInclude.Include.NON_NULL) - public Integer getRecordBufferSizeConfigured() + public Integer getRecordBufferSizeBytesConfigured() { return recordBufferSizeBytes; } @@ -257,7 +257,7 @@ public KinesisIndexTaskTuningConfig withBasePersistDirectory(File dir) getHandoffConditionTimeout(), isResetOffsetAutomatically(), isSkipSequenceNumberAvailabilityCheck(), - getRecordBufferSizeConfigured(), + getRecordBufferSizeBytesConfigured(), getRecordBufferOfferTimeout(), getRecordBufferFullWait(), getFetchThreads(), diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java index c9a321e3cc39..0e40ff633c1b 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -225,7 +225,7 @@ public String toString() ", chatRetries=" + chatRetries + ", httpTimeout=" + httpTimeout + ", shutdownTimeout=" + shutdownTimeout + - ", recordBufferSize=" + getRecordBufferSizeConfigured() + + ", recordBufferSizeBytes=" + getRecordBufferSizeBytesConfigured() + ", recordBufferOfferTimeout=" + getRecordBufferOfferTimeout() + ", recordBufferFullWait=" + getRecordBufferFullWait() + ", fetchThreads=" + getFetchThreads() + @@ -259,7 +259,7 @@ public KinesisIndexTaskTuningConfig convertToTaskTuningConfig() getHandoffConditionTimeout(), isResetOffsetAutomatically(), isSkipSequenceNumberAvailabilityCheck(), - getRecordBufferSizeConfigured(), + getRecordBufferSizeBytesConfigured(), getRecordBufferOfferTimeout(), getRecordBufferFullWait(), getFetchThreads(), diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java index bf262144b35d..3162b2ea0eee 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java @@ -112,11 +112,9 @@ public void testSerdeWithNonDefaults() throws Exception + " \"minimumMessageTime\": \"2016-05-31T12:00Z\",\n" + " \"maximumMessageTime\": \"2016-05-31T14:00Z\",\n" + " \"endpoint\": \"kinesis.us-east-2.amazonaws.com\",\n" - + " \"recordsPerFetch\": 1000,\n" + " \"fetchDelayMillis\": 1000,\n" + " \"awsAssumedRoleArn\": \"role\",\n" - + " \"awsExternalId\": \"awsexternalid\",\n" - + " \"deaggregate\": true\n" + + " \"awsExternalId\": \"awsexternalid\"\n" + "}"; KinesisIndexTaskIOConfig config = (KinesisIndexTaskIOConfig) mapper.readValue( @@ -315,11 +313,9 @@ public void testDeserializeFromOldIoConfig() throws IOException DateTimes.nowUtc(), DateTimes.nowUtc(), "endpoint", - 1000, 2000, "awsAssumedRoleArn", - "awsExternalId", - true + "awsExternalId" ); final byte[] json = oldMapper.writeValueAsBytes(oldConfig); @@ -355,12 +351,9 @@ private static class OldKinesisIndexTaskIoConfig implements IOConfig private final Optional minimumMessageTime; private final Optional maximumMessageTime; private final String endpoint; - private final Integer recordsPerFetch; private final Integer fetchDelayMillis; - private final String awsAssumedRoleArn; private final String awsExternalId; - private final boolean deaggregate; @JsonCreator private OldKinesisIndexTaskIoConfig( @@ -372,11 +365,9 @@ private OldKinesisIndexTaskIoConfig( @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, @JsonProperty("maximumMessageTime") DateTime maximumMessageTime, @JsonProperty("endpoint") String endpoint, - @JsonProperty("recordsPerFetch") Integer recordsPerFetch, @JsonProperty("fetchDelayMillis") Integer fetchDelayMillis, @JsonProperty("awsAssumedRoleArn") String awsAssumedRoleArn, - @JsonProperty("awsExternalId") String awsExternalId, - @JsonProperty("deaggregate") boolean deaggregate + @JsonProperty("awsExternalId") String awsExternalId ) { this.baseSequenceName = baseSequenceName; @@ -387,11 +378,9 @@ private OldKinesisIndexTaskIoConfig( this.minimumMessageTime = Optional.fromNullable(minimumMessageTime); this.maximumMessageTime = Optional.fromNullable(maximumMessageTime); this.endpoint = endpoint; - this.recordsPerFetch = recordsPerFetch; this.fetchDelayMillis = fetchDelayMillis; this.awsAssumedRoleArn = awsAssumedRoleArn; this.awsExternalId = awsExternalId; - this.deaggregate = deaggregate; } @JsonProperty @@ -442,12 +431,6 @@ public String getEndpoint() return endpoint; } - @JsonProperty - public int getRecordsPerFetch() - { - return recordsPerFetch; - } - @JsonProperty public int getFetchDelayMillis() { @@ -465,11 +448,5 @@ public String getAwsExternalId() { return awsExternalId; } - - @JsonProperty - public boolean isDeaggregate() - { - return deaggregate; - } } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 356dfa063eda..df0c31b615d9 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -2242,20 +2242,22 @@ public void testRunWithoutDataInserted() throws Exception public void testComputeFetchThreads() { final DruidProcessingConfigTest.MockRuntimeInfo runtimeInfo = - new DruidProcessingConfigTest.MockRuntimeInfo(3, 1000, 2000); + new DruidProcessingConfigTest.MockRuntimeInfo(3, 1000, 10_000_000_000L); - Assert.assertEquals(6, KinesisIndexTask.computeFetchThreads(runtimeInfo, 100_000_000, null)); - Assert.assertEquals(2, KinesisIndexTask.computeFetchThreads(runtimeInfo, 100_000_000, 2)); + Assert.assertEquals(6, KinesisIndexTask.computeFetchThreads(runtimeInfo, null)); + Assert.assertEquals(2, KinesisIndexTask.computeFetchThreads(runtimeInfo, 2)); - Assert.assertEquals(5, KinesisIndexTask.computeFetchThreads(runtimeInfo, 50_000_000, null)); - Assert.assertEquals(5, KinesisIndexTask.computeFetchThreads(runtimeInfo, 50_000_000, 6)); + final DruidProcessingConfigTest.MockRuntimeInfo runtimeInfo2 = + new DruidProcessingConfigTest.MockRuntimeInfo(3, 1000, 1_000_000_000); + Assert.assertEquals(5, KinesisIndexTask.computeFetchThreads(runtimeInfo2, null)); + Assert.assertEquals(5, KinesisIndexTask.computeFetchThreads(runtimeInfo2, 6)); Assert.assertThrows( IllegalArgumentException.class, - () -> KinesisIndexTask.computeFetchThreads(runtimeInfo, 100_000_000, 0) + () -> KinesisIndexTask.computeFetchThreads(runtimeInfo, 0) ); Assert.assertThrows( IllegalArgumentException.class, - () -> KinesisIndexTask.computeFetchThreads(runtimeInfo, 100_000_000, -1) + () -> KinesisIndexTask.computeFetchThreads(runtimeInfo, -1) ); } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index 3f064450426f..9055ee38b0a2 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -77,7 +77,7 @@ public void testSerdeWithDefaults() throws Exception Assert.assertEquals(IndexSpec.DEFAULT, config.getIndexSpec()); Assert.assertFalse(config.isReportParseExceptions()); Assert.assertEquals(Duration.ofMinutes(15).toMillis(), config.getHandoffConditionTimeout()); - Assert.assertNull(config.getRecordBufferSizeConfigured()); + Assert.assertNull(config.getRecordBufferSizeBytesConfigured()); Assert.assertEquals(100_000_000, config.getRecordBufferSizeBytesOrDefault(2_000_000_000)); Assert.assertEquals(100_000_000, config.getRecordBufferSizeBytesOrDefault(1_000_000_000)); Assert.assertEquals(10_000_000, config.getRecordBufferSizeBytesOrDefault(100_000_000)); @@ -127,7 +127,7 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(100, config.getMaxPendingPersists()); Assert.assertTrue(config.isReportParseExceptions()); Assert.assertEquals(100, config.getHandoffConditionTimeout()); - Assert.assertEquals(1000, (int) config.getRecordBufferSizeConfigured()); + Assert.assertEquals(1000, (int) config.getRecordBufferSizeBytesConfigured()); Assert.assertEquals(1000, config.getRecordBufferSizeBytesOrDefault(1_000_000_000)); Assert.assertEquals(500, config.getRecordBufferOfferTimeout()); Assert.assertEquals(500, config.getRecordBufferFullWait()); @@ -191,7 +191,7 @@ public void testSerdeWithModifiedTuningConfigAddedField() throws IOException Assert.assertEquals(base.getMaxSavedParseExceptions(), deserialized.getMaxSavedParseExceptions()); Assert.assertEquals(base.getRecordBufferFullWait(), deserialized.getRecordBufferFullWait()); Assert.assertEquals(base.getRecordBufferOfferTimeout(), deserialized.getRecordBufferOfferTimeout()); - Assert.assertEquals(base.getRecordBufferSizeConfigured(), deserialized.getRecordBufferSizeConfigured()); + Assert.assertEquals(base.getRecordBufferSizeBytesConfigured(), deserialized.getRecordBufferSizeBytesConfigured()); Assert.assertEquals(base.getMaxRecordsPerPollConfigured(), deserialized.getMaxRecordsPerPollConfigured()); } @@ -249,7 +249,7 @@ public void testSerdeWithModifiedTuningConfigRemovedField() throws IOException Assert.assertEquals(base.getMaxSavedParseExceptions(), deserialized.getMaxSavedParseExceptions()); Assert.assertEquals(base.getRecordBufferFullWait(), deserialized.getRecordBufferFullWait()); Assert.assertEquals(base.getRecordBufferOfferTimeout(), deserialized.getRecordBufferOfferTimeout()); - Assert.assertEquals(base.getRecordBufferSizeConfigured(), deserialized.getRecordBufferSizeConfigured()); + Assert.assertEquals(base.getRecordBufferSizeBytesConfigured(), deserialized.getRecordBufferSizeBytesConfigured()); Assert.assertEquals(base.getMaxRecordsPerPollConfigured(), deserialized.getMaxRecordsPerPollConfigured()); } @@ -329,7 +329,7 @@ public void testConvert() Assert.assertEquals(IndexSpec.DEFAULT, copy.getIndexSpec()); Assert.assertTrue(copy.isReportParseExceptions()); Assert.assertEquals(5L, copy.getHandoffConditionTimeout()); - Assert.assertEquals(1000, (int) copy.getRecordBufferSizeConfigured()); + Assert.assertEquals(1000, (int) copy.getRecordBufferSizeBytesConfigured()); Assert.assertEquals(500, copy.getRecordBufferOfferTimeout()); Assert.assertEquals(500, copy.getRecordBufferFullWait()); Assert.assertEquals(2, (int) copy.getFetchThreads()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java index d06ea43523c7..4de35cf5e5d0 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java @@ -92,11 +92,9 @@ public void testSerdeWithNonDefaults() throws Exception + " \"completionTimeout\": \"PT45M\",\n" + " \"lateMessageRejectionPeriod\": \"PT1H\",\n" + " \"earlyMessageRejectionPeriod\": \"PT1H\",\n" - + " \"recordsPerFetch\": 4000,\n" + " \"fetchDelayMillis\": 1000,\n" + " \"awsAssumedRoleArn\": \"role\",\n" - + " \"awsExternalId\": \"awsexternalid\",\n" - + " \"deaggregate\": true\n" + + " \"awsExternalId\": \"awsexternalid\"\n" + "}"; KinesisSupervisorIOConfig config = mapper.readValue( diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 67bc95b19c3c..7d278ca815a7 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -321,7 +321,6 @@ public void testNoInitialStateWithAutoScaleOut() throws Exception null, false, null, - null, autoScalerConfig ); KinesisSupervisorSpec kinesisSupervisorSpec = supervisor.getKinesisSupervisorSpec(); @@ -395,7 +394,6 @@ public void testNoInitialStateWithAutoScaleIn() throws Exception null, false, null, - null, autoScalerConfig ); @@ -3741,7 +3739,6 @@ public void testDoNotKillCompatibleTasks() new Period("P1D"), new Period("P1D"), false, - 42, 1000, true ); @@ -3839,7 +3836,6 @@ public void testKillIncompatibleTasks() new Period("P1D"), new Period("P1D"), false, - 42, 1000, false ); @@ -3925,7 +3921,6 @@ public void testIsTaskCurrent() new Period("P1D"), false, 42, - 42, dataSchema, tuningConfig ); @@ -5124,7 +5119,6 @@ private TestableKinesisSupervisor getTestableSupervisor( earlyMessageRejectionPeriod, false, null, - null, null ); } @@ -5137,7 +5131,6 @@ private TestableKinesisSupervisor getTestableSupervisor( Period lateMessageRejectionPeriod, Period earlyMessageRejectionPeriod, boolean suspended, - Integer recordsPerFetch, Integer fetchDelayMillis, AutoScalerConfig autoScalerConfig ) @@ -5225,7 +5218,6 @@ private TestableKinesisSupervisor getTestableSupervisorCustomIsTaskCurrent( Period lateMessageRejectionPeriod, Period earlyMessageRejectionPeriod, boolean suspended, - Integer recordsPerFetch, Integer fetchDelayMillis, boolean isTaskCurrentReturn ) @@ -5311,7 +5303,6 @@ private KinesisSupervisor getSupervisor( Period lateMessageRejectionPeriod, Period earlyMessageRejectionPeriod, boolean suspended, - Integer recordsPerFetch, Integer fetchDelayMillis, DataSchema dataSchema, KinesisSupervisorTuningConfig tuningConfig diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java index 9933d61a7df1..626e09104b91 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java @@ -112,7 +112,7 @@ public TestModifiedKinesisIndexTaskTuningConfig(KinesisIndexTaskTuningConfig bas base.getHandoffConditionTimeout(), base.isResetOffsetAutomatically(), base.isSkipSequenceNumberAvailabilityCheck(), - base.getRecordBufferSizeConfigured(), + base.getRecordBufferSizeBytesConfigured(), base.getRecordBufferOfferTimeout(), base.getRecordBufferFullWait(), base.getFetchThreads(), diff --git a/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx b/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx index 7dd6bd4e82fd..e2e59cecf44d 100644 --- a/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx +++ b/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx @@ -1219,13 +1219,6 @@ export function getIoConfigTuningFormFields( ), }, - { - name: 'recordsPerFetch', - type: 'number', - defaultValue: 4000, - defined: typeIsKnown(KNOWN_TYPES, 'kinesis'), - info: <>The number of records to request per GetRecords call to Kinesis., - }, { name: 'pollTimeout', type: 'number', @@ -1246,13 +1239,6 @@ export function getIoConfigTuningFormFields( defined: typeIsKnown(KNOWN_TYPES, 'kinesis'), info: <>Time in milliseconds to wait between subsequent GetRecords calls to Kinesis., }, - { - name: 'deaggregate', - type: 'boolean', - defaultValue: false, - defined: typeIsKnown(KNOWN_TYPES, 'kinesis'), - info: <>Whether to use the de-aggregate function of the KCL., - }, { name: 'startDelay', type: 'duration', @@ -1440,7 +1426,7 @@ export interface TuningConfig { offsetFetchPeriod?: string; maxParseExceptions?: number; maxSavedParseExceptions?: number; - recordBufferSize?: number; + recordBufferSizeBytes?: number; recordBufferOfferTimeout?: number; recordBufferFullWait?: number; fetchThreads?: number; @@ -2052,13 +2038,13 @@ const TUNING_FORM_FIELDS: Field[] = [ ), }, { - name: 'spec.tuningConfig.recordBufferSize', + name: 'spec.tuningConfig.recordBufferSizeBytes', type: 'number', - defaultValue: 10000, + defaultValue: 100000000, defined: typeIsKnown(KNOWN_TYPES, 'kinesis'), info: ( <> - Size of the buffer (number of events) used between the Kinesis fetch threads and the main + Size of the buffer (heap memory bytes) used between the Kinesis fetch threads and the main ingestion thread. ), From fd57dfb8ffe7a6636fe71b265d64d3fe5eab1554 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Wed, 15 Nov 2023 22:57:17 -0500 Subject: [PATCH 09/24] * fix spellcheck --- docs/development/extensions-core/kinesis-ingestion.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/development/extensions-core/kinesis-ingestion.md b/docs/development/extensions-core/kinesis-ingestion.md index c3d8143f8b66..ddf17bb0c40b 100644 --- a/docs/development/extensions-core/kinesis-ingestion.md +++ b/docs/development/extensions-core/kinesis-ingestion.md @@ -655,7 +655,7 @@ For more detail, see [Segment size optimization](../../operations/segment-optimi Kinesis indexing tasks fetch records using `fetchThreads` threads. If `fetchThreads` is higher than the number of Kinesis shards, the excess threads are unused. -Each fetch thread fetches up to 10MB of records at once from a Kinesis shard, with a delay between fetches +Each fetch thread fetches up to 10 MB of records at once from a Kinesis shard, with a delay between fetches of `fetchDelayMillis`. The records fetched by each thread are pushed into a shared queue of size `recordBufferSizeBytes`. The main runner thread for each task polls up to `maxRecordsPerPoll` records from the queue at once. @@ -665,7 +665,7 @@ The default values for these parameters are: - `fetchThreads`: Twice the number of processors available to the task. The number of processors available to the task is the total number of processors on the server, divided by `druid.worker.capacity` (the number of task slots on that particular server). This value is further limited so that the total data record data fetched at a given time does not -exceed 5% of the max heap configured, assuming that each thread fetches 10MB of recrods at once. If the value specified +exceed 5% of the max heap configured, assuming that each thread fetches 10 MB of records at once. If the value specified for this configuration is higher than this limit, no failure occurs, but a warning is logged, and the value is implicitly lowered to the max allowed by this constraint. - `fetchDelayMillis`: 0 (no delay between fetches). From 6666bc80985eaf270d25ed096467ae029a5b5c00 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Fri, 17 Nov 2023 14:30:55 -0500 Subject: [PATCH 10/24] * dont throw away the rest of the GetRecords result after recordBufferOfferTimeout * check return value of newQ::offer and fail if false --- .../kinesis/KinesisRecordSupplier.java | 38 ++++++++++--------- 1 file changed, 21 insertions(+), 17 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 6ce59d531bd0..38af484c2d2c 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -210,6 +210,7 @@ private Runnable fetchRecords() // used for retrying on InterruptedException GetRecordsResult recordsResult = null; OrderedPartitionableRecord currRecord; + long recordBufferOfferWaitMillis; try { if (shardIterator == null) { @@ -225,15 +226,16 @@ private Runnable fetchRecords() recordsResult = null; - if (!records.offer( + recordBufferOfferWaitMillis = recordBufferOfferTimeout; + while (!records.offer( new MemoryBoundLinkedBlockingQueue.ObjectContainer<>(currRecord, 0), - recordBufferOfferTimeout, + recordBufferOfferWaitMillis, TimeUnit.MILLISECONDS )) { log.warn("Kinesis records are being processed slower than they are fetched. " + "OrderedPartitionableRecord buffer full, retrying in [%,dms].", recordBufferFullWait); - scheduleBackgroundFetch(recordBufferFullWait); + recordBufferOfferWaitMillis = recordBufferFullWait; } return; @@ -293,9 +295,10 @@ private Runnable fetchRecords() // If the buffer was full and we weren't able to add the message, grab a new stream iterator starting // from this message and back off for a bit to let the buffer drain before retrying. - if (!records.offer( + recordBufferOfferWaitMillis = recordBufferOfferTimeout; + while (!records.offer( new MemoryBoundLinkedBlockingQueue.ObjectContainer<>(currRecord, recordSize), - recordBufferOfferTimeout, + recordBufferOfferWaitMillis, TimeUnit.MILLISECONDS )) { log.warn( @@ -303,16 +306,7 @@ private Runnable fetchRecords() + "OrderedPartitionableRecord buffer full, storing iterator and retrying in [%,dms].", recordBufferFullWait ); - - shardIterator = kinesis.getShardIterator( - currRecord.getStream(), - currRecord.getPartitionId(), - ShardIteratorType.AT_SEQUENCE_NUMBER.toString(), - currRecord.getSequenceNumber() - ).getShardIterator(); - - scheduleBackgroundFetch(recordBufferFullWait); - return; + recordBufferOfferWaitMillis = recordBufferFullWait; } } @@ -1055,8 +1049,18 @@ private void filterBufferAndResetBackgroundFetch(Set> pa new MemoryBoundLinkedBlockingQueue<>(recordBufferSizeBytes); records.stream() - .filter(x -> !partitions.contains(x.getData().getStreamPartition())) - .forEachOrdered(newQ::offer); + .filter(x -> !partitions.contains(x.getData().getStreamPartition())) + .forEachOrdered(x -> { + if (!newQ.offer(x)) { + throw new StreamException(new ISE(StringUtils.format( + "Failed to insert item to new queue when resetting background fetch. " + + "[stream: '%s', partitionId: '%s', sequenceNumber: '%s']", + x.getData().getStream(), + x.getData().getPartitionId(), + x.getData().getSequenceNumber() + ))); + } + }); records = newQ; From 8bd7c69100236b2608325da06a37e85f3eecd7a0 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Mon, 27 Nov 2023 18:47:45 -0500 Subject: [PATCH 11/24] * address review comments --- .../indexing/kinesis/KinesisIndexTask.java | 10 +++++- .../kinesis/KinesisIndexTaskTuningConfig.java | 23 +++++++++--- .../kinesis/KinesisRecordSupplier.java | 9 ++--- .../KinesisSupervisorTuningConfig.java | 4 +++ .../kinesis/KinesisIndexTaskSerdeTest.java | 1 + .../kinesis/KinesisIndexTaskTest.java | 1 + .../KinesisIndexTaskTuningConfigTest.java | 3 ++ .../supervisor/KinesisSupervisorTest.java | 3 ++ ...tModifiedKinesisIndexTaskTuningConfig.java | 2 ++ .../MemoryBoundLinkedBlockingQueue.java | 35 ++++++++----------- .../MemoryBoundLinkedBlockingQueueTest.java | 29 +++++++++++---- 11 files changed, 84 insertions(+), 36 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 219999be4ad4..42f8693733b4 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -49,6 +49,10 @@ public class KinesisIndexTask extends SeekableStreamIndexTask { private static final String TYPE = "index_kinesis"; + + // GetRecords returns maximum 10MB per call + // (https://docs.aws.amazon.com/streams/latest/dev/service-sizes-and-limits.html) + private static final long GET_RECORDS_MAX_BYTES_PER_CALL = 10_000_000L; private static final Logger log = new Logger(KinesisIndexTask.class); private final boolean useListShards; @@ -78,6 +82,10 @@ public KinesisIndexTask( ); this.useListShards = useListShards; this.awsCredentialsConfig = awsCredentialsConfig; + if (tuningConfig.getRecordBufferSizeConfigured() != null) { + log.warn("The 'recordBufferSize' config property of the kinesis tuning config has been deprecated. " + + "Please use 'recordBufferSizeBytes'."); + } } @Override @@ -197,7 +205,7 @@ static int computeFetchThreads( ); int maxFetchThreads = Math.max( 1, - (int) (memoryToUse / 10_000_000L) + (int) (memoryToUse / GET_RECORDS_MAX_BYTES_PER_CALL) ); if (fetchThreads > maxFetchThreads) { log.warn("fetchThreads [%d] being lowered to [%d]", fetchThreads, maxFetchThreads); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java index a703628303b5..ba1104b8398a 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java @@ -56,8 +56,7 @@ public class KinesisIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningC private static final int DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT = 5000; private static final int DEFAULT_RECORD_BUFFER_FULL_WAIT = 5000; private static final int DEFAULT_MAX_RECORDS_PER_POLL = 100; - private static final int DEFAULT_MAX_RECORDS_PER_POLL_AGGREGATE = 1; - + private final Integer recordBufferSize; private final Integer recordBufferSizeBytes; private final int recordBufferOfferTimeout; private final int recordBufferFullWait; @@ -80,6 +79,7 @@ public KinesisIndexTaskTuningConfig( Long handoffConditionTimeout, Boolean resetOffsetAutomatically, Boolean skipSequenceNumberAvailabilityCheck, + @Deprecated @Nullable Integer recordBufferSize, @Nullable Integer recordBufferSizeBytes, Integer recordBufferOfferTimeout, Integer recordBufferFullWait, @@ -114,6 +114,7 @@ public KinesisIndexTaskTuningConfig( maxParseExceptions, maxSavedParseExceptions ); + this.recordBufferSize = recordBufferSize; this.recordBufferSizeBytes = recordBufferSizeBytes; this.recordBufferOfferTimeout = recordBufferOfferTimeout == null ? DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT @@ -144,6 +145,7 @@ private KinesisIndexTaskTuningConfig( @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, @JsonProperty("skipSequenceNumberAvailabilityCheck") Boolean skipSequenceNumberAvailabilityCheck, + @JsonProperty("recordBufferSize") @Deprecated @Nullable Integer recordBufferSize, @JsonProperty("recordBufferSizeBytes") Integer recordBufferSizeBytes, @JsonProperty("recordBufferOfferTimeout") Integer recordBufferOfferTimeout, @JsonProperty("recordBufferFullWait") Integer recordBufferFullWait, @@ -172,6 +174,7 @@ private KinesisIndexTaskTuningConfig( handoffConditionTimeout, resetOffsetAutomatically, skipSequenceNumberAvailabilityCheck, + recordBufferSize, recordBufferSizeBytes, recordBufferOfferTimeout, recordBufferFullWait, @@ -185,6 +188,14 @@ private KinesisIndexTaskTuningConfig( ); } + @Nullable + @JsonProperty("recordBufferSize") + @JsonInclude(JsonInclude.Include.NON_NULL) + public Integer getRecordBufferSizeConfigured() + { + return recordBufferSize; + } + @Nullable @JsonProperty("recordBufferSizeBytes") @JsonInclude(JsonInclude.Include.NON_NULL) @@ -235,7 +246,7 @@ public Integer getMaxRecordsPerPollConfigured() public int getMaxRecordsPerPollOrDefault() { - return DEFAULT_MAX_RECORDS_PER_POLL_AGGREGATE; + return DEFAULT_MAX_RECORDS_PER_POLL; } @Override @@ -257,6 +268,7 @@ public KinesisIndexTaskTuningConfig withBasePersistDirectory(File dir) getHandoffConditionTimeout(), isResetOffsetAutomatically(), isSkipSequenceNumberAvailabilityCheck(), + getRecordBufferSizeConfigured(), getRecordBufferSizeBytesConfigured(), getRecordBufferOfferTimeout(), getRecordBufferFullWait(), @@ -283,7 +295,8 @@ public boolean equals(Object o) return false; } KinesisIndexTaskTuningConfig that = (KinesisIndexTaskTuningConfig) o; - return Objects.equals(recordBufferSizeBytes, that.recordBufferSizeBytes) && + return Objects.equals(recordBufferSize, that.recordBufferSize) && + Objects.equals(recordBufferSizeBytes, that.recordBufferSizeBytes) && recordBufferOfferTimeout == that.recordBufferOfferTimeout && recordBufferFullWait == that.recordBufferFullWait && Objects.equals(maxRecordsPerPoll, that.maxRecordsPerPoll) && @@ -295,6 +308,7 @@ public int hashCode() { return Objects.hash( super.hashCode(), + recordBufferSize, recordBufferSizeBytes, recordBufferOfferTimeout, recordBufferFullWait, @@ -319,6 +333,7 @@ public String toString() ", handoffConditionTimeout=" + getHandoffConditionTimeout() + ", resetOffsetAutomatically=" + isResetOffsetAutomatically() + ", skipSequenceNumberAvailabilityCheck=" + isSkipSequenceNumberAvailabilityCheck() + + ", recordBufferSize=" + recordBufferSize + ", recordBufferSizeBytes=" + recordBufferSizeBytes + ", recordBufferOfferTimeout=" + recordBufferOfferTimeout + ", recordBufferFullWait=" + recordBufferFullWait + diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 38af484c2d2c..3c0421cb6dbf 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -106,6 +106,7 @@ public class KinesisRecordSupplier implements RecordSupplier 0; - // the deaggregate function is implemented by the amazon-kinesis-client, whose license is not compatible with Apache. - // The work around here is to use reflection to find the deaggregate function in the classpath. See details on the - // docs page for more information on how to use deaggregation + // the deaggregate function is implemented by the amazon-kinesis-client. Use reflection to find the deaggregate + // function in the classpath. See details on the docs page for more information on how to use deaggregation try { Class kclUserRecordclass = Class.forName("com.amazonaws.services.kinesis.clientlibrary.types.UserRecord"); MethodHandles.Lookup lookup = MethodHandles.publicLookup(); @@ -626,7 +626,7 @@ public List> poll(long ti records.drain( polledRecords, - expectedSize, + MAX_BYTES_PER_POLL, timeout, TimeUnit.MILLISECONDS ); @@ -1052,6 +1052,7 @@ private void filterBufferAndResetBackgroundFetch(Set> pa .filter(x -> !partitions.contains(x.getData().getStreamPartition())) .forEachOrdered(x -> { if (!newQ.offer(x)) { + // this should never really happen in practice but adding check here for safety. throw new StreamException(new ISE(StringUtils.format( "Failed to insert item to new queue when resetting background fetch. " + "[stream: '%s', partitionId: '%s', sequenceNumber: '%s']", diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java index 0e40ff633c1b..801f78688df3 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -74,6 +74,7 @@ public static KinesisSupervisorTuningConfig defaultConfig() null, null, null, + null, null ); } @@ -98,6 +99,7 @@ public KinesisSupervisorTuningConfig( @JsonProperty("chatRetries") Long chatRetries, @JsonProperty("httpTimeout") Period httpTimeout, @JsonProperty("shutdownTimeout") Period shutdownTimeout, + @JsonProperty("recordBufferSize") @Deprecated @Nullable Integer recordBufferSize, @JsonProperty("recordBufferSizeBytes") Integer recordBufferSizeBytes, @JsonProperty("recordBufferOfferTimeout") Integer recordBufferOfferTimeout, @JsonProperty("recordBufferFullWait") Integer recordBufferFullWait, @@ -128,6 +130,7 @@ public KinesisSupervisorTuningConfig( handoffConditionTimeout, resetOffsetAutomatically, skipSequenceNumberAvailabilityCheck, + recordBufferSize, recordBufferSizeBytes, recordBufferOfferTimeout, recordBufferFullWait, @@ -259,6 +262,7 @@ public KinesisIndexTaskTuningConfig convertToTaskTuningConfig() getHandoffConditionTimeout(), isResetOffsetAutomatically(), isSkipSequenceNumberAvailabilityCheck(), + getRecordBufferSizeConfigured(), getRecordBufferSizeBytesConfigured(), getRecordBufferOfferTimeout(), getRecordBufferFullWait(), diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java index 09b70500556b..8f27adb5c435 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java @@ -76,6 +76,7 @@ public class KinesisIndexTaskSerdeTest null, null, null, + null, null ); private static final KinesisIndexTaskIOConfig IO_CONFIG = new KinesisIndexTaskIOConfig( diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index df0c31b615d9..54834efdbaf2 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -2353,6 +2353,7 @@ private KinesisIndexTask createTask( null, null, null, + null, logParseExceptions, maxParseExceptions, maxSavedParseExceptions, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index 9055ee38b0a2..b1a2193571c8 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -155,6 +155,7 @@ public void testSerdeWithModifiedTuningConfigAddedField() throws IOException 5L, true, false, + null, 1000, 1000, 500, @@ -214,6 +215,7 @@ public void testSerdeWithModifiedTuningConfigRemovedField() throws IOException 5L, true, false, + null, 1000, 1000, 500, @@ -303,6 +305,7 @@ public void testConvert() null, null, null, + null, 1000, 500, 500, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 7d278ca815a7..73ebdce881e2 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -211,6 +211,7 @@ public void setupTest() null, null, null, + null, null ); rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory(); @@ -3963,6 +3964,7 @@ public void testIsTaskCurrent() null, null, null, + null, 42, // This property is different from tuningConfig null, null, @@ -5070,6 +5072,7 @@ public SeekableStreamIndexTaskClient build( null, null, null, + null, null ); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java index 626e09104b91..ca4de4a99dd7 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java @@ -80,6 +80,7 @@ public TestModifiedKinesisIndexTaskTuningConfig( handoffConditionTimeout, resetOffsetAutomatically, skipSequenceNumberAvailabilityCheck, + null, recordBufferSizeBytes, recordBufferOfferTimeout, recordBufferFullWait, @@ -112,6 +113,7 @@ public TestModifiedKinesisIndexTaskTuningConfig(KinesisIndexTaskTuningConfig bas base.getHandoffConditionTimeout(), base.isResetOffsetAutomatically(), base.isSkipSequenceNumberAvailabilityCheck(), + null, base.getRecordBufferSizeBytesConfigured(), base.getRecordBufferOfferTimeout(), base.getRecordBufferFullWait(), diff --git a/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java b/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java index 322757259359..e7b202183119 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java @@ -97,34 +97,27 @@ public Stream> stream() return queue.stream(); } - // This code is taken from {om.google.common.collect.Queues#Drain}} - public int drain(Collection> buffer, int numElements, long timeout, TimeUnit unit) + public int drain(Collection> buffer, int bytesToDrain, long timeout, TimeUnit unit) throws InterruptedException { Preconditions.checkNotNull(buffer); long deadline = System.nanoTime() + unit.toNanos(timeout); int added = 0; - - while (added < numElements) { - added += queue.drainTo(buffer, numElements - added); - buffer.forEach( - i -> { - currentMemory.addAndGet(-((ObjectContainer) i).getSize()); - } - ); - if (added < numElements) { - ObjectContainer e = queue.poll(deadline - System.nanoTime(), TimeUnit.NANOSECONDS); - - if (e == null) { - break; - } - - currentMemory.addAndGet(-e.getSize()); - buffer.add(e); - ++added; + int bytesAdded = 0; + while (bytesAdded < bytesToDrain) { + ObjectContainer e = queue.poll(deadline - System.nanoTime(), TimeUnit.NANOSECONDS); + if (e == null) { + break; + } + currentMemory.addAndGet(-e.getSize()); + buffer.add(e); + ++added; + bytesAdded += e.getSize(); + e = queue.peek(); + if (e != null && (bytesAdded + e.getSize()) > bytesToDrain) { + break; } } - return added; } diff --git a/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java b/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java index c72ca62562b5..d113c2691b1c 100644 --- a/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java @@ -141,6 +141,22 @@ public void test_take_nonEmptyQueue_expected() throws InterruptedException Assert.assertEquals(byteCapacity - expectedByteSize, queue.remainingCapacity()); } + @Test + public void test_drain_emptyQueue_succeeds() throws InterruptedException + { + + long byteCapacity = 7L; + MemoryBoundLinkedBlockingQueue queue = setupQueue(byteCapacity, ImmutableList.of()); + Collection> buffer = new ArrayList<>(); + + int numAdded = queue.drain(buffer, 1, 1, TimeUnit.SECONDS); + + Assert.assertTrue(numAdded == 0 && numAdded == buffer.size()); + Assert.assertEquals(0, queue.size()); + Assert.assertEquals(0L, queue.byteSize()); + Assert.assertEquals(byteCapacity, queue.remainingCapacity()); + } + @Test public void test_drain_queueWithOneItem_succeeds() throws InterruptedException { @@ -165,21 +181,22 @@ public void test_drain_queueWithOneItem_succeeds() throws InterruptedException public void test_drain_queueWithMultipleItems_succeeds() throws InterruptedException { - long byteCapacity = 10L; + long byteCapacity = 15L; byte[] item1 = "item1".getBytes(StandardCharsets.UTF_8); byte[] item2 = "item2".getBytes(StandardCharsets.UTF_8); + byte[] item3 = "item3".getBytes(StandardCharsets.UTF_8); Collection> items = buildItemContainers( - ImmutableList.of(item1, item2) + ImmutableList.of(item1, item2, item3) ); MemoryBoundLinkedBlockingQueue queue = setupQueue(byteCapacity, items, new NotAllDrainedQueue()); Collection> buffer = new ArrayList<>(); - int numAdded = queue.drain(buffer, 2, 1, TimeUnit.MINUTES); + int numAdded = queue.drain(buffer, 10, 1, TimeUnit.MINUTES); Assert.assertTrue(numAdded == 2 && numAdded == buffer.size()); - Assert.assertEquals(0, queue.size()); - Assert.assertEquals(0L, queue.byteSize()); - Assert.assertEquals(byteCapacity, queue.remainingCapacity()); + Assert.assertEquals(1, queue.size()); + Assert.assertEquals(item3.length, queue.byteSize()); + Assert.assertEquals(byteCapacity - item3.length, queue.remainingCapacity()); } private static MemoryBoundLinkedBlockingQueue setupQueue( From b6349fbc1205e4f455ba3ea577f68f712b32a2aa Mon Sep 17 00:00:00 2001 From: zachjsh Date: Mon, 27 Nov 2023 18:54:27 -0500 Subject: [PATCH 12/24] * fix integration test compilation failure --- .../druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 38600d3f61a5..62ad857160ab 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -4070,7 +4070,6 @@ public void testSequenceNameDoesNotChangeWithTaskId() new Period("P1D"), false, 42, - 42, dataSchema, tuningConfig ); From 03000742d125a2794ab62cc735db06b54403f935 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Tue, 28 Nov 2023 02:05:06 -0500 Subject: [PATCH 13/24] * fix code scan failure --- .../druid/java/util/common/MemoryBoundLinkedBlockingQueue.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java b/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java index e7b202183119..d585477d046a 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java @@ -103,7 +103,7 @@ public int drain(Collection> buffer, int bytesToDrain Preconditions.checkNotNull(buffer); long deadline = System.nanoTime() + unit.toNanos(timeout); int added = 0; - int bytesAdded = 0; + long bytesAdded = 0; while (bytesAdded < bytesToDrain) { ObjectContainer e = queue.poll(deadline - System.nanoTime(), TimeUnit.NANOSECONDS); if (e == null) { From f4c06651d2c4b90cf9027de740172cf371619ea7 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Wed, 29 Nov 2023 14:59:48 -0500 Subject: [PATCH 14/24] * review comments --- .../extensions-core/kinesis-ingestion.md | 69 +++++++++---------- .../indexing/kinesis/KinesisIndexTask.java | 12 ++-- .../kinesis/KinesisIndexTaskTuningConfig.java | 28 ++++++-- .../kinesis/KinesisRecordSupplier.java | 33 ++++----- .../indexing/kinesis/KinesisSamplerSpec.java | 2 +- .../kinesis/supervisor/KinesisSupervisor.java | 2 +- .../KinesisSupervisorTuningConfig.java | 7 +- .../kinesis/KinesisIndexTaskSerdeTest.java | 1 + .../kinesis/KinesisIndexTaskTest.java | 5 ++ .../KinesisIndexTaskTuningConfigTest.java | 5 ++ .../kinesis/KinesisRecordSupplierTest.java | 43 ++++++------ .../supervisor/KinesisSupervisorTest.java | 3 + ...tModifiedKinesisIndexTaskTuningConfig.java | 8 ++- .../ingestion-spec/ingestion-spec.tsx | 8 +-- website/.spelling | 1 + 15 files changed, 134 insertions(+), 93 deletions(-) diff --git a/docs/development/extensions-core/kinesis-ingestion.md b/docs/development/extensions-core/kinesis-ingestion.md index b44409144948..6d9a6018988a 100644 --- a/docs/development/extensions-core/kinesis-ingestion.md +++ b/docs/development/extensions-core/kinesis-ingestion.md @@ -383,40 +383,39 @@ The `tuningConfig` object is optional. If you don't specify the `tuningConfig` o The following table outlines the configuration options for `tuningConfig`: -|Property|Type|Description|Required|Default| -|--------|----|-----------|--------|-------| -|`type`|String|The indexing task type. This should always be `kinesis`.|Yes|| -|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number represents the post-aggregation rows. It is not equivalent to the number of input events, but the resulting number of aggregated rows. Druid uses `maxRowsInMemory` to manage the required JVM heap size. The maximum heap memory usage for indexing scales is `maxRowsInMemory * (2 + maxPendingPersists)`.|No|100000| -|`maxBytesInMemory`|Long| The number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. Normally, this is computed internally. The maximum heap memory usage for indexing is `maxBytesInMemory * (2 + maxPendingPersists)`.|No|One-sixth of max JVM memory| -|`skipBytesInMemoryOverheadCheck`|Boolean|The calculation of `maxBytesInMemory` takes into account overhead objects created during ingestion and each intermediate persist. To exclude the bytes of these overhead objects from the `maxBytesInMemory` check, set `skipBytesInMemoryOverheadCheck` to `true`.|No|`false`| -|`maxRowsPerSegment`|Integer|The number of rows to aggregate into a segment; this number represents the post-aggregation rows. Handoff occurs when `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first.|No|5000000| -|`maxTotalRows`|Long|The number of rows to aggregate across all segments; this number represents the post-aggregation rows. Handoff occurs when `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first.|No|unlimited| -|`intermediateHandoffPeriod`|ISO 8601 period|The period that determines how often tasks hand off segments. Handoff occurs if `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first.|No|P2147483647D| -|`intermediatePersistPeriod`|ISO 8601 period|The period that determines the rate at which intermediate persists occur.|No|PT10M| -|`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If a new intermediate persist exceeds this limit, Druid blocks ingestion until the currently running persist finishes. One persist can be running concurrently with ingestion, and none can be queued up. The maximum heap memory usage for indexing scales is `maxRowsInMemory * (2 + maxPendingPersists)`.|No|0| -|`indexSpec`|Object|Defines how Druid indexes the data. See [IndexSpec](#indexspec) for more information.|No|| -|`indexSpecForIntermediatePersists`|Object|Defines segment storage format options to use at indexing time for intermediate persisted temporary segments. You can use `indexSpecForIntermediatePersists` to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. However, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published. See [IndexSpec](#indexspec) for possible values.|No|Same as `indexSpec`| -|`reportParseExceptions`|Boolean|If `true`, Druid throws exceptions encountered during parsing causing ingestion to halt. If `false`, Druid skips unparseable rows and fields.|No|`false`| -|`handoffConditionTimeout`|Long|Number of milliseconds to wait for segment handoff. Set to a value >= 0, where 0 means to wait indefinitely.|No|0| -|`resetOffsetAutomatically`|Boolean|Controls behavior when Druid needs to read Kinesis messages that are no longer available.
If `false`, the exception bubbles up causing tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation, potentially using the [Reset Supervisor API](../../api-reference/supervisor-api.md). This mode is useful for production, since it highlights issues with ingestion.
If `true`, Druid automatically resets to the earliest or latest sequence number available in Kinesis, based on the value of the `useEarliestSequenceNumber` property (earliest if `true`, latest if `false`). Note that this can lead to dropping data (if `useEarliestSequenceNumber` is `false`) or duplicating data (if `useEarliestSequenceNumber` is `true`) without your knowledge. Druid logs messages indicating that a reset has occurred without interrupting ingestion. This mode is useful for non-production situations since it enables Druid to recover from problems automatically, even if they lead to quiet dropping or duplicating of data.|No|`false`| -|`skipSequenceNumberAvailabilityCheck`|Boolean|Whether to enable checking if the current sequence number is still available in a particular Kinesis shard. If `false`, the indexing task attempts to reset the current sequence number, depending on the value of `resetOffsetAutomatically`.|No|`false`| -|`workerThreads`|Integer|The number of threads that the supervisor uses to handle requests/responses for worker tasks, along with any other internal asynchronous operation.|No| `min(10, taskCount)`| -|`chatRetries`|Integer|The number of times Druid retries HTTP requests to indexing tasks before considering tasks unresponsive.|No|8| -|`httpTimeout`|ISO 8601 period|The period of time to wait for a HTTP response from an indexing task.|No|PT10S| -|`shutdownTimeout`|ISO 8601 period|The period of time to wait for the supervisor to attempt a graceful shutdown of tasks before exiting.|No|PT80S| -|`recordBufferSizeBytes`|Integer| The size of the buffer (heap memory bytes) Druid uses between the Kinesis fetch threads and the main ingestion thread. |No|See [Determine fetch settings](#determine-fetch-settings) for defaults.| -|`recordBufferOfferTimeout`|Integer|The number of milliseconds to wait for space to become available in the buffer before timing out.|No|5000| -|`recordBufferFullWait`|Integer|The number of milliseconds to wait for the buffer to drain before Druid attempts to fetch records from Kinesis again.|No|5000| -|`fetchThreads`|Integer|The size of the pool of threads fetching data from Kinesis. There is no benefit in having more threads than Kinesis shards.|No| `procs * 2`, where `procs` is the number of processors available to the task.| -|`segmentWriteOutMediumFactory`|Object|The segment write-out medium to use when creating segments See [Additional Peon configuration: SegmentWriteOutMediumFactory](../../configuration/index.md#segmentwriteoutmediumfactory) for explanation and available options.|No|If not specified, Druid uses the value from `druid.peon.defaultSegmentWriteOutMediumFactory.type`.| -|`logParseExceptions`|Boolean|If `true`, Druid logs an error message when a parsing exception occurs, containing information about the row where the error occurred.|No|`false`| -|`maxParseExceptions`|Integer|The maximum number of parse exceptions that can occur before the task halts ingestion and fails. Overridden if `reportParseExceptions` is set.|No|unlimited| -|`maxSavedParseExceptions`|Integer|When a parse exception occurs, Druid keeps track of the most recent parse exceptions. `maxSavedParseExceptions` limits the number of saved exception instances. These saved exceptions are available after the task finishes in the [task completion report](../../ingestion/tasks.md#task-reports). Overridden if `reportParseExceptions` is set.|No|0| -|`maxRecordsPerPoll`|Integer|The maximum number of records to be fetched from buffer per poll. The actual maximum will be `Max(maxRecordsPerPoll, Max(bufferSize, 1))`.|No| See [Determine fetch settings](#determine-fetch-settings) for defaults.| -|`repartitionTransitionDuration`|ISO 8601 period|When shards are split or merged, the supervisor recomputes shard to task group mappings. The supervisor also signals any running tasks created under the old mappings to stop early at current time + `repartitionTransitionDuration`. Stopping the tasks early allows Druid to begin reading from the new shards more quickly. The repartition transition wait time controlled by this property gives the stream additional time to write records to the new shards after the split or merge, which helps avoid issues with [empty shard handling](https://github.com/apache/druid/issues/7600).|No|PT2M| -|`offsetFetchPeriod`|ISO 8601 period|Determines how often the supervisor queries Kinesis and the indexing tasks to fetch current offsets and calculate lag. If the user-specified value is below the minimum value of PT5S, the supervisor ignores the value and uses the minimum value instead.|No|PT30S| -|`useListShards`|Boolean|Indicates if `listShards` API of AWS Kinesis SDK can be used to prevent `LimitExceededException` during ingestion. You must set the necessary `IAM` permissions.|No|`false`| - +|Property|Type| Description |Required| Default | +|--------|----|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|--------|----------------------------------------------------------------------------------------------------| +|`type`|String| The indexing task type. This should always be `kinesis`. |Yes| | +|`maxRowsInMemory`|Integer| The number of rows to aggregate before persisting. This number represents the post-aggregation rows. It is not equivalent to the number of input events, but the resulting number of aggregated rows. Druid uses `maxRowsInMemory` to manage the required JVM heap size. The maximum heap memory usage for indexing scales is `maxRowsInMemory * (2 + maxPendingPersists)`. |No| 100000 | +|`maxBytesInMemory`|Long| The number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. Normally, this is computed internally. The maximum heap memory usage for indexing is `maxBytesInMemory * (2 + maxPendingPersists)`. |No| One-sixth of max JVM memory | +|`skipBytesInMemoryOverheadCheck`|Boolean| The calculation of `maxBytesInMemory` takes into account overhead objects created during ingestion and each intermediate persist. To exclude the bytes of these overhead objects from the `maxBytesInMemory` check, set `skipBytesInMemoryOverheadCheck` to `true`. |No| `false` | +|`maxRowsPerSegment`|Integer| The number of rows to aggregate into a segment; this number represents the post-aggregation rows. Handoff occurs when `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first. |No| 5000000 | +|`maxTotalRows`|Long| The number of rows to aggregate across all segments; this number represents the post-aggregation rows. Handoff occurs when `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first. |No| unlimited | +|`intermediateHandoffPeriod`|ISO 8601 period| The period that determines how often tasks hand off segments. Handoff occurs if `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first. |No| P2147483647D | +|`intermediatePersistPeriod`|ISO 8601 period| The period that determines the rate at which intermediate persists occur. |No| PT10M | +|`maxPendingPersists`|Integer| Maximum number of persists that can be pending but not started. If a new intermediate persist exceeds this limit, Druid blocks ingestion until the currently running persist finishes. One persist can be running concurrently with ingestion, and none can be queued up. The maximum heap memory usage for indexing scales is `maxRowsInMemory * (2 + maxPendingPersists)`. |No| 0 | +|`indexSpec`|Object| Defines how Druid indexes the data. See [IndexSpec](#indexspec) for more information. |No| | +|`indexSpecForIntermediatePersists`|Object| Defines segment storage format options to use at indexing time for intermediate persisted temporary segments. You can use `indexSpecForIntermediatePersists` to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. However, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published. See [IndexSpec](#indexspec) for possible values. |No| Same as `indexSpec` | +|`reportParseExceptions`|Boolean| If `true`, Druid throws exceptions encountered during parsing causing ingestion to halt. If `false`, Druid skips unparseable rows and fields. |No| `false` | +|`handoffConditionTimeout`|Long| Number of milliseconds to wait for segment handoff. Set to a value >= 0, where 0 means to wait indefinitely. |No| 0 | +|`resetOffsetAutomatically`|Boolean| Controls behavior when Druid needs to read Kinesis messages that are no longer available.
If `false`, the exception bubbles up causing tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation, potentially using the [Reset Supervisor API](../../api-reference/supervisor-api.md). This mode is useful for production, since it highlights issues with ingestion.
If `true`, Druid automatically resets to the earliest or latest sequence number available in Kinesis, based on the value of the `useEarliestSequenceNumber` property (earliest if `true`, latest if `false`). Note that this can lead to dropping data (if `useEarliestSequenceNumber` is `false`) or duplicating data (if `useEarliestSequenceNumber` is `true`) without your knowledge. Druid logs messages indicating that a reset has occurred without interrupting ingestion. This mode is useful for non-production situations since it enables Druid to recover from problems automatically, even if they lead to quiet dropping or duplicating of data. |No| `false` | +|`skipSequenceNumberAvailabilityCheck`|Boolean| Whether to enable checking if the current sequence number is still available in a particular Kinesis shard. If `false`, the indexing task attempts to reset the current sequence number, depending on the value of `resetOffsetAutomatically`. |No| `false` | +|`workerThreads`|Integer| The number of threads that the supervisor uses to handle requests/responses for worker tasks, along with any other internal asynchronous operation. |No| `min(10, taskCount)` | +|`chatRetries`|Integer| The number of times Druid retries HTTP requests to indexing tasks before considering tasks unresponsive. |No| 8 | +|`httpTimeout`|ISO 8601 period| The period of time to wait for a HTTP response from an indexing task. |No| PT10S | +|`shutdownTimeout`|ISO 8601 period| The period of time to wait for the supervisor to attempt a graceful shutdown of tasks before exiting. |No| PT80S | +|`recordBufferSizeBytes`|Integer| The size of the buffer (heap memory bytes) Druid uses between the Kinesis fetch threads and the main ingestion thread. |No| See [Determine fetch settings](#determine-fetch-settings) for defaults. | +|`recordBufferOfferTimeout`|Integer| The number of milliseconds to wait for space to become available in the buffer before timing out. |No| 5000 | +|`recordBufferFullWait`|Integer| The number of milliseconds to wait for the buffer to drain before Druid attempts to fetch records from Kinesis again. |No| 5000 | +|`fetchThreads`|Integer| The size of the pool of threads fetching data from Kinesis. There is no benefit in having more threads than Kinesis shards. |No| `procs * 2`, where `procs` is the number of processors available to the task. | +|`segmentWriteOutMediumFactory`|Object| The segment write-out medium to use when creating segments See [Additional Peon configuration: SegmentWriteOutMediumFactory](../../configuration/index.md#segmentwriteoutmediumfactory) for explanation and available options. |No| If not specified, Druid uses the value from `druid.peon.defaultSegmentWriteOutMediumFactory.type`. | +|`logParseExceptions`|Boolean| If `true`, Druid logs an error message when a parsing exception occurs, containing information about the row where the error occurred. |No| `false` | +|`maxParseExceptions`|Integer| The maximum number of parse exceptions that can occur before the task halts ingestion and fails. Overridden if `reportParseExceptions` is set. |No| unlimited | +|`maxSavedParseExceptions`|Integer| When a parse exception occurs, Druid keeps track of the most recent parse exceptions. `maxSavedParseExceptions` limits the number of saved exception instances. These saved exceptions are available after the task finishes in the [task completion report](../../ingestion/tasks.md#task-reports). Overridden if `reportParseExceptions` is set. |No| 0 | +|`maxBytesPerPoll`|Integer| The maximum number of bytes to be fetched from buffer per poll. At least one record is polled from the buffer regardless of this config. |No| 1000000 bytes | +|`repartitionTransitionDuration`|ISO 8601 period| When shards are split or merged, the supervisor recomputes shard to task group mappings. The supervisor also signals any running tasks created under the old mappings to stop early at current time + `repartitionTransitionDuration`. Stopping the tasks early allows Druid to begin reading from the new shards more quickly. The repartition transition wait time controlled by this property gives the stream additional time to write records to the new shards after the split or merge, which helps avoid issues with [empty shard handling](https://github.com/apache/druid/issues/7600). |No| PT2M | +|`offsetFetchPeriod`|ISO 8601 period| Determines how often the supervisor queries Kinesis and the indexing tasks to fetch current offsets and calculate lag. If the user-specified value is below the minimum value of PT5S, the supervisor ignores the value and uses the minimum value instead. |No| PT30S | +|`useListShards`|Boolean| Indicates if `listShards` API of AWS Kinesis SDK can be used to prevent `LimitExceededException` during ingestion. You must set the necessary `IAM` permissions. |No| `false` | ### IndexSpec @@ -670,7 +669,7 @@ for this configuration is higher than this limit, no failure occurs, but a warni implicitly lowered to the max allowed by this constraint. - `fetchDelayMillis`: 0 (no delay between fetches). - `recordBufferSizeBytes`: 100 MB or an estimated 10% of available heap, whichever is smaller. -- `maxRecordsPerPoll`: 1. +- `maxBytesPerPoll`: 1000000. Kinesis places the following restrictions on calls to fetch records: diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 42f8693733b4..ad5aaa6c7723 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -86,6 +86,10 @@ public KinesisIndexTask( log.warn("The 'recordBufferSize' config property of the kinesis tuning config has been deprecated. " + "Please use 'recordBufferSizeBytes'."); } + if (tuningConfig.getMaxRecordsPerPollConfigured() != null) { + log.warn("The 'maxRecordsPerPoll' config property of the kinesis tuning config has been deprecated. " + + "Please use 'maxBytesPerPoll'."); + } } @Override @@ -116,15 +120,15 @@ protected KinesisRecordSupplier newTaskRecordSupplier(final TaskToolbox toolbox) final int recordBufferSizeBytes = tuningConfig.getRecordBufferSizeBytesOrDefault(runtimeInfo.getMaxHeapSizeBytes()); final int fetchThreads = computeFetchThreads(runtimeInfo, tuningConfig.getFetchThreads()); - final int maxRecordsPerPoll = tuningConfig.getMaxRecordsPerPollOrDefault(); + final int maxBytesPerPoll = tuningConfig.getMaxBytesPerPollOrDefault(); log.info( "Starting record supplier with fetchThreads [%d], fetchDelayMillis [%d], " - + "recordBufferSizeBytes [%d], maxRecordsPerPoll [%d]", + + "recordBufferSizeBytes [%d], maxBytesPerPoll [%d]", fetchThreads, ioConfig.getFetchDelayMillis(), recordBufferSizeBytes, - maxRecordsPerPoll + maxBytesPerPoll ); return new KinesisRecordSupplier( @@ -139,7 +143,7 @@ protected KinesisRecordSupplier newTaskRecordSupplier(final TaskToolbox toolbox) recordBufferSizeBytes, tuningConfig.getRecordBufferOfferTimeout(), tuningConfig.getRecordBufferFullWait(), - maxRecordsPerPoll, + maxBytesPerPoll, false, useListShards ); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java index ba1104b8398a..0c3cd7177522 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java @@ -56,12 +56,14 @@ public class KinesisIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningC private static final int DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT = 5000; private static final int DEFAULT_RECORD_BUFFER_FULL_WAIT = 5000; private static final int DEFAULT_MAX_RECORDS_PER_POLL = 100; + private static final int DEFAULT_MAX_BYTES_PER_POLL = 1_000_000; private final Integer recordBufferSize; private final Integer recordBufferSizeBytes; private final int recordBufferOfferTimeout; private final int recordBufferFullWait; private final Integer fetchThreads; private final Integer maxRecordsPerPoll; + private final Integer maxBytesPerPoll; public KinesisIndexTaskTuningConfig( @Nullable AppendableIndexSpec appendableIndexSpec, @@ -88,7 +90,8 @@ public KinesisIndexTaskTuningConfig( @Nullable Boolean logParseExceptions, @Nullable Integer maxParseExceptions, @Nullable Integer maxSavedParseExceptions, - @Nullable Integer maxRecordsPerPoll, + @Deprecated @Nullable Integer maxRecordsPerPoll, + @Nullable Integer maxBytesPerPoll, @Nullable Period intermediateHandoffPeriod ) { @@ -122,6 +125,7 @@ public KinesisIndexTaskTuningConfig( this.recordBufferFullWait = recordBufferFullWait == null ? DEFAULT_RECORD_BUFFER_FULL_WAIT : recordBufferFullWait; this.fetchThreads = fetchThreads; // we handle this being null later this.maxRecordsPerPoll = maxRecordsPerPoll; + this.maxBytesPerPoll = maxBytesPerPoll; Preconditions.checkArgument( !(super.isResetOffsetAutomatically() && super.isSkipSequenceNumberAvailabilityCheck()), @@ -154,7 +158,8 @@ private KinesisIndexTaskTuningConfig( @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, - @JsonProperty("maxRecordsPerPoll") @Nullable Integer maxRecordsPerPoll, + @JsonProperty("maxRecordsPerPoll") @Deprecated @Nullable Integer maxRecordsPerPoll, + @JsonProperty("maxBytesPerPoll") @Nullable Integer maxBytesPerPoll, @JsonProperty("intermediateHandoffPeriod") @Nullable Period intermediateHandoffPeriod ) { @@ -184,6 +189,7 @@ private KinesisIndexTaskTuningConfig( maxParseExceptions, maxSavedParseExceptions, maxRecordsPerPoll, + maxBytesPerPoll, intermediateHandoffPeriod ); } @@ -244,9 +250,17 @@ public Integer getMaxRecordsPerPollConfigured() return maxRecordsPerPoll; } - public int getMaxRecordsPerPollOrDefault() + @Nullable + @JsonProperty("maxBytesPerPoll") + @JsonInclude(JsonInclude.Include.NON_NULL) + public Integer getMaxBytesPerPollConfigured() + { + return maxBytesPerPoll; + } + + public int getMaxBytesPerPollOrDefault() { - return DEFAULT_MAX_RECORDS_PER_POLL; + return maxBytesPerPoll != null ? maxBytesPerPoll : DEFAULT_MAX_BYTES_PER_POLL; } @Override @@ -278,6 +292,7 @@ public KinesisIndexTaskTuningConfig withBasePersistDirectory(File dir) getMaxParseExceptions(), getMaxSavedParseExceptions(), getMaxRecordsPerPollConfigured(), + getMaxBytesPerPollConfigured(), getIntermediateHandoffPeriod() ); } @@ -300,6 +315,7 @@ public boolean equals(Object o) recordBufferOfferTimeout == that.recordBufferOfferTimeout && recordBufferFullWait == that.recordBufferFullWait && Objects.equals(maxRecordsPerPoll, that.maxRecordsPerPoll) && + Objects.equals(maxBytesPerPoll, that.maxBytesPerPoll) && Objects.equals(fetchThreads, that.fetchThreads); } @@ -313,7 +329,8 @@ public int hashCode() recordBufferOfferTimeout, recordBufferFullWait, fetchThreads, - maxRecordsPerPoll + maxRecordsPerPoll, + maxBytesPerPoll ); } @@ -343,6 +360,7 @@ public String toString() ", maxParseExceptions=" + getMaxParseExceptions() + ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() + ", maxRecordsPerPoll=" + maxRecordsPerPoll + + ", maxBytesPerPoll=" + maxBytesPerPoll + ", intermediateHandoffPeriod=" + getIntermediateHandoffPeriod() + '}'; } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 3c0421cb6dbf..8a777119506e 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -50,6 +50,7 @@ import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.common.aws.AWSCredentialsUtils; import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.error.DruidException; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisor; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; @@ -73,6 +74,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; @@ -106,7 +108,6 @@ public class KinesisRecordSupplier implements RecordSupplier(currRecord, recordSize), @@ -400,7 +399,7 @@ private long getPartitionTimeLag() private final int fetchDelayMillis; private final int recordBufferOfferTimeout; private final int recordBufferFullWait; - private final int maxRecordsPerPoll; + private final int maxBytesPerPoll; private final int fetchThreads; private final int recordBufferSizeBytes; private final boolean useEarliestSequenceNumber; @@ -423,7 +422,7 @@ public KinesisRecordSupplier( int recordBufferSizeBytes, int recordBufferOfferTimeout, int recordBufferFullWait, - int maxRecordsPerPoll, + int maxBytesPerPoll, boolean useEarliestSequenceNumber, boolean useListShards ) @@ -433,15 +432,17 @@ public KinesisRecordSupplier( this.fetchDelayMillis = fetchDelayMillis; this.recordBufferOfferTimeout = recordBufferOfferTimeout; this.recordBufferFullWait = recordBufferFullWait; - this.maxRecordsPerPoll = maxRecordsPerPoll; + this.maxBytesPerPoll = maxBytesPerPoll; this.fetchThreads = fetchThreads; this.recordBufferSizeBytes = recordBufferSizeBytes; this.useEarliestSequenceNumber = useEarliestSequenceNumber; this.useListShards = useListShards; this.backgroundFetchEnabled = fetchThreads > 0; - // the deaggregate function is implemented by the amazon-kinesis-client. Use reflection to find the deaggregate - // function in the classpath. See details on the docs page for more information on how to use deaggregation + // The deaggregate function is implemented by the amazon-kinesis-client, whose license was formerly not compatible + // with Apache. The code here avoids the license issue by using reflection, but is no longer necessary since + // amazon-kinesis-client is now Apache-licensed and is now a dependency of Druid. This code could safely be + // modified to use regular calls rather than reflection. try { Class kclUserRecordclass = Class.forName("com.amazonaws.services.kinesis.clientlibrary.types.UserRecord"); MethodHandles.Lookup lookup = MethodHandles.publicLookup(); @@ -620,22 +621,17 @@ public List> poll(long ti start(); try { - int expectedSize = Math.min(Math.max(records.size(), 1), maxRecordsPerPoll); - - List>> polledRecords = new ArrayList<>(expectedSize); + List>> polledRecords = new LinkedList<>(); records.drain( polledRecords, - MAX_BYTES_PER_POLL, + maxBytesPerPoll, timeout, TimeUnit.MILLISECONDS ); - polledRecords = polledRecords.stream() - .filter(x -> partitionResources.containsKey(x.getData().getStreamPartition())) - .collect(Collectors.toList()); - return polledRecords.stream() + .filter(x -> partitionResources.containsKey(x.getData().getStreamPartition())) .map(MemoryBoundLinkedBlockingQueue.ObjectContainer::getData) .collect(Collectors.toList()); } @@ -1053,13 +1049,12 @@ private void filterBufferAndResetBackgroundFetch(Set> pa .forEachOrdered(x -> { if (!newQ.offer(x)) { // this should never really happen in practice but adding check here for safety. - throw new StreamException(new ISE(StringUtils.format( - "Failed to insert item to new queue when resetting background fetch. " + throw DruidException.defensive("Failed to insert item to new queue when resetting background fetch. " + "[stream: '%s', partitionId: '%s', sequenceNumber: '%s']", x.getData().getStream(), x.getData().getPartitionId(), x.getData().getSequenceNumber() - ))); + ); } }); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpec.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpec.java index 08d9474479b4..81f8b774f042 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpec.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpec.java @@ -78,7 +78,7 @@ protected KinesisRecordSupplier createRecordSupplier() tuningConfig.getRecordBufferSizeBytesOrDefault(Runtime.getRuntime().maxMemory()), tuningConfig.getRecordBufferOfferTimeout(), tuningConfig.getRecordBufferFullWait(), - tuningConfig.getMaxRecordsPerPollOrDefault(), + tuningConfig.getMaxBytesPerPollOrDefault(), ioConfig.isUseEarliestSequenceNumber(), tuningConfig.isUseListShards() ); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 5aae6af06fb2..a142f4147627 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -200,7 +200,7 @@ protected RecordSupplier setupRecordSupplier() throw taskTuningConfig.getRecordBufferSizeBytesOrDefault(Runtime.getRuntime().maxMemory()), taskTuningConfig.getRecordBufferOfferTimeout(), taskTuningConfig.getRecordBufferFullWait(), - taskTuningConfig.getMaxRecordsPerPollOrDefault(), + taskTuningConfig.getMaxBytesPerPollOrDefault(), ioConfig.isUseEarliestSequenceNumber(), spec.getSpec().getTuningConfig().isUseListShards() ); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java index 801f78688df3..a0a68c14bc0b 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -75,6 +75,7 @@ public static KinesisSupervisorTuningConfig defaultConfig() null, null, null, + null, null ); } @@ -107,7 +108,8 @@ public KinesisSupervisorTuningConfig( @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, - @JsonProperty("maxRecordsPerPoll") @Nullable Integer maxRecordsPerPoll, + @JsonProperty("maxRecordsPerPoll") @Deprecated @Nullable Integer maxRecordsPerPoll, + @JsonProperty("maxBytesPerPoll") @Nullable Integer maxBytesPerPoll, @JsonProperty("intermediateHandoffPeriod") Period intermediateHandoffPeriod, @JsonProperty("repartitionTransitionDuration") Period repartitionTransitionDuration, @JsonProperty("offsetFetchPeriod") Period offsetFetchPeriod, @@ -140,6 +142,7 @@ public KinesisSupervisorTuningConfig( maxParseExceptions, maxSavedParseExceptions, maxRecordsPerPoll, + maxBytesPerPoll, intermediateHandoffPeriod ); @@ -237,6 +240,7 @@ public String toString() ", maxParseExceptions=" + getMaxParseExceptions() + ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() + ", maxRecordsPerPoll=" + getMaxRecordsPerPollConfigured() + + ", maxBytesPerPoll=" + getMaxBytesPerPollConfigured() + ", intermediateHandoffPeriod=" + getIntermediateHandoffPeriod() + ", repartitionTransitionDuration=" + getRepartitionTransitionDuration() + ", useListShards=" + isUseListShards() + @@ -272,6 +276,7 @@ public KinesisIndexTaskTuningConfig convertToTaskTuningConfig() getMaxParseExceptions(), getMaxSavedParseExceptions(), getMaxRecordsPerPollConfigured(), + getMaxBytesPerPollConfigured(), getIntermediateHandoffPeriod() ); } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java index 8f27adb5c435..da04e3ab0a6d 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java @@ -77,6 +77,7 @@ public class KinesisIndexTaskSerdeTest null, null, null, + null, null ); private static final KinesisIndexTaskIOConfig IO_CONFIG = new KinesisIndexTaskIOConfig( diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 54834efdbaf2..e14b6679b09e 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -184,6 +184,7 @@ public static Iterable constructorFeeder() private Long maxTotalRows = null; private final Period intermediateHandoffPeriod = null; private int maxRecordsPerPoll; + private int maxBytesPerPoll; @BeforeClass public static void setupClass() @@ -218,6 +219,7 @@ public void setupTest() throws IOException, InterruptedException doHandoff = true; reportsFile = File.createTempFile("KinesisIndexTaskTestReports-" + System.currentTimeMillis(), "json"); maxRecordsPerPoll = 1; + maxBytesPerPoll = 1_000_000; recordSupplier = mock(KinesisRecordSupplier.class); @@ -562,6 +564,7 @@ public void testIncrementalHandOff() throws Exception // as soon as any segment has more than one record, incremental publishing should happen maxRowsPerSegment = 2; maxRecordsPerPoll = 1; + maxBytesPerPoll = 1_000_000; recordSupplier.assign(EasyMock.anyObject()); EasyMock.expectLastCall().anyTimes(); @@ -1693,6 +1696,7 @@ public void testRestoreAfterPersistingSequences() throws Exception { maxRowsPerSegment = 2; maxRecordsPerPoll = 1; + maxBytesPerPoll = 1_000_000; List> records = clone(SINGLE_PARTITION_RECORDS); @@ -2358,6 +2362,7 @@ private KinesisIndexTask createTask( maxParseExceptions, maxSavedParseExceptions, maxRecordsPerPoll, + maxBytesPerPoll, intermediateHandoffPeriod ); return createTask(taskId, dataSchema, ioConfig, tuningConfig, context); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index b1a2193571c8..b61c5cf2ae48 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -165,6 +165,7 @@ public void testSerdeWithModifiedTuningConfigAddedField() throws IOException 500, 500, 6000, + 1_000_000, new Period("P3D") ); @@ -192,8 +193,10 @@ public void testSerdeWithModifiedTuningConfigAddedField() throws IOException Assert.assertEquals(base.getMaxSavedParseExceptions(), deserialized.getMaxSavedParseExceptions()); Assert.assertEquals(base.getRecordBufferFullWait(), deserialized.getRecordBufferFullWait()); Assert.assertEquals(base.getRecordBufferOfferTimeout(), deserialized.getRecordBufferOfferTimeout()); + Assert.assertEquals(base.getRecordBufferSizeConfigured(), deserialized.getRecordBufferSizeConfigured()); Assert.assertEquals(base.getRecordBufferSizeBytesConfigured(), deserialized.getRecordBufferSizeBytesConfigured()); Assert.assertEquals(base.getMaxRecordsPerPollConfigured(), deserialized.getMaxRecordsPerPollConfigured()); + Assert.assertEquals(base.getMaxBytesPerPollConfigured(), deserialized.getMaxBytesPerPollConfigured()); } @Test @@ -224,6 +227,7 @@ public void testSerdeWithModifiedTuningConfigRemovedField() throws IOException false, 500, 500, + 1_000_000, 6000, new Period("P3D") ); @@ -314,6 +318,7 @@ public void testConvert() null, null, 10, + 1_000_000, null, null, null, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java index 5d1b77f38584..5fcf81139eb6 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java @@ -221,7 +221,7 @@ public void testSupplierSetup_withoutListShards() 100, 5000, 5000, - 5, + 1_000_000, true, false ); @@ -278,7 +278,7 @@ public void testSupplierSetup_withListShards() 100, 5000, 5000, - 5, + 1_000_000, true, true ); @@ -385,7 +385,7 @@ public void testPollWithKinesisInternalFailure() throws InterruptedException 10_000, 5000, 5000, - 100, + 1_000_000, true, false ); @@ -442,7 +442,7 @@ public void testPollWithKinesisNonRetryableFailure() throws InterruptedException 100, 5000, 5000, - 100, + 1_000_000, true, false ); @@ -517,7 +517,7 @@ public void testSeek() 10_000, 5000, 5000, - 100, + 1_000_000, true, false ); @@ -581,7 +581,7 @@ public void testSeekToLatest() 100, 5000, 5000, - 100, + 1_000_000, true, false ); @@ -614,7 +614,7 @@ public void testSeekUnassigned() throws InterruptedException 100, 5000, 5000, - 5, + 1_000_000, true, false ); @@ -674,7 +674,7 @@ public void testPollAfterSeek() 10_000, 5000, 5000, - 1, + 1_000_000, true, false ); @@ -763,7 +763,7 @@ public void testPollDeaggregate() throws InterruptedException 10_000, 5000, 5000, - 100, + 1_000_000, true, false ); @@ -837,7 +837,7 @@ public void getLatestSequenceNumberWhenKinesisRetryableException() 10_000, 5000, 5000, - 100, + 1_000_000, true, false ); @@ -872,7 +872,7 @@ private KinesisRecordSupplier getSequenceNumberWhenNoRecordsHelperForOpenShard() 10_000, 5000, 5000, - 100, + 1_000_000, true, false ); @@ -959,7 +959,7 @@ public void getPartitionTimeLag() throws InterruptedException 10_000, 5000, 5000, - 100, + 1_000_000, true, false ); @@ -1007,15 +1007,16 @@ public void getPartitionTimeLag() throws InterruptedException public void testIsOffsetAvailable() { AmazonKinesis mockKinesis = EasyMock.mock(AmazonKinesis.class); - KinesisRecordSupplier target = new KinesisRecordSupplier(mockKinesis, - 0, - 2, - 100, - 5000, - 5000, - 5, - true, - false + KinesisRecordSupplier target = new KinesisRecordSupplier( + mockKinesis, + 0, + 2, + 100, + 5000, + 5000, + 1_000_000, + true, + false ); StreamPartition partition = new StreamPartition<>(STREAM, SHARD_ID0); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 62ad857160ab..e25d57fcf679 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -211,6 +211,7 @@ public void setupTest() null, null, null, + null, null ); rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory(); @@ -3964,6 +3965,7 @@ public void testIsTaskCurrent() null, null, 42, // This property is different from tuningConfig + 1_000_000, null, null, null, @@ -5141,6 +5143,7 @@ public SeekableStreamIndexTaskClient build( null, null, null, + null, null ); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java index ca4de4a99dd7..ac84d2105cd1 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java @@ -51,6 +51,7 @@ public TestModifiedKinesisIndexTaskTuningConfig( @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, @JsonProperty("skipSequenceNumberAvailabilityCheck") Boolean skipSequenceNumberAvailabilityCheck, + @JsonProperty("recordBufferSize") Integer recordBufferSize, @JsonProperty("recordBufferSizeBytes") Integer recordBufferSizeBytes, @JsonProperty("recordBufferOfferTimeout") Integer recordBufferOfferTimeout, @JsonProperty("recordBufferFullWait") Integer recordBufferFullWait, @@ -60,6 +61,7 @@ public TestModifiedKinesisIndexTaskTuningConfig( @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, @JsonProperty("maxRecordsPerPoll") @Nullable Integer maxRecordsPerPoll, + @JsonProperty("maxBytesPerPoll") @Nullable Integer maxBytesPerPoll, @JsonProperty("intermediateHandoffPeriod") @Nullable Period intermediateHandoffPeriod, @JsonProperty("extra") String extra ) @@ -80,7 +82,7 @@ public TestModifiedKinesisIndexTaskTuningConfig( handoffConditionTimeout, resetOffsetAutomatically, skipSequenceNumberAvailabilityCheck, - null, + recordBufferSize, recordBufferSizeBytes, recordBufferOfferTimeout, recordBufferFullWait, @@ -90,6 +92,7 @@ public TestModifiedKinesisIndexTaskTuningConfig( maxParseExceptions, maxSavedParseExceptions, maxRecordsPerPoll, + maxBytesPerPoll, intermediateHandoffPeriod ); this.extra = extra; @@ -113,7 +116,7 @@ public TestModifiedKinesisIndexTaskTuningConfig(KinesisIndexTaskTuningConfig bas base.getHandoffConditionTimeout(), base.isResetOffsetAutomatically(), base.isSkipSequenceNumberAvailabilityCheck(), - null, + base.getRecordBufferSizeConfigured(), base.getRecordBufferSizeBytesConfigured(), base.getRecordBufferOfferTimeout(), base.getRecordBufferFullWait(), @@ -123,6 +126,7 @@ public TestModifiedKinesisIndexTaskTuningConfig(KinesisIndexTaskTuningConfig bas base.getMaxParseExceptions(), base.getMaxSavedParseExceptions(), base.getMaxRecordsPerPollConfigured(), + base.getMaxBytesPerPollConfigured(), base.getIntermediateHandoffPeriod() ); this.extra = extra; diff --git a/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx b/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx index e2e59cecf44d..c96784d5efdc 100644 --- a/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx +++ b/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx @@ -2089,15 +2089,15 @@ const TUNING_FORM_FIELDS: Field[] = [ ), }, { - name: 'spec.tuningConfig.maxRecordsPerPoll', + name: 'spec.tuningConfig.maxBytesPerPoll', type: 'number', - defaultValue: 100, + defaultValue: 1000000, defined: typeIsKnown(KNOWN_TYPES, 'kinesis'), hideInMore: true, info: ( <> - The maximum number of records/events to be fetched from buffer per poll. The actual maximum - will be max(maxRecordsPerPoll, max(bufferSize, 1)). + The maximum number of bytes to be fetched from buffer per poll. At least one + record will be fetched regardless of config.. ), }, diff --git a/website/.spelling b/website/.spelling index b6c304da1f6d..49468469da54 100644 --- a/website/.spelling +++ b/website/.spelling @@ -1097,6 +1097,7 @@ LZ4LZFuncompressedLZ4LZ4LZFuncompressednoneLZ4autolongsautolongslongstypeconcise deaggregate druid-kinesis-indexing-service maxRecordsPerPoll +maxBytesPerPoll maxRecordsPerPollrecordsPerFetchfetchDelayMillisreplicasfetchDelayMillisrecordsPerFetchfetchDelayMillismaxRecordsPerPollamazon-kinesis-client1 numKinesisShards numProcessors From bca26e944e905d42da36ba6abcd9a41c3fd65473 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Wed, 29 Nov 2023 15:08:42 -0500 Subject: [PATCH 15/24] * fix ingestion-spec.tsx --- web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx b/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx index c96784d5efdc..1856bfa6cab9 100644 --- a/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx +++ b/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx @@ -2097,7 +2097,7 @@ const TUNING_FORM_FIELDS: Field[] = [ info: ( <> The maximum number of bytes to be fetched from buffer per poll. At least one - record will be fetched regardless of config.. + record will be fetched regardless of config. ), }, From 127bf0d07371b1bc1ce71b2c640bb94908dc9d03 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Wed, 29 Nov 2023 15:47:31 -0500 Subject: [PATCH 16/24] * change back to ArrayList --- .../apache/druid/indexing/kinesis/KinesisRecordSupplier.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 8a777119506e..36047ce429db 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -74,7 +74,6 @@ import java.util.Collection; import java.util.Collections; import java.util.Iterator; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; @@ -621,7 +620,7 @@ public List> poll(long ti start(); try { - List>> polledRecords = new LinkedList<>(); + List>> polledRecords = new ArrayList<>(); records.drain( polledRecords, From 07840a79a47d8b6b622d2cc0790e0644f82ba4f9 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Wed, 29 Nov 2023 17:35:14 -0500 Subject: [PATCH 17/24] * fix ingestion-spec.tsx again --- .../src/druid-models/ingestion-spec/ingestion-spec.tsx | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx b/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx index 1856bfa6cab9..1f5b3c148920 100644 --- a/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx +++ b/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx @@ -2096,8 +2096,8 @@ const TUNING_FORM_FIELDS: Field[] = [ hideInMore: true, info: ( <> - The maximum number of bytes to be fetched from buffer per poll. At least one - record will be fetched regardless of config. + The maximum number of bytes to be fetched from buffer per poll. At least one record will be + fetched regardless of config. ), }, From 0287eb19648f5a813991eb5c2fe62e9e345e93bb Mon Sep 17 00:00:00 2001 From: zachjsh Date: Fri, 1 Dec 2023 12:45:20 -0500 Subject: [PATCH 18/24] * more review comments --- .../extensions-core/kinesis-ingestion.md | 66 ++++++------ .../indexing/kinesis/KinesisIndexTask.java | 27 +++-- .../supervisor/KinesisSupervisorIOConfig.java | 25 ++++- .../kinesis/KinesisSamplerSpecTest.java | 12 ++- .../supervisor/KinesisSupervisorTest.java | 100 ++++++++++-------- .../MemoryBoundLinkedBlockingQueue.java | 12 +++ .../MemoryBoundLinkedBlockingQueueTest.java | 22 ++++ 7 files changed, 176 insertions(+), 88 deletions(-) diff --git a/docs/development/extensions-core/kinesis-ingestion.md b/docs/development/extensions-core/kinesis-ingestion.md index 6d9a6018988a..7d0709d99cf9 100644 --- a/docs/development/extensions-core/kinesis-ingestion.md +++ b/docs/development/extensions-core/kinesis-ingestion.md @@ -383,39 +383,39 @@ The `tuningConfig` object is optional. If you don't specify the `tuningConfig` o The following table outlines the configuration options for `tuningConfig`: -|Property|Type| Description |Required| Default | -|--------|----|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|--------|----------------------------------------------------------------------------------------------------| -|`type`|String| The indexing task type. This should always be `kinesis`. |Yes| | -|`maxRowsInMemory`|Integer| The number of rows to aggregate before persisting. This number represents the post-aggregation rows. It is not equivalent to the number of input events, but the resulting number of aggregated rows. Druid uses `maxRowsInMemory` to manage the required JVM heap size. The maximum heap memory usage for indexing scales is `maxRowsInMemory * (2 + maxPendingPersists)`. |No| 100000 | -|`maxBytesInMemory`|Long| The number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. Normally, this is computed internally. The maximum heap memory usage for indexing is `maxBytesInMemory * (2 + maxPendingPersists)`. |No| One-sixth of max JVM memory | -|`skipBytesInMemoryOverheadCheck`|Boolean| The calculation of `maxBytesInMemory` takes into account overhead objects created during ingestion and each intermediate persist. To exclude the bytes of these overhead objects from the `maxBytesInMemory` check, set `skipBytesInMemoryOverheadCheck` to `true`. |No| `false` | -|`maxRowsPerSegment`|Integer| The number of rows to aggregate into a segment; this number represents the post-aggregation rows. Handoff occurs when `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first. |No| 5000000 | -|`maxTotalRows`|Long| The number of rows to aggregate across all segments; this number represents the post-aggregation rows. Handoff occurs when `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first. |No| unlimited | -|`intermediateHandoffPeriod`|ISO 8601 period| The period that determines how often tasks hand off segments. Handoff occurs if `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first. |No| P2147483647D | -|`intermediatePersistPeriod`|ISO 8601 period| The period that determines the rate at which intermediate persists occur. |No| PT10M | -|`maxPendingPersists`|Integer| Maximum number of persists that can be pending but not started. If a new intermediate persist exceeds this limit, Druid blocks ingestion until the currently running persist finishes. One persist can be running concurrently with ingestion, and none can be queued up. The maximum heap memory usage for indexing scales is `maxRowsInMemory * (2 + maxPendingPersists)`. |No| 0 | -|`indexSpec`|Object| Defines how Druid indexes the data. See [IndexSpec](#indexspec) for more information. |No| | -|`indexSpecForIntermediatePersists`|Object| Defines segment storage format options to use at indexing time for intermediate persisted temporary segments. You can use `indexSpecForIntermediatePersists` to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. However, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published. See [IndexSpec](#indexspec) for possible values. |No| Same as `indexSpec` | -|`reportParseExceptions`|Boolean| If `true`, Druid throws exceptions encountered during parsing causing ingestion to halt. If `false`, Druid skips unparseable rows and fields. |No| `false` | -|`handoffConditionTimeout`|Long| Number of milliseconds to wait for segment handoff. Set to a value >= 0, where 0 means to wait indefinitely. |No| 0 | -|`resetOffsetAutomatically`|Boolean| Controls behavior when Druid needs to read Kinesis messages that are no longer available.
If `false`, the exception bubbles up causing tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation, potentially using the [Reset Supervisor API](../../api-reference/supervisor-api.md). This mode is useful for production, since it highlights issues with ingestion.
If `true`, Druid automatically resets to the earliest or latest sequence number available in Kinesis, based on the value of the `useEarliestSequenceNumber` property (earliest if `true`, latest if `false`). Note that this can lead to dropping data (if `useEarliestSequenceNumber` is `false`) or duplicating data (if `useEarliestSequenceNumber` is `true`) without your knowledge. Druid logs messages indicating that a reset has occurred without interrupting ingestion. This mode is useful for non-production situations since it enables Druid to recover from problems automatically, even if they lead to quiet dropping or duplicating of data. |No| `false` | -|`skipSequenceNumberAvailabilityCheck`|Boolean| Whether to enable checking if the current sequence number is still available in a particular Kinesis shard. If `false`, the indexing task attempts to reset the current sequence number, depending on the value of `resetOffsetAutomatically`. |No| `false` | -|`workerThreads`|Integer| The number of threads that the supervisor uses to handle requests/responses for worker tasks, along with any other internal asynchronous operation. |No| `min(10, taskCount)` | -|`chatRetries`|Integer| The number of times Druid retries HTTP requests to indexing tasks before considering tasks unresponsive. |No| 8 | -|`httpTimeout`|ISO 8601 period| The period of time to wait for a HTTP response from an indexing task. |No| PT10S | -|`shutdownTimeout`|ISO 8601 period| The period of time to wait for the supervisor to attempt a graceful shutdown of tasks before exiting. |No| PT80S | -|`recordBufferSizeBytes`|Integer| The size of the buffer (heap memory bytes) Druid uses between the Kinesis fetch threads and the main ingestion thread. |No| See [Determine fetch settings](#determine-fetch-settings) for defaults. | -|`recordBufferOfferTimeout`|Integer| The number of milliseconds to wait for space to become available in the buffer before timing out. |No| 5000 | -|`recordBufferFullWait`|Integer| The number of milliseconds to wait for the buffer to drain before Druid attempts to fetch records from Kinesis again. |No| 5000 | -|`fetchThreads`|Integer| The size of the pool of threads fetching data from Kinesis. There is no benefit in having more threads than Kinesis shards. |No| `procs * 2`, where `procs` is the number of processors available to the task. | -|`segmentWriteOutMediumFactory`|Object| The segment write-out medium to use when creating segments See [Additional Peon configuration: SegmentWriteOutMediumFactory](../../configuration/index.md#segmentwriteoutmediumfactory) for explanation and available options. |No| If not specified, Druid uses the value from `druid.peon.defaultSegmentWriteOutMediumFactory.type`. | -|`logParseExceptions`|Boolean| If `true`, Druid logs an error message when a parsing exception occurs, containing information about the row where the error occurred. |No| `false` | -|`maxParseExceptions`|Integer| The maximum number of parse exceptions that can occur before the task halts ingestion and fails. Overridden if `reportParseExceptions` is set. |No| unlimited | -|`maxSavedParseExceptions`|Integer| When a parse exception occurs, Druid keeps track of the most recent parse exceptions. `maxSavedParseExceptions` limits the number of saved exception instances. These saved exceptions are available after the task finishes in the [task completion report](../../ingestion/tasks.md#task-reports). Overridden if `reportParseExceptions` is set. |No| 0 | -|`maxBytesPerPoll`|Integer| The maximum number of bytes to be fetched from buffer per poll. At least one record is polled from the buffer regardless of this config. |No| 1000000 bytes | -|`repartitionTransitionDuration`|ISO 8601 period| When shards are split or merged, the supervisor recomputes shard to task group mappings. The supervisor also signals any running tasks created under the old mappings to stop early at current time + `repartitionTransitionDuration`. Stopping the tasks early allows Druid to begin reading from the new shards more quickly. The repartition transition wait time controlled by this property gives the stream additional time to write records to the new shards after the split or merge, which helps avoid issues with [empty shard handling](https://github.com/apache/druid/issues/7600). |No| PT2M | -|`offsetFetchPeriod`|ISO 8601 period| Determines how often the supervisor queries Kinesis and the indexing tasks to fetch current offsets and calculate lag. If the user-specified value is below the minimum value of PT5S, the supervisor ignores the value and uses the minimum value instead. |No| PT30S | -|`useListShards`|Boolean| Indicates if `listShards` API of AWS Kinesis SDK can be used to prevent `LimitExceededException` during ingestion. You must set the necessary `IAM` permissions. |No| `false` | +|Property|Type|Description|Required|Default| +|--------|----|-----------|--------|-------| +|`type`|String|The indexing task type. This should always be `kinesis`.|Yes|| +|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number represents the post-aggregation rows. It is not equivalent to the number of input events, but the resulting number of aggregated rows. Druid uses `maxRowsInMemory` to manage the required JVM heap size. The maximum heap memory usage for indexing scales is `maxRowsInMemory * (2 + maxPendingPersists)`.|No|100000| +|`maxBytesInMemory`|Long| The number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. Normally, this is computed internally. The maximum heap memory usage for indexing is `maxBytesInMemory * (2 + maxPendingPersists)`.|No|One-sixth of max JVM memory| +|`skipBytesInMemoryOverheadCheck`|Boolean|The calculation of `maxBytesInMemory` takes into account overhead objects created during ingestion and each intermediate persist. To exclude the bytes of these overhead objects from the `maxBytesInMemory` check, set `skipBytesInMemoryOverheadCheck` to `true`.|No|`false`| +|`maxRowsPerSegment`|Integer|The number of rows to aggregate into a segment; this number represents the post-aggregation rows. Handoff occurs when `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first.|No|5000000| +|`maxTotalRows`|Long|The number of rows to aggregate across all segments; this number represents the post-aggregation rows. Handoff occurs when `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first.|No|unlimited| +|`intermediateHandoffPeriod`|ISO 8601 period|The period that determines how often tasks hand off segments. Handoff occurs if `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first.|No|P2147483647D| +|`intermediatePersistPeriod`|ISO 8601 period|The period that determines the rate at which intermediate persists occur.|No|PT10M| +|`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If a new intermediate persist exceeds this limit, Druid blocks ingestion until the currently running persist finishes. One persist can be running concurrently with ingestion, and none can be queued up. The maximum heap memory usage for indexing scales is `maxRowsInMemory * (2 + maxPendingPersists)`.|No|0| +|`indexSpec`|Object|Defines how Druid indexes the data. See [IndexSpec](#indexspec) for more information.|No|| +|`indexSpecForIntermediatePersists`|Object|Defines segment storage format options to use at indexing time for intermediate persisted temporary segments. You can use `indexSpecForIntermediatePersists` to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. However, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published. See [IndexSpec](#indexspec) for possible values.|No|Same as `indexSpec`| +|`reportParseExceptions`|Boolean|If `true`, Druid throws exceptions encountered during parsing causing ingestion to halt. If `false`, Druid skips unparseable rows and fields.|No|`false`| +|`handoffConditionTimeout`|Long|Number of milliseconds to wait for segment handoff. Set to a value >= 0, where 0 means to wait indefinitely.|No|0| +|`resetOffsetAutomatically`|Boolean|Controls behavior when Druid needs to read Kinesis messages that are no longer available.
If `false`, the exception bubbles up causing tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation, potentially using the [Reset Supervisor API](../../api-reference/supervisor-api.md). This mode is useful for production, since it highlights issues with ingestion.
If `true`, Druid automatically resets to the earliest or latest sequence number available in Kinesis, based on the value of the `useEarliestSequenceNumber` property (earliest if `true`, latest if `false`). Note that this can lead to dropping data (if `useEarliestSequenceNumber` is `false`) or duplicating data (if `useEarliestSequenceNumber` is `true`) without your knowledge. Druid logs messages indicating that a reset has occurred without interrupting ingestion. This mode is useful for non-production situations since it enables Druid to recover from problems automatically, even if they lead to quiet dropping or duplicating of data.|No|`false`| +|`skipSequenceNumberAvailabilityCheck`|Boolean|Whether to enable checking if the current sequence number is still available in a particular Kinesis shard. If `false`, the indexing task attempts to reset the current sequence number, depending on the value of `resetOffsetAutomatically`.|No|`false`| +|`workerThreads`|Integer|The number of threads that the supervisor uses to handle requests/responses for worker tasks, along with any other internal asynchronous operation.|No| `min(10, taskCount)`| +|`chatRetries`|Integer|The number of times Druid retries HTTP requests to indexing tasks before considering tasks unresponsive.|No|8| +|`httpTimeout`|ISO 8601 period|The period of time to wait for a HTTP response from an indexing task.|No|PT10S| +|`shutdownTimeout`|ISO 8601 period|The period of time to wait for the supervisor to attempt a graceful shutdown of tasks before exiting.|No|PT80S| +|`recordBufferSizeBytes`|Integer| The size of the buffer (heap memory bytes) Druid uses between the Kinesis fetch threads and the main ingestion thread.|No| See [Determine fetch settings](#determine-fetch-settings) for defaults.| +|`recordBufferOfferTimeout`|Integer|The number of milliseconds to wait for space to become available in the buffer before timing out.|No|5000| +|`recordBufferFullWait`|Integer|The number of milliseconds to wait for the buffer to drain before Druid attempts to fetch records from Kinesis again.|No|5000| +|`fetchThreads`|Integer|The size of the pool of threads fetching data from Kinesis. There is no benefit in having more threads than Kinesis shards.|No| `procs * 2`, where `procs` is the number of processors available to the task.| +|`segmentWriteOutMediumFactory`|Object|The segment write-out medium to use when creating segments See [Additional Peon configuration: SegmentWriteOutMediumFactory](../../configuration/index.md#segmentwriteoutmediumfactory) for explanation and available options.|No|If not specified, Druid uses the value from `druid.peon.defaultSegmentWriteOutMediumFactory.type`.| +|`logParseExceptions`|Boolean|If `true`, Druid logs an error message when a parsing exception occurs, containing information about the row where the error occurred.|No|`false`| +|`maxParseExceptions`|Integer|The maximum number of parse exceptions that can occur before the task halts ingestion and fails. Overridden if `reportParseExceptions` is set.|No|unlimited| +|`maxSavedParseExceptions`|Integer|When a parse exception occurs, Druid keeps track of the most recent parse exceptions. `maxSavedParseExceptions` limits the number of saved exception instances. These saved exceptions are available after the task finishes in the [task completion report](../../ingestion/tasks.md#task-reports). Overridden if `reportParseExceptions` is set.|No|0| +|`maxBytesPerPoll`|Integer| The maximum number of bytes to be fetched from buffer per poll. At least one record is polled from the buffer regardless of this config.|No| 1000000 bytes| +|`repartitionTransitionDuration`|ISO 8601 period|When shards are split or merged, the supervisor recomputes shard to task group mappings. The supervisor also signals any running tasks created under the old mappings to stop early at current time + `repartitionTransitionDuration`. Stopping the tasks early allows Druid to begin reading from the new shards more quickly. The repartition transition wait time controlled by this property gives the stream additional time to write records to the new shards after the split or merge, which helps avoid issues with [empty shard handling](https://github.com/apache/druid/issues/7600).|No|PT2M| +|`offsetFetchPeriod`|ISO 8601 period|Determines how often the supervisor queries Kinesis and the indexing tasks to fetch current offsets and calculate lag. If the user-specified value is below the minimum value of PT5S, the supervisor ignores the value and uses the minimum value instead.|No|PT30S| +|`useListShards`|Boolean|Indicates if `listShards` API of AWS Kinesis SDK can be used to prevent `LimitExceededException` during ingestion. You must set the necessary `IAM` permissions.|No|`false`| ### IndexSpec diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index ad5aaa6c7723..fb019f10030b 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -82,20 +82,20 @@ public KinesisIndexTask( ); this.useListShards = useListShards; this.awsCredentialsConfig = awsCredentialsConfig; - if (tuningConfig.getRecordBufferSizeConfigured() != null) { - log.warn("The 'recordBufferSize' config property of the kinesis tuning config has been deprecated. " - + "Please use 'recordBufferSizeBytes'."); - } - if (tuningConfig.getMaxRecordsPerPollConfigured() != null) { - log.warn("The 'maxRecordsPerPoll' config property of the kinesis tuning config has been deprecated. " - + "Please use 'maxBytesPerPoll'."); - } } @Override public TaskStatus runTask(TaskToolbox toolbox) { this.runtimeInfo = toolbox.getAdjustedRuntimeInfo(); + if (getTuningConfig().getRecordBufferSizeConfigured() != null) { + log.warn("The 'recordBufferSize' config property of the kinesis tuning config has been deprecated. " + + "Please use 'recordBufferSizeBytes'."); + } + if (getTuningConfig().getMaxRecordsPerPollConfigured() != null) { + log.warn("The 'maxRecordsPerPoll' config property of the kinesis tuning config has been deprecated. " + + "Please use 'maxBytesPerPoll'."); + } return super.runTask(toolbox); } @@ -149,6 +149,13 @@ protected KinesisRecordSupplier newTaskRecordSupplier(final TaskToolbox toolbox) ); } + @Override + @JsonProperty + public KinesisIndexTaskTuningConfig getTuningConfig() + { + return (KinesisIndexTaskTuningConfig) super.getTuningConfig(); + } + @Override @JsonProperty("ioConfig") public KinesisIndexTaskIOConfig getIOConfig() @@ -212,7 +219,9 @@ static int computeFetchThreads( (int) (memoryToUse / GET_RECORDS_MAX_BYTES_PER_CALL) ); if (fetchThreads > maxFetchThreads) { - log.warn("fetchThreads [%d] being lowered to [%d]", fetchThreads, maxFetchThreads); + if (configuredFetchThreads != null) { + log.warn("fetchThreads [%d] being lowered to [%d]", configuredFetchThreads, maxFetchThreads); + } fetchThreads = maxFetchThreads; } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java index daaceed3a1e4..9910f22a349b 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java @@ -48,10 +48,12 @@ public class KinesisSupervisorIOConfig extends SeekableStreamSupervisorIOConfig // throw ProvisionedThroughputExceededException. Note that GetRecords won't return any data when it throws an // exception. For this reason, we recommend that you wait one second between calls to GetRecords; however, it's // possible that the application will get exceptions for longer than 1 second. + private final Integer recordsPerFetch; private final int fetchDelayMillis; private final String awsAssumedRoleArn; private final String awsExternalId; + private final boolean deaggregate; @JsonCreator public KinesisSupervisorIOConfig( @@ -69,10 +71,12 @@ public KinesisSupervisorIOConfig( @JsonProperty("lateMessageRejectionPeriod") Period lateMessageRejectionPeriod, @JsonProperty("earlyMessageRejectionPeriod") Period earlyMessageRejectionPeriod, @JsonProperty("lateMessageRejectionStartDateTime") DateTime lateMessageRejectionStartDateTime, + @JsonProperty("recordsPerFetch") @Deprecated Integer recordsPerFetch, @JsonProperty("fetchDelayMillis") Integer fetchDelayMillis, @JsonProperty("awsAssumedRoleArn") String awsAssumedRoleArn, @JsonProperty("awsExternalId") String awsExternalId, - @Nullable @JsonProperty("autoScalerConfig") AutoScalerConfig autoScalerConfig + @Nullable @JsonProperty("autoScalerConfig") AutoScalerConfig autoScalerConfig, + @JsonProperty("deaggregate") @Deprecated boolean deaggregate ) { super( @@ -96,11 +100,13 @@ public KinesisSupervisorIOConfig( this.endpoint = endpoint != null ? endpoint : (region != null ? region.getEndpoint() : KinesisRegion.US_EAST_1.getEndpoint()); + this.recordsPerFetch = recordsPerFetch; this.fetchDelayMillis = fetchDelayMillis != null ? fetchDelayMillis : KinesisIndexTaskIOConfig.DEFAULT_FETCH_DELAY_MILLIS; this.awsAssumedRoleArn = awsAssumedRoleArn; this.awsExternalId = awsExternalId; + this.deaggregate = deaggregate; } @JsonProperty @@ -109,6 +115,14 @@ public String getEndpoint() return endpoint; } + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Integer getRecordsPerFetch() + { + return recordsPerFetch; + } + @JsonProperty @JsonInclude(JsonInclude.Include.NON_DEFAULT) public int getFetchDelayMillis() @@ -130,6 +144,13 @@ public String getAwsExternalId() return awsExternalId; } + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public boolean isDeaggregate() + { + return deaggregate; + } + @Override public String toString() { @@ -147,9 +168,11 @@ public String toString() ", lateMessageRejectionPeriod=" + getLateMessageRejectionPeriod() + ", earlyMessageRejectionPeriod=" + getEarlyMessageRejectionPeriod() + ", lateMessageRejectionStartDateTime=" + getLateMessageRejectionStartDateTime() + + ", recordsPerFetch=" + recordsPerFetch + ", fetchDelayMillis=" + fetchDelayMillis + ", awsAssumedRoleArn='" + awsAssumedRoleArn + '\'' + ", awsExternalId='" + awsExternalId + '\'' + + ", deaggregate=" + deaggregate + '}'; } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java index f95c4c6f3841..b0ba730a3502 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java @@ -146,7 +146,9 @@ public void testSample() throws InterruptedException null, null, null, - null + null, + null, + false ), null, null, @@ -222,7 +224,9 @@ public void testSampleWithInputRowParser() throws IOException, InterruptedExcept null, null, null, - null + null, + null, + false ), null, null, @@ -273,7 +277,9 @@ public void testGetInputSourceResources() null, null, null, - null + null, + null, + false ), null, null, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index e25d57fcf679..9bec68669369 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -461,10 +461,12 @@ public void testRecordSupplier() null, null, null, + 100, 1000, null, null, - null + null, + false ); KinesisIndexTaskClientFactory clientFactory = new KinesisIndexTaskClientFactory(null, OBJECT_MAPPER); KinesisSupervisor supervisor = new KinesisSupervisor( @@ -509,24 +511,26 @@ public void testKinesisIOConfigInitAndAutoscalerConfigCreation() { // create KinesisSupervisorIOConfig with autoScalerConfig null KinesisSupervisorIOConfig kinesisSupervisorIOConfigWithNullAutoScalerConfig = new KinesisSupervisorIOConfig( - STREAM, - INPUT_FORMAT, - "awsEndpoint", - null, - 1, - 1, - new Period("PT30M"), - new Period("P1D"), - new Period("PT30S"), - false, - new Period("PT30M"), - null, - null, - null, - 1000, - null, - null, - null + STREAM, + INPUT_FORMAT, + "awsEndpoint", + null, + 1, + 1, + new Period("PT30M"), + new Period("P1D"), + new Period("PT30S"), + false, + new Period("PT30M"), + null, + null, + null, + 100, + 1000, + null, + null, + null, + false ); AutoScalerConfig autoscalerConfigNull = kinesisSupervisorIOConfigWithNullAutoScalerConfig.getAutoScalerConfig(); @@ -534,24 +538,26 @@ public void testKinesisIOConfigInitAndAutoscalerConfigCreation() // create KinesisSupervisorIOConfig with autoScalerConfig Empty KinesisSupervisorIOConfig kinesisSupervisorIOConfigWithEmptyAutoScalerConfig = new KinesisSupervisorIOConfig( - STREAM, - INPUT_FORMAT, - "awsEndpoint", - null, - 1, - 1, - new Period("PT30M"), - new Period("P1D"), - new Period("PT30S"), - false, - new Period("PT30M"), - null, - null, - null, - 1000, - null, - null, - OBJECT_MAPPER.convertValue(new HashMap<>(), AutoScalerConfig.class) + STREAM, + INPUT_FORMAT, + "awsEndpoint", + null, + 1, + 1, + new Period("PT30M"), + new Period("P1D"), + new Period("PT30S"), + false, + new Period("PT30M"), + null, + null, + null, + 100, + 1000, + null, + null, + OBJECT_MAPPER.convertValue(new HashMap<>(), AutoScalerConfig.class), + false ); AutoScalerConfig autoscalerConfig = kinesisSupervisorIOConfigWithEmptyAutoScalerConfig.getAutoScalerConfig(); @@ -4165,7 +4171,9 @@ public void testCorrectInputSources() null, null, null, - null + null, + null, + false ), null, null, @@ -5088,7 +5096,9 @@ private TestableKinesisSupervisor getTestableSupervisor( null, null, null, - null + null, + null, + false ); KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory( @@ -5224,10 +5234,12 @@ private TestableKinesisSupervisor getTestableSupervisor( lateMessageRejectionPeriod, earlyMessageRejectionPeriod, null, + null, fetchDelayMillis, null, null, - autoScalerConfig + autoScalerConfig, + false ); KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory( @@ -5311,10 +5323,12 @@ private TestableKinesisSupervisor getTestableSupervisorCustomIsTaskCurrent( lateMessageRejectionPeriod, earlyMessageRejectionPeriod, null, + null, fetchDelayMillis, null, null, - null + null, + false ); KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory( @@ -5397,10 +5411,12 @@ private KinesisSupervisor createSupervisor( lateMessageRejectionPeriod, earlyMessageRejectionPeriod, null, + null, fetchDelayMillis, null, null, - null + null, + false ); KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory( diff --git a/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java b/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java index d585477d046a..d51d8ac9f104 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java @@ -97,6 +97,18 @@ public Stream> stream() return queue.stream(); } + /** + * Drain up to specified bytes worth of items from the queue into the provided buffer. At least one record is + * drained from the queue, regardless of the value of bytes specified. + * + * @param buffer The buffer to drain queue items into. + * @param bytesToDrain The amount of bytes to drain from the queue + * @param timeout The maximum time allowed to drain the queue + * @param unit The time unit of the timeout. + * + * @return The number of items drained from the queue. + * @throws InterruptedException + */ public int drain(Collection> buffer, int bytesToDrain, long timeout, TimeUnit unit) throws InterruptedException { diff --git a/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java b/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java index d113c2691b1c..72dc5d33cc9f 100644 --- a/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java @@ -199,6 +199,28 @@ public void test_drain_queueWithMultipleItems_succeeds() throws InterruptedExcep Assert.assertEquals(byteCapacity - item3.length, queue.remainingCapacity()); } + @Test + public void test_drain_queueWithFirstItemSizeGreaterThanLimit_succeeds() throws InterruptedException + { + + long byteCapacity = 15L; + byte[] item1 = "item1".getBytes(StandardCharsets.UTF_8); + byte[] item2 = "item2".getBytes(StandardCharsets.UTF_8); + byte[] item3 = "item3".getBytes(StandardCharsets.UTF_8); + Collection> items = buildItemContainers( + ImmutableList.of(item1, item2, item3) + ); + MemoryBoundLinkedBlockingQueue queue = setupQueue(byteCapacity, items, new NotAllDrainedQueue()); + Collection> buffer = new ArrayList<>(); + + int numAdded = queue.drain(buffer, item1.length - 1, 1, TimeUnit.MINUTES); + + Assert.assertTrue(numAdded == 1 && numAdded == buffer.size()); + Assert.assertEquals(2, queue.size()); + Assert.assertEquals(item2.length + item3.length, queue.byteSize()); + Assert.assertEquals(byteCapacity - (item2.length + item3.length), queue.remainingCapacity()); + } + private static MemoryBoundLinkedBlockingQueue setupQueue( long byteCapacity, Collection> items From 1a6d83a2ed8ec510c5533525241f41331c05b950 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Fri, 15 Dec 2023 10:53:27 -0800 Subject: [PATCH 19/24] * make blocking queue actually block for time specified --- .../MemoryBoundLinkedBlockingQueue.java | 15 ++++++++++ .../MemoryBoundLinkedBlockingQueueTest.java | 30 +++++++++++++++++++ 2 files changed, 45 insertions(+) diff --git a/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java b/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java index d51d8ac9f104..91a763facd64 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java @@ -26,6 +26,8 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Stream; /** @@ -37,6 +39,8 @@ public class MemoryBoundLinkedBlockingQueue private final long memoryBound; private final AtomicLong currentMemory; private final LinkedBlockingQueue> queue; + private final ReentrantLock putLock = new ReentrantLock(); + private final Condition notFull = putLock.newCondition(); public MemoryBoundLinkedBlockingQueue(long memoryBound) { @@ -69,7 +73,15 @@ public boolean offer(ObjectContainer item, long timeout, TimeUnit unit) throw { final long itemLength = item.getSize(); + long nanos = unit.toNanos(timeout); + final ReentrantLock putLock = this.putLock; + putLock.lockInterruptibly(); try { + while (currentMemory.get() + itemLength > memoryBound) { + if (nanos <= 0L) + return false; + nanos = notFull.awaitNanos(nanos); + } if (currentMemory.addAndGet(itemLength) <= memoryBound) { if (queue.offer(item, timeout, unit)) { return true; @@ -80,6 +92,9 @@ public boolean offer(ObjectContainer item, long timeout, TimeUnit unit) throw currentMemory.addAndGet(-itemLength); throw e; } + finally { + putLock.unlock(); + } currentMemory.addAndGet(-itemLength); return false; } diff --git a/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java b/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java index 72dc5d33cc9f..894948f731ae 100644 --- a/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java @@ -117,6 +117,36 @@ public void test_offerWithTimeLimit_interruptedExceptinThrown_throws() Assert.assertEquals(byteCapacity, queue.remainingCapacity()); } + @Test + public void test_offerWithTimeLimit_fullQueue_waitsTime() throws InterruptedException { + long timeoutMillis = 5000L; + long byteCapacity = 10L; + byte[] item1 = "item1".getBytes(StandardCharsets.UTF_8); + byte[] item2 = "item2".getBytes(StandardCharsets.UTF_8); + Collection> items = buildItemContainers( + ImmutableList.of(item1, item2) + ); + MemoryBoundLinkedBlockingQueue queue = setupQueue( + byteCapacity, + items, + new InterruptedExceptionThrowingQueue() + ); + byte[] item = "item".getBytes(StandardCharsets.UTF_8); + long start = System.currentTimeMillis(); + boolean succeeds = queue.offer( + new MemoryBoundLinkedBlockingQueue.ObjectContainer<>(item, item.length), + timeoutMillis, + TimeUnit.MILLISECONDS + ); + long end = System.currentTimeMillis(); + + Assert.assertFalse(succeeds); + Assert.assertTrue((end - start) > timeoutMillis); + Assert.assertEquals(2, queue.size()); + Assert.assertEquals(10L, queue.byteSize()); + Assert.assertEquals(0L, queue.remainingCapacity()); + } + @Test public void test_take_nonEmptyQueue_expected() throws InterruptedException { From bce053008c538103e8be345fdfaeb7c5ddffb993 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Fri, 15 Dec 2023 10:56:44 -0800 Subject: [PATCH 20/24] * fix checkstyle --- .../druid/java/util/common/MemoryBoundLinkedBlockingQueue.java | 3 ++- .../java/util/common/MemoryBoundLinkedBlockingQueueTest.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java b/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java index 91a763facd64..6b28fdb59305 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java @@ -78,8 +78,9 @@ public boolean offer(ObjectContainer item, long timeout, TimeUnit unit) throw putLock.lockInterruptibly(); try { while (currentMemory.get() + itemLength > memoryBound) { - if (nanos <= 0L) + if (nanos <= 0L) { return false; + } nanos = notFull.awaitNanos(nanos); } if (currentMemory.addAndGet(itemLength) <= memoryBound) { diff --git a/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java b/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java index 894948f731ae..08d26817a274 100644 --- a/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java @@ -118,7 +118,8 @@ public void test_offerWithTimeLimit_interruptedExceptinThrown_throws() } @Test - public void test_offerWithTimeLimit_fullQueue_waitsTime() throws InterruptedException { + public void test_offerWithTimeLimit_fullQueue_waitsTime() throws InterruptedException + { long timeoutMillis = 5000L; long byteCapacity = 10L; byte[] item1 = "item1".getBytes(StandardCharsets.UTF_8); From 047c266d3c689a973fa7d32ee963886830c957e3 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Fri, 15 Dec 2023 13:39:38 -0800 Subject: [PATCH 21/24] * fix failing test --- .../util/common/MemoryBoundLinkedBlockingQueueTest.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java b/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java index 08d26817a274..9db65f7dccbe 100644 --- a/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java @@ -120,7 +120,7 @@ public void test_offerWithTimeLimit_interruptedExceptinThrown_throws() @Test public void test_offerWithTimeLimit_fullQueue_waitsTime() throws InterruptedException { - long timeoutMillis = 5000L; + long timeoutMillis = 2000L; long byteCapacity = 10L; byte[] item1 = "item1".getBytes(StandardCharsets.UTF_8); byte[] item2 = "item2".getBytes(StandardCharsets.UTF_8); @@ -142,7 +142,11 @@ public void test_offerWithTimeLimit_fullQueue_waitsTime() throws InterruptedExce long end = System.currentTimeMillis(); Assert.assertFalse(succeeds); - Assert.assertTrue((end - start) > timeoutMillis); + Assert.assertTrue(StringUtils.format( + "offer only waited at most [%d] millis instead of expected [%d] millis", + (end - start), + timeoutMillis), + (end - start) > timeoutMillis); Assert.assertEquals(2, queue.size()); Assert.assertEquals(10L, queue.byteSize()); Assert.assertEquals(0L, queue.remainingCapacity()); From 765efc4d36a2df3ea70499fc33920ee7cc668c34 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Fri, 15 Dec 2023 14:33:27 -0800 Subject: [PATCH 22/24] * signal not empty --- .../MemoryBoundLinkedBlockingQueue.java | 52 +++++++++++++------ 1 file changed, 37 insertions(+), 15 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java b/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java index 6b28fdb59305..49105f3a8337 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueue.java @@ -105,6 +105,7 @@ public ObjectContainer take() throws InterruptedException { final ObjectContainer ret = queue.take(); currentMemory.addAndGet(-ret.getSize()); + signalNotFull(); return ret; } @@ -129,24 +130,33 @@ public int drain(Collection> buffer, int bytesToDrain throws InterruptedException { Preconditions.checkNotNull(buffer); - long deadline = System.nanoTime() + unit.toNanos(timeout); - int added = 0; - long bytesAdded = 0; - while (bytesAdded < bytesToDrain) { - ObjectContainer e = queue.poll(deadline - System.nanoTime(), TimeUnit.NANOSECONDS); - if (e == null) { - break; + boolean signalNotFull = false; + try { + long deadline = System.nanoTime() + unit.toNanos(timeout); + int added = 0; + long bytesAdded = 0; + while (bytesAdded < bytesToDrain) { + ObjectContainer e = queue.poll(deadline - System.nanoTime(), TimeUnit.NANOSECONDS); + if (e == null) { + break; + } + currentMemory.addAndGet(-e.getSize()); + signalNotFull = true; + buffer.add(e); + ++added; + bytesAdded += e.getSize(); + e = queue.peek(); + if (e != null && (bytesAdded + e.getSize()) > bytesToDrain) { + break; + } } - currentMemory.addAndGet(-e.getSize()); - buffer.add(e); - ++added; - bytesAdded += e.getSize(); - e = queue.peek(); - if (e != null && (bytesAdded + e.getSize()) > bytesToDrain) { - break; + return added; + } + finally { + if (signalNotFull) { + signalNotFull(); } } - return added; } public int size() @@ -164,6 +174,18 @@ public long remainingCapacity() return memoryBound - currentMemory.get(); } + private void signalNotFull() + { + final ReentrantLock putLock = this.putLock; + putLock.lock(); + try { + notFull.signal(); + } + finally { + putLock.unlock(); + } + } + public static class ObjectContainer { private final T data; From 30e0148a7ed815da072770720978b5d4c421fb9f Mon Sep 17 00:00:00 2001 From: zachjsh Date: Mon, 18 Dec 2023 12:30:47 -0800 Subject: [PATCH 23/24] * fix test --- .../common/MemoryBoundLinkedBlockingQueueTest.java | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java b/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java index 9db65f7dccbe..619821833117 100644 --- a/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java @@ -142,11 +142,14 @@ public void test_offerWithTimeLimit_fullQueue_waitsTime() throws InterruptedExce long end = System.currentTimeMillis(); Assert.assertFalse(succeeds); - Assert.assertTrue(StringUtils.format( - "offer only waited at most [%d] millis instead of expected [%d] millis", - (end - start), - timeoutMillis), - (end - start) > timeoutMillis); + Assert.assertTrue( + StringUtils.format( + "offer only waited at most [%d] nanos instead of expected [%d] nanos", + TimeUnit.MILLISECONDS.toNanos(end - start), + TimeUnit.MILLISECONDS.toNanos(timeoutMillis) + ), + TimeUnit.MILLISECONDS.toNanos(end - start) > TimeUnit.MILLISECONDS.toNanos(timeoutMillis) + ); Assert.assertEquals(2, queue.size()); Assert.assertEquals(10L, queue.byteSize()); Assert.assertEquals(0L, queue.remainingCapacity()); From 5e5a84e5314391eec0b3c9998fea20fd9ebae360 Mon Sep 17 00:00:00 2001 From: zachjsh Date: Mon, 18 Dec 2023 14:49:12 -0800 Subject: [PATCH 24/24] * fix test finally --- .../java/util/common/MemoryBoundLinkedBlockingQueueTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java b/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java index 619821833117..ec36d83f250c 100644 --- a/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/common/MemoryBoundLinkedBlockingQueueTest.java @@ -148,7 +148,7 @@ public void test_offerWithTimeLimit_fullQueue_waitsTime() throws InterruptedExce TimeUnit.MILLISECONDS.toNanos(end - start), TimeUnit.MILLISECONDS.toNanos(timeoutMillis) ), - TimeUnit.MILLISECONDS.toNanos(end - start) > TimeUnit.MILLISECONDS.toNanos(timeoutMillis) + TimeUnit.MILLISECONDS.toNanos(end - start) >= TimeUnit.MILLISECONDS.toNanos(timeoutMillis) ); Assert.assertEquals(2, queue.size()); Assert.assertEquals(10L, queue.byteSize());