From 3d9735cc3aedc353c0e48297e38aae88b6396ccb Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Tue, 14 Apr 2026 22:09:08 +0530 Subject: [PATCH 01/49] Add AcknowledgingRecordSupplier interface and AcknowledgeType enum for queue-semantics ingestion --- .../common/AcknowledgeType.java | 48 +++++++ .../common/AcknowledgingRecordSupplier.java | 126 ++++++++++++++++++ 2 files changed, 174 insertions(+) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgeType.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgingRecordSupplier.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgeType.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgeType.java new file mode 100644 index 000000000000..2edd2cdaef5f --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgeType.java @@ -0,0 +1,48 @@ +/* + * 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.indexing.seekablestream.common; + +/** + * Acknowledgement types for queue-semantics record suppliers where the broker + * manages delivery state (e.g., Kafka Share Groups). + * + * Maps directly to Kafka's {@code AcknowledgeType} enum values. + */ +public enum AcknowledgeType +{ + /** + * The record was consumed and processed successfully. + * The broker will mark the record as committed. + */ + ACCEPT, + + /** + * Release the record for redelivery to another consumer. + * Used when the consumer cannot process the record right now + * but another consumer might be able to. + */ + RELEASE, + + /** + * Reject the record permanently. The broker will not redeliver it. + * Used for poison-pill records that can never be processed. + */ + REJECT +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgingRecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgingRecordSupplier.java new file mode 100644 index 000000000000..4734537236fe --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgingRecordSupplier.java @@ -0,0 +1,126 @@ +/* + * 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.indexing.seekablestream.common; + +import org.apache.druid.data.input.impl.ByteEntity; + +import javax.validation.constraints.NotNull; +import java.io.Closeable; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * Record supplier for queue-semantics streams where the broker manages delivery + * state and consumers explicitly acknowledge records after processing. + * + * Unlike {@link RecordSupplier}, this interface does not support assign/seek + * operations because the broker controls partition assignment and offset tracking. + * Instead, consumers subscribe to topics and acknowledge individual records. + * + * Designed for Kafka Share Groups (KIP-932) but generic enough for other + * queue-semantics systems. + * + * @param partition identifier type + * @param sequence/offset number type + * @param record entity type + */ +public interface AcknowledgingRecordSupplier + extends Closeable +{ + /** + * Subscribe to the given topics. The broker manages partition assignment. + * + * @param topics topics to subscribe to + */ + void subscribe(Set topics); + + /** + * Unsubscribe from all topics. + */ + void unsubscribe(); + + /** + * Returns the current subscription. + * + * @return set of subscribed topic names + */ + Set subscription(); + + /** + * Poll for records from the subscribed topics. The broker delivers records + * with acquisition locks; records must be acknowledged before the lock expires. + * + * @param timeoutMs poll timeout in milliseconds + * @return list of records, never null + */ + @NotNull + List> poll(long timeoutMs); + + /** + * Acknowledge a single record with the default type (ACCEPT). + * + * @param partitionId partition of the record + * @param offset offset of the record + */ + void acknowledge(PartitionIdType partitionId, SequenceOffsetType offset); + + /** + * Acknowledge a single record with a specific type. + * + * @param partitionId partition of the record + * @param offset offset of the record + * @param type acknowledgement type (ACCEPT, RELEASE, REJECT) + */ + void acknowledge(PartitionIdType partitionId, SequenceOffsetType offset, AcknowledgeType type); + + /** + * Acknowledge a batch of records with a specific type. + * + * @param offsets map from partition to collection of offsets + * @param type acknowledgement type + */ + void acknowledge(Map> offsets, AcknowledgeType type); + + /** + * Commit all pending acknowledgements synchronously. + * Returns a map from partition to an optional exception if the commit + * failed for that partition. + * + * @return commit results per partition + */ + Map> commitSync(); + + /** + * Returns the set of partition IDs for the given stream/topic. + * + * @param stream topic name + * @return set of partition identifiers + */ + Set getPartitionIds(String stream); + + /** + * Close this supplier and release all resources. + */ + @Override + void close(); +} From 8351a824ca2597f1ad7aecf1016764d0485f97c5 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Tue, 14 Apr 2026 22:09:19 +0530 Subject: [PATCH 02/49] Add ShareGroupIndexTask and ShareGroupIndexTaskIOConfig for Kafka share group ingestion --- .../indexing/kafka/ShareGroupIndexTask.java | 154 ++++++++++++++++++ .../kafka/ShareGroupIndexTaskIOConfig.java | 109 +++++++++++++ 2 files changed, 263 insertions(+) create mode 100644 extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java create mode 100644 extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskIOConfig.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java new file mode 100644 index 000000000000..16634085dc03 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java @@ -0,0 +1,154 @@ +/* + * 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.indexing.kafka; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.task.AbstractTask; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.server.security.AuthorizationUtils; +import org.apache.druid.server.security.ResourceAction; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Indexing task that consumes from a Kafka topic using share group semantics + * (KIP-932). Unlike {@link KafkaIndexTask}, this task does not manage offsets + * client-side. The Kafka broker tracks delivery state; the task explicitly + * acknowledges records after segments are published. + * + * Phase 1: Single-threaded, no supervisor, no dedup cache. + * The task polls, parses, builds segments, publishes, ACKs, and commits. + */ +public class ShareGroupIndexTask extends AbstractTask +{ + private static final Logger log = new Logger(ShareGroupIndexTask.class); + private static final String TYPE = "index_kafka_share_group"; + + public static final Set INPUT_SOURCE_RESOURCES = Set.of( + AuthorizationUtils.createExternalResourceReadAction(KafkaIndexTaskModule.SCHEME) + ); + + private final DataSchema dataSchema; + private final KafkaIndexTaskTuningConfig tuningConfig; + private final ShareGroupIndexTaskIOConfig ioConfig; + private final ObjectMapper configMapper; + + private final AtomicBoolean stopRequested = new AtomicBoolean(false); + + @JsonCreator + public ShareGroupIndexTask( + @JsonProperty("id") @Nullable String id, + @JsonProperty("resource") @Nullable TaskResource taskResource, + @JsonProperty("dataSchema") DataSchema dataSchema, + @JsonProperty("tuningConfig") KafkaIndexTaskTuningConfig tuningConfig, + @JsonProperty("ioConfig") ShareGroupIndexTaskIOConfig ioConfig, + @JsonProperty("context") @Nullable Map context, + @JacksonInject ObjectMapper configMapper + ) + { + super( + getOrMakeId(id, TYPE, dataSchema.getDataSource()), + null, + taskResource, + dataSchema.getDataSource(), + context, + IngestionMode.APPEND + ); + this.dataSchema = dataSchema; + this.tuningConfig = tuningConfig; + this.ioConfig = ioConfig; + this.configMapper = configMapper; + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) + { + return true; + } + + @Nonnull + @JsonIgnore + @Override + public Set getInputSourceResources() + { + return INPUT_SOURCE_RESOURCES; + } + + @Override + public TaskStatus runTask(TaskToolbox toolbox) throws Exception + { + return new ShareGroupIndexTaskRunner(this, toolbox, configMapper).run(); + } + + @Override + public void stopGracefully(TaskConfig taskConfig) + { + log.info("Graceful stop requested for task[%s].", getId()); + stopRequested.set(true); + } + + @JsonProperty + public DataSchema getDataSchema() + { + return dataSchema; + } + + @JsonProperty + public KafkaIndexTaskTuningConfig getTuningConfig() + { + return tuningConfig; + } + + @JsonProperty("ioConfig") + public ShareGroupIndexTaskIOConfig getIOConfig() + { + return ioConfig; + } + + boolean isStopRequested() + { + return stopRequested.get(); + } + + ObjectMapper getConfigMapper() + { + return configMapper; + } +} diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskIOConfig.java new file mode 100644 index 000000000000..2b00a5529cbf --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskIOConfig.java @@ -0,0 +1,109 @@ +/* + * 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.indexing.kafka; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; +import org.apache.druid.segment.indexing.IOConfig; + +import javax.annotation.Nullable; +import java.util.Map; + +/** + * IO configuration for {@link ShareGroupIndexTask}. + * + * Unlike {@link KafkaIndexTaskIOConfig}, this config does not carry start/end + * offsets because the Kafka broker manages offset tracking for share groups. + * The task only needs the topic, group ID, consumer properties, and input format. + */ +public class ShareGroupIndexTaskIOConfig implements IOConfig +{ + private static final long DEFAULT_POLL_TIMEOUT_MILLIS = KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS; + + private final String topic; + private final String groupId; + private final Map consumerProperties; + private final InputFormat inputFormat; + private final long pollTimeout; + + @JsonCreator + public ShareGroupIndexTaskIOConfig( + @JsonProperty("topic") String topic, + @JsonProperty("groupId") String groupId, + @JsonProperty("consumerProperties") Map consumerProperties, + @JsonProperty("inputFormat") @Nullable InputFormat inputFormat, + @JsonProperty("pollTimeout") @Nullable Long pollTimeout + ) + { + this.topic = Preconditions.checkNotNull(topic, "topic"); + this.groupId = Preconditions.checkNotNull(groupId, "groupId"); + this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties"); + this.inputFormat = inputFormat; + this.pollTimeout = pollTimeout != null ? pollTimeout : DEFAULT_POLL_TIMEOUT_MILLIS; + } + + @JsonProperty + public String getTopic() + { + return topic; + } + + @JsonProperty + public String getGroupId() + { + return groupId; + } + + @JsonProperty + public Map getConsumerProperties() + { + return consumerProperties; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public InputFormat getInputFormat() + { + return inputFormat; + } + + @JsonProperty + public long getPollTimeout() + { + return pollTimeout; + } + + @Override + public String toString() + { + return "ShareGroupIndexTaskIOConfig{" + + "topic='" + topic + '\'' + + ", groupId='" + groupId + '\'' + + ", consumerProperties=" + consumerProperties + + ", inputFormat=" + inputFormat + + ", pollTimeout=" + pollTimeout + + '}'; + } +} From ac827e71f569a5ca94d5b412d5d8de052e777729 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Tue, 14 Apr 2026 22:10:43 +0530 Subject: [PATCH 03/49] Add KafkaShareGroupRecordSupplier wrapping KafkaShareConsumer with explicit acknowledgement --- .../kafka/KafkaShareGroupRecordSupplier.java | 231 ++++++++++++++++++ 1 file changed, 231 insertions(+) create mode 100644 extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java new file mode 100644 index 000000000000..ffe859d82fb2 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java @@ -0,0 +1,231 @@ +/* + * 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.indexing.kafka; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.kafka.KafkaRecordEntity; +import org.apache.druid.data.input.kafka.KafkaTopicPartition; +import org.apache.druid.indexing.seekablestream.common.AcknowledgeType; +import org.apache.druid.indexing.seekablestream.common.AcknowledgingRecordSupplier; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.KafkaShareConsumer; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; + +import javax.annotation.Nonnull; +import javax.validation.constraints.NotNull; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Wraps a {@link KafkaShareConsumer} to implement {@link AcknowledgingRecordSupplier}. + * + * The share consumer uses broker-managed offset tracking with explicit + * acknowledgement. Records are delivered with acquisition locks; unacknowledged + * records are redelivered after lock timeout. + * + * This supplier sets {@code group.id} to the share group name provided in config. + */ +public class KafkaShareGroupRecordSupplier + implements AcknowledgingRecordSupplier +{ + private static final Logger log = new Logger(KafkaShareGroupRecordSupplier.class); + + private final KafkaShareConsumer consumer; + private boolean closed; + + public KafkaShareGroupRecordSupplier( + Map consumerProperties, + ObjectMapper sortingMapper, + String groupId + ) + { + this(createShareConsumer(consumerProperties, sortingMapper, groupId)); + } + + @VisibleForTesting + public KafkaShareGroupRecordSupplier(KafkaShareConsumer consumer) + { + this.consumer = consumer; + } + + @Override + public void subscribe(Set topics) + { + consumer.subscribe(topics); + } + + @Override + public void unsubscribe() + { + consumer.unsubscribe(); + } + + @Override + public Set subscription() + { + return consumer.subscription(); + } + + @NotNull + @Override + public List> poll(long timeoutMs) + { + final List> polledRecords = + new ArrayList<>(); + for (ConsumerRecord record : consumer.poll(Duration.ofMillis(timeoutMs))) { + polledRecords.add(new OrderedPartitionableRecord<>( + record.topic(), + new KafkaTopicPartition(false, record.topic(), record.partition()), + record.offset(), + record.value() == null ? null : ImmutableList.of(new KafkaRecordEntity(record)), + record.timestamp() + )); + } + return polledRecords; + } + + @Override + public void acknowledge(KafkaTopicPartition partitionId, Long offset) + { + acknowledge(partitionId, offset, AcknowledgeType.ACCEPT); + } + + @Override + public void acknowledge(KafkaTopicPartition partitionId, Long offset, AcknowledgeType type) + { + final String topic = partitionId.topic().orElseThrow( + () -> new IllegalArgumentException("Cannot acknowledge record without topic") + ); + consumer.acknowledge( + new ConsumerRecord<>(topic, partitionId.partition(), offset, null, null), + toKafkaAcknowledgeType(type) + ); + } + + @Override + public void acknowledge( + Map> offsets, + AcknowledgeType type + ) + { + final org.apache.kafka.clients.consumer.AcknowledgeType kafkaType = toKafkaAcknowledgeType(type); + for (Map.Entry> entry : offsets.entrySet()) { + final KafkaTopicPartition partition = entry.getKey(); + final String topic = partition.topic().orElseThrow( + () -> new IllegalArgumentException("Cannot acknowledge record without topic") + ); + for (Long offset : entry.getValue()) { + consumer.acknowledge( + new ConsumerRecord<>(topic, partition.partition(), offset, null, null), + kafkaType + ); + } + } + } + + @Override + public Map> commitSync() + { + final Map> result = consumer.commitSync(); + final Map> mapped = new HashMap<>(); + for (Map.Entry> entry : result.entrySet()) { + final TopicIdPartition tip = entry.getKey(); + mapped.put( + new KafkaTopicPartition(false, tip.topic(), tip.partition()), + entry.getValue().map(e -> (Exception) e) + ); + } + return mapped; + } + + @Override + public Set getPartitionIds(String stream) + { + // Share consumer does not expose partitionsFor; use admin client if needed. + // For Phase 1, return empty set as the broker manages assignment. + return Collections.emptySet(); + } + + @Override + public void close() + { + if (closed) { + return; + } + closed = true; + try { + consumer.close(); + } + catch (Exception e) { + log.warn(e, "Exception closing KafkaShareConsumer"); + } + } + + @Nonnull + private static org.apache.kafka.clients.consumer.AcknowledgeType toKafkaAcknowledgeType(AcknowledgeType type) + { + switch (type) { + case ACCEPT: + return org.apache.kafka.clients.consumer.AcknowledgeType.ACCEPT; + case RELEASE: + return org.apache.kafka.clients.consumer.AcknowledgeType.RELEASE; + case REJECT: + return org.apache.kafka.clients.consumer.AcknowledgeType.REJECT; + default: + throw new IllegalArgumentException("Unknown acknowledge type: " + type); + } + } + + private static KafkaShareConsumer createShareConsumer( + Map consumerProperties, + ObjectMapper sortingMapper, + String groupId + ) + { + final Properties props = new Properties(); + KafkaRecordSupplier.addConsumerPropertiesFromConfig(props, sortingMapper, consumerProperties); + props.setProperty("group.id", groupId); + + final ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(KafkaShareGroupRecordSupplier.class.getClassLoader()); + return new KafkaShareConsumer<>(props, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + } + finally { + Thread.currentThread().setContextClassLoader(currCtxCl); + } + } +} From da77a47092ba9864731fa77d3820e5a461d64e35 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Tue, 14 Apr 2026 22:10:43 +0530 Subject: [PATCH 04/49] Add ShareGroupIndexTaskRunner with poll-parse-publish-ack ingestion loop --- .../kafka/ShareGroupIndexTaskRunner.java | 363 ++++++++++++++++++ 1 file changed, 363 insertions(+) create mode 100644 extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java new file mode 100644 index 000000000000..d7f65e586479 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java @@ -0,0 +1,363 @@ +/* + * 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.indexing.kafka; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.Committer; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.data.input.kafka.KafkaRecordEntity; +import org.apache.druid.data.input.kafka.KafkaTopicPartition; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.appenderator.ActionBasedPublishedSegmentRetriever; +import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.TaskLocks; +import org.apache.druid.indexing.common.actions.LockReleaseAction; +import org.apache.druid.indexing.common.actions.SegmentAllocateAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; +import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.indexing.input.InputRowSchemas; +import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.indexing.seekablestream.SeekableStreamAppenderatorConfig; +import org.apache.druid.indexing.seekablestream.common.AcknowledgeType; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.common.config.Configs; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.segment.SegmentSchemaMapping; +import org.apache.druid.segment.incremental.ParseExceptionHandler; +import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; +import org.apache.druid.segment.realtime.appenderator.Appenderator; +import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; +import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; +import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; +import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NumberedPartialShardSpec; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import com.google.common.base.Supplier; + +/** + * Ingestion loop for {@link ShareGroupIndexTask}. + * + * Phase 1: single-threaded. Polls from KafkaShareConsumer, parses rows, + * adds to appenderator, persists, publishes segments, then ACKs records + * and commits. Records are only acknowledged AFTER segment publication + * succeeds (INVARIANT 1). + * + * On failure, unacknowledged records are redelivered by the broker. + */ +public class ShareGroupIndexTaskRunner +{ + private static final Logger log = new Logger(ShareGroupIndexTaskRunner.class); + private static final String SEQUENCE_NAME = "share_group_seq_0"; + + private final ShareGroupIndexTask task; + private final TaskToolbox toolbox; + private final ObjectMapper configMapper; + + ShareGroupIndexTaskRunner( + ShareGroupIndexTask task, + TaskToolbox toolbox, + ObjectMapper configMapper + ) + { + this.task = task; + this.toolbox = toolbox; + this.configMapper = configMapper; + } + + public TaskStatus run() throws Exception + { + final DataSchema dataSchema = task.getDataSchema(); + final KafkaIndexTaskTuningConfig tuningConfig = task.getTuningConfig(); + final ShareGroupIndexTaskIOConfig ioConfig = task.getIOConfig(); + + final RowIngestionMeters rowIngestionMeters = + toolbox.getRowIngestionMetersFactory().createRowIngestionMeters(); + final ParseExceptionHandler parseExceptionHandler = new ParseExceptionHandler( + rowIngestionMeters, + tuningConfig.isLogParseExceptions(), + tuningConfig.getMaxParseExceptions(), + tuningConfig.getMaxSavedParseExceptions() + ); + final SegmentGenerationMetrics segmentGenerationMetrics = new SegmentGenerationMetrics(); + + final InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(dataSchema); + final InputFormat inputFormat = ioConfig.getInputFormat(); + if (inputFormat == null) { + throw new ISE("inputFormat must be specified in ioConfig"); + } + + final LockGranularity lockGranularity = Configs.valueOrDefault( + task.getContextValue(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK), + Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK + ) + ? LockGranularity.TIME_CHUNK + : LockGranularity.SEGMENT; + final TaskLockType lockType = TaskLocks.determineLockTypeForAppend(task.getContext()); + + // Create appenderator and driver + final Appenderator appenderator = toolbox.getAppenderatorsManager().createRealtimeAppenderatorForTask( + toolbox.getSegmentLoaderConfig(), + task.getId(), + dataSchema, + SeekableStreamAppenderatorConfig.fromTuningConfig( + tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), + toolbox.getProcessingConfig() + ), + toolbox.getConfig(), + segmentGenerationMetrics, + toolbox.getSegmentPusher(), + toolbox.getJsonMapper(), + toolbox.getIndexIO(), + toolbox.getIndexMerger(), + toolbox.getQueryRunnerFactoryConglomerate(), + toolbox.getSegmentAnnouncer(), + toolbox.getEmitter(), + toolbox.getQueryProcessingPool(), + toolbox.getJoinableFactory(), + toolbox.getCache(), + toolbox.getCacheConfig(), + toolbox.getCachePopulatorStats(), + toolbox.getPolicyEnforcer(), + rowIngestionMeters, + parseExceptionHandler, + toolbox.getCentralizedTableSchemaConfig(), + interval -> { + toolbox.getTaskActionClient().submit(new LockReleaseAction(interval)); + } + ); + + final StreamAppenderatorDriver driver = new StreamAppenderatorDriver( + appenderator, + new ActionBasedSegmentAllocator( + toolbox.getTaskActionClient(), + dataSchema, + (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction( + schema.getDataSource(), + row.getTimestamp(), + schema.getGranularitySpec().getQueryGranularity(), + schema.getGranularitySpec().getSegmentGranularity(), + sequenceName, + previousSegmentId, + skipSegmentLineageCheck, + NumberedPartialShardSpec.instance(), + lockGranularity, + lockType + ) + ), + toolbox.getSegmentHandoffNotifierFactory(), + new ActionBasedPublishedSegmentRetriever(toolbox.getTaskActionClient()), + toolbox.getDataSegmentKiller(), + toolbox.getJsonMapper(), + segmentGenerationMetrics + ); + + try (final KafkaShareGroupRecordSupplier recordSupplier = createRecordSupplier(ioConfig)) { + recordSupplier.subscribe(Collections.singleton(ioConfig.getTopic())); + + driver.startJob(segmentId -> true); + + final Supplier committerSupplier = () -> new Committer() + { + @Override + public Object getMetadata() + { + return ImmutableMap.of("type", "share_group"); + } + + @Override + public void run() + { + // no-op: share group does not need client-side offset persistence + } + }; + + log.info("Starting share group ingestion loop for topic[%s], group[%s].", + ioConfig.getTopic(), ioConfig.getGroupId()); + + long totalRowsIngested = 0; + + while (!task.isStopRequested()) { + + final List> records = + recordSupplier.poll(ioConfig.getPollTimeout()); + + if (records.isEmpty()) { + continue; + } + + // Track offsets of records in this batch for acknowledgement after publish + final Map> batchOffsets = new HashMap<>(); + + for (OrderedPartitionableRecord record : records) { + batchOffsets.computeIfAbsent(record.getPartitionId(), k -> new ArrayList<>()) + .add(record.getSequenceNumber()); + + if (record.getData() == null) { + continue; + } + + for (ByteEntity entity : record.getData()) { + try { + final InputRow row = parseRow(entity, inputFormat, inputRowSchema, toolbox); + + final AppenderatorDriverAddResult addResult = driver.add( + row, + SEQUENCE_NAME, + committerSupplier, + true, + false + ); + + if (addResult.isOk()) { + totalRowsIngested++; + rowIngestionMeters.incrementProcessed(); + } else { + throw new ISE("Could not allocate segment for row with timestamp[%s]", row.getTimestamp()); + } + } + catch (ParseException e) { + parseExceptionHandler.handle(e); + } + } + } + + // Persist before publish + driver.persist(committerSupplier.get()); + + // Publish all segments for this sequence + final TransactionalSegmentPublisher publisher = new TransactionalSegmentPublisher() + { + @Override + public SegmentPublishResult publishAnnotatedSegments( + @Nullable Set mustBeNullOrEmptyOverwriteSegments, + Set segmentsToPush, + @Nullable Object commitMetadata, + SegmentSchemaMapping segmentSchemaMapping + ) throws IOException + { + if (segmentsToPush.isEmpty()) { + return SegmentPublishResult.ok(segmentsToPush); + } + return toolbox.getTaskActionClient().submit( + SegmentTransactionalAppendAction.forSegments(segmentsToPush, segmentSchemaMapping) + ); + } + + @Override + public boolean supportsEmptyPublish() + { + return false; + } + }; + + final SegmentsAndCommitMetadata published = driver.publish( + publisher, + committerSupplier.get(), + Collections.singletonList(SEQUENCE_NAME) + ).get(); + + final int segmentCount = published.getSegments().size(); + if (segmentCount > 0) { + log.info("Published %d segments with %d total rows.", segmentCount, totalRowsIngested); + } + + // INVARIANT 1: ACK only after successful publish + for (Map.Entry> entry : batchOffsets.entrySet()) { + for (Long offset : entry.getValue()) { + recordSupplier.acknowledge(entry.getKey(), offset, AcknowledgeType.ACCEPT); + } + } + + // Commit acknowledgements to the broker + final Map> commitResult = recordSupplier.commitSync(); + for (Map.Entry> entry : commitResult.entrySet()) { + if (entry.getValue().isPresent()) { + log.warn( + entry.getValue().get(), + "Commit failed for partition[%s]. Records may be redelivered.", + entry.getKey() + ); + } + } + } + + // Final persist and close + driver.persist(committerSupplier.get()); + + log.info("Share group ingestion complete. Total rows ingested: %d", totalRowsIngested); + appenderator.close(); + } + finally { + driver.close(); + } + + return TaskStatus.success(task.getId()); + } + + private KafkaShareGroupRecordSupplier createRecordSupplier(ShareGroupIndexTaskIOConfig ioConfig) + { + final ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); + return new KafkaShareGroupRecordSupplier( + ioConfig.getConsumerProperties(), + configMapper, + ioConfig.getGroupId() + ); + } + finally { + Thread.currentThread().setContextClassLoader(currCtxCl); + } + } + + private static InputRow parseRow( + ByteEntity entity, + InputFormat inputFormat, + InputRowSchema inputRowSchema, + TaskToolbox toolbox + ) throws IOException + { + try (org.apache.druid.java.util.common.parsers.CloseableIterator iterator = + inputFormat.createReader(inputRowSchema, entity, toolbox.getIndexingTmpDir()).read()) { + return iterator.hasNext() ? iterator.next() : null; + } + } +} From 10fa462dd61c8a5de6dea53e6d3de582eb97bda2 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Tue, 14 Apr 2026 22:10:43 +0530 Subject: [PATCH 05/49] Register ShareGroupIndexTask in KafkaIndexTaskModule and bump Kafka client to 4.2.0 --- .../org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java index 6a364fc5f031..559f867f4be4 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java @@ -43,8 +43,10 @@ public List getJacksonModules() new SimpleModule(getClass().getSimpleName()) .registerSubtypes( new NamedType(KafkaIndexTask.class, "index_kafka"), + new NamedType(ShareGroupIndexTask.class, "index_kafka_share_group"), new NamedType(KafkaDataSourceMetadata.class, SCHEME), new NamedType(KafkaIndexTaskIOConfig.class, SCHEME), + new NamedType(ShareGroupIndexTaskIOConfig.class, "kafka_share_group"), // "KafkaTuningConfig" is not the ideal name, but is needed for backwards compatibility. // (Older versions of Druid didn't specify a type name and got this one by default.) new NamedType(KafkaIndexTaskTuningConfig.class, "KafkaTuningConfig"), From 24f1b193b37a7c43d92508ce2c41209e390638b1 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Tue, 14 Apr 2026 22:10:43 +0530 Subject: [PATCH 06/49] Add unit tests for ShareGroupIndexTaskIOConfig, KafkaShareGroupRecordSupplier, and ShareGroupIndexTask --- .../kafka-indexing-service/pom.xml | 5 + .../KafkaShareGroupRecordSupplierTest.java | 265 ++++++++++++++++++ .../ShareGroupIndexTaskIOConfigTest.java | 161 +++++++++++ .../kafka/ShareGroupIndexTaskTest.java | 161 +++++++++++ 4 files changed, 592 insertions(+) create mode 100644 extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java create mode 100644 extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskIOConfigTest.java create mode 100644 extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskTest.java diff --git a/extensions-core/kafka-indexing-service/pom.xml b/extensions-core/kafka-indexing-service/pom.xml index f86138c8742a..203abb5e31dd 100644 --- a/extensions-core/kafka-indexing-service/pom.xml +++ b/extensions-core/kafka-indexing-service/pom.xml @@ -189,6 +189,11 @@ test-jar test + + org.mockito + mockito-core + test + org.easymock easymock diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java new file mode 100644 index 000000000000..ffcf47c68c14 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java @@ -0,0 +1,265 @@ +/* + * 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.indexing.kafka; + +import org.apache.druid.data.input.kafka.KafkaRecordEntity; +import org.apache.druid.data.input.kafka.KafkaTopicPartition; +import org.apache.druid.indexing.seekablestream.common.AcknowledgeType; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaShareConsumer; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Unit tests for {@link KafkaShareGroupRecordSupplier} using a mocked + * KafkaShareConsumer. These tests verify the wrapping logic without + * requiring a real Kafka broker. + */ +public class KafkaShareGroupRecordSupplierTest +{ + private KafkaShareConsumer mockConsumer; + private KafkaShareGroupRecordSupplier supplier; + + @SuppressWarnings("unchecked") + @Before + public void setUp() + { + mockConsumer = mock(KafkaShareConsumer.class); + supplier = new KafkaShareGroupRecordSupplier(mockConsumer); + } + + @After + public void tearDown() + { + supplier.close(); + } + + @Test + public void testSubscribeAndSubscription() + { + final Set topics = Set.of("topic-a", "topic-b"); + when(mockConsumer.subscription()).thenReturn(topics); + + supplier.subscribe(topics); + verify(mockConsumer).subscribe(topics); + + Assert.assertEquals(topics, supplier.subscription()); + } + + @Test + public void testUnsubscribe() + { + supplier.unsubscribe(); + verify(mockConsumer).unsubscribe(); + } + + @Test + public void testPollWrapsRecords() + { + final ConsumerRecord record1 = new ConsumerRecord<>( + "test-topic", 0, 100L, "key1".getBytes(), "value1".getBytes() + ); + final ConsumerRecord record2 = new ConsumerRecord<>( + "test-topic", 1, 200L, "key2".getBytes(), "value2".getBytes() + ); + + final Map>> recordMap = new HashMap<>(); + recordMap.put(new TopicPartition("test-topic", 0), List.of(record1)); + recordMap.put(new TopicPartition("test-topic", 1), List.of(record2)); + final ConsumerRecords consumerRecords = new ConsumerRecords<>(recordMap); + + when(mockConsumer.poll(any(Duration.class))).thenReturn(consumerRecords); + + final List> result = + supplier.poll(1000); + + Assert.assertEquals(2, result.size()); + + // Verify first record + final OrderedPartitionableRecord polled1 = + result.stream().filter(r -> r.getSequenceNumber() == 100L).findFirst().orElse(null); + Assert.assertNotNull(polled1); + Assert.assertEquals("test-topic", polled1.getStream()); + Assert.assertEquals(0, polled1.getPartitionId().partition()); + Assert.assertNotNull(polled1.getData()); + Assert.assertEquals(1, polled1.getData().size()); + + // Verify second record + final OrderedPartitionableRecord polled2 = + result.stream().filter(r -> r.getSequenceNumber() == 200L).findFirst().orElse(null); + Assert.assertNotNull(polled2); + Assert.assertEquals(1, polled2.getPartitionId().partition()); + } + + @Test + public void testPollWithNullValue() + { + final ConsumerRecord record = new ConsumerRecord<>( + "test-topic", 0, 50L, "key".getBytes(), null + ); + final Map>> recordMap = new HashMap<>(); + recordMap.put(new TopicPartition("test-topic", 0), List.of(record)); + when(mockConsumer.poll(any(Duration.class))).thenReturn(new ConsumerRecords<>(recordMap)); + + final List> result = + supplier.poll(1000); + + Assert.assertEquals(1, result.size()); + Assert.assertNull(result.get(0).getData()); + } + + @Test + public void testPollReturnsEmptyOnTimeout() + { + when(mockConsumer.poll(any(Duration.class))).thenReturn(ConsumerRecords.empty()); + final List> result = + supplier.poll(100); + Assert.assertTrue(result.isEmpty()); + } + + @Test + public void testAcknowledgeDefaultAccept() + { + final KafkaTopicPartition partition = new KafkaTopicPartition(false, "test-topic", 0); + supplier.acknowledge(partition, 42L); + + final ArgumentCaptor recordCaptor = ArgumentCaptor.forClass(ConsumerRecord.class); + final ArgumentCaptor typeCaptor = + ArgumentCaptor.forClass(org.apache.kafka.clients.consumer.AcknowledgeType.class); + + verify(mockConsumer).acknowledge(recordCaptor.capture(), typeCaptor.capture()); + Assert.assertEquals(42L, recordCaptor.getValue().offset()); + Assert.assertEquals("test-topic", recordCaptor.getValue().topic()); + Assert.assertEquals(0, recordCaptor.getValue().partition()); + Assert.assertEquals(org.apache.kafka.clients.consumer.AcknowledgeType.ACCEPT, typeCaptor.getValue()); + } + + @Test + public void testAcknowledgeWithRelease() + { + final KafkaTopicPartition partition = new KafkaTopicPartition(false, "test-topic", 0); + supplier.acknowledge(partition, 10L, AcknowledgeType.RELEASE); + + final ArgumentCaptor typeCaptor = + ArgumentCaptor.forClass(org.apache.kafka.clients.consumer.AcknowledgeType.class); + verify(mockConsumer).acknowledge(any(ConsumerRecord.class), typeCaptor.capture()); + Assert.assertEquals(org.apache.kafka.clients.consumer.AcknowledgeType.RELEASE, typeCaptor.getValue()); + } + + @Test + public void testAcknowledgeWithReject() + { + final KafkaTopicPartition partition = new KafkaTopicPartition(false, "test-topic", 0); + supplier.acknowledge(partition, 10L, AcknowledgeType.REJECT); + + final ArgumentCaptor typeCaptor = + ArgumentCaptor.forClass(org.apache.kafka.clients.consumer.AcknowledgeType.class); + verify(mockConsumer).acknowledge(any(ConsumerRecord.class), typeCaptor.capture()); + Assert.assertEquals(org.apache.kafka.clients.consumer.AcknowledgeType.REJECT, typeCaptor.getValue()); + } + + @Test + public void testAcknowledgeBatch() + { + final KafkaTopicPartition p0 = new KafkaTopicPartition(false, "test-topic", 0); + final KafkaTopicPartition p1 = new KafkaTopicPartition(false, "test-topic", 1); + + final Map> offsets = new HashMap<>(); + offsets.put(p0, Arrays.asList(1L, 2L, 3L)); + offsets.put(p1, Arrays.asList(10L, 11L)); + + supplier.acknowledge(offsets, AcknowledgeType.ACCEPT); + + // 3 + 2 = 5 individual acknowledge calls + verify(mockConsumer, Mockito.times(5)).acknowledge( + any(ConsumerRecord.class), + eq(org.apache.kafka.clients.consumer.AcknowledgeType.ACCEPT) + ); + } + + @Test + public void testCommitSync() + { + final TopicIdPartition tip = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("test-topic", 0)); + final Map> kafkaResult = new HashMap<>(); + kafkaResult.put(tip, Optional.empty()); + + when(mockConsumer.commitSync()).thenReturn(kafkaResult); + + final Map> result = supplier.commitSync(); + Assert.assertEquals(1, result.size()); + + final Map.Entry> entry = result.entrySet().iterator().next(); + Assert.assertEquals("test-topic", entry.getKey().topic()); + Assert.assertEquals(0, entry.getKey().partition()); + Assert.assertFalse(entry.getValue().isPresent()); + } + + @Test + public void testCommitSyncWithError() + { + final TopicIdPartition tip = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("test-topic", 0)); + final KafkaException error = new KafkaException("commit failed"); + final Map> kafkaResult = new HashMap<>(); + kafkaResult.put(tip, Optional.of(error)); + + when(mockConsumer.commitSync()).thenReturn(kafkaResult); + + final Map> result = supplier.commitSync(); + Assert.assertEquals(1, result.size()); + + final Optional maybeError = result.values().iterator().next(); + Assert.assertTrue(maybeError.isPresent()); + Assert.assertEquals("commit failed", maybeError.get().getMessage()); + } + + @Test + public void testCloseIsIdempotent() + { + supplier.close(); + supplier.close(); + verify(mockConsumer, Mockito.times(1)).close(); + } +} diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskIOConfigTest.java new file mode 100644 index 000000000000..352a2ee5aab9 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskIOConfigTest.java @@ -0,0 +1,161 @@ +/* + * 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.indexing.kafka; + +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Map; + +public class ShareGroupIndexTaskIOConfigTest +{ + private ObjectMapper mapper; + + @Before + public void setUp() + { + mapper = new DefaultObjectMapper(); + mapper.registerSubtypes( + new NamedType(ShareGroupIndexTaskIOConfig.class, "kafka_share_group") + ); + } + + @Test + public void testSerdeWithAllFields() throws IOException + { + final Map consumerProps = ImmutableMap.of( + "bootstrap.servers", "localhost:9092" + ); + final ShareGroupIndexTaskIOConfig config = new ShareGroupIndexTaskIOConfig( + "test-topic", + "my-share-group", + consumerProps, + null, + 5000L + ); + + final String json = mapper.writeValueAsString(config); + final ShareGroupIndexTaskIOConfig deserialized = mapper.readValue(json, ShareGroupIndexTaskIOConfig.class); + + Assert.assertEquals("test-topic", deserialized.getTopic()); + Assert.assertEquals("my-share-group", deserialized.getGroupId()); + Assert.assertEquals(consumerProps, deserialized.getConsumerProperties()); + Assert.assertNull(deserialized.getInputFormat()); + Assert.assertEquals(5000L, deserialized.getPollTimeout()); + } + + @Test + public void testSerdeWithDefaultPollTimeout() throws IOException + { + final Map consumerProps = ImmutableMap.of( + "bootstrap.servers", "localhost:9092" + ); + final ShareGroupIndexTaskIOConfig config = new ShareGroupIndexTaskIOConfig( + "test-topic", + "my-share-group", + consumerProps, + null, + null + ); + + final String json = mapper.writeValueAsString(config); + final ShareGroupIndexTaskIOConfig deserialized = mapper.readValue(json, ShareGroupIndexTaskIOConfig.class); + + Assert.assertEquals("test-topic", deserialized.getTopic()); + Assert.assertEquals("my-share-group", deserialized.getGroupId()); + // Default poll timeout from KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS + Assert.assertTrue(deserialized.getPollTimeout() > 0); + } + + @Test + public void testDeserializationFromJson() throws IOException + { + final String json = "{" + + "\"topic\": \"events\"," + + "\"groupId\": \"druid-share\"," + + "\"consumerProperties\": {\"bootstrap.servers\": \"broker:9092\"}," + + "\"pollTimeout\": 2000" + + "}"; + + final ShareGroupIndexTaskIOConfig config = mapper.readValue(json, ShareGroupIndexTaskIOConfig.class); + Assert.assertEquals("events", config.getTopic()); + Assert.assertEquals("druid-share", config.getGroupId()); + Assert.assertEquals("broker:9092", config.getConsumerProperties().get("bootstrap.servers")); + Assert.assertEquals(2000L, config.getPollTimeout()); + } + + @Test(expected = NullPointerException.class) + public void testTopicRequired() + { + new ShareGroupIndexTaskIOConfig( + null, + "my-share-group", + ImmutableMap.of("bootstrap.servers", "localhost:9092"), + null, + null + ); + } + + @Test(expected = NullPointerException.class) + public void testGroupIdRequired() + { + new ShareGroupIndexTaskIOConfig( + "test-topic", + null, + ImmutableMap.of("bootstrap.servers", "localhost:9092"), + null, + null + ); + } + + @Test(expected = NullPointerException.class) + public void testConsumerPropertiesRequired() + { + new ShareGroupIndexTaskIOConfig( + "test-topic", + "my-share-group", + null, + null, + null + ); + } + + @Test + public void testToString() + { + final ShareGroupIndexTaskIOConfig config = new ShareGroupIndexTaskIOConfig( + "test-topic", + "my-share-group", + ImmutableMap.of("bootstrap.servers", "localhost:9092"), + null, + null + ); + final String str = config.toString(); + Assert.assertTrue(str.contains("test-topic")); + Assert.assertTrue(str.contains("my-share-group")); + } +} diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskTest.java new file mode 100644 index 000000000000..ae22f3b17bc5 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskTest.java @@ -0,0 +1,161 @@ +/* + * 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.indexing.kafka; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.segment.indexing.DataSchema; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Map; + +/** + * Unit tests for {@link ShareGroupIndexTask} focusing on task type, + * serialization/deserialization, and basic properties. + */ +public class ShareGroupIndexTaskTest +{ + private ObjectMapper mapper; + + @Before + public void setUp() + { + mapper = new DefaultObjectMapper(); + mapper.registerSubtypes( + new NamedType(ShareGroupIndexTask.class, "index_kafka_share_group"), + new NamedType(ShareGroupIndexTaskIOConfig.class, "kafka_share_group"), + new NamedType(KafkaIndexTaskTuningConfig.class, "KafkaTuningConfig") + ); + // Inject the ObjectMapper itself since ShareGroupIndexTask uses @JacksonInject + mapper.setInjectableValues(new com.fasterxml.jackson.databind.InjectableValues.Std() + .addValue(ObjectMapper.class, mapper)); + } + + @Test + public void testTaskType() + { + final ShareGroupIndexTask task = createTask("task_1"); + Assert.assertEquals("index_kafka_share_group", task.getType()); + } + + @Test + public void testTaskIsAlwaysReady() throws Exception + { + final ShareGroupIndexTask task = createTask("task_2"); + Assert.assertTrue(task.isReady(null)); + } + + @Test + public void testTaskDataSource() + { + final ShareGroupIndexTask task = createTask("task_3"); + Assert.assertEquals("test_datasource", task.getDataSource()); + } + + @Test + public void testTaskIdGeneration() + { + final ShareGroupIndexTask task = createTask(null); + Assert.assertNotNull(task.getId()); + Assert.assertTrue(task.getId().contains("index_kafka_share_group")); + } + + @Test + public void testGracefulStop() + { + final ShareGroupIndexTask task = createTask("task_4"); + Assert.assertFalse(task.isStopRequested()); + task.stopGracefully(null); + Assert.assertTrue(task.isStopRequested()); + } + + @Test + public void testIOConfigAccessor() + { + final ShareGroupIndexTask task = createTask("task_5"); + final ShareGroupIndexTaskIOConfig ioConfig = task.getIOConfig(); + Assert.assertEquals("test-topic", ioConfig.getTopic()); + Assert.assertEquals("test-share-group", ioConfig.getGroupId()); + } + + @Test + public void testSerdeRoundTrip() throws IOException + { + final ShareGroupIndexTask task = createTask("task_serde"); + final String json = mapper.writeValueAsString(task); + + // Verify the type field is present + Assert.assertTrue(json.contains("index_kafka_share_group")); + + final ShareGroupIndexTask deserialized = mapper.readValue(json, ShareGroupIndexTask.class); + Assert.assertEquals(task.getId(), deserialized.getId()); + Assert.assertEquals(task.getDataSource(), deserialized.getDataSource()); + Assert.assertEquals(task.getType(), deserialized.getType()); + Assert.assertEquals(task.getIOConfig().getTopic(), deserialized.getIOConfig().getTopic()); + Assert.assertEquals(task.getIOConfig().getGroupId(), deserialized.getIOConfig().getGroupId()); + } + + @Test + public void testInputSourceResources() + { + final ShareGroupIndexTask task = createTask("task_6"); + Assert.assertNotNull(task.getInputSourceResources()); + Assert.assertFalse(task.getInputSourceResources().isEmpty()); + } + + private ShareGroupIndexTask createTask(String id) + { + final DataSchema dataSchema = DataSchema.builder() + .withDataSource("test_datasource") + .build(); + + final Map consumerProps = ImmutableMap.of( + "bootstrap.servers", "localhost:9092" + ); + + final ShareGroupIndexTaskIOConfig ioConfig = new ShareGroupIndexTaskIOConfig( + "test-topic", + "test-share-group", + consumerProps, + null, + null + ); + + final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig( + null, null, null, null, null, null, null, null, null, null, + null, null, null, null, null, null, null, null, null, null + ); + + return new ShareGroupIndexTask( + id, + null, + dataSchema, + tuningConfig, + ioConfig, + null, + mapper + ); + } +} From 6b4a40e33eb9df53425b048b105f4d115a5effa7 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Tue, 14 Apr 2026 22:10:43 +0530 Subject: [PATCH 07/49] Add E2E test for share group ingestion with embedded Druid cluster and Kafka --- .../EmbeddedShareGroupIngestionTest.java | 191 ++++++++++++++++++ 1 file changed, 191 insertions(+) create mode 100644 extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedShareGroupIngestionTest.java diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedShareGroupIngestionTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedShareGroupIngestionTest.java new file mode 100644 index 000000000000..75a411348f4d --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedShareGroupIngestionTest.java @@ -0,0 +1,191 @@ +/* + * 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.indexing.kafka.simulate; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.KafkaIndexTaskTuningConfig; +import org.apache.druid.indexing.kafka.ShareGroupIndexTask; +import org.apache.druid.indexing.kafka.ShareGroupIndexTaskIOConfig; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedIndexer; +import org.apache.druid.testing.embedded.EmbeddedOverlord; +import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.joda.time.DateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; + +/** + * End-to-end integration test for Kafka share group ingestion. + * + * Demonstrates the full workflow: + *
    + *
  1. Start an embedded Druid cluster with Kafka extension
  2. + *
  3. Create a Kafka topic and produce CSV records
  4. + *
  5. Submit a {@link ShareGroupIndexTask} via the Overlord API
  6. + *
  7. Wait for records to be ingested
  8. + *
  9. Verify data via SQL query
  10. + *
+ * + * This test requires a Kafka 4.0+ broker with share group support. + * The KafkaResource Testcontainer uses apache/kafka:4.1.1 by default. + */ +public class EmbeddedShareGroupIngestionTest extends EmbeddedClusterTestBase +{ + private static final String COL_TIMESTAMP = "__time"; + private static final String COL_ITEM = "item"; + private static final String COL_VALUE = "value"; + + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedIndexer indexer = new EmbeddedIndexer(); + private final EmbeddedHistorical historical = new EmbeddedHistorical(); + private final EmbeddedBroker broker = new EmbeddedBroker(); + + private KafkaResource kafkaServer; + + @Override + public EmbeddedDruidCluster createCluster() + { + kafkaServer = new KafkaResource(); + + final EmbeddedDruidCluster cluster = EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper(); + indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s"); + + cluster.addExtension(KafkaIndexTaskModule.class) + .addResource(kafkaServer) + .useLatchableEmitter() + .addServer(coordinator) + .addServer(overlord) + .addServer(indexer) + .addServer(historical) + .addServer(broker); + + return cluster; + } + + /** + * Showcase test: Submit a ShareGroupIndexTask, ingest data from Kafka + * using share group semantics, and verify via SQL. + */ + @Test + public void test_shareGroupIngestion_basicEndToEnd() + { + final String topic = dataSource + "_topic"; + kafkaServer.createTopicWithPartitions(topic, 2); + + final int numRecords = 10; + kafkaServer.produceRecordsToTopic( + generateRecords(topic, numRecords, DateTimes.of("2025-06-01")) + ); + + // Build the task spec + final DataSchema dataSchema = DataSchema.builder() + .withDataSource(dataSource) + .withTimestamp(new TimestampSpec(COL_TIMESTAMP, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .withGranularity(new UniformGranularitySpec( + org.apache.druid.java.util.common.granularity.Granularities.DAY, + org.apache.druid.java.util.common.granularity.Granularities.NONE, + null + )) + .build(); + + final ShareGroupIndexTaskIOConfig ioConfig = new ShareGroupIndexTaskIOConfig( + topic, + "druid-share-group-test", + kafkaServer.consumerProperties(), + new CsvInputFormat(List.of(COL_TIMESTAMP, COL_ITEM, COL_VALUE), null, null, false, 0, false), + null + ); + + final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig( + null, null, null, null, null, null, null, null, null, null, + null, null, null, null, null, null, null, null, null, null + ); + + final ObjectMapper mapper = cluster.callApi().objectMapper(); + final ShareGroupIndexTask task = new ShareGroupIndexTask( + null, + null, + dataSchema, + tuningConfig, + ioConfig, + null, + mapper + ); + + // Submit the task + cluster.callApi().submitTask(task); + + // Wait for records to be processed + indexer.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("ingest/events/processed") + .hasDimension(DruidMetrics.DATASOURCE, dataSource), + agg -> agg.hasSumAtLeast(numRecords) + ); + + // Gracefully stop the task so it publishes and exits + // (In production, a supervisor would manage this lifecycle) + + // Verify ingested row count via SQL + Assertions.assertEquals( + String.valueOf(numRecords), + cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) + ); + } + + private List> generateRecords( + String topic, + int numRecords, + DateTime startTime + ) + { + final List> records = new ArrayList<>(); + for (int i = 0; i < numRecords; i++) { + final String csv = StringUtils.format( + "%s,item_%d,%d", + startTime.plusDays(i), + i, + ThreadLocalRandom.current().nextInt(1000) + ); + records.add( + new ProducerRecord<>(topic, i % 2, null, StringUtils.toUtf8(csv)) + ); + } + return records; + } +} From 3187c961d43b3fff1ce21030638ffcd09e10e6a0 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Tue, 14 Apr 2026 22:18:41 +0530 Subject: [PATCH 08/49] Add share group ingestion documentation with usage guide and examples --- docs/ingestion/kafka-share-group-ingestion.md | 155 ++++++++++++++++++ 1 file changed, 155 insertions(+) create mode 100644 docs/ingestion/kafka-share-group-ingestion.md diff --git a/docs/ingestion/kafka-share-group-ingestion.md b/docs/ingestion/kafka-share-group-ingestion.md new file mode 100644 index 000000000000..f1c3ddda13c6 --- /dev/null +++ b/docs/ingestion/kafka-share-group-ingestion.md @@ -0,0 +1,155 @@ +--- +id: kafka-share-group-ingestion +title: "Kafka share group ingestion" +sidebar_label: "Kafka share group ingestion" +description: "Queue-semantics ingestion from Apache Kafka using share groups (KIP-932). Scale consumers beyond partition count with at-least-once delivery." +--- + + + +:::info +Requires Apache Kafka 4.0 or higher. Share groups (KIP-932) must be enabled on the broker. +::: + +## Overview + +Standard Kafka consumer groups bind each partition to exactly one consumer. This creates a hard scaling ceiling: you cannot have more consumers than partitions. Rebalancing when consumers join or leave pauses all consumers in the group. A single slow message blocks all subsequent messages in its partition. + +Kafka share groups (KIP-932) eliminate these constraints. The broker manages record delivery across consumers with per-record acquisition locks and explicit acknowledgement. Multiple consumers can read from the same partition concurrently. There is no rebalancing pause. Slow records do not block other records. + +Druid's share group ingestion uses `ShareGroupIndexTask` to consume from Kafka share groups and publish segments with at-least-once delivery guarantees. Records are acknowledged only after segments are atomically registered in the metadata store. + +## When to use share group ingestion + +| Scenario | Consumer group | Share group | +|----------|---------------|-------------| +| Workers needed exceed partition count | Idle workers | All workers active | +| Elastic scaling (auto-scale events) | Rebalancing pause (30-60s) | Zero pause | +| Per-message processing time varies | Head-of-line blocking | Independent processing | +| Ordered processing required per partition | Yes | No (delivery order not guaranteed) | + +Use share group ingestion when throughput and elastic scaling matter more than strict per-partition ordering. + +## Task spec + +Submit a `ShareGroupIndexTask` to the Overlord. Unlike standard Kafka ingestion, there are no start/end offsets -- the broker manages offset tracking. + +```json +{ + "type": "index_kafka_share_group", + "dataSchema": { + "dataSource": "my_datasource", + "timestampSpec": { + "column": "__time", + "format": "auto" + }, + "dimensionsSpec": { + "useSchemaDiscovery": true + }, + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "NONE" + } + }, + "ioConfig": { + "topic": "my_topic", + "groupId": "druid-share-group", + "consumerProperties": { + "bootstrap.servers": "kafka-broker:9092" + }, + "inputFormat": { + "type": "json" + }, + "pollTimeout": 2000 + }, + "tuningConfig": { + "type": "KafkaTuningConfig", + "maxRowsPerSegment": 5000000 + } +} +``` + +## IO configuration + +| Property | Type | Required | Default | Description | +|----------|------|----------|---------|-------------| +| `topic` | String | Yes | -- | Kafka topic to consume from. | +| `groupId` | String | Yes | -- | Share group identifier. Multiple tasks with the same `groupId` share the workload. | +| `consumerProperties` | Map | Yes | -- | Kafka consumer properties. Must include `bootstrap.servers`. | +| `inputFormat` | Object | Yes | -- | Input format for parsing records (json, csv, avro, etc.). | +| `pollTimeout` | Long | No | 2000 | Poll timeout in milliseconds. | + +## How it works + +1. The task subscribes to the topic using a `KafkaShareConsumer` with the configured `groupId`. +2. The broker delivers batches of records with acquisition locks. +3. The task parses records, adds rows to an appenderator, and persists segments. +4. Segments are published atomically to the metadata store. +5. After successful publish, the task acknowledges all records in the batch with `ACCEPT`. +6. The task calls `commitSync()` to commit acknowledgements to the broker. +7. On task failure, unacknowledged records are redelivered by the broker to another consumer in the share group. + +## Safety invariants + +1. Records are acknowledged with `ACCEPT` only after the segment containing them is atomically registered in the metadata store. No data loss on task failure. +2. Every polled record reaches exactly one terminal state: `ACCEPT` (processed), `RELEASE` (redelivered), or task crash (broker redelivers after lock timeout). + +## Scaling + +Multiple tasks with the same `groupId` distribute the workload automatically. Unlike consumer groups, you can run more tasks than partitions: + +``` +Topic: 4 partitions +Tasks with same groupId: 20 +Result: All 20 tasks actively consuming (broker distributes records) +``` + +Adding or removing tasks does not trigger a rebalancing pause. New tasks begin consuming immediately. + +## Delivery semantics + +Share group ingestion provides **at-least-once** delivery. On task failure, records between the last committed acknowledgement and the failure point are redelivered. Duplicate records may be ingested across task restarts. A deduplication cache is planned for a future release. + +## Limitations (current release) + +- Single-threaded ingestion per task. Two-thread architecture with background lock renewal is planned. +- No supervisor integration. Tasks must be submitted manually via the Overlord API. +- No deduplication cache. Redelivered records after task failure may produce duplicates. +- Delivery order within a partition is not guaranteed. + +## Running tests + +Unit tests: + +```bash +mvn test -pl extensions-core/kafka-indexing-service \ + -Dtest="org.apache.druid.indexing.kafka.ShareGroupIndexTaskIOConfigTest,org.apache.druid.indexing.kafka.KafkaShareGroupRecordSupplierTest,org.apache.druid.indexing.kafka.ShareGroupIndexTaskTest" \ + -Dsurefire.failIfNoSpecifiedTests=false \ + -Pskip-static-checks -Dweb.console.skip=true -T1C +``` + +E2E test (requires Docker for Kafka Testcontainer): + +```bash +mvn test -pl extensions-core/kafka-indexing-service \ + -Dtest="org.apache.druid.indexing.kafka.simulate.EmbeddedShareGroupIngestionTest" \ + -Dsurefire.failIfNoSpecifiedTests=false \ + -Pskip-static-checks -Dweb.console.skip=true -T1C +``` From 6ffbfcf1ce375d5563ea5502e906374b8777954f Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Tue, 14 Apr 2026 22:21:19 +0530 Subject: [PATCH 09/49] Add ShareGroupKafkaResource with share group broker config for E2E tests --- docs/ingestion/kafka-share-group-ingestion.md | 4 +- .../EmbeddedShareGroupIngestionTest.java | 4 +- .../simulate/ShareGroupKafkaResource.java | 38 +++++++++++++++++++ 3 files changed, 43 insertions(+), 3 deletions(-) create mode 100644 extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/ShareGroupKafkaResource.java diff --git a/docs/ingestion/kafka-share-group-ingestion.md b/docs/ingestion/kafka-share-group-ingestion.md index f1c3ddda13c6..430662f525cb 100644 --- a/docs/ingestion/kafka-share-group-ingestion.md +++ b/docs/ingestion/kafka-share-group-ingestion.md @@ -145,7 +145,7 @@ mvn test -pl extensions-core/kafka-indexing-service \ -Pskip-static-checks -Dweb.console.skip=true -T1C ``` -E2E test (requires Docker for Kafka Testcontainer): +E2E test (requires Docker running -- Testcontainers starts a Kafka 4.1.1 broker with share groups enabled automatically): ```bash mvn test -pl extensions-core/kafka-indexing-service \ @@ -153,3 +153,5 @@ mvn test -pl extensions-core/kafka-indexing-service \ -Dsurefire.failIfNoSpecifiedTests=false \ -Pskip-static-checks -Dweb.console.skip=true -T1C ``` + +The E2E test uses `ShareGroupKafkaResource` which starts an `apache/kafka:4.1.1` container with `group.share.enable=true`. No manual Kafka setup is needed. diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedShareGroupIngestionTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedShareGroupIngestionTest.java index 75a411348f4d..ce7001ddfab4 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedShareGroupIngestionTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedShareGroupIngestionTest.java @@ -75,12 +75,12 @@ public class EmbeddedShareGroupIngestionTest extends EmbeddedClusterTestBase private final EmbeddedHistorical historical = new EmbeddedHistorical(); private final EmbeddedBroker broker = new EmbeddedBroker(); - private KafkaResource kafkaServer; + private ShareGroupKafkaResource kafkaServer; @Override public EmbeddedDruidCluster createCluster() { - kafkaServer = new KafkaResource(); + kafkaServer = new ShareGroupKafkaResource(); final EmbeddedDruidCluster cluster = EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper(); indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s"); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/ShareGroupKafkaResource.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/ShareGroupKafkaResource.java new file mode 100644 index 000000000000..fd59edcced5a --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/ShareGroupKafkaResource.java @@ -0,0 +1,38 @@ +/* + * 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.indexing.kafka.simulate; + +import org.testcontainers.kafka.KafkaContainer; + +/** + * A {@link KafkaResource} that starts a Kafka broker with share groups enabled. + * Share groups (KIP-932) require the broker config {@code group.share.enable=true}. + */ +public class ShareGroupKafkaResource extends KafkaResource +{ + @Override + protected KafkaContainer createContainer() + { + final KafkaContainer container = super.createContainer(); + container.withEnv("KAFKA_GROUP_SHARE_ENABLE", "true"); + container.withEnv("KAFKA_GROUP_SHARE_RECORD_LOCK_DURATION_MS", "30000"); + return container; + } +} From 57fbdfde658a78dcfb7620725f6e102205151469 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Tue, 14 Apr 2026 23:17:01 +0530 Subject: [PATCH 10/49] Fix IOConfig test: add type field to JSON deserialization test --- .../druid/indexing/kafka/ShareGroupIndexTaskIOConfigTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskIOConfigTest.java index 352a2ee5aab9..4f3ac94bc65a 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskIOConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskIOConfigTest.java @@ -95,6 +95,7 @@ public void testSerdeWithDefaultPollTimeout() throws IOException public void testDeserializationFromJson() throws IOException { final String json = "{" + + "\"type\": \"kafka_share_group\"," + "\"topic\": \"events\"," + "\"groupId\": \"druid-share\"," + "\"consumerProperties\": {\"bootstrap.servers\": \"broker:9092\"}," From cb8c96d60a656b8bc4635c1038f788f7da4bbded Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Tue, 14 Apr 2026 23:27:15 +0530 Subject: [PATCH 11/49] Fix unit tests: add TimestampSpec to DataSchema, use multi-topic KafkaTopicPartition --- .../indexing/kafka/KafkaShareGroupRecordSupplier.java | 4 ++-- .../kafka/KafkaShareGroupRecordSupplierTest.java | 10 +++++----- .../druid/indexing/kafka/ShareGroupIndexTaskTest.java | 4 ++++ 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java index ffe859d82fb2..c4dff0bf7137 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java @@ -108,7 +108,7 @@ public List record : consumer.poll(Duration.ofMillis(timeoutMs))) { polledRecords.add(new OrderedPartitionableRecord<>( record.topic(), - new KafkaTopicPartition(false, record.topic(), record.partition()), + new KafkaTopicPartition(true, record.topic(), record.partition()), record.offset(), record.value() == null ? null : ImmutableList.of(new KafkaRecordEntity(record)), record.timestamp() @@ -164,7 +164,7 @@ public Map> commitSync() for (Map.Entry> entry : result.entrySet()) { final TopicIdPartition tip = entry.getKey(); mapped.put( - new KafkaTopicPartition(false, tip.topic(), tip.partition()), + new KafkaTopicPartition(true, tip.topic(), tip.partition()), entry.getValue().map(e -> (Exception) e) ); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java index ffcf47c68c14..74a4a720901f 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java @@ -162,7 +162,7 @@ public void testPollReturnsEmptyOnTimeout() @Test public void testAcknowledgeDefaultAccept() { - final KafkaTopicPartition partition = new KafkaTopicPartition(false, "test-topic", 0); + final KafkaTopicPartition partition = new KafkaTopicPartition(true, "test-topic", 0); supplier.acknowledge(partition, 42L); final ArgumentCaptor recordCaptor = ArgumentCaptor.forClass(ConsumerRecord.class); @@ -179,7 +179,7 @@ public void testAcknowledgeDefaultAccept() @Test public void testAcknowledgeWithRelease() { - final KafkaTopicPartition partition = new KafkaTopicPartition(false, "test-topic", 0); + final KafkaTopicPartition partition = new KafkaTopicPartition(true, "test-topic", 0); supplier.acknowledge(partition, 10L, AcknowledgeType.RELEASE); final ArgumentCaptor typeCaptor = @@ -191,7 +191,7 @@ public void testAcknowledgeWithRelease() @Test public void testAcknowledgeWithReject() { - final KafkaTopicPartition partition = new KafkaTopicPartition(false, "test-topic", 0); + final KafkaTopicPartition partition = new KafkaTopicPartition(true, "test-topic", 0); supplier.acknowledge(partition, 10L, AcknowledgeType.REJECT); final ArgumentCaptor typeCaptor = @@ -203,8 +203,8 @@ public void testAcknowledgeWithReject() @Test public void testAcknowledgeBatch() { - final KafkaTopicPartition p0 = new KafkaTopicPartition(false, "test-topic", 0); - final KafkaTopicPartition p1 = new KafkaTopicPartition(false, "test-topic", 1); + final KafkaTopicPartition p0 = new KafkaTopicPartition(true, "test-topic", 0); + final KafkaTopicPartition p1 = new KafkaTopicPartition(true, "test-topic", 1); final Map> offsets = new HashMap<>(); offsets.put(p0, Arrays.asList(1L, 2L, 3L)); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskTest.java index ae22f3b17bc5..74746b3d7f2c 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskTest.java @@ -22,6 +22,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.segment.indexing.DataSchema; import org.junit.Assert; @@ -129,6 +131,8 @@ private ShareGroupIndexTask createTask(String id) { final DataSchema dataSchema = DataSchema.builder() .withDataSource("test_datasource") + .withTimestamp(new TimestampSpec("__time", null, null)) + .withDimensions(DimensionsSpec.EMPTY) .build(); final Map consumerProps = ImmutableMap.of( From 6593b579e0945821de1f081ab99f43cf4d00096d Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Wed, 15 Apr 2026 16:43:16 +0530 Subject: [PATCH 12/49] Add end-to-end demo runbook for share group ingestion with Druid UI --- docs/ingestion/kafka-share-group-ingestion.md | 103 ++++++++++++++++++ 1 file changed, 103 insertions(+) diff --git a/docs/ingestion/kafka-share-group-ingestion.md b/docs/ingestion/kafka-share-group-ingestion.md index 430662f525cb..9289403d2322 100644 --- a/docs/ingestion/kafka-share-group-ingestion.md +++ b/docs/ingestion/kafka-share-group-ingestion.md @@ -134,6 +134,109 @@ Share group ingestion provides **at-least-once** delivery. On task failure, reco - No deduplication cache. Redelivered records after task failure may produce duplicates. - Delivery order within a partition is not guaranteed. +## Demo: end-to-end validation with Druid UI + +### Prerequisites + +- Java 17 +- Kafka 4.2.0 (with share groups enabled) +- Druid 31.0.0 release (downloaded) + +### Step 1: Start Kafka with share groups + +```bash +cd kafka_2.13-4.2.0 + +KAFKA_CLUSTER_ID="$(bin/kafka-storage.sh random-uuid)" +bin/kafka-storage.sh format --standalone -t $KAFKA_CLUSTER_ID -c config/server.properties + +# Enable share groups +echo "group.share.enable=true" >> config/server.properties +echo "group.share.record.lock.duration.ms=30000" >> config/server.properties + +bin/kafka-server-start.sh config/server.properties +``` + +### Step 2: Create topic and produce messages + +```bash +cd kafka_2.13-4.2.0 + +bin/kafka-topics.sh --create --topic druid-share-test --partitions 4 --bootstrap-server localhost:9092 + +bin/kafka-console-producer.sh --topic druid-share-test --bootstrap-server localhost:9092 +``` + +Paste these JSON records: + +```json +{"__time":"2025-06-01T00:00:00.000Z","item":"widget_a","value":100,"category":"electronics"} +{"__time":"2025-06-01T01:00:00.000Z","item":"widget_b","value":250,"category":"clothing"} +{"__time":"2025-06-01T02:00:00.000Z","item":"widget_c","value":50,"category":"electronics"} +{"__time":"2025-06-01T03:00:00.000Z","item":"widget_d","value":175,"category":"food"} +{"__time":"2025-06-01T04:00:00.000Z","item":"widget_e","value":320,"category":"electronics"} +``` + +### Step 3: Build the extension and set up Druid + +```bash +# Build the kafka-indexing-service extension +cd /path/to/druid +JAVA_HOME=$(/usr/libexec/java_home -v 17) mvn package \ + -pl extensions-core/kafka-indexing-service -am \ + -Pskip-static-checks -Dmaven.test.skip=true -T1C -q + +# Download and extract Druid release +cd ~/Downloads +curl -O https://dlcdn.apache.org/druid/31.0.0/apache-druid-31.0.0-bin.tar.gz +tar -xzf apache-druid-31.0.0-bin.tar.gz +cd apache-druid-31.0.0 + +# Replace the kafka extension with our build +rm extensions/druid-kafka-indexing-service/*.jar +cp /path/to/druid/extensions-core/kafka-indexing-service/target/druid-kafka-indexing-service-*.jar \ + extensions/druid-kafka-indexing-service/ +cp ~/.m2/repository/org/apache/kafka/kafka-clients/4.2.0/kafka-clients-4.2.0.jar \ + extensions/druid-kafka-indexing-service/ + +# Start Druid +bin/start-druid +``` + +### Step 4: Submit task via Druid console + +Open `http://localhost:8888`, go to the **Ingestion** tab, click **Submit JSON task**, and paste: + +```json +{ + "type": "index_kafka_share_group", + "dataSchema": { + "dataSource": "share_group_demo", + "timestampSpec": {"column": "__time", "format": "auto"}, + "dimensionsSpec": {"useSchemaDiscovery": true}, + "granularitySpec": {"segmentGranularity": "DAY", "queryGranularity": "NONE"} + }, + "ioConfig": { + "type": "kafka_share_group", + "topic": "druid-share-test", + "groupId": "druid-demo-share-group", + "consumerProperties": {"bootstrap.servers": "localhost:9092"}, + "inputFormat": {"type": "json"}, + "pollTimeout": 2000 + }, + "tuningConfig": {"type": "KafkaTuningConfig"} +} +``` + +### Step 5: Query data + +Go to the **Query** tab and run: + +```sql +SELECT COUNT(*) AS total_rows FROM share_group_demo; +SELECT category, COUNT(*) AS cnt, SUM(value) AS total FROM share_group_demo GROUP BY category; +``` + ## Running tests Unit tests: From 1e13898e291c85de25cda92afa959fa29c1f4d14 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Wed, 15 Apr 2026 17:52:34 +0530 Subject: [PATCH 13/49] Implement PendingSegmentAllocatingTask for segment allocation compat --- .../apache/druid/indexing/kafka/ShareGroupIndexTask.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java index 16634085dc03..cf70172d8ab8 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java @@ -29,6 +29,7 @@ import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.AbstractTask; +import org.apache.druid.indexing.common.task.PendingSegmentAllocatingTask; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.indexing.DataSchema; @@ -50,7 +51,7 @@ * Phase 1: Single-threaded, no supervisor, no dedup cache. * The task polls, parses, builds segments, publishes, ACKs, and commits. */ -public class ShareGroupIndexTask extends AbstractTask +public class ShareGroupIndexTask extends AbstractTask implements PendingSegmentAllocatingTask { private static final Logger log = new Logger(ShareGroupIndexTask.class); private static final String TYPE = "index_kafka_share_group"; @@ -142,6 +143,12 @@ public ShareGroupIndexTaskIOConfig getIOConfig() return ioConfig; } + @Override + public String getTaskAllocatorId() + { + return getTaskResource().getAvailabilityGroup(); + } + boolean isStopRequested() { return stopRequested.get(); From f0aecb1d37f467ff31b5d9018ba39c66ab11d811 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Wed, 15 Apr 2026 18:07:14 +0530 Subject: [PATCH 14/49] Enable APPEND lock type for share group task to support concurrent segment allocation --- .../druid/indexing/kafka/ShareGroupIndexTask.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java index cf70172d8ab8..51d0258a0b17 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java @@ -31,6 +31,7 @@ import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.PendingSegmentAllocatingTask; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.security.AuthorizationUtils; @@ -38,6 +39,7 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.util.HashMap; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; @@ -83,7 +85,7 @@ public ShareGroupIndexTask( null, taskResource, dataSchema.getDataSource(), - context, + addConcurrentLocksContext(context), IngestionMode.APPEND ); this.dataSchema = dataSchema; @@ -158,4 +160,14 @@ ObjectMapper getConfigMapper() { return configMapper; } + + private static Map addConcurrentLocksContext(@Nullable Map context) + { + final Map merged = new HashMap<>(); + if (context != null) { + merged.putAll(context); + } + merged.putIfAbsent(Tasks.USE_CONCURRENT_LOCKS, true); + return merged; + } } From 5703d9820020a52f58449d9a0fea6574fb783740 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Wed, 15 Apr 2026 18:20:38 +0530 Subject: [PATCH 15/49] Set share.acknowledgement.mode=explicit for ShareConsumer to enable explicit ACK after publish --- .../druid/indexing/kafka/KafkaShareGroupRecordSupplier.java | 1 + 1 file changed, 1 insertion(+) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java index c4dff0bf7137..3b49d961d269 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java @@ -218,6 +218,7 @@ private static KafkaShareConsumer createShareConsumer( final Properties props = new Properties(); KafkaRecordSupplier.addConsumerPropertiesFromConfig(props, sortingMapper, consumerProperties); props.setProperty("group.id", groupId); + props.setProperty("share.acknowledgement.mode", "explicit"); final ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); try { From 928c0e5d48f87ce7dc74b36de1cf0f8d30afbecd Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Thu, 7 May 2026 18:02:37 +0530 Subject: [PATCH 16/49] Add RENEW ack type, wakeup() and acquisitionLockTimeoutMs() to AcknowledgingRecordSupplier --- .../common/AcknowledgeType.java | 10 +++++- .../common/AcknowledgingRecordSupplier.java | 33 +++++++++++++++++++ 2 files changed, 42 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgeType.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgeType.java index 2edd2cdaef5f..b152b0c5948e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgeType.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgeType.java @@ -44,5 +44,13 @@ public enum AcknowledgeType * Reject the record permanently. The broker will not redeliver it. * Used for poison-pill records that can never be processed. */ - REJECT + REJECT, + + /** + * Extend the acquisition lock on the record without releasing or accepting it. + * Used by background lock-renewal workers to keep records under lock while + * the foreground processing thread continues building segments. Maps to + * Kafka's {@code AcknowledgeType.RENEW}. + */ + RENEW } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgingRecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgingRecordSupplier.java index 4734537236fe..2a10cbd5b887 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgingRecordSupplier.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgingRecordSupplier.java @@ -118,6 +118,39 @@ public interface AcknowledgingRecordSupplier getPartitionIds(String stream); + /** + * Wakes up a blocked {@link #poll(long)} call so the ingestion loop can + * exit promptly on graceful stop. Implementations that wrap a + * {@link org.apache.kafka.clients.consumer.ShareConsumer} should call its + * {@code wakeup()} method here. The contract for the caller is that the + * next {@link #poll(long)} (or the in-flight one) will throw an + * implementation-specific wake-up exception + * (e.g. {@link org.apache.kafka.common.errors.WakeupException}). + * + *

Default implementation is a no-op for suppliers that do not support + * wake-ups; the ingestion loop will fall back to polling the + * {@code stopRequested} flag at the next poll boundary.

+ */ + default void wakeup() + { + } + + /** + * Returns the broker-effective acquisition lock timeout, if known. For + * Kafka share groups this is the {@code group.share.record.lock.duration.ms} + * the broker is using. The value is only meaningful after the first + * successful {@link #poll(long)} call (Kafka does not push it until the + * client has joined the share group). + * + *

Default implementation returns {@link Optional#empty()}.

+ * + * @return acquisition lock timeout in milliseconds, or empty if unknown + */ + default Optional acquisitionLockTimeoutMs() + { + return Optional.empty(); + } + /** * Close this supplier and release all resources. */ From 0a529997a085ce1feaf2112ef0802bf40bfc8bc0 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Thu, 7 May 2026 18:02:43 +0530 Subject: [PATCH 17/49] Promote StreamChunkReader to public and widen parse() with PECS for cross-package reuse --- .../indexing/seekablestream/StreamChunkReader.java | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkReader.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkReader.java index a0ac1f01ea5a..27d60feb5434 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkReader.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkReader.java @@ -42,8 +42,15 @@ /** * Abstraction for parsing stream data which internally uses {@link org.apache.druid.data.input.InputEntityReader}. + * + *

Public so it can be reused by stream-ingestion implementations outside the + * {@code seekablestream} package (notably the Kafka share-group runner). The + * class is intentionally narrow in scope: given a chunk of {@link ByteEntity} + * bytes from one stream record, parse it into zero-or-more {@link InputRow}s + * while honoring the configured {@link InputRowFilter}, + * {@link RowIngestionMeters}, and {@link ParseExceptionHandler}.

*/ -class StreamChunkReader +public class StreamChunkReader { private final SettableByteEntityReader byteEntityReader; private final InputRowFilter rowFilter; @@ -53,7 +60,7 @@ class StreamChunkReader /** * Either parser or inputFormat shouldn't be null. */ - StreamChunkReader( + public StreamChunkReader( InputFormat inputFormat, InputRowSchema inputRowSchema, TransformSpec transformSpec, @@ -89,7 +96,8 @@ class StreamChunkReader this.parseExceptionHandler = parseExceptionHandler; } - List parse(@Nullable List streamChunk, boolean isEndOfShard) throws IOException + public List parse(@Nullable List streamChunk, boolean isEndOfShard) + throws IOException { if (streamChunk == null || streamChunk.isEmpty()) { if (!isEndOfShard) { From 9cc61a715baea178462260a150b721bdf2ba74a2 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Thu, 7 May 2026 18:02:51 +0530 Subject: [PATCH 18/49] Add ShareGroupConsumerProperties.sanitize() to strip Kafka 4.2.0 share-unsupported configs --- .../kafka/ShareGroupConsumerProperties.java | 94 +++++++++++++++ .../ShareGroupConsumerPropertiesTest.java | 110 ++++++++++++++++++ 2 files changed, 204 insertions(+) create mode 100644 extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupConsumerProperties.java create mode 100644 extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupConsumerPropertiesTest.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupConsumerProperties.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupConsumerProperties.java new file mode 100644 index 000000000000..d621bb5b695d --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupConsumerProperties.java @@ -0,0 +1,94 @@ +/* + * 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.indexing.kafka; + +import com.google.common.collect.ImmutableSet; +import org.apache.druid.java.util.common.logger.Logger; + +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; + +/** + * Helpers for handling Kafka consumer properties supplied to the share-group + * task. Single Responsibility: filter / validate properties before they reach + * the {@link org.apache.kafka.clients.consumer.KafkaShareConsumer}. + * + *

Kafka 4.2.0 rejects a fixed set of properties when the consumer is a + * share consumer (see {@code ShareConsumerConfig.SHARE_GROUP_UNSUPPORTED_CONFIGS}). + * If users copy a regular consumer config wholesale, the broker raises + * {@code ConfigException} at construction time. We strip those keys with a + * warning so the task can still start; users get a clear log line telling them + * exactly which keys were dropped.

+ * + *

The list is kept in sync with Kafka 4.2.0; if the upstream list changes + * in a future release, this code should be updated.

+ */ +public final class ShareGroupConsumerProperties +{ + private static final Logger log = new Logger(ShareGroupConsumerProperties.class); + + /** + * Mirror of {@code ShareConsumerConfig.SHARE_GROUP_UNSUPPORTED_CONFIGS} from + * Kafka 4.2.0. Setting any of these on a {@code KafkaShareConsumer} causes + * a {@code ConfigException} at construction. We strip them defensively. + */ + static final Set UNSUPPORTED_CONFIGS = ImmutableSet.of( + "auto.offset.reset", + "enable.auto.commit", + "group.instance.id", + "isolation.level", + "partition.assignment.strategy", + "interceptor.classes", + "session.timeout.ms", + "heartbeat.interval.ms", + "group.protocol", + "group.remote.assignor" + ); + + private ShareGroupConsumerProperties() + { + } + + /** + * Returns a copy of {@code consumerProperties} with all share-group-unsupported + * keys removed. Each removed key is logged at WARN level once. Insertion + * order is preserved (uses {@link LinkedHashMap}). + * + * @param consumerProperties caller-supplied properties; never null + * @return new map with unsupported keys removed + */ + public static Map sanitize(Map consumerProperties) + { + final Map sanitized = new LinkedHashMap<>(consumerProperties.size()); + for (Map.Entry entry : consumerProperties.entrySet()) { + if (UNSUPPORTED_CONFIGS.contains(entry.getKey())) { + log.warn( + "Stripping unsupported consumer property [%s] for share-group consumer " + + "(see Kafka ShareConsumerConfig).", + entry.getKey() + ); + } else { + sanitized.put(entry.getKey(), entry.getValue()); + } + } + return sanitized; + } +} diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupConsumerPropertiesTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupConsumerPropertiesTest.java new file mode 100644 index 000000000000..f56f3e1a84b8 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupConsumerPropertiesTest.java @@ -0,0 +1,110 @@ +/* + * 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.indexing.kafka; + +import com.google.common.collect.ImmutableMap; +import org.junit.Assert; +import org.junit.Test; + +import java.util.LinkedHashMap; +import java.util.Map; + +/** + * Unit tests for {@link ShareGroupConsumerProperties}. Validates that the + * sanitize method strips exactly the share-group-unsupported keys without + * touching legitimate ones, and preserves insertion order. + */ +public class ShareGroupConsumerPropertiesTest +{ + @Test + public void testSanitizeKeepsSupportedProperties() + { + final Map input = ImmutableMap.of( + "bootstrap.servers", "broker:9092", + "max.poll.records", 100, + "ssl.protocol", "TLSv1.3" + ); + final Map sanitized = ShareGroupConsumerProperties.sanitize(input); + Assert.assertEquals(input, sanitized); + } + + @Test + public void testSanitizeStripsAutoOffsetReset() + { + final Map input = ImmutableMap.of( + "bootstrap.servers", "broker:9092", + "auto.offset.reset", "earliest" + ); + final Map sanitized = ShareGroupConsumerProperties.sanitize(input); + Assert.assertEquals(1, sanitized.size()); + Assert.assertEquals("broker:9092", sanitized.get("bootstrap.servers")); + Assert.assertFalse(sanitized.containsKey("auto.offset.reset")); + } + + @Test + public void testSanitizeStripsAllUnsupportedKeys() + { + final Map input = new LinkedHashMap<>(); + input.put("bootstrap.servers", "broker:9092"); + for (String key : ShareGroupConsumerProperties.UNSUPPORTED_CONFIGS) { + input.put(key, "some-value"); + } + final Map sanitized = ShareGroupConsumerProperties.sanitize(input); + Assert.assertEquals(1, sanitized.size()); + Assert.assertEquals("broker:9092", sanitized.get("bootstrap.servers")); + } + + @Test + public void testSanitizePreservesInsertionOrder() + { + final Map input = new LinkedHashMap<>(); + input.put("a.first", 1); + input.put("group.protocol", "consumer"); + input.put("b.second", 2); + input.put("c.third", 3); + + final Map sanitized = ShareGroupConsumerProperties.sanitize(input); + Assert.assertArrayEquals( + new String[]{"a.first", "b.second", "c.third"}, + sanitized.keySet().toArray(new String[0]) + ); + } + + @Test + public void testSanitizeOnEmptyMap() + { + Assert.assertTrue(ShareGroupConsumerProperties.sanitize(ImmutableMap.of()).isEmpty()); + } + + @Test + public void testUnsupportedConfigsContainsKnownKafka42Keys() + { + Assert.assertTrue(ShareGroupConsumerProperties.UNSUPPORTED_CONFIGS.contains("auto.offset.reset")); + Assert.assertTrue(ShareGroupConsumerProperties.UNSUPPORTED_CONFIGS.contains("enable.auto.commit")); + Assert.assertTrue(ShareGroupConsumerProperties.UNSUPPORTED_CONFIGS.contains("group.instance.id")); + Assert.assertTrue(ShareGroupConsumerProperties.UNSUPPORTED_CONFIGS.contains("isolation.level")); + Assert.assertTrue(ShareGroupConsumerProperties.UNSUPPORTED_CONFIGS.contains("partition.assignment.strategy")); + Assert.assertTrue(ShareGroupConsumerProperties.UNSUPPORTED_CONFIGS.contains("interceptor.classes")); + Assert.assertTrue(ShareGroupConsumerProperties.UNSUPPORTED_CONFIGS.contains("session.timeout.ms")); + Assert.assertTrue(ShareGroupConsumerProperties.UNSUPPORTED_CONFIGS.contains("heartbeat.interval.ms")); + Assert.assertTrue(ShareGroupConsumerProperties.UNSUPPORTED_CONFIGS.contains("group.protocol")); + Assert.assertTrue(ShareGroupConsumerProperties.UNSUPPORTED_CONFIGS.contains("group.remote.assignor")); + } +} From 7b9ea57a5786c401e6c70337c4ddadba23e12f95 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Thu, 7 May 2026 18:03:02 +0530 Subject: [PATCH 19/49] Use clean acknowledge overload, add RENEW/wakeup/lockTimeout/sanitize wiring in share supplier --- .../kafka/KafkaShareGroupRecordSupplier.java | 28 ++++--- .../KafkaShareGroupRecordSupplierTest.java | 81 ++++++++++++++----- 2 files changed, 79 insertions(+), 30 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java index 3b49d961d269..0e02c827caad 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java @@ -129,10 +129,7 @@ public void acknowledge(KafkaTopicPartition partitionId, Long offset, Acknowledg final String topic = partitionId.topic().orElseThrow( () -> new IllegalArgumentException("Cannot acknowledge record without topic") ); - consumer.acknowledge( - new ConsumerRecord<>(topic, partitionId.partition(), offset, null, null), - toKafkaAcknowledgeType(type) - ); + consumer.acknowledge(topic, partitionId.partition(), offset, toKafkaAcknowledgeType(type)); } @Override @@ -148,10 +145,7 @@ public void acknowledge( () -> new IllegalArgumentException("Cannot acknowledge record without topic") ); for (Long offset : entry.getValue()) { - consumer.acknowledge( - new ConsumerRecord<>(topic, partition.partition(), offset, null, null), - kafkaType - ); + consumer.acknowledge(topic, partition.partition(), offset, kafkaType); } } } @@ -179,6 +173,18 @@ public Set getPartitionIds(String stream) return Collections.emptySet(); } + @Override + public void wakeup() + { + consumer.wakeup(); + } + + @Override + public Optional acquisitionLockTimeoutMs() + { + return consumer.acquisitionLockTimeoutMs(); + } + @Override public void close() { @@ -204,6 +210,8 @@ private static org.apache.kafka.clients.consumer.AcknowledgeType toKafkaAcknowle return org.apache.kafka.clients.consumer.AcknowledgeType.RELEASE; case REJECT: return org.apache.kafka.clients.consumer.AcknowledgeType.REJECT; + case RENEW: + return org.apache.kafka.clients.consumer.AcknowledgeType.RENEW; default: throw new IllegalArgumentException("Unknown acknowledge type: " + type); } @@ -215,8 +223,10 @@ private static KafkaShareConsumer createShareConsumer( String groupId ) { + final Map sanitized = ShareGroupConsumerProperties.sanitize(consumerProperties); + final Properties props = new Properties(); - KafkaRecordSupplier.addConsumerPropertiesFromConfig(props, sortingMapper, consumerProperties); + KafkaRecordSupplier.addConsumerPropertiesFromConfig(props, sortingMapper, sanitized); props.setProperty("group.id", groupId); props.setProperty("share.acknowledgement.mode", "explicit"); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java index 74a4a720901f..fd4a19a7e23c 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java @@ -165,15 +165,12 @@ public void testAcknowledgeDefaultAccept() final KafkaTopicPartition partition = new KafkaTopicPartition(true, "test-topic", 0); supplier.acknowledge(partition, 42L); - final ArgumentCaptor recordCaptor = ArgumentCaptor.forClass(ConsumerRecord.class); - final ArgumentCaptor typeCaptor = - ArgumentCaptor.forClass(org.apache.kafka.clients.consumer.AcknowledgeType.class); - - verify(mockConsumer).acknowledge(recordCaptor.capture(), typeCaptor.capture()); - Assert.assertEquals(42L, recordCaptor.getValue().offset()); - Assert.assertEquals("test-topic", recordCaptor.getValue().topic()); - Assert.assertEquals(0, recordCaptor.getValue().partition()); - Assert.assertEquals(org.apache.kafka.clients.consumer.AcknowledgeType.ACCEPT, typeCaptor.getValue()); + Mockito.verify(mockConsumer).acknowledge( + Mockito.eq("test-topic"), + Mockito.eq(0), + Mockito.eq(42L), + Mockito.eq(org.apache.kafka.clients.consumer.AcknowledgeType.ACCEPT) + ); } @Test @@ -182,10 +179,12 @@ public void testAcknowledgeWithRelease() final KafkaTopicPartition partition = new KafkaTopicPartition(true, "test-topic", 0); supplier.acknowledge(partition, 10L, AcknowledgeType.RELEASE); - final ArgumentCaptor typeCaptor = - ArgumentCaptor.forClass(org.apache.kafka.clients.consumer.AcknowledgeType.class); - verify(mockConsumer).acknowledge(any(ConsumerRecord.class), typeCaptor.capture()); - Assert.assertEquals(org.apache.kafka.clients.consumer.AcknowledgeType.RELEASE, typeCaptor.getValue()); + Mockito.verify(mockConsumer).acknowledge( + Mockito.eq("test-topic"), + Mockito.eq(0), + Mockito.eq(10L), + Mockito.eq(org.apache.kafka.clients.consumer.AcknowledgeType.RELEASE) + ); } @Test @@ -194,10 +193,26 @@ public void testAcknowledgeWithReject() final KafkaTopicPartition partition = new KafkaTopicPartition(true, "test-topic", 0); supplier.acknowledge(partition, 10L, AcknowledgeType.REJECT); - final ArgumentCaptor typeCaptor = - ArgumentCaptor.forClass(org.apache.kafka.clients.consumer.AcknowledgeType.class); - verify(mockConsumer).acknowledge(any(ConsumerRecord.class), typeCaptor.capture()); - Assert.assertEquals(org.apache.kafka.clients.consumer.AcknowledgeType.REJECT, typeCaptor.getValue()); + Mockito.verify(mockConsumer).acknowledge( + Mockito.eq("test-topic"), + Mockito.eq(0), + Mockito.eq(10L), + Mockito.eq(org.apache.kafka.clients.consumer.AcknowledgeType.REJECT) + ); + } + + @Test + public void testAcknowledgeWithRenew() + { + final KafkaTopicPartition partition = new KafkaTopicPartition(true, "test-topic", 0); + supplier.acknowledge(partition, 99L, AcknowledgeType.RENEW); + + Mockito.verify(mockConsumer).acknowledge( + Mockito.eq("test-topic"), + Mockito.eq(0), + Mockito.eq(99L), + Mockito.eq(org.apache.kafka.clients.consumer.AcknowledgeType.RENEW) + ); } @Test @@ -212,10 +227,11 @@ public void testAcknowledgeBatch() supplier.acknowledge(offsets, AcknowledgeType.ACCEPT); - // 3 + 2 = 5 individual acknowledge calls - verify(mockConsumer, Mockito.times(5)).acknowledge( - any(ConsumerRecord.class), - eq(org.apache.kafka.clients.consumer.AcknowledgeType.ACCEPT) + Mockito.verify(mockConsumer, Mockito.times(5)).acknowledge( + Mockito.anyString(), + Mockito.anyInt(), + Mockito.anyLong(), + Mockito.eq(org.apache.kafka.clients.consumer.AcknowledgeType.ACCEPT) ); } @@ -262,4 +278,27 @@ public void testCloseIsIdempotent() supplier.close(); verify(mockConsumer, Mockito.times(1)).close(); } + + @Test + public void testWakeupForwardsToConsumer() + { + supplier.wakeup(); + Mockito.verify(mockConsumer).wakeup(); + } + + @Test + public void testAcquisitionLockTimeoutMsForwardsToConsumer() + { + Mockito.when(mockConsumer.acquisitionLockTimeoutMs()).thenReturn(Optional.of(45_000)); + final Optional lockMs = supplier.acquisitionLockTimeoutMs(); + Assert.assertTrue(lockMs.isPresent()); + Assert.assertEquals(Integer.valueOf(45_000), lockMs.get()); + } + + @Test + public void testAcquisitionLockTimeoutMsEmpty() + { + Mockito.when(mockConsumer.acquisitionLockTimeoutMs()).thenReturn(Optional.empty()); + Assert.assertTrue(supplier.acquisitionLockTimeoutMs().isEmpty()); + } } From 0f9bb3c686629faf05a95a101c506bc50db89b2c Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Thu, 7 May 2026 18:03:18 +0530 Subject: [PATCH 20/49] Fix multi-row data loss, add DIP factory, push/persist, try-finally, commit-failure metric, graceful-stop wakeup --- .../indexing/kafka/ShareGroupIndexTask.java | 15 +- .../kafka/ShareGroupIndexTaskRunner.java | 267 ++++++++++++++---- .../kafka/ShareGroupIndexTaskRunnerTest.java | 153 ++++++++++ 3 files changed, 385 insertions(+), 50 deletions(-) create mode 100644 extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunnerTest.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java index 51d0258a0b17..329e32617808 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java @@ -43,6 +43,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; /** * Indexing task that consumes from a Kafka topic using share group semantics @@ -68,6 +69,7 @@ public class ShareGroupIndexTask extends AbstractTask implements PendingSegmentA private final ObjectMapper configMapper; private final AtomicBoolean stopRequested = new AtomicBoolean(false); + private final AtomicReference activeRunner = new AtomicReference<>(); @JsonCreator public ShareGroupIndexTask( @@ -117,7 +119,14 @@ public Set getInputSourceResources() @Override public TaskStatus runTask(TaskToolbox toolbox) throws Exception { - return new ShareGroupIndexTaskRunner(this, toolbox, configMapper).run(); + final ShareGroupIndexTaskRunner runner = new ShareGroupIndexTaskRunner(this, toolbox, configMapper); + activeRunner.set(runner); + try { + return runner.run(); + } + finally { + activeRunner.set(null); + } } @Override @@ -125,6 +134,10 @@ public void stopGracefully(TaskConfig taskConfig) { log.info("Graceful stop requested for task[%s].", getId()); stopRequested.set(true); + final ShareGroupIndexTaskRunner runner = activeRunner.get(); + if (runner != null) { + runner.requestWakeup(); + } } @JsonProperty diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java index d7f65e586479..a517cef11b77 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java @@ -25,10 +25,10 @@ import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowSchema; -import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.kafka.KafkaRecordEntity; import org.apache.druid.data.input.kafka.KafkaTopicPartition; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexing.appenderator.ActionBasedPublishedSegmentRetriever; import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; import org.apache.druid.indexing.common.LockGranularity; @@ -38,16 +38,18 @@ import org.apache.druid.indexing.common.actions.LockReleaseAction; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; +import org.apache.druid.indexing.common.task.InputRowFilter; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.input.InputRowSchemas; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.seekablestream.SeekableStreamAppenderatorConfig; +import org.apache.druid.indexing.seekablestream.StreamChunkReader; import org.apache.druid.indexing.seekablestream.common.AcknowledgeType; +import org.apache.druid.indexing.seekablestream.common.AcknowledgingRecordSupplier; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.common.config.Configs; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; @@ -61,6 +63,8 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedPartialShardSpec; +import org.apache.kafka.common.errors.WakeupException; + import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; @@ -70,36 +74,103 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Supplier; /** * Ingestion loop for {@link ShareGroupIndexTask}. * - * Phase 1: single-threaded. Polls from KafkaShareConsumer, parses rows, - * adds to appenderator, persists, publishes segments, then ACKs records - * and commits. Records are only acknowledged AFTER segment publication - * succeeds (INVARIANT 1). + *

Phase 1 invariants: + *

    + *
  1. ACK after publish (data safety): records are acknowledged with + * {@code ACCEPT} only after the segment containing them is atomically + * registered in the metadata store via + * {@link SegmentTransactionalAppendAction}. Any failure between poll and + * publish leaves the records unacknowledged and the broker will + * redeliver them after the acquisition lock expires.
  2. + *
  3. Multi-row safe: records are parsed via + * {@link StreamChunkReader} which returns 0..N rows per record (handles + * JSON arrays, null/empty records, and ParseException routing). Every + * row produced from a record is added to the appenderator before the + * record is acknowledged.
  4. + *
  5. Resource safe: {@code Appenderator} and {@code RecordSupplier} + * are released on every exit path, including + * {@link org.apache.kafka.common.errors.WakeupException} during graceful + * stop and any in-loop ISE.
  6. + *
  7. Graceful stop: {@link ShareGroupIndexTask#stopGracefully} calls + * {@link #requestWakeup()} which forwards to + * {@link AcknowledgingRecordSupplier#wakeup()}, interrupting an + * in-flight {@code poll()} so the loop exits at the next iteration.
  8. + *
  9. Observability: per-partition commit failures are emitted as + * {@code ingest/shareGroup/commitFailures}; broker-effective acquisition + * lock duration is logged once after the first poll.
  10. + *
+ * + *

Single-threaded: all {@code Appenderator} interactions happen on + * the run thread. Background lock-renewal is a Phase 2 enhancement (see + * {@link AcknowledgeType#RENEW}).

* - * On failure, unacknowledged records are redelivered by the broker. + *

Testability (DIP): the supplier factory is constructor-injected + * so unit tests can plug in a mock {@link AcknowledgingRecordSupplier} without + * spinning up a Kafka broker.

*/ public class ShareGroupIndexTaskRunner { private static final Logger log = new Logger(ShareGroupIndexTaskRunner.class); private static final String SEQUENCE_NAME = "share_group_seq_0"; + /** + * Metric for per-partition acknowledgement commit failures during share-group + * ingestion. Emitted with the standard task dimensions (datasource, taskId, + * etc.) so operators can alert on sustained broker commit issues. A non-zero + * value implies the corresponding records will be redelivered by the broker. + */ + static final String METRIC_COMMIT_FAILURES = "ingest/shareGroup/commitFailures"; + private final ShareGroupIndexTask task; private final TaskToolbox toolbox; private final ObjectMapper configMapper; + private final Function> supplierFactory; + + /** + * Active supplier reference used by {@link #requestWakeup()} so the task's + * {@code stopGracefully} hook can interrupt a blocking poll. Set on entry + * to the run loop and cleared on exit. + */ + private final AtomicReference> + activeSupplier = new AtomicReference<>(); ShareGroupIndexTaskRunner( ShareGroupIndexTask task, TaskToolbox toolbox, ObjectMapper configMapper ) + { + this(task, toolbox, configMapper, null); + } + + /** + * DIP-friendly constructor: accepts a factory that builds the + * {@link AcknowledgingRecordSupplier} from the IO config. Tests can pass a + * factory returning a mock supplier; production passes {@code null} and the + * runner falls back to building a {@link KafkaShareGroupRecordSupplier}. + */ + @VisibleForTesting + ShareGroupIndexTaskRunner( + ShareGroupIndexTask task, + TaskToolbox toolbox, + ObjectMapper configMapper, + @Nullable Function> supplierFactory + ) { this.task = task; this.toolbox = toolbox; this.configMapper = configMapper; + this.supplierFactory = supplierFactory != null ? supplierFactory : this::createDefaultRecordSupplier; } public TaskStatus run() throws Exception @@ -124,6 +195,25 @@ public TaskStatus run() throws Exception throw new ISE("inputFormat must be specified in ioConfig"); } + // Single canonical multi-row parser, shared with the seekable-stream runner. + // Handles null/empty records, multi-row records (e.g. JSON arrays), parse + // exceptions (via ParseExceptionHandler), and processedBytes metrics. + // + // Raw type matches the SeekableStreamIndexTaskRunner idiom: works around + // OrderedPartitionableRecord.getData() returning List + // (rather than List); changing that signature would + // be a much larger refactor for marginal benefit. + @SuppressWarnings({"rawtypes", "unchecked"}) + final StreamChunkReader chunkReader = new StreamChunkReader( + inputFormat, + inputRowSchema, + dataSchema.getTransformSpec(), + toolbox.getIndexingTmpDir(), + InputRowFilter.allowAll(), + rowIngestionMeters, + parseExceptionHandler + ); + final LockGranularity lockGranularity = Configs.valueOrDefault( task.getContextValue(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK), Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK @@ -189,7 +279,10 @@ public TaskStatus run() throws Exception segmentGenerationMetrics ); - try (final KafkaShareGroupRecordSupplier recordSupplier = createRecordSupplier(ioConfig)) { + boolean appenderatorClosedNormally = false; + try (final AcknowledgingRecordSupplier recordSupplier = + supplierFactory.apply(ioConfig)) { + activeSupplier.set(recordSupplier); recordSupplier.subscribe(Collections.singleton(ioConfig.getTopic())); driver.startJob(segmentId -> true); @@ -213,11 +306,31 @@ public void run() ioConfig.getTopic(), ioConfig.getGroupId()); long totalRowsIngested = 0; + boolean lockTimeoutLogged = false; while (!task.isStopRequested()) { - final List> records = - recordSupplier.poll(ioConfig.getPollTimeout()); + final List> records; + try { + records = recordSupplier.poll(ioConfig.getPollTimeout()); + } + catch (WakeupException e) { + if (task.isStopRequested()) { + log.info("Wake-up received while stopGracefully is in progress; exiting ingestion loop."); + break; + } + throw e; + } + + if (!lockTimeoutLogged) { + final Optional effectiveLockMs = recordSupplier.acquisitionLockTimeoutMs(); + log.info( + "Effective broker acquisition lock timeout for share-group[%s]: %s ms", + ioConfig.getGroupId(), + effectiveLockMs.map(String::valueOf).orElse("") + ); + lockTimeoutLogged = true; + } if (records.isEmpty()) { continue; @@ -225,41 +338,62 @@ public void run() // Track offsets of records in this batch for acknowledgement after publish final Map> batchOffsets = new HashMap<>(); + boolean midBatchPersistNeeded = false; + boolean pushThresholdLogged = false; for (OrderedPartitionableRecord record : records) { batchOffsets.computeIfAbsent(record.getPartitionId(), k -> new ArrayList<>()) .add(record.getSequenceNumber()); - if (record.getData() == null) { - continue; - } - - for (ByteEntity entity : record.getData()) { - try { - final InputRow row = parseRow(entity, inputFormat, inputRowSchema, toolbox); + // Multi-row safe: chunkReader returns 0..N rows per record, increments + // processedBytes/thrownAway/unparseable internally, and routes parse + // exceptions through parseExceptionHandler (so maxParseExceptions/etc. + // are honored). Null record.data() is handled inside parse(). + final List rows = chunkReader.parse(record.getData(), false); + + for (InputRow row : rows) { + final AppenderatorDriverAddResult addResult = driver.add( + row, + SEQUENCE_NAME, + committerSupplier, + true, + false + ); - final AppenderatorDriverAddResult addResult = driver.add( - row, - SEQUENCE_NAME, - committerSupplier, - true, - false + if (!addResult.isOk()) { + // Failure to allocate segment puts data integrity at risk: bail out + // so the records remain unacknowledged and the broker redelivers. + throw new ISE( + "Could not allocate segment for row with timestamp[%s]", + row.getTimestamp() ); - - if (addResult.isOk()) { - totalRowsIngested++; - rowIngestionMeters.incrementProcessed(); - } else { - throw new ISE("Could not allocate segment for row with timestamp[%s]", row.getTimestamp()); - } } - catch (ParseException e) { - parseExceptionHandler.handle(e); + + totalRowsIngested++; + midBatchPersistNeeded |= addResult.isPersistRequired(); + + if (!pushThresholdLogged && addResult.isPushRequired( + tuningConfig.getPartitionsSpec().getMaxRowsPerSegment(), + tuningConfig.getPartitionsSpec() + .getMaxTotalRowsOr(DynamicPartitionsSpec.DEFAULT_MAX_TOTAL_ROWS) + )) { + pushThresholdLogged = true; + log.info( + "Segment row threshold reached during batch (current segment[%s], rows[%d]); " + + "will publish at end-of-batch boundary.", + addResult.getSegmentIdentifier(), + addResult.getNumRowsInSegment() + ); } } } - // Persist before publish + // Mid-batch persist if memory pressure dictates (max{Bytes,Rows}InMemory). + if (midBatchPersistNeeded) { + driver.persist(committerSupplier.get()); + } + + // End-of-batch persist before publish driver.persist(committerSupplier.get()); // Publish all segments for this sequence @@ -306,10 +440,14 @@ public boolean supportsEmptyPublish() } } - // Commit acknowledgements to the broker + // Commit acknowledgements to the broker. Failures here mean the broker + // will redeliver the affected records (at-least-once); we count and emit + // a metric so operators can alert on sustained commit issues. final Map> commitResult = recordSupplier.commitSync(); + long commitFailures = 0L; for (Map.Entry> entry : commitResult.entrySet()) { if (entry.getValue().isPresent()) { + commitFailures++; log.warn( entry.getValue().get(), "Commit failed for partition[%s]. Records may be redelivered.", @@ -317,22 +455,65 @@ public boolean supportsEmptyPublish() ); } } + if (commitFailures > 0) { + task.emitMetric(toolbox.getEmitter(), METRIC_COMMIT_FAILURES, commitFailures); + } } - // Final persist and close + // Final persist and graceful close driver.persist(committerSupplier.get()); log.info("Share group ingestion complete. Total rows ingested: %d", totalRowsIngested); appenderator.close(); + appenderatorClosedNormally = true; } finally { - driver.close(); + activeSupplier.set(null); + try { + driver.close(); + } + catch (Exception e) { + log.warn(e, "Exception closing StreamAppenderatorDriver; continuing teardown."); + } + // Guarantees the appenderator is released on every exception path + // (incl. WakeupException, ISE during add, persist/publish failures). + // closeNow() is the immediate variant; safe to call after a successful + // close() since Appenderator implementations make close idempotent. + if (!appenderatorClosedNormally) { + try { + appenderator.closeNow(); + } + catch (Exception e) { + log.warn(e, "Exception during emergency closeNow() of Appenderator; continuing teardown."); + } + } } return TaskStatus.success(task.getId()); } - private KafkaShareGroupRecordSupplier createRecordSupplier(ShareGroupIndexTaskIOConfig ioConfig) + /** + * Wake up an in-flight {@link AcknowledgingRecordSupplier#poll(long)} call. + * Called by {@link ShareGroupIndexTask#stopGracefully(org.apache.druid.indexing.common.config.TaskConfig)}. + * Safe to call from any thread; no-op if the loop is not currently running. + */ + void requestWakeup() + { + final AcknowledgingRecordSupplier supplier = activeSupplier.get(); + if (supplier == null) { + return; + } + try { + supplier.wakeup(); + } + catch (Exception e) { + log.warn(e, "Exception calling wakeup() on the active record supplier; ignoring."); + } + } + + private AcknowledgingRecordSupplier createDefaultRecordSupplier( + ShareGroupIndexTaskIOConfig ioConfig + ) { final ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); try { @@ -348,16 +529,4 @@ private KafkaShareGroupRecordSupplier createRecordSupplier(ShareGroupIndexTaskIO } } - private static InputRow parseRow( - ByteEntity entity, - InputFormat inputFormat, - InputRowSchema inputRowSchema, - TaskToolbox toolbox - ) throws IOException - { - try (org.apache.druid.java.util.common.parsers.CloseableIterator iterator = - inputFormat.createReader(inputRowSchema, entity, toolbox.getIndexingTmpDir()).read()) { - return iterator.hasNext() ? iterator.next() : null; - } - } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunnerTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunnerTest.java new file mode 100644 index 000000000000..8cb2f78f0c65 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunnerTest.java @@ -0,0 +1,153 @@ +/* + * 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.indexing.kafka; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.segment.indexing.DataSchema; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.Map; + +/** + * Targeted unit tests for {@link ShareGroupIndexTaskRunner} that do not + * require a full Druid toolbox. These cover behaviors that are isolatable + * outside of {@link ShareGroupIndexTaskRunner#run()} (which itself is + * exercised end-to-end by {@code EmbeddedShareGroupIngestionTest}). + * + *

Specifically validates:

+ *
    + *
  • {@link ShareGroupIndexTaskRunner#requestWakeup()} is null-safe when the + * run loop is not active (called by {@code stopGracefully} before/after + * {@code runTask}).
  • + *
  • The DIP-friendly constructor correctly accepts a custom + * {@code AcknowledgingRecordSupplier} factory; the factory is the only + * seam needed to write higher-fidelity runner tests in the future + * (Phase 2 supervisor work).
  • + *
  • The commit-failure metric name is the agreed contract + * ({@code ingest/shareGroup/commitFailures}).
  • + *
+ */ +public class ShareGroupIndexTaskRunnerTest +{ + private ObjectMapper mapper; + private ShareGroupIndexTask task; + private TaskToolbox toolbox; + + @Before + public void setUp() + { + mapper = new DefaultObjectMapper(); + toolbox = Mockito.mock(TaskToolbox.class); + + final DataSchema dataSchema = DataSchema.builder() + .withDataSource("test_datasource") + .withTimestamp(new TimestampSpec("__time", null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .build(); + + final Map consumerProps = ImmutableMap.of("bootstrap.servers", "localhost:9092"); + final ShareGroupIndexTaskIOConfig ioConfig = new ShareGroupIndexTaskIOConfig( + "test-topic", + "test-share-group", + consumerProps, + null, + null + ); + final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig( + null, null, null, null, null, null, null, null, null, null, + null, null, null, null, null, null, null, null, null, null, + null, null + ); + task = new ShareGroupIndexTask( + "task_runner_test", + null, + dataSchema, + tuningConfig, + ioConfig, + null, + mapper + ); + } + + /** + * {@code stopGracefully} may be called before {@code runTask} or after it + * exits. Both windows leave {@code activeSupplier} == null, so + * {@code requestWakeup} must be a safe no-op. + */ + @Test + public void testRequestWakeupIsNullSafeWhenNoActiveSupplier() + { + final ShareGroupIndexTaskRunner runner = new ShareGroupIndexTaskRunner(task, toolbox, mapper); + runner.requestWakeup(); + } + + /** + * A task lifecycle assertion: calling {@code stopGracefully} when no + * runner is active (i.e. before {@code runTask} ran or after it exited) + * must not throw and must set {@code stopRequested}. + */ + @Test + public void testStopGracefullyBeforeRunTaskIsSafe() + { + Assert.assertFalse(task.isStopRequested()); + task.stopGracefully(null); + Assert.assertTrue(task.isStopRequested()); + } + + /** + * Sanity check that the DIP factory constructor is wired and does not crash + * when only constructing the runner (run() is exercised by IT). + */ + @Test + public void testRunnerAcceptsCustomSupplierFactory() + { + final ShareGroupIndexTaskRunner runner = new ShareGroupIndexTaskRunner( + task, + toolbox, + mapper, + ioConfig -> Mockito.mock( + org.apache.druid.indexing.seekablestream.common.AcknowledgingRecordSupplier.class + ) + ); + Assert.assertNotNull(runner); + runner.requestWakeup(); + } + + /** + * Locks the commit-failure metric name as a public contract: it is the + * single signal operators use to alert on broker commit issues. + */ + @Test + public void testCommitFailureMetricName() + { + Assert.assertEquals( + "ingest/shareGroup/commitFailures", + ShareGroupIndexTaskRunner.METRIC_COMMIT_FAILURES + ); + } +} From 5b33214a69877262ce9a5d2cfc3e47c65a8b55df Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Thu, 7 May 2026 18:03:24 +0530 Subject: [PATCH 21/49] Add embedded IT validating multi-row JSON-array ingestion via share group --- .../EmbeddedShareGroupIngestionTest.java | 114 ++++++++++++++++++ 1 file changed, 114 insertions(+) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedShareGroupIngestionTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedShareGroupIngestionTest.java index ce7001ddfab4..3e8f109ccfa5 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedShareGroupIngestionTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedShareGroupIngestionTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; import org.apache.druid.indexing.kafka.KafkaIndexTaskTuningConfig; @@ -45,6 +46,7 @@ import org.junit.jupiter.api.Test; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.concurrent.ThreadLocalRandom; @@ -168,6 +170,82 @@ public void test_shareGroupIngestion_basicEndToEnd() ); } + /** + * Validates the P1 multi-row fix (FrankChen021's review on PR 19311): + * a single Kafka record encoding a JSON array must produce N rows in Druid. + * Before the fix, the runner read only the first row from each record and + * then ACKed the record, silently dropping rows 2..K. With the fix, the + * runner uses {@link org.apache.druid.indexing.seekablestream.StreamChunkReader} + * which iterates all rows from a record before ACK. + * + *

Producing {@code numRecords} records each with a {@code rowsPerRecord}-element + * array; the assertion is that exactly {@code numRecords * rowsPerRecord} rows + * land in the datasource.

+ */ + @Test + public void test_shareGroupIngestion_multiRowJsonArray() + { + final String topic = dataSource + "_topic"; + kafkaServer.createTopicWithPartitions(topic, 2); + + final int numRecords = 5; + final int rowsPerRecord = 4; + kafkaServer.produceRecordsToTopic( + generateJsonArrayRecords(topic, numRecords, rowsPerRecord, DateTimes.of("2025-07-01")) + ); + + final DataSchema dataSchema = DataSchema.builder() + .withDataSource(dataSource) + .withTimestamp(new TimestampSpec(COL_TIMESTAMP, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .withGranularity(new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + null + )) + .build(); + + final ShareGroupIndexTaskIOConfig ioConfig = new ShareGroupIndexTaskIOConfig( + topic, + "druid-share-group-multirow-test", + kafkaServer.consumerProperties(), + new JsonInputFormat(null, Collections.emptyMap(), null, null, null), + null + ); + + final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig( + null, null, null, null, null, null, null, null, null, null, + null, null, null, null, null, null, null, null, null, null, + null, null + ); + + final ObjectMapper mapper = new DefaultObjectMapper(); + final ShareGroupIndexTask task = new ShareGroupIndexTask( + null, + null, + dataSchema, + tuningConfig, + ioConfig, + null, + mapper + ); + + cluster.callApi().submitTask(task); + + final long expectedRows = (long) numRecords * rowsPerRecord; + + indexer.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("ingest/events/processed") + .hasDimension(DruidMetrics.DATASOURCE, dataSource), + agg -> agg.hasSumAtLeast(expectedRows) + ); + + Assertions.assertEquals( + String.valueOf(expectedRows), + cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) + ); + } + private List> generateRecords( String topic, int numRecords, @@ -188,4 +266,40 @@ private List> generateRecords( } return records; } + + /** + * Produces {@code numRecords} Kafka records, each with a JSON array payload + * containing {@code rowsPerRecord} elements. Used to exercise the multi-row + * path in {@link org.apache.druid.indexing.seekablestream.StreamChunkReader}. + */ + private List> generateJsonArrayRecords( + String topic, + int numRecords, + int rowsPerRecord, + DateTime startTime + ) + { + final List> records = new ArrayList<>(); + for (int i = 0; i < numRecords; i++) { + final StringBuilder json = new StringBuilder("["); + for (int j = 0; j < rowsPerRecord; j++) { + if (j > 0) { + json.append(','); + } + json.append(StringUtils.format( + "{\"%s\":\"%s\",\"%s\":\"item_%d_%d\",\"%s\":%d}", + COL_TIMESTAMP, + startTime.plusHours(i * rowsPerRecord + j), + COL_ITEM, + i, + j, + COL_VALUE, + ThreadLocalRandom.current().nextInt(1000) + )); + } + json.append(']'); + records.add(new ProducerRecord<>(topic, i % 2, null, StringUtils.toUtf8(json.toString()))); + } + return records; + } } From 1413d397a97dd82d1868079b6b351bf3c1e23ef6 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Thu, 7 May 2026 18:03:29 +0530 Subject: [PATCH 22/49] Document consumer-property restrictions, graceful stop, lock duration, commit-failure metric --- docs/ingestion/kafka-share-group-ingestion.md | 83 ++++++++++++++++--- 1 file changed, 70 insertions(+), 13 deletions(-) diff --git a/docs/ingestion/kafka-share-group-ingestion.md b/docs/ingestion/kafka-share-group-ingestion.md index 9289403d2322..dae4614e0348 100644 --- a/docs/ingestion/kafka-share-group-ingestion.md +++ b/docs/ingestion/kafka-share-group-ingestion.md @@ -92,24 +92,68 @@ Submit a `ShareGroupIndexTask` to the Overlord. Unlike standard Kafka ingestion, |----------|------|----------|---------|-------------| | `topic` | String | Yes | -- | Kafka topic to consume from. | | `groupId` | String | Yes | -- | Share group identifier. Multiple tasks with the same `groupId` share the workload. | -| `consumerProperties` | Map | Yes | -- | Kafka consumer properties. Must include `bootstrap.servers`. | +| `consumerProperties` | Map | Yes | -- | Kafka consumer properties. Must include `bootstrap.servers`. See [Consumer property restrictions](#consumer-property-restrictions) below. | | `inputFormat` | Object | Yes | -- | Input format for parsing records (json, csv, avro, etc.). | | `pollTimeout` | Long | No | 2000 | Poll timeout in milliseconds. | +### Consumer property restrictions + +Kafka share consumers (KIP-932) reject a fixed set of consumer properties that are valid for regular consumer groups. To avoid broker-side `ConfigException` at task startup, Druid silently strips the following keys from `consumerProperties` (with a `WARN` log line per stripped key) before constructing the `KafkaShareConsumer`: + +| Stripped key | Why | +|--------------|-----| +| `auto.offset.reset` | Initial position is broker-controlled for share groups. | +| `enable.auto.commit` | Share consumers always require explicit `acknowledge()` + `commitSync()`. | +| `group.instance.id` | Share groups do not support static membership. | +| `isolation.level` | Always read-committed for share groups. | +| `partition.assignment.strategy` | Broker controls per-record delivery for share groups. | +| `interceptor.classes` | Not supported for share consumers. | +| `session.timeout.ms` | Share groups have no consumer-group session model. | +| `heartbeat.interval.ms` | Share groups have no heartbeat. | +| `group.protocol` | Always `SHARE` for share consumers. | +| `group.remote.assignor` | Not applicable to share groups. | + +The internal property `share.acknowledgement.mode=explicit` is set automatically and must not be overridden. + +### Tuning configuration + +`tuningConfig` accepts the standard `KafkaTuningConfig` fields (e.g. `maxRowsPerSegment`, `maxRowsInMemory`, `intermediatePersistPeriod`, etc.). Phase 1 ingestion respects: + +- `maxRowsInMemory` / `maxBytesInMemory`: triggers a mid-batch persist when the appenderator signals `isPersistRequired`. +- `maxRowsPerSegment`: when reached during a batch the runner logs the event; the over-threshold segments are pushed at the end-of-batch publish boundary. + +Phase 1 does not yet support mid-batch checkpoint/sequence rollover; that is a Phase 2 enhancement. + ## How it works 1. The task subscribes to the topic using a `KafkaShareConsumer` with the configured `groupId`. -2. The broker delivers batches of records with acquisition locks. -3. The task parses records, adds rows to an appenderator, and persists segments. -4. Segments are published atomically to the metadata store. -5. After successful publish, the task acknowledges all records in the batch with `ACCEPT`. -6. The task calls `commitSync()` to commit acknowledgements to the broker. -7. On task failure, unacknowledged records are redelivered by the broker to another consumer in the share group. +2. The broker delivers batches of records with per-record acquisition locks. +3. For each polled record the task uses the same multi-row parser that powers `KafkaIndexTask` (`StreamChunkReader`). A single Kafka record may produce zero (e.g. tombstones), one, or many `InputRow`s (e.g. JSON arrays). All resulting rows are added to the appenderator before the record is acknowledged. +4. Parse failures are routed through `ParseExceptionHandler` (so `maxParseExceptions` etc. are honored). Bytes/processed/unparseable/thrown-away counters are incremented exactly once per row. +5. Segments are persisted (mid-batch on memory pressure, end-of-batch unconditionally) and then published atomically to the metadata store via `SegmentTransactionalAppendAction`. +6. After successful publish, the task acknowledges every offset polled in the batch with `ACCEPT` and calls `commitSync()` to commit acknowledgements to the broker. +7. On task failure or graceful stop before publish, unacknowledged records are redelivered by the broker to another consumer in the share group after the acquisition lock expires. ## Safety invariants -1. Records are acknowledged with `ACCEPT` only after the segment containing them is atomically registered in the metadata store. No data loss on task failure. -2. Every polled record reaches exactly one terminal state: `ACCEPT` (processed), `RELEASE` (redelivered), or task crash (broker redelivers after lock timeout). +1. **ACK after publish:** records are acknowledged with `ACCEPT` only after the segment containing them is atomically registered in the metadata store. No data loss on task failure. +2. **Multi-row safe:** every row produced from a record (including all elements of a JSON-array record) is added to the appenderator before the record is acknowledged. +3. **Resource safe:** `Appenderator` and `KafkaShareConsumer` are released on every exit path, including graceful stop and exceptions. +4. **Terminal state:** every polled record reaches exactly one terminal state: `ACCEPT` (processed), `RELEASE` (redelivered), or task crash (broker redelivers after the acquisition lock expires). + +## Graceful stop + +When the Overlord asks a task to stop (rolling restart, supervisor reconfiguration, etc.), the task forwards the request to the runner which calls `KafkaShareConsumer.wakeup()`. The in-flight `poll()` throws `WakeupException`; the runner observes that `stopRequested` is set and exits the loop after committing any in-flight batch. Records polled but not yet published remain unacknowledged and are redelivered by the broker after the acquisition lock expires. + +## Acquisition lock duration + +The broker controls the per-record acquisition lock duration via `group.share.record.lock.duration.ms`. The runner logs the broker-effective value once after the first poll, e.g.: + +``` +Effective broker acquisition lock timeout for share-group[my-group]: 30000 ms +``` + +In Phase 1 the foreground ingestion thread does both poll and publish. If a single batch takes longer than the broker lock duration, in-flight records may be redelivered to another consumer (resulting in duplicates). To stay well under the lock duration, tune `pollTimeout`, `maxRowsInMemory`, and `maxRowsPerSegment` so each poll-publish cycle completes comfortably within the broker lock window. Background lock renewal via the `RENEW` acknowledge type is a Phase 2 enhancement. ## Scaling @@ -127,12 +171,21 @@ Adding or removing tasks does not trigger a rebalancing pause. New tasks begin c Share group ingestion provides **at-least-once** delivery. On task failure, records between the last committed acknowledgement and the failure point are redelivered. Duplicate records may be ingested across task restarts. A deduplication cache is planned for a future release. +## Metrics + +In addition to the standard ingestion metrics (`ingest/events/processed`, `ingest/events/unparseable`, `ingest/persists/count`, etc.), share-group ingestion emits: + +| Metric | Description | +|--------|-------------| +| `ingest/shareGroup/commitFailures` | Per-batch count of partitions whose `commitSync()` failed. A non-zero value implies the affected records will be redelivered by the broker. Operators should alert on sustained non-zero values. | + ## Limitations (current release) -- Single-threaded ingestion per task. Two-thread architecture with background lock renewal is planned. -- No supervisor integration. Tasks must be submitted manually via the Overlord API. -- No deduplication cache. Redelivered records after task failure may produce duplicates. +- Single-threaded ingestion per task. Two-thread architecture with background lock renewal (using `AcknowledgeType.RENEW`) is a Phase 2 enhancement. +- No supervisor integration. Tasks must be submitted manually via the Overlord API. A `KafkaShareGroupSupervisor` is planned for Phase 2. +- No deduplication cache. Records redelivered after task failure or graceful stop before publish may produce duplicates (at-least-once). - Delivery order within a partition is not guaranteed. +- Mid-batch checkpoint / sequence rollover is not supported. If a single batch grossly exceeds `maxRowsPerSegment`, the runner still publishes correctly (multiple segments per batch), but the per-segment row threshold is only checked at end-of-batch boundaries. ## Demo: end-to-end validation with Druid UI @@ -243,7 +296,11 @@ Unit tests: ```bash mvn test -pl extensions-core/kafka-indexing-service \ - -Dtest="org.apache.druid.indexing.kafka.ShareGroupIndexTaskIOConfigTest,org.apache.druid.indexing.kafka.KafkaShareGroupRecordSupplierTest,org.apache.druid.indexing.kafka.ShareGroupIndexTaskTest" \ + -Dtest="org.apache.druid.indexing.kafka.ShareGroupIndexTaskIOConfigTest,\ +org.apache.druid.indexing.kafka.KafkaShareGroupRecordSupplierTest,\ +org.apache.druid.indexing.kafka.ShareGroupIndexTaskTest,\ +org.apache.druid.indexing.kafka.ShareGroupIndexTaskRunnerTest,\ +org.apache.druid.indexing.kafka.ShareGroupConsumerPropertiesTest" \ -Dsurefire.failIfNoSpecifiedTests=false \ -Pskip-static-checks -Dweb.console.skip=true -T1C ``` From 431e1920876033aab92b28c11c688d4173a08423 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Fri, 8 May 2026 10:54:34 +0530 Subject: [PATCH 23/49] Fix share-consumer ack to use cached ConsumerRecord reference --- .../kafka/KafkaShareGroupRecordSupplier.java | 85 ++++++++++++++++--- 1 file changed, 74 insertions(+), 11 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java index 0e02c827caad..b263a55a53c6 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java @@ -50,13 +50,9 @@ import java.util.stream.Collectors; /** - * Wraps a {@link KafkaShareConsumer} to implement {@link AcknowledgingRecordSupplier}. - * - * The share consumer uses broker-managed offset tracking with explicit - * acknowledgement. Records are delivered with acquisition locks; unacknowledged - * records are redelivered after lock timeout. - * - * This supplier sets {@code group.id} to the share group name provided in config. + * Adapts {@link KafkaShareConsumer} to {@link AcknowledgingRecordSupplier}. + * Delivery state lives on the broker; the supplier sets {@code group.id} to + * the configured share group name. */ public class KafkaShareGroupRecordSupplier implements AcknowledgingRecordSupplier @@ -64,6 +60,16 @@ public class KafkaShareGroupRecordSupplier private static final Logger log = new Logger(KafkaShareGroupRecordSupplier.class); private final KafkaShareConsumer consumer; + /** + * Tracks {@link ConsumerRecord} references from the most recent {@link #poll} + * so {@link #acknowledge} can be invoked with the original record. Kafka's + * {@code KafkaShareConsumer.acknowledge(topic, partition, offset, type)} + * variant is restricted to records still resident in the consumer's current + * fetch and can throw {@code IllegalStateException("The record cannot be + * acknowledged.")} once the fetch buffer has rolled over; passing the + * record reference is the safer pattern recommended by KIP-932. + */ + private final Map> deliveredRecords = new HashMap<>(); private boolean closed; public KafkaShareGroupRecordSupplier( @@ -103,9 +109,11 @@ public Set subscription() @Override public List> poll(long timeoutMs) { + deliveredRecords.clear(); final List> polledRecords = new ArrayList<>(); for (ConsumerRecord record : consumer.poll(Duration.ofMillis(timeoutMs))) { + deliveredRecords.put(new RecordKey(record.topic(), record.partition(), record.offset()), record); polledRecords.add(new OrderedPartitionableRecord<>( record.topic(), new KafkaTopicPartition(true, record.topic(), record.partition()), @@ -129,7 +137,17 @@ public void acknowledge(KafkaTopicPartition partitionId, Long offset, Acknowledg final String topic = partitionId.topic().orElseThrow( () -> new IllegalArgumentException("Cannot acknowledge record without topic") ); - consumer.acknowledge(topic, partitionId.partition(), offset, toKafkaAcknowledgeType(type)); + final ConsumerRecord record = deliveredRecords.get( + new RecordKey(topic, partitionId.partition(), offset) + ); + if (record == null) { + throw new IllegalStateException(StringUtils.format( + "Cannot acknowledge unknown record at topic[%s] partition[%d] offset[%d]; " + + "either it was not delivered by the most recent poll() or it has already been acknowledged.", + topic, partitionId.partition(), offset + )); + } + consumer.acknowledge(record, toKafkaAcknowledgeType(type)); } @Override @@ -145,7 +163,17 @@ public void acknowledge( () -> new IllegalArgumentException("Cannot acknowledge record without topic") ); for (Long offset : entry.getValue()) { - consumer.acknowledge(topic, partition.partition(), offset, kafkaType); + final ConsumerRecord record = deliveredRecords.get( + new RecordKey(topic, partition.partition(), offset) + ); + if (record == null) { + throw new IllegalStateException(StringUtils.format( + "Cannot acknowledge unknown record at topic[%s] partition[%d] offset[%d]; " + + "either it was not delivered by the most recent poll() or it has already been acknowledged.", + topic, partition.partition(), offset + )); + } + consumer.acknowledge(record, kafkaType); } } } @@ -168,8 +196,7 @@ public Map> commitSync() @Override public Set getPartitionIds(String stream) { - // Share consumer does not expose partitionsFor; use admin client if needed. - // For Phase 1, return empty set as the broker manages assignment. + // Share consumer does not expose partition assignment; broker manages it. return Collections.emptySet(); } @@ -239,4 +266,40 @@ private static KafkaShareConsumer createShareConsumer( Thread.currentThread().setContextClassLoader(currCtxCl); } } + + private static final class RecordKey + { + private final String topic; + private final int partition; + private final long offset; + + RecordKey(String topic, int partition, long offset) + { + this.topic = topic; + this.partition = partition; + this.offset = offset; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (!(o instanceof RecordKey)) { + return false; + } + final RecordKey that = (RecordKey) o; + return partition == that.partition && offset == that.offset && topic.equals(that.topic); + } + + @Override + public int hashCode() + { + int result = topic.hashCode(); + result = 31 * result + partition; + result = 31 * result + Long.hashCode(offset); + return result; + } + } } From 436f818d0ef0295de47ab6ca6ad3232b0c559cb4 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Fri, 8 May 2026 10:54:39 +0530 Subject: [PATCH 24/49] Register TaskRealtimeMetricsMonitor in share-group runner --- .../kafka/ShareGroupIndexTaskRunner.java | 124 +++++++----------- 1 file changed, 46 insertions(+), 78 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java index a517cef11b77..471b3e7f0c45 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java @@ -82,51 +82,31 @@ /** * Ingestion loop for {@link ShareGroupIndexTask}. * - *

Phase 1 invariants: + *

Invariants: *

    - *
  1. ACK after publish (data safety): records are acknowledged with - * {@code ACCEPT} only after the segment containing them is atomically - * registered in the metadata store via - * {@link SegmentTransactionalAppendAction}. Any failure between poll and - * publish leaves the records unacknowledged and the broker will - * redeliver them after the acquisition lock expires.
  2. - *
  3. Multi-row safe: records are parsed via - * {@link StreamChunkReader} which returns 0..N rows per record (handles - * JSON arrays, null/empty records, and ParseException routing). Every - * row produced from a record is added to the appenderator before the - * record is acknowledged.
  4. - *
  5. Resource safe: {@code Appenderator} and {@code RecordSupplier} - * are released on every exit path, including - * {@link org.apache.kafka.common.errors.WakeupException} during graceful - * stop and any in-loop ISE.
  6. - *
  7. Graceful stop: {@link ShareGroupIndexTask#stopGracefully} calls - * {@link #requestWakeup()} which forwards to - * {@link AcknowledgingRecordSupplier#wakeup()}, interrupting an - * in-flight {@code poll()} so the loop exits at the next iteration.
  8. - *
  9. Observability: per-partition commit failures are emitted as - * {@code ingest/shareGroup/commitFailures}; broker-effective acquisition - * lock duration is logged once after the first poll.
  10. + *
  11. Records are {@code ACCEPT}-acknowledged only after the segment they + * belong to has been atomically published; any failure before that + * leaves them unacknowledged and the broker redelivers them after the + * acquisition lock expires.
  12. + *
  13. Multi-row records (e.g. JSON arrays) are fully drained through + * {@link StreamChunkReader} before the source record is acknowledged.
  14. + *
  15. {@code Appenderator} and {@link AcknowledgingRecordSupplier} are + * released on every exit path, including + * {@link org.apache.kafka.common.errors.WakeupException}.
  16. + *
  17. {@link ShareGroupIndexTask#stopGracefully} drives + * {@link AcknowledgingRecordSupplier#wakeup()} via {@link #requestWakeup()} + * to interrupt an in-flight {@code poll()}.
  18. *
* - *

Single-threaded: all {@code Appenderator} interactions happen on - * the run thread. Background lock-renewal is a Phase 2 enhancement (see - * {@link AcknowledgeType#RENEW}).

- * - *

Testability (DIP): the supplier factory is constructor-injected - * so unit tests can plug in a mock {@link AcknowledgingRecordSupplier} without - * spinning up a Kafka broker.

+ *

The loop is single-threaded. The supplier factory is constructor-injected + * so tests can plug in a mock {@link AcknowledgingRecordSupplier}. */ public class ShareGroupIndexTaskRunner { private static final Logger log = new Logger(ShareGroupIndexTaskRunner.class); private static final String SEQUENCE_NAME = "share_group_seq_0"; - /** - * Metric for per-partition acknowledgement commit failures during share-group - * ingestion. Emitted with the standard task dimensions (datasource, taskId, - * etc.) so operators can alert on sustained broker commit issues. A non-zero - * value implies the corresponding records will be redelivered by the broker. - */ + /** Per-partition acknowledgement commit failures; non-zero implies redelivery. */ static final String METRIC_COMMIT_FAILURES = "ingest/shareGroup/commitFailures"; private final ShareGroupIndexTask task; @@ -135,11 +115,7 @@ public class ShareGroupIndexTaskRunner private final Function> supplierFactory; - /** - * Active supplier reference used by {@link #requestWakeup()} so the task's - * {@code stopGracefully} hook can interrupt a blocking poll. Set on entry - * to the run loop and cleared on exit. - */ + // Set on entry to the run loop and cleared on exit; used by requestWakeup(). private final AtomicReference> activeSupplier = new AtomicReference<>(); @@ -152,12 +128,7 @@ public class ShareGroupIndexTaskRunner this(task, toolbox, configMapper, null); } - /** - * DIP-friendly constructor: accepts a factory that builds the - * {@link AcknowledgingRecordSupplier} from the IO config. Tests can pass a - * factory returning a mock supplier; production passes {@code null} and the - * runner falls back to building a {@link KafkaShareGroupRecordSupplier}. - */ + /** Test-only ctor; pass a factory to inject a mock {@link AcknowledgingRecordSupplier}. */ @VisibleForTesting ShareGroupIndexTaskRunner( ShareGroupIndexTask task, @@ -195,14 +166,8 @@ public TaskStatus run() throws Exception throw new ISE("inputFormat must be specified in ioConfig"); } - // Single canonical multi-row parser, shared with the seekable-stream runner. - // Handles null/empty records, multi-row records (e.g. JSON arrays), parse - // exceptions (via ParseExceptionHandler), and processedBytes metrics. - // - // Raw type matches the SeekableStreamIndexTaskRunner idiom: works around - // OrderedPartitionableRecord.getData() returning List - // (rather than List); changing that signature would - // be a much larger refactor for marginal benefit. + // Raw type mirrors SeekableStreamIndexTaskRunner: works around + // OrderedPartitionableRecord.getData() returning List. @SuppressWarnings({"rawtypes", "unchecked"}) final StreamChunkReader chunkReader = new StreamChunkReader( inputFormat, @@ -279,6 +244,14 @@ public TaskStatus run() throws Exception segmentGenerationMetrics ); + final org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor metricsMonitor = + new org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor( + segmentGenerationMetrics, + rowIngestionMeters, + task.getMetricBuilder() + ); + toolbox.addMonitor(metricsMonitor); + boolean appenderatorClosedNormally = false; try (final AcknowledgingRecordSupplier recordSupplier = supplierFactory.apply(ioConfig)) { @@ -298,7 +271,7 @@ public Object getMetadata() @Override public void run() { - // no-op: share group does not need client-side offset persistence + // Share groups manage delivery state on the broker; no client commit. } }; @@ -336,7 +309,7 @@ public void run() continue; } - // Track offsets of records in this batch for acknowledgement after publish + // Offsets to acknowledge after the batch is published. final Map> batchOffsets = new HashMap<>(); boolean midBatchPersistNeeded = false; boolean pushThresholdLogged = false; @@ -345,10 +318,8 @@ public void run() batchOffsets.computeIfAbsent(record.getPartitionId(), k -> new ArrayList<>()) .add(record.getSequenceNumber()); - // Multi-row safe: chunkReader returns 0..N rows per record, increments - // processedBytes/thrownAway/unparseable internally, and routes parse - // exceptions through parseExceptionHandler (so maxParseExceptions/etc. - // are honored). Null record.data() is handled inside parse(). + // 0..N rows per record; null/empty data and parse exceptions are + // handled inside parse() and routed through parseExceptionHandler. final List rows = chunkReader.parse(record.getData(), false); for (InputRow row : rows) { @@ -361,8 +332,7 @@ public void run() ); if (!addResult.isOk()) { - // Failure to allocate segment puts data integrity at risk: bail out - // so the records remain unacknowledged and the broker redelivers. + // Bail without acknowledging so the broker redelivers the batch. throw new ISE( "Could not allocate segment for row with timestamp[%s]", row.getTimestamp() @@ -388,15 +358,13 @@ public void run() } } - // Mid-batch persist if memory pressure dictates (max{Bytes,Rows}InMemory). + // Mid-batch persist when memory thresholds were tripped during add(). if (midBatchPersistNeeded) { driver.persist(committerSupplier.get()); } - // End-of-batch persist before publish driver.persist(committerSupplier.get()); - // Publish all segments for this sequence final TransactionalSegmentPublisher publisher = new TransactionalSegmentPublisher() { @Override @@ -433,16 +401,14 @@ public boolean supportsEmptyPublish() log.info("Published %d segments with %d total rows.", segmentCount, totalRowsIngested); } - // INVARIANT 1: ACK only after successful publish + // ACK only after publish succeeded (data-safety invariant). for (Map.Entry> entry : batchOffsets.entrySet()) { for (Long offset : entry.getValue()) { recordSupplier.acknowledge(entry.getKey(), offset, AcknowledgeType.ACCEPT); } } - // Commit acknowledgements to the broker. Failures here mean the broker - // will redeliver the affected records (at-least-once); we count and emit - // a metric so operators can alert on sustained commit issues. + // Commit failures cause broker redelivery; emit a metric for alerting. final Map> commitResult = recordSupplier.commitSync(); long commitFailures = 0L; for (Map.Entry> entry : commitResult.entrySet()) { @@ -460,7 +426,6 @@ public boolean supportsEmptyPublish() } } - // Final persist and graceful close driver.persist(committerSupplier.get()); log.info("Share group ingestion complete. Total rows ingested: %d", totalRowsIngested); @@ -469,16 +434,20 @@ public boolean supportsEmptyPublish() } finally { activeSupplier.set(null); + try { + toolbox.removeMonitor(metricsMonitor); + } + catch (Exception e) { + log.warn(e, "Exception removing TaskRealtimeMetricsMonitor; continuing teardown."); + } try { driver.close(); } catch (Exception e) { log.warn(e, "Exception closing StreamAppenderatorDriver; continuing teardown."); } - // Guarantees the appenderator is released on every exception path - // (incl. WakeupException, ISE during add, persist/publish failures). - // closeNow() is the immediate variant; safe to call after a successful - // close() since Appenderator implementations make close idempotent. + // Release the appenderator on every abnormal exit path; closeNow() is + // idempotent and safe even if close() already ran. if (!appenderatorClosedNormally) { try { appenderator.closeNow(); @@ -493,9 +462,8 @@ public boolean supportsEmptyPublish() } /** - * Wake up an in-flight {@link AcknowledgingRecordSupplier#poll(long)} call. - * Called by {@link ShareGroupIndexTask#stopGracefully(org.apache.druid.indexing.common.config.TaskConfig)}. - * Safe to call from any thread; no-op if the loop is not currently running. + * Interrupt an in-flight {@link AcknowledgingRecordSupplier#poll(long)}. + * Thread-safe; no-op when the loop is not running. */ void requestWakeup() { From 31579e6e832df3816d15c95d64299526becccb01 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Fri, 8 May 2026 10:54:45 +0530 Subject: [PATCH 25/49] Tighten share-group task and consumer-property javadocs --- .../kafka/ShareGroupConsumerProperties.java | 32 +++++-------------- .../indexing/kafka/ShareGroupIndexTask.java | 11 +++---- 2 files changed, 12 insertions(+), 31 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupConsumerProperties.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupConsumerProperties.java index d621bb5b695d..0ab719292ee1 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupConsumerProperties.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupConsumerProperties.java @@ -27,29 +27,17 @@ import java.util.Set; /** - * Helpers for handling Kafka consumer properties supplied to the share-group - * task. Single Responsibility: filter / validate properties before they reach - * the {@link org.apache.kafka.clients.consumer.KafkaShareConsumer}. - * - *

Kafka 4.2.0 rejects a fixed set of properties when the consumer is a - * share consumer (see {@code ShareConsumerConfig.SHARE_GROUP_UNSUPPORTED_CONFIGS}). - * If users copy a regular consumer config wholesale, the broker raises - * {@code ConfigException} at construction time. We strip those keys with a - * warning so the task can still start; users get a clear log line telling them - * exactly which keys were dropped.

- * - *

The list is kept in sync with Kafka 4.2.0; if the upstream list changes - * in a future release, this code should be updated.

+ * Sanitises Kafka consumer properties before they reach + * {@link org.apache.kafka.clients.consumer.KafkaShareConsumer}: strips keys + * forbidden by {@code ShareConsumerConfig.SHARE_GROUP_UNSUPPORTED_CONFIGS} + * (Kafka 4.2.0) so the task starts cleanly when users supply a regular + * consumer config. */ public final class ShareGroupConsumerProperties { private static final Logger log = new Logger(ShareGroupConsumerProperties.class); - /** - * Mirror of {@code ShareConsumerConfig.SHARE_GROUP_UNSUPPORTED_CONFIGS} from - * Kafka 4.2.0. Setting any of these on a {@code KafkaShareConsumer} causes - * a {@code ConfigException} at construction. We strip them defensively. - */ + /** Mirrors {@code ShareConsumerConfig.SHARE_GROUP_UNSUPPORTED_CONFIGS} (Kafka 4.2.0). */ static final Set UNSUPPORTED_CONFIGS = ImmutableSet.of( "auto.offset.reset", "enable.auto.commit", @@ -68,12 +56,8 @@ private ShareGroupConsumerProperties() } /** - * Returns a copy of {@code consumerProperties} with all share-group-unsupported - * keys removed. Each removed key is logged at WARN level once. Insertion - * order is preserved (uses {@link LinkedHashMap}). - * - * @param consumerProperties caller-supplied properties; never null - * @return new map with unsupported keys removed + * Returns a copy of {@code consumerProperties} with unsupported keys + * removed; each removed key is logged at WARN. Iteration order is preserved. */ public static Map sanitize(Map consumerProperties) { diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java index 329e32617808..42ce7833785e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java @@ -46,13 +46,10 @@ import java.util.concurrent.atomic.AtomicReference; /** - * Indexing task that consumes from a Kafka topic using share group semantics - * (KIP-932). Unlike {@link KafkaIndexTask}, this task does not manage offsets - * client-side. The Kafka broker tracks delivery state; the task explicitly - * acknowledges records after segments are published. - * - * Phase 1: Single-threaded, no supervisor, no dedup cache. - * The task polls, parses, builds segments, publishes, ACKs, and commits. + * Indexing task that consumes from a Kafka topic with share-group semantics + * (KIP-932). The broker tracks delivery state; the task acknowledges records + * after the segments containing them are published. Phase 1: single-threaded, + * no supervisor, no dedup cache. */ public class ShareGroupIndexTask extends AbstractTask implements PendingSegmentAllocatingTask { From 7d7bd13d0f3317c98f632e96382729c24fd07498 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Fri, 8 May 2026 10:54:52 +0530 Subject: [PATCH 26/49] Tighten seekablestream common javadocs --- .../seekablestream/StreamChunkReader.java | 13 +-- .../common/AcknowledgeType.java | 29 ++--- .../common/AcknowledgingRecordSupplier.java | 100 ++++-------------- 3 files changed, 31 insertions(+), 111 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkReader.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkReader.java index 27d60feb5434..0d4a51ff7cee 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkReader.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkReader.java @@ -41,14 +41,11 @@ import java.util.List; /** - * Abstraction for parsing stream data which internally uses {@link org.apache.druid.data.input.InputEntityReader}. - * - *

Public so it can be reused by stream-ingestion implementations outside the - * {@code seekablestream} package (notably the Kafka share-group runner). The - * class is intentionally narrow in scope: given a chunk of {@link ByteEntity} - * bytes from one stream record, parse it into zero-or-more {@link InputRow}s - * while honoring the configured {@link InputRowFilter}, - * {@link RowIngestionMeters}, and {@link ParseExceptionHandler}.

+ * Parses one stream record's worth of {@link ByteEntity} chunks into zero-or-more + * {@link InputRow}s, honouring the configured {@link InputRowFilter}, + * {@link RowIngestionMeters}, and {@link ParseExceptionHandler}. Public so it + * can be reused by stream-ingestion implementations outside the + * {@code seekablestream} package. */ public class StreamChunkReader { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgeType.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgeType.java index b152b0c5948e..7b41d8657198 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgeType.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgeType.java @@ -20,37 +20,20 @@ package org.apache.druid.indexing.seekablestream.common; /** - * Acknowledgement types for queue-semantics record suppliers where the broker - * manages delivery state (e.g., Kafka Share Groups). - * - * Maps directly to Kafka's {@code AcknowledgeType} enum values. + * Acknowledgement types for queue-semantics record suppliers; mirrors Kafka's + * {@code AcknowledgeType}. */ public enum AcknowledgeType { - /** - * The record was consumed and processed successfully. - * The broker will mark the record as committed. - */ + /** Record processed successfully; broker commits it. */ ACCEPT, - /** - * Release the record for redelivery to another consumer. - * Used when the consumer cannot process the record right now - * but another consumer might be able to. - */ + /** Release the record back to the broker for redelivery. */ RELEASE, - /** - * Reject the record permanently. The broker will not redeliver it. - * Used for poison-pill records that can never be processed. - */ + /** Reject the record permanently (poison pill); never redelivered. */ REJECT, - /** - * Extend the acquisition lock on the record without releasing or accepting it. - * Used by background lock-renewal workers to keep records under lock while - * the foreground processing thread continues building segments. Maps to - * Kafka's {@code AcknowledgeType.RENEW}. - */ + /** Extend the acquisition lock without accepting or releasing. */ RENEW } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgingRecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgingRecordSupplier.java index 2a10cbd5b887..57a4ef4a7118 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgingRecordSupplier.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/AcknowledgingRecordSupplier.java @@ -30,15 +30,10 @@ import java.util.Set; /** - * Record supplier for queue-semantics streams where the broker manages delivery - * state and consumers explicitly acknowledge records after processing. - * - * Unlike {@link RecordSupplier}, this interface does not support assign/seek - * operations because the broker controls partition assignment and offset tracking. - * Instead, consumers subscribe to topics and acknowledge individual records. - * - * Designed for Kafka Share Groups (KIP-932) but generic enough for other - * queue-semantics systems. + * Record supplier for queue-semantics streams (e.g. Kafka share groups, + * KIP-932) where the broker owns delivery state and consumers acknowledge + * records explicitly. Unlike {@link RecordSupplier}, callers do not assign + * or seek partitions; they subscribe to topics and ack individual records. * * @param partition identifier type * @param sequence/offset number type @@ -47,113 +42,58 @@ public interface AcknowledgingRecordSupplier extends Closeable { - /** - * Subscribe to the given topics. The broker manages partition assignment. - * - * @param topics topics to subscribe to - */ void subscribe(Set topics); - /** - * Unsubscribe from all topics. - */ void unsubscribe(); - /** - * Returns the current subscription. - * - * @return set of subscribed topic names - */ Set subscription(); /** - * Poll for records from the subscribed topics. The broker delivers records - * with acquisition locks; records must be acknowledged before the lock expires. - * - * @param timeoutMs poll timeout in milliseconds - * @return list of records, never null + * Poll for records. Records carry acquisition locks and must be + * acknowledged before the lock expires. */ @NotNull List> poll(long timeoutMs); - /** - * Acknowledge a single record with the default type (ACCEPT). - * - * @param partitionId partition of the record - * @param offset offset of the record - */ + /** Acknowledge a single record with the default type ({@code ACCEPT}). */ void acknowledge(PartitionIdType partitionId, SequenceOffsetType offset); - /** - * Acknowledge a single record with a specific type. - * - * @param partitionId partition of the record - * @param offset offset of the record - * @param type acknowledgement type (ACCEPT, RELEASE, REJECT) - */ + /** Acknowledge a single record with the given type. */ void acknowledge(PartitionIdType partitionId, SequenceOffsetType offset, AcknowledgeType type); - /** - * Acknowledge a batch of records with a specific type. - * - * @param offsets map from partition to collection of offsets - * @param type acknowledgement type - */ + /** Acknowledge a batch of records with the given type. */ void acknowledge(Map> offsets, AcknowledgeType type); /** - * Commit all pending acknowledgements synchronously. - * Returns a map from partition to an optional exception if the commit - * failed for that partition. - * - * @return commit results per partition + * Commit pending acknowledgements; returns a per-partition exception when + * the commit failed for that partition. */ Map> commitSync(); - /** - * Returns the set of partition IDs for the given stream/topic. - * - * @param stream topic name - * @return set of partition identifiers - */ Set getPartitionIds(String stream); /** - * Wakes up a blocked {@link #poll(long)} call so the ingestion loop can - * exit promptly on graceful stop. Implementations that wrap a - * {@link org.apache.kafka.clients.consumer.ShareConsumer} should call its - * {@code wakeup()} method here. The contract for the caller is that the - * next {@link #poll(long)} (or the in-flight one) will throw an - * implementation-specific wake-up exception - * (e.g. {@link org.apache.kafka.common.errors.WakeupException}). - * - *

Default implementation is a no-op for suppliers that do not support - * wake-ups; the ingestion loop will fall back to polling the - * {@code stopRequested} flag at the next poll boundary.

+ * Interrupt a blocked {@link #poll(long)} so the loop can exit promptly. + * Implementations wrapping {@link org.apache.kafka.clients.consumer.ShareConsumer} + * should delegate to its {@code wakeup()}, after which the in-flight or + * next {@link #poll(long)} throws an implementation-specific wake-up + * exception (e.g. {@link org.apache.kafka.common.errors.WakeupException}). + * Default is a no-op. */ default void wakeup() { } /** - * Returns the broker-effective acquisition lock timeout, if known. For - * Kafka share groups this is the {@code group.share.record.lock.duration.ms} - * the broker is using. The value is only meaningful after the first - * successful {@link #poll(long)} call (Kafka does not push it until the - * client has joined the share group). - * - *

Default implementation returns {@link Optional#empty()}.

- * - * @return acquisition lock timeout in milliseconds, or empty if unknown + * Broker-effective acquisition lock timeout, if known. For Kafka share + * groups this surfaces {@code group.share.record.lock.duration.ms} after + * the first {@link #poll(long)}; default is {@link Optional#empty()}. */ default Optional acquisitionLockTimeoutMs() { return Optional.empty(); } - /** - * Close this supplier and release all resources. - */ @Override void close(); } From fd09eca32b2070aaccb45dd700bf0a505161b58b Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Fri, 8 May 2026 10:54:58 +0530 Subject: [PATCH 27/49] Update share-group unit tests for new ack contract --- .../KafkaShareGroupRecordSupplierTest.java | 82 +++++++++++++------ .../ShareGroupConsumerPropertiesTest.java | 5 -- .../kafka/ShareGroupIndexTaskRunnerTest.java | 37 +-------- 3 files changed, 58 insertions(+), 66 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java index fd4a19a7e23c..68f451cef78d 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java @@ -37,6 +37,12 @@ import org.mockito.ArgumentCaptor; import org.mockito.Mockito; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.Arrays; import java.util.Collections; @@ -46,17 +52,6 @@ import java.util.Optional; import java.util.Set; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -/** - * Unit tests for {@link KafkaShareGroupRecordSupplier} using a mocked - * KafkaShareConsumer. These tests verify the wrapping logic without - * requiring a real Kafka broker. - */ public class KafkaShareGroupRecordSupplierTest { private KafkaShareConsumer mockConsumer; @@ -162,13 +157,16 @@ public void testPollReturnsEmptyOnTimeout() @Test public void testAcknowledgeDefaultAccept() { + final ConsumerRecord record = new ConsumerRecord<>( + "test-topic", 0, 42L, "key".getBytes(), "value".getBytes() + ); + pollSingleRecord(record); + final KafkaTopicPartition partition = new KafkaTopicPartition(true, "test-topic", 0); supplier.acknowledge(partition, 42L); Mockito.verify(mockConsumer).acknowledge( - Mockito.eq("test-topic"), - Mockito.eq(0), - Mockito.eq(42L), + Mockito.same(record), Mockito.eq(org.apache.kafka.clients.consumer.AcknowledgeType.ACCEPT) ); } @@ -176,13 +174,16 @@ public void testAcknowledgeDefaultAccept() @Test public void testAcknowledgeWithRelease() { + final ConsumerRecord record = new ConsumerRecord<>( + "test-topic", 0, 10L, "key".getBytes(), "value".getBytes() + ); + pollSingleRecord(record); + final KafkaTopicPartition partition = new KafkaTopicPartition(true, "test-topic", 0); supplier.acknowledge(partition, 10L, AcknowledgeType.RELEASE); Mockito.verify(mockConsumer).acknowledge( - Mockito.eq("test-topic"), - Mockito.eq(0), - Mockito.eq(10L), + Mockito.same(record), Mockito.eq(org.apache.kafka.clients.consumer.AcknowledgeType.RELEASE) ); } @@ -190,13 +191,16 @@ public void testAcknowledgeWithRelease() @Test public void testAcknowledgeWithReject() { + final ConsumerRecord record = new ConsumerRecord<>( + "test-topic", 0, 10L, "key".getBytes(), "value".getBytes() + ); + pollSingleRecord(record); + final KafkaTopicPartition partition = new KafkaTopicPartition(true, "test-topic", 0); supplier.acknowledge(partition, 10L, AcknowledgeType.REJECT); Mockito.verify(mockConsumer).acknowledge( - Mockito.eq("test-topic"), - Mockito.eq(0), - Mockito.eq(10L), + Mockito.same(record), Mockito.eq(org.apache.kafka.clients.consumer.AcknowledgeType.REJECT) ); } @@ -204,13 +208,16 @@ public void testAcknowledgeWithReject() @Test public void testAcknowledgeWithRenew() { + final ConsumerRecord record = new ConsumerRecord<>( + "test-topic", 0, 99L, "key".getBytes(), "value".getBytes() + ); + pollSingleRecord(record); + final KafkaTopicPartition partition = new KafkaTopicPartition(true, "test-topic", 0); supplier.acknowledge(partition, 99L, AcknowledgeType.RENEW); Mockito.verify(mockConsumer).acknowledge( - Mockito.eq("test-topic"), - Mockito.eq(0), - Mockito.eq(99L), + Mockito.same(record), Mockito.eq(org.apache.kafka.clients.consumer.AcknowledgeType.RENEW) ); } @@ -218,6 +225,22 @@ public void testAcknowledgeWithRenew() @Test public void testAcknowledgeBatch() { + final List> recordsP0 = Arrays.asList( + new ConsumerRecord<>("test-topic", 0, 1L, "k".getBytes(), "v".getBytes()), + new ConsumerRecord<>("test-topic", 0, 2L, "k".getBytes(), "v".getBytes()), + new ConsumerRecord<>("test-topic", 0, 3L, "k".getBytes(), "v".getBytes()) + ); + final List> recordsP1 = Arrays.asList( + new ConsumerRecord<>("test-topic", 1, 10L, "k".getBytes(), "v".getBytes()), + new ConsumerRecord<>("test-topic", 1, 11L, "k".getBytes(), "v".getBytes()) + ); + final Map>> recordMap = new HashMap<>(); + recordMap.put(new TopicPartition("test-topic", 0), recordsP0); + recordMap.put(new TopicPartition("test-topic", 1), recordsP1); + Mockito.when(mockConsumer.poll(Mockito.any(Duration.class))) + .thenReturn(new ConsumerRecords<>(recordMap)); + supplier.poll(1000); + final KafkaTopicPartition p0 = new KafkaTopicPartition(true, "test-topic", 0); final KafkaTopicPartition p1 = new KafkaTopicPartition(true, "test-topic", 1); @@ -228,13 +251,20 @@ public void testAcknowledgeBatch() supplier.acknowledge(offsets, AcknowledgeType.ACCEPT); Mockito.verify(mockConsumer, Mockito.times(5)).acknowledge( - Mockito.anyString(), - Mockito.anyInt(), - Mockito.anyLong(), + Mockito.>any(), Mockito.eq(org.apache.kafka.clients.consumer.AcknowledgeType.ACCEPT) ); } + private void pollSingleRecord(ConsumerRecord record) + { + final Map>> recordMap = new HashMap<>(); + recordMap.put(new TopicPartition(record.topic(), record.partition()), List.of(record)); + Mockito.when(mockConsumer.poll(Mockito.any(Duration.class))) + .thenReturn(new ConsumerRecords<>(recordMap)); + supplier.poll(1000); + } + @Test public void testCommitSync() { diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupConsumerPropertiesTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupConsumerPropertiesTest.java index f56f3e1a84b8..595eaa425af5 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupConsumerPropertiesTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupConsumerPropertiesTest.java @@ -26,11 +26,6 @@ import java.util.LinkedHashMap; import java.util.Map; -/** - * Unit tests for {@link ShareGroupConsumerProperties}. Validates that the - * sanitize method strips exactly the share-group-unsupported keys without - * touching legitimate ones, and preserves insertion order. - */ public class ShareGroupConsumerPropertiesTest { @Test diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunnerTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunnerTest.java index 8cb2f78f0c65..35571f9606a8 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunnerTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunnerTest.java @@ -34,23 +34,8 @@ import java.util.Map; /** - * Targeted unit tests for {@link ShareGroupIndexTaskRunner} that do not - * require a full Druid toolbox. These cover behaviors that are isolatable - * outside of {@link ShareGroupIndexTaskRunner#run()} (which itself is - * exercised end-to-end by {@code EmbeddedShareGroupIngestionTest}). - * - *

Specifically validates:

- *
    - *
  • {@link ShareGroupIndexTaskRunner#requestWakeup()} is null-safe when the - * run loop is not active (called by {@code stopGracefully} before/after - * {@code runTask}).
  • - *
  • The DIP-friendly constructor correctly accepts a custom - * {@code AcknowledgingRecordSupplier} factory; the factory is the only - * seam needed to write higher-fidelity runner tests in the future - * (Phase 2 supervisor work).
  • - *
  • The commit-failure metric name is the agreed contract - * ({@code ingest/shareGroup/commitFailures}).
  • - *
+ * Lightweight unit tests for {@link ShareGroupIndexTaskRunner}; the run loop + * itself is covered end-to-end by {@code EmbeddedShareGroupIngestionTest}. */ public class ShareGroupIndexTaskRunnerTest { @@ -94,11 +79,6 @@ public void setUp() ); } - /** - * {@code stopGracefully} may be called before {@code runTask} or after it - * exits. Both windows leave {@code activeSupplier} == null, so - * {@code requestWakeup} must be a safe no-op. - */ @Test public void testRequestWakeupIsNullSafeWhenNoActiveSupplier() { @@ -106,11 +86,6 @@ public void testRequestWakeupIsNullSafeWhenNoActiveSupplier() runner.requestWakeup(); } - /** - * A task lifecycle assertion: calling {@code stopGracefully} when no - * runner is active (i.e. before {@code runTask} ran or after it exited) - * must not throw and must set {@code stopRequested}. - */ @Test public void testStopGracefullyBeforeRunTaskIsSafe() { @@ -119,10 +94,6 @@ public void testStopGracefullyBeforeRunTaskIsSafe() Assert.assertTrue(task.isStopRequested()); } - /** - * Sanity check that the DIP factory constructor is wired and does not crash - * when only constructing the runner (run() is exercised by IT). - */ @Test public void testRunnerAcceptsCustomSupplierFactory() { @@ -138,10 +109,6 @@ public void testRunnerAcceptsCustomSupplierFactory() runner.requestWakeup(); } - /** - * Locks the commit-failure metric name as a public contract: it is the - * single signal operators use to alert on broker commit issues. - */ @Test public void testCommitFailureMetricName() { From bc61817b89863e6549749cc93db926335766cf39 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Fri, 8 May 2026 10:55:07 +0530 Subject: [PATCH 28/49] Move share-group ITs to embedded-tests and add probe test --- .../EmbeddedShareGroupIngestionTest.java | 474 ++++++++++++++++++ .../indexing/ShareGroupKafkaResource.java | 73 +++ .../indexing/ShareGroupProbeTest.java | 249 +++++++++ .../EmbeddedShareGroupIngestionTest.java | 305 ----------- .../simulate/ShareGroupKafkaResource.java | 38 -- 5 files changed, 796 insertions(+), 343 deletions(-) create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupKafkaResource.java create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupProbeTest.java delete mode 100644 extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedShareGroupIngestionTest.java delete mode 100644 extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/ShareGroupKafkaResource.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java new file mode 100644 index 000000000000..a66302cd177c --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java @@ -0,0 +1,474 @@ +/* + * 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.testing.embedded.indexing; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.KafkaIndexTaskTuningConfig; +import org.apache.druid.indexing.kafka.ShareGroupIndexTask; +import org.apache.druid.indexing.kafka.ShareGroupIndexTaskIOConfig; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedIndexer; +import org.apache.druid.testing.embedded.EmbeddedOverlord; +import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.joda.time.DateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; + +/** + * End-to-end integration tests for Kafka share group ingestion. Requires + * Docker for the Testcontainers Kafka broker. + */ +public class EmbeddedShareGroupIngestionTest extends EmbeddedClusterTestBase +{ + private static final String COL_TIMESTAMP = "__time"; + private static final String COL_ITEM = "item"; + private static final String COL_VALUE = "value"; + + private static final InputFormat DEFAULT_CSV_FORMAT = + new CsvInputFormat(List.of(COL_TIMESTAMP, COL_ITEM, COL_VALUE), null, null, false, 0, false); + + // Wall-clock pause to let a freshly submitted share-group task reach STABLE + // before records are produced; STABLE is observed within ~300 ms locally. + private static final long SHARE_CONSUMER_READY_DELAY_MS = 3_000L; + + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedIndexer indexer = new EmbeddedIndexer(); + private final EmbeddedHistorical historical = new EmbeddedHistorical(); + private final EmbeddedBroker broker = new EmbeddedBroker(); + + private ShareGroupKafkaResource kafkaServer; + + @Override + public EmbeddedDruidCluster createCluster() + { + kafkaServer = new ShareGroupKafkaResource(); + + final EmbeddedDruidCluster cluster = EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper(); + indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s"); + + cluster.addExtension(KafkaIndexTaskModule.class) + .addResource(kafkaServer) + .useLatchableEmitter() + .useDefaultTimeoutForLatchableEmitter(30) + .addCommonProperty("druid.monitoring.emissionPeriod", "PT0.1s") + .addServer(coordinator) + .addServer(overlord) + .addServer(indexer) + .addServer(historical) + .addServer(broker); + + return cluster; + } + + @Test + public void test_shareGroupIngestion_basicEndToEnd() throws InterruptedException + { + final String topic = dataSource + "_topic"; + kafkaServer.createTopicWithPartitions(topic, 2); + + final String taskId = submitShareGroupTask( + topic, + "druid-share-group-test", + kafkaServer.consumerProperties(), + DEFAULT_CSV_FORMAT, + defaultTuningConfig() + ); + + Thread.sleep(SHARE_CONSUMER_READY_DELAY_MS); + + final int numRecords = 10; + kafkaServer.produceRecordsToTopic( + generateRecords(topic, numRecords, DateTimes.of("2025-06-01")) + ); + + waitForRowsProcessed(numRecords); + + Assertions.assertEquals( + String.valueOf(numRecords), + cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) + ); + + cancelAndAwaitTermination(taskId); + } + + /** + * A single Kafka record carrying multiple JSON objects must produce all its + * rows in Druid (multi-row records are routed through {@link + * org.apache.druid.indexing.seekablestream.StreamChunkReader}). The producer + * emits whitespace-separated JSON objects per record, which {@link + * org.apache.druid.data.input.impl.JsonNodeReader} splits into one row per + * object. + */ + @Test + public void test_shareGroupIngestion_multiRowPerKafkaRecord() throws InterruptedException + { + final String topic = dataSource + "_topic"; + kafkaServer.createTopicWithPartitions(topic, 2); + + final String taskId = submitShareGroupTask( + topic, + "druid-share-group-multirow-test", + kafkaServer.consumerProperties(), + new JsonInputFormat(null, Collections.emptyMap(), null, null, true), + defaultTuningConfig() + ); + + Thread.sleep(SHARE_CONSUMER_READY_DELAY_MS); + + final int numRecords = 5; + final int rowsPerRecord = 4; + kafkaServer.produceRecordsToTopic( + generateMultiObjectJsonRecords(topic, numRecords, rowsPerRecord, DateTimes.of("2025-07-01")) + ); + + final long expectedRows = (long) numRecords * rowsPerRecord; + waitForRowsProcessed(expectedRows); + + Assertions.assertEquals( + String.valueOf(expectedRows), + cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) + ); + + cancelAndAwaitTermination(taskId); + } + + /** + * Forbidden share-group consumer properties must be stripped silently so the + * task starts and ingests instead of failing with a Kafka {@code ConfigException}. + */ + @Test + public void test_shareGroupIngestion_unsupportedConsumerProperty_isSanitized() throws InterruptedException + { + final String topic = dataSource + "_topic"; + kafkaServer.createTopicWithPartitions(topic, 2); + + final String taskId = submitShareGroupTask( + topic, + "druid-share-group-sanitize-test", + consumerPropsWithUnsupportedKeys(), + DEFAULT_CSV_FORMAT, + defaultTuningConfig() + ); + + Thread.sleep(SHARE_CONSUMER_READY_DELAY_MS); + + final int numRecords = 10; + kafkaServer.produceRecordsToTopic( + generateRecords(topic, numRecords, DateTimes.of("2025-08-01")) + ); + + waitForRowsProcessed(numRecords); + + Assertions.assertEquals( + String.valueOf(numRecords), + cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) + ); + + cancelAndAwaitTermination(taskId); + } + + /** + * Cancelling a share-group task via the Overlord must drain the in-flight + * batch (segments published, records acknowledged) before terminating, so + * rows ingested before the cancel remain queryable. + */ + @Test + public void test_shareGroupIngestion_gracefulStop_publishesInflightBatch() throws InterruptedException + { + final String topic = dataSource + "_topic"; + kafkaServer.createTopicWithPartitions(topic, 2); + + final String taskId = submitShareGroupTask( + topic, + "druid-share-group-graceful-stop", + kafkaServer.consumerProperties(), + DEFAULT_CSV_FORMAT, + defaultTuningConfig() + ); + + Thread.sleep(SHARE_CONSUMER_READY_DELAY_MS); + + final int numRecords = 5; + kafkaServer.produceRecordsToTopic( + generateRecords(topic, numRecords, DateTimes.of("2025-09-01")) + ); + + waitForRowsProcessed(numRecords); + + cancelAndAwaitTermination(taskId); + + Assertions.assertEquals( + String.valueOf(numRecords), + cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) + ); + } + + /** + * Three share-group tasks sharing one {@code consumerGroupId} on a 2-partition + * topic must collectively ingest every produced record exactly once + * (KIP-932 fan-out: no duplicates, no loss). + */ + @Test + public void test_shareGroupIngestion_multiTaskShareGroup_noDuplicatesNoLoss() throws InterruptedException + { + final String topic = dataSource + "_topic"; + kafkaServer.createTopicWithPartitions(topic, 2); + + final String groupId = "druid-share-group-fanout"; + final String taskA = submitShareGroupTask( + topic, groupId, kafkaServer.consumerProperties(), DEFAULT_CSV_FORMAT, defaultTuningConfig() + ); + final String taskB = submitShareGroupTask( + topic, groupId, kafkaServer.consumerProperties(), DEFAULT_CSV_FORMAT, defaultTuningConfig() + ); + final String taskC = submitShareGroupTask( + topic, groupId, kafkaServer.consumerProperties(), DEFAULT_CSV_FORMAT, defaultTuningConfig() + ); + + Thread.sleep(SHARE_CONSUMER_READY_DELAY_MS); + + final int numRecords = 30; + kafkaServer.produceRecordsToTopic( + generateRecords(topic, numRecords, DateTimes.of("2025-10-01")) + ); + + waitForRowsProcessed(numRecords); + + cancelAndAwaitTermination(taskA); + cancelAndAwaitTermination(taskB); + cancelAndAwaitTermination(taskC); + + Assertions.assertEquals( + String.valueOf(numRecords), + cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) + ); + } + + /** + * Ingestion must be data-correct under memory pressure: 1000 rows with + * {@code maxRowsInMemory=100} forces mid-batch persists yet still yields + * the full row count. + */ + @Test + public void test_shareGroupIngestion_lowMaxRowsInMemory_dataCorrect() throws InterruptedException + { + final String topic = dataSource + "_topic"; + kafkaServer.createTopicWithPartitions(topic, 2); + + final String taskId = submitShareGroupTask( + topic, + "druid-share-group-persist-test", + kafkaServer.consumerProperties(), + DEFAULT_CSV_FORMAT, + tuningConfigWithMaxRowsInMemory(100) + ); + + Thread.sleep(SHARE_CONSUMER_READY_DELAY_MS); + + final int numRecords = 1000; + kafkaServer.produceRecordsToTopic( + generateRecords(topic, numRecords, DateTimes.of("2025-11-01")) + ); + + waitForRowsProcessed(numRecords); + + cancelAndAwaitTermination(taskId); + + Assertions.assertEquals( + String.valueOf(numRecords), + cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) + ); + } + + private DataSchema buildDataSchema() + { + return DataSchema.builder() + .withDataSource(dataSource) + .withTimestamp(new TimestampSpec(COL_TIMESTAMP, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .withGranularity(new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + null + )) + .build(); + } + + private static KafkaIndexTaskTuningConfig defaultTuningConfig() + { + return new KafkaIndexTaskTuningConfig( + null, null, null, null, null, null, null, null, null, null, + null, null, null, null, null, null, null, null, null, null, + null, null + ); + } + + private static KafkaIndexTaskTuningConfig tuningConfigWithMaxRowsInMemory(int maxRowsInMemory) + { + return new KafkaIndexTaskTuningConfig( + null, maxRowsInMemory, null, null, null, null, null, null, null, null, + null, null, null, null, null, null, null, null, null, null, + null, null + ); + } + + private String submitShareGroupTask( + String topic, + String groupId, + Map consumerProperties, + InputFormat inputFormat, + KafkaIndexTaskTuningConfig tuningConfig + ) + { + // Broker default is LATEST; switch this share group to EARLIEST so any + // pre-existing records are delivered. + kafkaServer.setShareGroupAutoOffsetReset(groupId, "earliest"); + + final ShareGroupIndexTaskIOConfig ioConfig = new ShareGroupIndexTaskIOConfig( + topic, + groupId, + consumerProperties, + inputFormat, + null + ); + + final ObjectMapper mapper = new DefaultObjectMapper(); + final ShareGroupIndexTask task = new ShareGroupIndexTask( + null, + null, + buildDataSchema(), + tuningConfig, + ioConfig, + null, + mapper + ); + + cluster.callApi().submitTask(task); + return task.getId(); + } + + /** + * Cancels the task and waits for it to reach a terminal state. The Overlord + * always reports user-initiated cancels as FAILED ("Shutdown request from + * user"); supervisor-driven graceful stops use {@code shutdownWithSuccess}, + * but standalone tasks cannot opt into that path. Data correctness is + * verified separately via SQL row counts. + */ + private void cancelAndAwaitTermination(String taskId) + { + cluster.callApi().onLeaderOverlord(o -> o.cancelTask(taskId)); + cluster.callApi().waitForTaskToFinish(taskId, overlord.latchableEmitter()); + } + + private void waitForRowsProcessed(long expected) + { + indexer.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("ingest/events/processed") + .hasDimension(DruidMetrics.DATASOURCE, dataSource), + agg -> agg.hasSumAtLeast(expected) + ); + } + + private Map consumerPropsWithUnsupportedKeys() + { + // Keys forbidden by ShareConsumerConfig.SHARE_GROUP_UNSUPPORTED_CONFIGS. + final Map props = new HashMap<>(kafkaServer.consumerProperties()); + props.put("enable.auto.commit", "false"); + props.put("auto.offset.reset", "earliest"); + props.put("group.instance.id", "share-it-instance-0"); + return props; + } + + private List> generateRecords( + String topic, + int numRecords, + DateTime startTime + ) + { + final List> records = new ArrayList<>(); + for (int i = 0; i < numRecords; i++) { + final String csv = StringUtils.format( + "%s,item_%d,%d", + startTime.plusDays(i), + i, + ThreadLocalRandom.current().nextInt(1000) + ); + records.add( + new ProducerRecord<>(topic, i % 2, null, StringUtils.toUtf8(csv)) + ); + } + return records; + } + + private List> generateMultiObjectJsonRecords( + String topic, + int numRecords, + int rowsPerRecord, + DateTime startTime + ) + { + final List> records = new ArrayList<>(); + for (int i = 0; i < numRecords; i++) { + final StringBuilder json = new StringBuilder(); + for (int j = 0; j < rowsPerRecord; j++) { + if (j > 0) { + json.append(' '); + } + json.append(StringUtils.format( + "{\"%s\":\"%s\",\"%s\":\"item_%d_%d\",\"%s\":%d}", + COL_TIMESTAMP, + startTime.plusHours(i * rowsPerRecord + j), + COL_ITEM, + i, + j, + COL_VALUE, + ThreadLocalRandom.current().nextInt(1000) + )); + } + records.add(new ProducerRecord<>(topic, i % 2, null, StringUtils.toUtf8(json.toString()))); + } + return records; + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupKafkaResource.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupKafkaResource.java new file mode 100644 index 000000000000..64731e850bfb --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupKafkaResource.java @@ -0,0 +1,73 @@ +/* + * 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.testing.embedded.indexing; + +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AlterConfigOp; +import org.apache.kafka.clients.admin.ConfigEntry; +import org.apache.kafka.common.config.ConfigResource; +import org.testcontainers.kafka.KafkaContainer; + +import java.util.Collection; +import java.util.List; +import java.util.Map; + +/** + * A {@link KafkaResource} that starts a Kafka broker with share groups enabled. + * Share groups (KIP-932) require the broker config {@code group.share.enable=true}. + */ +public class ShareGroupKafkaResource extends KafkaResource +{ + @Override + protected KafkaContainer createContainer() + { + final KafkaContainer container = super.createContainer(); + container.withEnv("KAFKA_GROUP_SHARE_ENABLE", "true"); + container.withEnv("KAFKA_GROUP_SHARE_RECORD_LOCK_DURATION_MS", "30000"); + // Single-broker test cluster: override replication for the internal + // __share_group_state topic so the share coordinator can initialize. + container.withEnv("KAFKA_SHARE_COORDINATOR_STATE_TOPIC_REPLICATION_FACTOR", "1"); + container.withEnv("KAFKA_SHARE_COORDINATOR_STATE_TOPIC_MIN_ISR", "1"); + return container; + } + + /** + * Sets {@code share.auto.offset.reset} for the named share group via + * AdminClient. Broker default is {@code LATEST}; tests that produce + * records before subscribing need {@code earliest}. + */ + public void setShareGroupAutoOffsetReset(String groupId, String value) + { + try (Admin admin = newAdminClient()) { + final ConfigResource resource = new ConfigResource(ConfigResource.Type.GROUP, groupId); + final Collection ops = List.of( + new AlterConfigOp( + new ConfigEntry("share.auto.offset.reset", value), + AlterConfigOp.OpType.SET + ) + ); + admin.incrementalAlterConfigs(Map.of(resource, ops)).all().get(); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupProbeTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupProbeTest.java new file mode 100644 index 000000000000..29d4f25bb412 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupProbeTest.java @@ -0,0 +1,249 @@ +/* + * 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.testing.embedded.indexing; + +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.ConfigEntry; +import org.apache.kafka.clients.admin.DescribeConfigsResult; +import org.apache.kafka.clients.consumer.AcknowledgeType; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaShareConsumer; +import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Diagnostic probe: bypasses Druid entirely and uses {@link KafkaShareConsumer} + * directly against the testcontainers broker to verify share groups work. + */ +public class ShareGroupProbeTest +{ + private ShareGroupKafkaResource kafkaServer; + + @BeforeEach + public void setUp() + { + kafkaServer = new ShareGroupKafkaResource(); + final EmbeddedDruidCluster cluster = EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper(); + cluster.addResource(kafkaServer); + kafkaServer.beforeStart(cluster); + kafkaServer.start(); + kafkaServer.onStarted(cluster); + } + + @AfterEach + public void tearDown() + { + if (kafkaServer != null) { + kafkaServer.stop(); + } + } + + @Test + public void probe_mimicDruid_pollLoop100ms_areConsumed() throws Exception + { + final String topic = "probe_topic_mimic_" + System.currentTimeMillis(); + final String groupId = "probe_group_mimic_" + System.currentTimeMillis(); + + kafkaServer.createTopicWithPartitions(topic, 2); + kafkaServer.setShareGroupAutoOffsetReset(groupId, "earliest"); + + // Build props EXACTLY like Druid's KafkaShareGroupRecordSupplier does: + // - start from kafkaServer.consumerProperties() (KafkaConsumerConfigs.getConsumerProperties + bootstrap) + // - sanitize (strips enable.auto.commit/auto.offset.reset/group.instance.id) + // - set group.id and share.acknowledgement.mode + final Map raw = new HashMap<>(kafkaServer.consumerProperties()); + final Map sanitized = + org.apache.druid.indexing.kafka.ShareGroupConsumerProperties.sanitize(raw); + final Properties props = new Properties(); + for (Map.Entry e : sanitized.entrySet()) { + props.setProperty(e.getKey(), String.valueOf(e.getValue())); + } + props.setProperty("group.id", groupId); + props.setProperty("share.acknowledgement.mode", "explicit"); + + final AtomicInteger received = new AtomicInteger(); + + Thread producerThread = new Thread(() -> { + try { + Thread.sleep(3_000); + final java.util.ArrayList> records = + new java.util.ArrayList<>(); + for (int i = 0; i < 10; i++) { + records.add(new org.apache.kafka.clients.producer.ProducerRecord<>(topic, i % 2, null, ("v-" + i).getBytes())); + } + kafkaServer.produceRecordsToTopic(records); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + }, "probe-producer"); + producerThread.start(); + + try (KafkaShareConsumer consumer = + new KafkaShareConsumer<>(props, new ByteArrayDeserializer(), new ByteArrayDeserializer())) { + consumer.subscribe(List.of(topic)); + + final long deadlineMs = System.currentTimeMillis() + 30_000; + while (received.get() < 10 && System.currentTimeMillis() < deadlineMs) { + final ConsumerRecords polled = consumer.poll(Duration.ofMillis(100)); + for (ConsumerRecord r : polled) { + received.incrementAndGet(); + consumer.acknowledge(r, AcknowledgeType.ACCEPT); + } + consumer.commitSync(); + } + } + + producerThread.join(2_000); + Assertions.assertEquals(10, received.get(), "Expected 10 records via Druid-mimicking probe"); + } + + @Test + public void probe_noWarmup_transactionalProducer_areConsumed() throws Exception + { + final String topic = "probe_topic_nowarmup_" + System.currentTimeMillis(); + final String groupId = "probe_group_nowarmup_" + System.currentTimeMillis(); + + kafkaServer.createTopicWithPartitions(topic, 2); + kafkaServer.setShareGroupAutoOffsetReset(groupId, "earliest"); + + final Properties props = new Properties(); + props.put("bootstrap.servers", kafkaServer.getBootstrapServerUrl()); + props.put("group.id", groupId); + props.put("share.acknowledgement.mode", "explicit"); + + final AtomicInteger received = new AtomicInteger(); + + try (KafkaShareConsumer consumer = + new KafkaShareConsumer<>(props, new ByteArrayDeserializer(), new ByteArrayDeserializer())) { + consumer.subscribe(List.of(topic)); + + Thread.sleep(3_000); + + final java.util.ArrayList> records = + new java.util.ArrayList<>(); + for (int i = 0; i < 10; i++) { + records.add(new org.apache.kafka.clients.producer.ProducerRecord<>(topic, i % 2, null, ("value-" + i).getBytes())); + } + kafkaServer.produceRecordsToTopic(records); + + final long deadlineMs = System.currentTimeMillis() + 30_000; + while (received.get() < 10 && System.currentTimeMillis() < deadlineMs) { + final ConsumerRecords polled = consumer.poll(Duration.ofMillis(100)); + for (ConsumerRecord r : polled) { + received.incrementAndGet(); + consumer.acknowledge(r, AcknowledgeType.ACCEPT); + } + consumer.commitSync(); + } + } + + Assertions.assertEquals(10, received.get(), "Expected 10 records via KafkaShareConsumer (no warmup, transactional producer)"); + } + + @Test + public void probe_recordsProducedAfterSubscribe_areConsumed() throws Exception + { + final String topic = "probe_topic_" + System.currentTimeMillis(); + final String groupId = "probe_group_" + System.currentTimeMillis(); + + kafkaServer.createTopicWithPartitions(topic, 2); + + try (Admin admin = kafkaServer.newAdminClient()) { + final ConfigResource brokerCfg = new ConfigResource(ConfigResource.Type.BROKER, "1"); + final DescribeConfigsResult res = admin.describeConfigs(List.of(brokerCfg)); + for (ConfigEntry e : res.all().get().get(brokerCfg).entries()) { + if (e.name().startsWith("group.share")) { + System.out.println("BROKER_CFG: " + e.name() + "=" + e.value() + " source=" + e.source()); + } + } + } + + kafkaServer.setShareGroupAutoOffsetReset(groupId, "earliest"); + + try (Admin admin = kafkaServer.newAdminClient()) { + final ConfigResource grp = new ConfigResource(ConfigResource.Type.GROUP, groupId); + final DescribeConfigsResult res = admin.describeConfigs(List.of(grp)); + for (ConfigEntry e : res.all().get().get(grp).entries()) { + if (e.name().startsWith("share")) { + System.out.println("GROUP_CFG: " + e.name() + "=" + e.value() + " source=" + e.source()); + } + } + } + + final Properties props = new Properties(); + props.put("bootstrap.servers", kafkaServer.getBootstrapServerUrl()); + props.put("group.id", groupId); + props.put("share.acknowledgement.mode", "explicit"); + + final AtomicInteger received = new AtomicInteger(); + + try (KafkaShareConsumer consumer = + new KafkaShareConsumer<>(props, new ByteArrayDeserializer(), new ByteArrayDeserializer())) { + consumer.subscribe(List.of(topic)); + + final long subscribeAt = System.currentTimeMillis(); + while (System.currentTimeMillis() - subscribeAt < 3_000) { + consumer.poll(Duration.ofMillis(200)); + } + + final Map producerProps = new HashMap<>(); + producerProps.put("bootstrap.servers", kafkaServer.getBootstrapServerUrl()); + producerProps.put("key.serializer", ByteArraySerializer.class.getName()); + producerProps.put("value.serializer", ByteArraySerializer.class.getName()); + producerProps.put("acks", "all"); + + try (KafkaProducer producer = new KafkaProducer<>(producerProps)) { + for (int i = 0; i < 10; i++) { + producer.send(new ProducerRecord<>(topic, i % 2, null, ("value-" + i).getBytes())); + } + producer.flush(); + } + + final long deadlineMs = System.currentTimeMillis() + 30_000; + while (received.get() < 10 && System.currentTimeMillis() < deadlineMs) { + final ConsumerRecords records = consumer.poll(Duration.ofMillis(500)); + for (ConsumerRecord r : records) { + received.incrementAndGet(); + consumer.acknowledge(r, AcknowledgeType.ACCEPT); + } + consumer.commitSync(); + } + } + + Assertions.assertEquals(10, received.get(), "Expected 10 records via KafkaShareConsumer"); + } +} diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedShareGroupIngestionTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedShareGroupIngestionTest.java deleted file mode 100644 index 3e8f109ccfa5..000000000000 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedShareGroupIngestionTest.java +++ /dev/null @@ -1,305 +0,0 @@ -/* - * 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.indexing.kafka.simulate; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.data.input.impl.CsvInputFormat; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JsonInputFormat; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; -import org.apache.druid.indexing.kafka.KafkaIndexTaskTuningConfig; -import org.apache.druid.indexing.kafka.ShareGroupIndexTask; -import org.apache.druid.indexing.kafka.ShareGroupIndexTaskIOConfig; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.query.DruidMetrics; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.testing.embedded.EmbeddedBroker; -import org.apache.druid.testing.embedded.EmbeddedCoordinator; -import org.apache.druid.testing.embedded.EmbeddedDruidCluster; -import org.apache.druid.testing.embedded.EmbeddedHistorical; -import org.apache.druid.testing.embedded.EmbeddedIndexer; -import org.apache.druid.testing.embedded.EmbeddedOverlord; -import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.joda.time.DateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.ThreadLocalRandom; - -/** - * End-to-end integration test for Kafka share group ingestion. - * - * Demonstrates the full workflow: - *
    - *
  1. Start an embedded Druid cluster with Kafka extension
  2. - *
  3. Create a Kafka topic and produce CSV records
  4. - *
  5. Submit a {@link ShareGroupIndexTask} via the Overlord API
  6. - *
  7. Wait for records to be ingested
  8. - *
  9. Verify data via SQL query
  10. - *
- * - * This test requires a Kafka 4.0+ broker with share group support. - * The KafkaResource Testcontainer uses apache/kafka:4.1.1 by default. - */ -public class EmbeddedShareGroupIngestionTest extends EmbeddedClusterTestBase -{ - private static final String COL_TIMESTAMP = "__time"; - private static final String COL_ITEM = "item"; - private static final String COL_VALUE = "value"; - - private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); - private final EmbeddedOverlord overlord = new EmbeddedOverlord(); - private final EmbeddedIndexer indexer = new EmbeddedIndexer(); - private final EmbeddedHistorical historical = new EmbeddedHistorical(); - private final EmbeddedBroker broker = new EmbeddedBroker(); - - private ShareGroupKafkaResource kafkaServer; - - @Override - public EmbeddedDruidCluster createCluster() - { - kafkaServer = new ShareGroupKafkaResource(); - - final EmbeddedDruidCluster cluster = EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper(); - indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s"); - - cluster.addExtension(KafkaIndexTaskModule.class) - .addResource(kafkaServer) - .useLatchableEmitter() - .addServer(coordinator) - .addServer(overlord) - .addServer(indexer) - .addServer(historical) - .addServer(broker); - - return cluster; - } - - /** - * Showcase test: Submit a ShareGroupIndexTask, ingest data from Kafka - * using share group semantics, and verify via SQL. - */ - @Test - public void test_shareGroupIngestion_basicEndToEnd() - { - final String topic = dataSource + "_topic"; - kafkaServer.createTopicWithPartitions(topic, 2); - - final int numRecords = 10; - kafkaServer.produceRecordsToTopic( - generateRecords(topic, numRecords, DateTimes.of("2025-06-01")) - ); - - // Build the task spec - final DataSchema dataSchema = DataSchema.builder() - .withDataSource(dataSource) - .withTimestamp(new TimestampSpec(COL_TIMESTAMP, null, null)) - .withDimensions(DimensionsSpec.EMPTY) - .withGranularity(new UniformGranularitySpec( - org.apache.druid.java.util.common.granularity.Granularities.DAY, - org.apache.druid.java.util.common.granularity.Granularities.NONE, - null - )) - .build(); - - final ShareGroupIndexTaskIOConfig ioConfig = new ShareGroupIndexTaskIOConfig( - topic, - "druid-share-group-test", - kafkaServer.consumerProperties(), - new CsvInputFormat(List.of(COL_TIMESTAMP, COL_ITEM, COL_VALUE), null, null, false, 0, false), - null - ); - - final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig( - null, null, null, null, null, null, null, null, null, null, - null, null, null, null, null, null, null, null, null, null - ); - - final ObjectMapper mapper = cluster.callApi().objectMapper(); - final ShareGroupIndexTask task = new ShareGroupIndexTask( - null, - null, - dataSchema, - tuningConfig, - ioConfig, - null, - mapper - ); - - // Submit the task - cluster.callApi().submitTask(task); - - // Wait for records to be processed - indexer.latchableEmitter().waitForEventAggregate( - event -> event.hasMetricName("ingest/events/processed") - .hasDimension(DruidMetrics.DATASOURCE, dataSource), - agg -> agg.hasSumAtLeast(numRecords) - ); - - // Gracefully stop the task so it publishes and exits - // (In production, a supervisor would manage this lifecycle) - - // Verify ingested row count via SQL - Assertions.assertEquals( - String.valueOf(numRecords), - cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) - ); - } - - /** - * Validates the P1 multi-row fix (FrankChen021's review on PR 19311): - * a single Kafka record encoding a JSON array must produce N rows in Druid. - * Before the fix, the runner read only the first row from each record and - * then ACKed the record, silently dropping rows 2..K. With the fix, the - * runner uses {@link org.apache.druid.indexing.seekablestream.StreamChunkReader} - * which iterates all rows from a record before ACK. - * - *

Producing {@code numRecords} records each with a {@code rowsPerRecord}-element - * array; the assertion is that exactly {@code numRecords * rowsPerRecord} rows - * land in the datasource.

- */ - @Test - public void test_shareGroupIngestion_multiRowJsonArray() - { - final String topic = dataSource + "_topic"; - kafkaServer.createTopicWithPartitions(topic, 2); - - final int numRecords = 5; - final int rowsPerRecord = 4; - kafkaServer.produceRecordsToTopic( - generateJsonArrayRecords(topic, numRecords, rowsPerRecord, DateTimes.of("2025-07-01")) - ); - - final DataSchema dataSchema = DataSchema.builder() - .withDataSource(dataSource) - .withTimestamp(new TimestampSpec(COL_TIMESTAMP, null, null)) - .withDimensions(DimensionsSpec.EMPTY) - .withGranularity(new UniformGranularitySpec( - Granularities.DAY, - Granularities.NONE, - null - )) - .build(); - - final ShareGroupIndexTaskIOConfig ioConfig = new ShareGroupIndexTaskIOConfig( - topic, - "druid-share-group-multirow-test", - kafkaServer.consumerProperties(), - new JsonInputFormat(null, Collections.emptyMap(), null, null, null), - null - ); - - final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig( - null, null, null, null, null, null, null, null, null, null, - null, null, null, null, null, null, null, null, null, null, - null, null - ); - - final ObjectMapper mapper = new DefaultObjectMapper(); - final ShareGroupIndexTask task = new ShareGroupIndexTask( - null, - null, - dataSchema, - tuningConfig, - ioConfig, - null, - mapper - ); - - cluster.callApi().submitTask(task); - - final long expectedRows = (long) numRecords * rowsPerRecord; - - indexer.latchableEmitter().waitForEventAggregate( - event -> event.hasMetricName("ingest/events/processed") - .hasDimension(DruidMetrics.DATASOURCE, dataSource), - agg -> agg.hasSumAtLeast(expectedRows) - ); - - Assertions.assertEquals( - String.valueOf(expectedRows), - cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) - ); - } - - private List> generateRecords( - String topic, - int numRecords, - DateTime startTime - ) - { - final List> records = new ArrayList<>(); - for (int i = 0; i < numRecords; i++) { - final String csv = StringUtils.format( - "%s,item_%d,%d", - startTime.plusDays(i), - i, - ThreadLocalRandom.current().nextInt(1000) - ); - records.add( - new ProducerRecord<>(topic, i % 2, null, StringUtils.toUtf8(csv)) - ); - } - return records; - } - - /** - * Produces {@code numRecords} Kafka records, each with a JSON array payload - * containing {@code rowsPerRecord} elements. Used to exercise the multi-row - * path in {@link org.apache.druid.indexing.seekablestream.StreamChunkReader}. - */ - private List> generateJsonArrayRecords( - String topic, - int numRecords, - int rowsPerRecord, - DateTime startTime - ) - { - final List> records = new ArrayList<>(); - for (int i = 0; i < numRecords; i++) { - final StringBuilder json = new StringBuilder("["); - for (int j = 0; j < rowsPerRecord; j++) { - if (j > 0) { - json.append(','); - } - json.append(StringUtils.format( - "{\"%s\":\"%s\",\"%s\":\"item_%d_%d\",\"%s\":%d}", - COL_TIMESTAMP, - startTime.plusHours(i * rowsPerRecord + j), - COL_ITEM, - i, - j, - COL_VALUE, - ThreadLocalRandom.current().nextInt(1000) - )); - } - json.append(']'); - records.add(new ProducerRecord<>(topic, i % 2, null, StringUtils.toUtf8(json.toString()))); - } - return records; - } -} diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/ShareGroupKafkaResource.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/ShareGroupKafkaResource.java deleted file mode 100644 index fd59edcced5a..000000000000 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/ShareGroupKafkaResource.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.indexing.kafka.simulate; - -import org.testcontainers.kafka.KafkaContainer; - -/** - * A {@link KafkaResource} that starts a Kafka broker with share groups enabled. - * Share groups (KIP-932) require the broker config {@code group.share.enable=true}. - */ -public class ShareGroupKafkaResource extends KafkaResource -{ - @Override - protected KafkaContainer createContainer() - { - final KafkaContainer container = super.createContainer(); - container.withEnv("KAFKA_GROUP_SHARE_ENABLE", "true"); - container.withEnv("KAFKA_GROUP_SHARE_RECORD_LOCK_DURATION_MS", "30000"); - return container; - } -} From 210cd0472ec3b2598fcd66a9ecf7f168f211e3f8 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Fri, 8 May 2026 10:55:12 +0530 Subject: [PATCH 29/49] Pin kafka-clients 4.2.0 in embedded-tests --- embedded-tests/pom.xml | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/embedded-tests/pom.xml b/embedded-tests/pom.xml index 1c77fe29744b..8747bf9332da 100644 --- a/embedded-tests/pom.xml +++ b/embedded-tests/pom.xml @@ -40,6 +40,17 @@ + + + org.apache.kafka + kafka-clients + 4.2.0 + test + + org.apache.druid @@ -532,12 +543,6 @@ commons-codec test - - org.apache.kafka - kafka-clients - ${apache.kafka.version} - test - org.testcontainers testcontainers From 2356d6527f867e890d1dca259a8fbdc3a64c93a8 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Fri, 8 May 2026 10:55:17 +0530 Subject: [PATCH 30/49] Update docs for embedded-tests share-group IT location --- docs/ingestion/kafka-share-group-ingestion.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ingestion/kafka-share-group-ingestion.md b/docs/ingestion/kafka-share-group-ingestion.md index dae4614e0348..09042f00a209 100644 --- a/docs/ingestion/kafka-share-group-ingestion.md +++ b/docs/ingestion/kafka-share-group-ingestion.md @@ -308,8 +308,8 @@ org.apache.druid.indexing.kafka.ShareGroupConsumerPropertiesTest" \ E2E test (requires Docker running -- Testcontainers starts a Kafka 4.1.1 broker with share groups enabled automatically): ```bash -mvn test -pl extensions-core/kafka-indexing-service \ - -Dtest="org.apache.druid.indexing.kafka.simulate.EmbeddedShareGroupIngestionTest" \ +mvn test -pl embedded-tests -am \ + -Dtest="org.apache.druid.testing.embedded.indexing.EmbeddedShareGroupIngestionTest" \ -Dsurefire.failIfNoSpecifiedTests=false \ -Pskip-static-checks -Dweb.console.skip=true -T1C ``` From d2e94debe39cb5e790315d6cfea612984d2938d4 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Fri, 8 May 2026 11:05:45 +0530 Subject: [PATCH 31/49] Trim share-group code comments to essentials --- .../EmbeddedShareGroupIngestionTest.java | 54 +++++-------------- .../indexing/ShareGroupProbeTest.java | 5 +- .../kafka/KafkaShareGroupRecordSupplier.java | 12 ++--- .../kafka/ShareGroupIndexTaskRunner.java | 52 +++++------------- 4 files changed, 31 insertions(+), 92 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java index a66302cd177c..4956b3b1118e 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java @@ -55,10 +55,7 @@ import java.util.Map; import java.util.concurrent.ThreadLocalRandom; -/** - * End-to-end integration tests for Kafka share group ingestion. Requires - * Docker for the Testcontainers Kafka broker. - */ +/** End-to-end IT for Kafka share-group ingestion; requires Docker. */ public class EmbeddedShareGroupIngestionTest extends EmbeddedClusterTestBase { private static final String COL_TIMESTAMP = "__time"; @@ -68,8 +65,7 @@ public class EmbeddedShareGroupIngestionTest extends EmbeddedClusterTestBase private static final InputFormat DEFAULT_CSV_FORMAT = new CsvInputFormat(List.of(COL_TIMESTAMP, COL_ITEM, COL_VALUE), null, null, false, 0, false); - // Wall-clock pause to let a freshly submitted share-group task reach STABLE - // before records are produced; STABLE is observed within ~300 ms locally. + /** Lets a freshly submitted share-group task reach STABLE before records are produced. */ private static final long SHARE_CONSUMER_READY_DELAY_MS = 3_000L; private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); @@ -133,14 +129,7 @@ public void test_shareGroupIngestion_basicEndToEnd() throws InterruptedException cancelAndAwaitTermination(taskId); } - /** - * A single Kafka record carrying multiple JSON objects must produce all its - * rows in Druid (multi-row records are routed through {@link - * org.apache.druid.indexing.seekablestream.StreamChunkReader}). The producer - * emits whitespace-separated JSON objects per record, which {@link - * org.apache.druid.data.input.impl.JsonNodeReader} splits into one row per - * object. - */ + /** Verifies that one Kafka record carrying multiple JSON objects yields one row per object. */ @Test public void test_shareGroupIngestion_multiRowPerKafkaRecord() throws InterruptedException { @@ -174,10 +163,7 @@ public void test_shareGroupIngestion_multiRowPerKafkaRecord() throws Interrupted cancelAndAwaitTermination(taskId); } - /** - * Forbidden share-group consumer properties must be stripped silently so the - * task starts and ingests instead of failing with a Kafka {@code ConfigException}. - */ + /** Forbidden share-group consumer properties are stripped so the task starts and ingests. */ @Test public void test_shareGroupIngestion_unsupportedConsumerProperty_isSanitized() throws InterruptedException { @@ -209,11 +195,7 @@ public void test_shareGroupIngestion_unsupportedConsumerProperty_isSanitized() t cancelAndAwaitTermination(taskId); } - /** - * Cancelling a share-group task via the Overlord must drain the in-flight - * batch (segments published, records acknowledged) before terminating, so - * rows ingested before the cancel remain queryable. - */ + /** Cancel must drain the in-flight batch so rows ingested before cancel remain queryable. */ @Test public void test_shareGroupIngestion_gracefulStop_publishesInflightBatch() throws InterruptedException { @@ -245,11 +227,7 @@ public void test_shareGroupIngestion_gracefulStop_publishesInflightBatch() throw ); } - /** - * Three share-group tasks sharing one {@code consumerGroupId} on a 2-partition - * topic must collectively ingest every produced record exactly once - * (KIP-932 fan-out: no duplicates, no loss). - */ + /** Three tasks sharing one group on a 2-partition topic ingest every record exactly once (KIP-932). */ @Test public void test_shareGroupIngestion_multiTaskShareGroup_noDuplicatesNoLoss() throws InterruptedException { @@ -286,11 +264,7 @@ public void test_shareGroupIngestion_multiTaskShareGroup_noDuplicatesNoLoss() th ); } - /** - * Ingestion must be data-correct under memory pressure: 1000 rows with - * {@code maxRowsInMemory=100} forces mid-batch persists yet still yields - * the full row count. - */ + /** 1000 rows with {@code maxRowsInMemory=100} forces mid-batch persists yet ingests all rows. */ @Test public void test_shareGroupIngestion_lowMaxRowsInMemory_dataCorrect() throws InterruptedException { @@ -362,8 +336,7 @@ private String submitShareGroupTask( KafkaIndexTaskTuningConfig tuningConfig ) { - // Broker default is LATEST; switch this share group to EARLIEST so any - // pre-existing records are delivered. + // Broker default is LATEST; deliver pre-existing records to this group. kafkaServer.setShareGroupAutoOffsetReset(groupId, "earliest"); final ShareGroupIndexTaskIOConfig ioConfig = new ShareGroupIndexTaskIOConfig( @@ -390,11 +363,10 @@ private String submitShareGroupTask( } /** - * Cancels the task and waits for it to reach a terminal state. The Overlord - * always reports user-initiated cancels as FAILED ("Shutdown request from - * user"); supervisor-driven graceful stops use {@code shutdownWithSuccess}, - * but standalone tasks cannot opt into that path. Data correctness is - * verified separately via SQL row counts. + * Cancels and waits for any terminal state. User-initiated cancels are + * always reported as FAILED by the Overlord; only supervisors can drive + * SUCCESS via {@code shutdownWithSuccess}, so data correctness is verified + * separately via SQL row counts. */ private void cancelAndAwaitTermination(String taskId) { @@ -411,9 +383,9 @@ private void waitForRowsProcessed(long expected) ); } + /** Builds a property map containing keys forbidden by {@code SHARE_GROUP_UNSUPPORTED_CONFIGS}. */ private Map consumerPropsWithUnsupportedKeys() { - // Keys forbidden by ShareConsumerConfig.SHARE_GROUP_UNSUPPORTED_CONFIGS. final Map props = new HashMap<>(kafkaServer.consumerProperties()); props.put("enable.auto.commit", "false"); props.put("auto.offset.reset", "earliest"); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupProbeTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupProbeTest.java index 29d4f25bb412..010f9ca1f788 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupProbeTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupProbeTest.java @@ -80,10 +80,7 @@ public void probe_mimicDruid_pollLoop100ms_areConsumed() throws Exception kafkaServer.createTopicWithPartitions(topic, 2); kafkaServer.setShareGroupAutoOffsetReset(groupId, "earliest"); - // Build props EXACTLY like Druid's KafkaShareGroupRecordSupplier does: - // - start from kafkaServer.consumerProperties() (KafkaConsumerConfigs.getConsumerProperties + bootstrap) - // - sanitize (strips enable.auto.commit/auto.offset.reset/group.instance.id) - // - set group.id and share.acknowledgement.mode + // Mirror Druid's KafkaShareGroupRecordSupplier props pipeline. final Map raw = new HashMap<>(kafkaServer.consumerProperties()); final Map sanitized = org.apache.druid.indexing.kafka.ShareGroupConsumerProperties.sanitize(raw); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java index b263a55a53c6..a88f593950b0 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java @@ -60,14 +60,12 @@ public class KafkaShareGroupRecordSupplier private static final Logger log = new Logger(KafkaShareGroupRecordSupplier.class); private final KafkaShareConsumer consumer; + /** - * Tracks {@link ConsumerRecord} references from the most recent {@link #poll} - * so {@link #acknowledge} can be invoked with the original record. Kafka's - * {@code KafkaShareConsumer.acknowledge(topic, partition, offset, type)} - * variant is restricted to records still resident in the consumer's current - * fetch and can throw {@code IllegalStateException("The record cannot be - * acknowledged.")} once the fetch buffer has rolled over; passing the - * record reference is the safer pattern recommended by KIP-932. + * Records returned by the most recent {@link #poll(long)}, retained so + * {@link #acknowledge} can pass the original {@link ConsumerRecord} to the + * Kafka client. The {@code (topic, partition, offset)} ack variant is fragile + * once the share-fetch buffer rolls over (KIP-932). */ private final Map> deliveredRecords = new HashMap<>(); private boolean closed; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java index 471b3e7f0c45..a2c45d83209f 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java @@ -80,33 +80,16 @@ import com.google.common.base.Supplier; /** - * Ingestion loop for {@link ShareGroupIndexTask}. - * - *

Invariants: - *

    - *
  1. Records are {@code ACCEPT}-acknowledged only after the segment they - * belong to has been atomically published; any failure before that - * leaves them unacknowledged and the broker redelivers them after the - * acquisition lock expires.
  2. - *
  3. Multi-row records (e.g. JSON arrays) are fully drained through - * {@link StreamChunkReader} before the source record is acknowledged.
  4. - *
  5. {@code Appenderator} and {@link AcknowledgingRecordSupplier} are - * released on every exit path, including - * {@link org.apache.kafka.common.errors.WakeupException}.
  6. - *
  7. {@link ShareGroupIndexTask#stopGracefully} drives - * {@link AcknowledgingRecordSupplier#wakeup()} via {@link #requestWakeup()} - * to interrupt an in-flight {@code poll()}.
  8. - *
- * - *

The loop is single-threaded. The supplier factory is constructor-injected - * so tests can plug in a mock {@link AcknowledgingRecordSupplier}. + * Single-threaded ingestion loop for {@link ShareGroupIndexTask}. Records are + * {@code ACCEPT}-acknowledged only after the containing segment is published; + * any earlier failure leaves them unacknowledged so the broker redelivers + * them once the acquisition lock expires. */ public class ShareGroupIndexTaskRunner { private static final Logger log = new Logger(ShareGroupIndexTaskRunner.class); private static final String SEQUENCE_NAME = "share_group_seq_0"; - /** Per-partition acknowledgement commit failures; non-zero implies redelivery. */ static final String METRIC_COMMIT_FAILURES = "ingest/shareGroup/commitFailures"; private final ShareGroupIndexTask task; @@ -115,7 +98,6 @@ public class ShareGroupIndexTaskRunner private final Function> supplierFactory; - // Set on entry to the run loop and cleared on exit; used by requestWakeup(). private final AtomicReference> activeSupplier = new AtomicReference<>(); @@ -128,7 +110,6 @@ public class ShareGroupIndexTaskRunner this(task, toolbox, configMapper, null); } - /** Test-only ctor; pass a factory to inject a mock {@link AcknowledgingRecordSupplier}. */ @VisibleForTesting ShareGroupIndexTaskRunner( ShareGroupIndexTask task, @@ -166,8 +147,8 @@ public TaskStatus run() throws Exception throw new ISE("inputFormat must be specified in ioConfig"); } - // Raw type mirrors SeekableStreamIndexTaskRunner: works around - // OrderedPartitionableRecord.getData() returning List. + // Raw type mirrors SeekableStreamIndexTaskRunner; works around + // OrderedPartitionableRecord.getData() returning a wildcard list. @SuppressWarnings({"rawtypes", "unchecked"}) final StreamChunkReader chunkReader = new StreamChunkReader( inputFormat, @@ -187,7 +168,6 @@ public TaskStatus run() throws Exception : LockGranularity.SEGMENT; final TaskLockType lockType = TaskLocks.determineLockTypeForAppend(task.getContext()); - // Create appenderator and driver final Appenderator appenderator = toolbox.getAppenderatorsManager().createRealtimeAppenderatorForTask( toolbox.getSegmentLoaderConfig(), task.getId(), @@ -260,6 +240,8 @@ public TaskStatus run() throws Exception driver.startJob(segmentId -> true); + // Share groups manage delivery state on the broker; the Committer is a + // no-op placeholder required by the appenderator driver contract. final Supplier committerSupplier = () -> new Committer() { @Override @@ -271,7 +253,6 @@ public Object getMetadata() @Override public void run() { - // Share groups manage delivery state on the broker; no client commit. } }; @@ -309,7 +290,6 @@ public void run() continue; } - // Offsets to acknowledge after the batch is published. final Map> batchOffsets = new HashMap<>(); boolean midBatchPersistNeeded = false; boolean pushThresholdLogged = false; @@ -318,8 +298,6 @@ public void run() batchOffsets.computeIfAbsent(record.getPartitionId(), k -> new ArrayList<>()) .add(record.getSequenceNumber()); - // 0..N rows per record; null/empty data and parse exceptions are - // handled inside parse() and routed through parseExceptionHandler. final List rows = chunkReader.parse(record.getData(), false); for (InputRow row : rows) { @@ -332,7 +310,7 @@ public void run() ); if (!addResult.isOk()) { - // Bail without acknowledging so the broker redelivers the batch. + // Throw without acknowledging so the broker redelivers this batch. throw new ISE( "Could not allocate segment for row with timestamp[%s]", row.getTimestamp() @@ -358,7 +336,6 @@ public void run() } } - // Mid-batch persist when memory thresholds were tripped during add(). if (midBatchPersistNeeded) { driver.persist(committerSupplier.get()); } @@ -401,14 +378,14 @@ public boolean supportsEmptyPublish() log.info("Published %d segments with %d total rows.", segmentCount, totalRowsIngested); } - // ACK only after publish succeeded (data-safety invariant). + // Acknowledge only after the segment is published; any earlier failure + // leaves records unacknowledged so the broker redelivers them. for (Map.Entry> entry : batchOffsets.entrySet()) { for (Long offset : entry.getValue()) { recordSupplier.acknowledge(entry.getKey(), offset, AcknowledgeType.ACCEPT); } } - // Commit failures cause broker redelivery; emit a metric for alerting. final Map> commitResult = recordSupplier.commitSync(); long commitFailures = 0L; for (Map.Entry> entry : commitResult.entrySet()) { @@ -446,8 +423,6 @@ public boolean supportsEmptyPublish() catch (Exception e) { log.warn(e, "Exception closing StreamAppenderatorDriver; continuing teardown."); } - // Release the appenderator on every abnormal exit path; closeNow() is - // idempotent and safe even if close() already ran. if (!appenderatorClosedNormally) { try { appenderator.closeNow(); @@ -461,10 +436,7 @@ public boolean supportsEmptyPublish() return TaskStatus.success(task.getId()); } - /** - * Interrupt an in-flight {@link AcknowledgingRecordSupplier#poll(long)}. - * Thread-safe; no-op when the loop is not running. - */ + /** Interrupts an in-flight poll so the loop can exit on graceful stop. */ void requestWakeup() { final AcknowledgingRecordSupplier supplier = activeSupplier.get(); From 446e004bdcae26485d74a15add8b5c55a8c48779 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Fri, 8 May 2026 12:08:51 +0530 Subject: [PATCH 32/49] Tighten share-group docs and fix spellcheck --- docs/ingestion/kafka-share-group-ingestion.md | 75 ++++++++----------- website/.spelling | 12 +++ 2 files changed, 45 insertions(+), 42 deletions(-) diff --git a/docs/ingestion/kafka-share-group-ingestion.md b/docs/ingestion/kafka-share-group-ingestion.md index 09042f00a209..288f8cc0a29a 100644 --- a/docs/ingestion/kafka-share-group-ingestion.md +++ b/docs/ingestion/kafka-share-group-ingestion.md @@ -25,16 +25,14 @@ description: "Queue-semantics ingestion from Apache Kafka using share groups (KI --> :::info -Requires Apache Kafka 4.0 or higher. Share groups (KIP-932) must be enabled on the broker. +Requires Apache Kafka 4.0 or higher with share groups (KIP-932) enabled on the broker. ::: ## Overview -Standard Kafka consumer groups bind each partition to exactly one consumer. This creates a hard scaling ceiling: you cannot have more consumers than partitions. Rebalancing when consumers join or leave pauses all consumers in the group. A single slow message blocks all subsequent messages in its partition. +Kafka share groups (KIP-932) let multiple consumers read from the same partition concurrently. The broker manages per-record acquisition locks and explicit acknowledgement, so consumer count is not capped by partition count, joining or leaving consumers does not pause the group, and a slow record does not block its partition. -Kafka share groups (KIP-932) eliminate these constraints. The broker manages record delivery across consumers with per-record acquisition locks and explicit acknowledgement. Multiple consumers can read from the same partition concurrently. There is no rebalancing pause. Slow records do not block other records. - -Druid's share group ingestion uses `ShareGroupIndexTask` to consume from Kafka share groups and publish segments with at-least-once delivery guarantees. Records are acknowledged only after segments are atomically registered in the metadata store. +Druid's `ShareGroupIndexTask` consumes from a share group and publishes segments with at-least-once delivery: records are acknowledged only after their segments are atomically registered in the metadata store. ## When to use share group ingestion @@ -45,11 +43,11 @@ Druid's share group ingestion uses `ShareGroupIndexTask` to consume from Kafka s | Per-message processing time varies | Head-of-line blocking | Independent processing | | Ordered processing required per partition | Yes | No (delivery order not guaranteed) | -Use share group ingestion when throughput and elastic scaling matter more than strict per-partition ordering. +Choose share groups when throughput and elastic scaling matter more than strict per-partition ordering. ## Task spec -Submit a `ShareGroupIndexTask` to the Overlord. Unlike standard Kafka ingestion, there are no start/end offsets -- the broker manages offset tracking. +Submit a `ShareGroupIndexTask` to the Overlord. There are no start/end offsets -- the broker tracks them. ```json { @@ -92,13 +90,13 @@ Submit a `ShareGroupIndexTask` to the Overlord. Unlike standard Kafka ingestion, |----------|------|----------|---------|-------------| | `topic` | String | Yes | -- | Kafka topic to consume from. | | `groupId` | String | Yes | -- | Share group identifier. Multiple tasks with the same `groupId` share the workload. | -| `consumerProperties` | Map | Yes | -- | Kafka consumer properties. Must include `bootstrap.servers`. See [Consumer property restrictions](#consumer-property-restrictions) below. | +| `consumerProperties` | Map | Yes | -- | Kafka consumer properties. Must include `bootstrap.servers`. See [Consumer property restrictions](#consumer-property-restrictions). | | `inputFormat` | Object | Yes | -- | Input format for parsing records (json, csv, avro, etc.). | | `pollTimeout` | Long | No | 2000 | Poll timeout in milliseconds. | ### Consumer property restrictions -Kafka share consumers (KIP-932) reject a fixed set of consumer properties that are valid for regular consumer groups. To avoid broker-side `ConfigException` at task startup, Druid silently strips the following keys from `consumerProperties` (with a `WARN` log line per stripped key) before constructing the `KafkaShareConsumer`: +Share consumers (KIP-932) reject some keys that are valid for regular consumer groups. Druid silently strips the keys below from `consumerProperties` (with a `WARN` log per stripped key) before constructing the `KafkaShareConsumer`: | Stripped key | Why | |--------------|-----| @@ -113,51 +111,51 @@ Kafka share consumers (KIP-932) reject a fixed set of consumer properties that a | `group.protocol` | Always `SHARE` for share consumers. | | `group.remote.assignor` | Not applicable to share groups. | -The internal property `share.acknowledgement.mode=explicit` is set automatically and must not be overridden. +`share.acknowledgement.mode=explicit` is set automatically and must not be overridden. ### Tuning configuration -`tuningConfig` accepts the standard `KafkaTuningConfig` fields (e.g. `maxRowsPerSegment`, `maxRowsInMemory`, `intermediatePersistPeriod`, etc.). Phase 1 ingestion respects: +`tuningConfig` accepts the standard `KafkaTuningConfig` fields. Phase 1 honors: - `maxRowsInMemory` / `maxBytesInMemory`: triggers a mid-batch persist when the appenderator signals `isPersistRequired`. -- `maxRowsPerSegment`: when reached during a batch the runner logs the event; the over-threshold segments are pushed at the end-of-batch publish boundary. +- `maxRowsPerSegment`: when reached during a batch the runner logs the event; over-threshold segments are pushed at the end-of-batch publish boundary. -Phase 1 does not yet support mid-batch checkpoint/sequence rollover; that is a Phase 2 enhancement. +Mid-batch checkpoint and sequence rollover are deferred to Phase 2. ## How it works -1. The task subscribes to the topic using a `KafkaShareConsumer` with the configured `groupId`. +1. The task subscribes to the topic with a `KafkaShareConsumer` using the configured `groupId`. 2. The broker delivers batches of records with per-record acquisition locks. -3. For each polled record the task uses the same multi-row parser that powers `KafkaIndexTask` (`StreamChunkReader`). A single Kafka record may produce zero (e.g. tombstones), one, or many `InputRow`s (e.g. JSON arrays). All resulting rows are added to the appenderator before the record is acknowledged. -4. Parse failures are routed through `ParseExceptionHandler` (so `maxParseExceptions` etc. are honored). Bytes/processed/unparseable/thrown-away counters are incremented exactly once per row. -5. Segments are persisted (mid-batch on memory pressure, end-of-batch unconditionally) and then published atomically to the metadata store via `SegmentTransactionalAppendAction`. -6. After successful publish, the task acknowledges every offset polled in the batch with `ACCEPT` and calls `commitSync()` to commit acknowledgements to the broker. -7. On task failure or graceful stop before publish, unacknowledged records are redelivered by the broker to another consumer in the share group after the acquisition lock expires. +3. Each polled record is parsed by `StreamChunkReader` (the same multi-row parser as `KafkaIndexTask`); a record may produce zero, one, or many `InputRow`s. All resulting rows are added to the appenderator before the record is acknowledged. +4. Parse failures go through `ParseExceptionHandler` (so `maxParseExceptions` is honored). Bytes/processed/unparseable counters are incremented exactly once per row. +5. Segments persist mid-batch on memory pressure and unconditionally at end-of-batch, then publish atomically via `SegmentTransactionalAppendAction`. +6. After a successful publish, every offset in the batch is acknowledged with `ACCEPT` and a `commitSync()` flushes acknowledgements to the broker. +7. On task failure or graceful stop before publish, unacknowledged records are redelivered by the broker after the acquisition lock expires. ## Safety invariants -1. **ACK after publish:** records are acknowledged with `ACCEPT` only after the segment containing them is atomically registered in the metadata store. No data loss on task failure. -2. **Multi-row safe:** every row produced from a record (including all elements of a JSON-array record) is added to the appenderator before the record is acknowledged. -3. **Resource safe:** `Appenderator` and `KafkaShareConsumer` are released on every exit path, including graceful stop and exceptions. -4. **Terminal state:** every polled record reaches exactly one terminal state: `ACCEPT` (processed), `RELEASE` (redelivered), or task crash (broker redelivers after the acquisition lock expires). +1. **ACK after publish:** `ACCEPT` is sent only after the segment is registered in the metadata store. No data loss on task failure. +2. **Multi-row safe:** every row produced from a record is added to the appenderator before that record is acknowledged. +3. **Resource safe:** `Appenderator` and `KafkaShareConsumer` are released on every exit path. +4. **Terminal state:** every polled record reaches exactly one terminal state -- `ACCEPT`, `RELEASE`, or broker redelivery after lock expiry. ## Graceful stop -When the Overlord asks a task to stop (rolling restart, supervisor reconfiguration, etc.), the task forwards the request to the runner which calls `KafkaShareConsumer.wakeup()`. The in-flight `poll()` throws `WakeupException`; the runner observes that `stopRequested` is set and exits the loop after committing any in-flight batch. Records polled but not yet published remain unacknowledged and are redelivered by the broker after the acquisition lock expires. +When the Overlord asks a task to stop, the runner calls `KafkaShareConsumer.wakeup()`. The in-flight `poll()` throws `WakeupException`; the runner exits the loop after committing any in-flight batch. Records polled but not yet published remain unacknowledged and are redelivered by the broker after the acquisition lock expires. ## Acquisition lock duration -The broker controls the per-record acquisition lock duration via `group.share.record.lock.duration.ms`. The runner logs the broker-effective value once after the first poll, e.g.: +The broker controls the lock via `group.share.record.lock.duration.ms`. The runner logs the effective value once after the first poll: ``` Effective broker acquisition lock timeout for share-group[my-group]: 30000 ms ``` -In Phase 1 the foreground ingestion thread does both poll and publish. If a single batch takes longer than the broker lock duration, in-flight records may be redelivered to another consumer (resulting in duplicates). To stay well under the lock duration, tune `pollTimeout`, `maxRowsInMemory`, and `maxRowsPerSegment` so each poll-publish cycle completes comfortably within the broker lock window. Background lock renewal via the `RENEW` acknowledge type is a Phase 2 enhancement. +In Phase 1 a single thread does both poll and publish. If a batch exceeds the lock duration, in-flight records may be redelivered (duplicates). Tune `pollTimeout`, `maxRowsInMemory`, and `maxRowsPerSegment` so each cycle stays well under the lock window. Background renewal via `RENEW` is a Phase 2 enhancement. ## Scaling -Multiple tasks with the same `groupId` distribute the workload automatically. Unlike consumer groups, you can run more tasks than partitions: +Tasks with the same `groupId` share the workload automatically; you can run more tasks than partitions: ``` Topic: 4 partitions @@ -165,11 +163,11 @@ Tasks with same groupId: 20 Result: All 20 tasks actively consuming (broker distributes records) ``` -Adding or removing tasks does not trigger a rebalancing pause. New tasks begin consuming immediately. +Adding or removing tasks does not trigger a rebalancing pause. ## Delivery semantics -Share group ingestion provides **at-least-once** delivery. On task failure, records between the last committed acknowledgement and the failure point are redelivered. Duplicate records may be ingested across task restarts. A deduplication cache is planned for a future release. +At-least-once. On task failure, records between the last committed acknowledgement and the failure point are redelivered, which may produce duplicates across restarts. A deduplication cache is planned. ## Metrics @@ -177,15 +175,15 @@ In addition to the standard ingestion metrics (`ingest/events/processed`, `inges | Metric | Description | |--------|-------------| -| `ingest/shareGroup/commitFailures` | Per-batch count of partitions whose `commitSync()` failed. A non-zero value implies the affected records will be redelivered by the broker. Operators should alert on sustained non-zero values. | +| `ingest/shareGroup/commitFailures` | Per-batch count of partitions whose `commitSync()` failed. A non-zero value means the affected records will be redelivered; alert on sustained non-zero values. | ## Limitations (current release) -- Single-threaded ingestion per task. Two-thread architecture with background lock renewal (using `AcknowledgeType.RENEW`) is a Phase 2 enhancement. -- No supervisor integration. Tasks must be submitted manually via the Overlord API. A `KafkaShareGroupSupervisor` is planned for Phase 2. -- No deduplication cache. Records redelivered after task failure or graceful stop before publish may produce duplicates (at-least-once). +- Single-threaded ingestion per task. Two-thread architecture with background `RENEW` is a Phase 2 enhancement. +- No supervisor integration; tasks are submitted manually via the Overlord API. A `KafkaShareGroupSupervisor` is planned for Phase 2. +- No deduplication cache (at-least-once). - Delivery order within a partition is not guaranteed. -- Mid-batch checkpoint / sequence rollover is not supported. If a single batch grossly exceeds `maxRowsPerSegment`, the runner still publishes correctly (multiple segments per batch), but the per-segment row threshold is only checked at end-of-batch boundaries. +- Mid-batch checkpoint / sequence rollover is not supported. If a batch grossly exceeds `maxRowsPerSegment` the runner still publishes correctly (multiple segments per batch), but the threshold is only checked at end-of-batch boundaries. ## Demo: end-to-end validation with Druid UI @@ -203,7 +201,6 @@ cd kafka_2.13-4.2.0 KAFKA_CLUSTER_ID="$(bin/kafka-storage.sh random-uuid)" bin/kafka-storage.sh format --standalone -t $KAFKA_CLUSTER_ID -c config/server.properties -# Enable share groups echo "group.share.enable=true" >> config/server.properties echo "group.share.record.lock.duration.ms=30000" >> config/server.properties @@ -233,26 +230,22 @@ Paste these JSON records: ### Step 3: Build the extension and set up Druid ```bash -# Build the kafka-indexing-service extension cd /path/to/druid JAVA_HOME=$(/usr/libexec/java_home -v 17) mvn package \ -pl extensions-core/kafka-indexing-service -am \ -Pskip-static-checks -Dmaven.test.skip=true -T1C -q -# Download and extract Druid release cd ~/Downloads curl -O https://dlcdn.apache.org/druid/31.0.0/apache-druid-31.0.0-bin.tar.gz tar -xzf apache-druid-31.0.0-bin.tar.gz cd apache-druid-31.0.0 -# Replace the kafka extension with our build rm extensions/druid-kafka-indexing-service/*.jar cp /path/to/druid/extensions-core/kafka-indexing-service/target/druid-kafka-indexing-service-*.jar \ extensions/druid-kafka-indexing-service/ cp ~/.m2/repository/org/apache/kafka/kafka-clients/4.2.0/kafka-clients-4.2.0.jar \ extensions/druid-kafka-indexing-service/ -# Start Druid bin/start-druid ``` @@ -305,7 +298,7 @@ org.apache.druid.indexing.kafka.ShareGroupConsumerPropertiesTest" \ -Pskip-static-checks -Dweb.console.skip=true -T1C ``` -E2E test (requires Docker running -- Testcontainers starts a Kafka 4.1.1 broker with share groups enabled automatically): +E2E test (requires Docker; Testcontainers starts an `apache/kafka:4.1.1` broker with `group.share.enable=true`): ```bash mvn test -pl embedded-tests -am \ @@ -313,5 +306,3 @@ mvn test -pl embedded-tests -am \ -Dsurefire.failIfNoSpecifiedTests=false \ -Pskip-static-checks -Dweb.console.skip=true -T1C ``` - -The E2E test uses `ShareGroupKafkaResource` which starts an `apache/kafka:4.1.1` container with `group.share.enable=true`. No manual Kafka setup is needed. diff --git a/website/.spelling b/website/.spelling index 28701817f362..cf6abf244670 100644 --- a/website/.spelling +++ b/website/.spelling @@ -2646,3 +2646,15 @@ nginx - ../docs/development/extensions-core/s3.md NIO + +- ../docs/ingestion/kafka-share-group-ingestion.md +KIP +KIP-932 +appenderator +isPersistRequired +ACK +rebalancing +deduplication +Testcontainers +30-60s +E2E From 461b50b27a2593ec4dcbd94721b0258327da6cba Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Fri, 8 May 2026 13:16:33 +0530 Subject: [PATCH 33/49] Wait for segments before SQL in share-group IT --- .../embedded/indexing/EmbeddedShareGroupIngestionTest.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java index 4956b3b1118e..25969c87e99c 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java @@ -120,6 +120,7 @@ public void test_shareGroupIngestion_basicEndToEnd() throws InterruptedException ); waitForRowsProcessed(numRecords); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); Assertions.assertEquals( String.valueOf(numRecords), @@ -154,6 +155,7 @@ public void test_shareGroupIngestion_multiRowPerKafkaRecord() throws Interrupted final long expectedRows = (long) numRecords * rowsPerRecord; waitForRowsProcessed(expectedRows); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); Assertions.assertEquals( String.valueOf(expectedRows), @@ -186,6 +188,7 @@ public void test_shareGroupIngestion_unsupportedConsumerProperty_isSanitized() t ); waitForRowsProcessed(numRecords); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); Assertions.assertEquals( String.valueOf(numRecords), @@ -220,6 +223,7 @@ public void test_shareGroupIngestion_gracefulStop_publishesInflightBatch() throw waitForRowsProcessed(numRecords); cancelAndAwaitTermination(taskId); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); Assertions.assertEquals( String.valueOf(numRecords), @@ -257,6 +261,7 @@ public void test_shareGroupIngestion_multiTaskShareGroup_noDuplicatesNoLoss() th cancelAndAwaitTermination(taskA); cancelAndAwaitTermination(taskB); cancelAndAwaitTermination(taskC); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); Assertions.assertEquals( String.valueOf(numRecords), @@ -289,6 +294,7 @@ public void test_shareGroupIngestion_lowMaxRowsInMemory_dataCorrect() throws Int waitForRowsProcessed(numRecords); cancelAndAwaitTermination(taskId); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); Assertions.assertEquals( String.valueOf(numRecords), From 1226944f10bbd9e77f81017ee07af41ceb97afdb Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Fri, 8 May 2026 13:42:03 +0530 Subject: [PATCH 34/49] Retry SQL row count to absorb broker catalog refresh lag --- .../EmbeddedShareGroupIngestionTest.java | 72 +++++++++++-------- 1 file changed, 42 insertions(+), 30 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java index 25969c87e99c..89730a6ef1c2 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java @@ -121,11 +121,7 @@ public void test_shareGroupIngestion_basicEndToEnd() throws InterruptedException waitForRowsProcessed(numRecords); cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); - - Assertions.assertEquals( - String.valueOf(numRecords), - cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) - ); + assertRowCountEventually(numRecords); cancelAndAwaitTermination(taskId); } @@ -156,11 +152,7 @@ public void test_shareGroupIngestion_multiRowPerKafkaRecord() throws Interrupted final long expectedRows = (long) numRecords * rowsPerRecord; waitForRowsProcessed(expectedRows); cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); - - Assertions.assertEquals( - String.valueOf(expectedRows), - cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) - ); + assertRowCountEventually(expectedRows); cancelAndAwaitTermination(taskId); } @@ -189,11 +181,7 @@ public void test_shareGroupIngestion_unsupportedConsumerProperty_isSanitized() t waitForRowsProcessed(numRecords); cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); - - Assertions.assertEquals( - String.valueOf(numRecords), - cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) - ); + assertRowCountEventually(numRecords); cancelAndAwaitTermination(taskId); } @@ -224,11 +212,7 @@ public void test_shareGroupIngestion_gracefulStop_publishesInflightBatch() throw cancelAndAwaitTermination(taskId); cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); - - Assertions.assertEquals( - String.valueOf(numRecords), - cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) - ); + assertRowCountEventually(numRecords); } /** Three tasks sharing one group on a 2-partition topic ingest every record exactly once (KIP-932). */ @@ -262,11 +246,7 @@ public void test_shareGroupIngestion_multiTaskShareGroup_noDuplicatesNoLoss() th cancelAndAwaitTermination(taskB); cancelAndAwaitTermination(taskC); cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); - - Assertions.assertEquals( - String.valueOf(numRecords), - cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) - ); + assertRowCountEventually(numRecords); } /** 1000 rows with {@code maxRowsInMemory=100} forces mid-batch persists yet ingests all rows. */ @@ -295,11 +275,7 @@ public void test_shareGroupIngestion_lowMaxRowsInMemory_dataCorrect() throws Int cancelAndAwaitTermination(taskId); cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); - - Assertions.assertEquals( - String.valueOf(numRecords), - cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) - ); + assertRowCountEventually(numRecords); } private DataSchema buildDataSchema() @@ -389,6 +365,42 @@ private void waitForRowsProcessed(long expected) ); } + /** + * Polls SQL for the expected row count, swallowing transient + * "datasource not found" errors that can arise immediately after a task + * cancel before the broker's SQL catalog has refreshed. + */ + private void assertRowCountEventually(long expected) + { + final String expectedStr = String.valueOf(expected); + final long deadlineMillis = System.currentTimeMillis() + 30_000L; + String last = null; + Exception lastException = null; + while (System.currentTimeMillis() < deadlineMillis) { + try { + last = cluster.runSql("SELECT COUNT(*) FROM %s", dataSource); + if (expectedStr.equals(last)) { + return; + } + lastException = null; + } + catch (Exception e) { + lastException = e; + } + try { + Thread.sleep(250L); + } + catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + break; + } + } + Assertions.fail( + "Expected row count [" + expectedStr + "] but last result was [" + last + + "], lastException=" + lastException + ); + } + /** Builds a property map containing keys forbidden by {@code SHARE_GROUP_UNSUPPORTED_CONFIGS}. */ private Map consumerPropsWithUnsupportedKeys() { From 52dfecc0c5883f994a21ee2194f7886410a11d51 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Fri, 8 May 2026 23:30:48 +0530 Subject: [PATCH 35/49] fix: checkstyle violations in share-group sources and tests --- .../EmbeddedShareGroupIngestionTest.java | 68 ++++++++++++++++--- .../indexing/ShareGroupProbeTest.java | 2 +- .../kafka/KafkaShareGroupRecordSupplier.java | 1 - .../kafka/ShareGroupIndexTaskRunner.java | 9 ++- .../ShareGroupIndexTaskIOConfigTest.java | 1 - .../kafka/ShareGroupIndexTaskRunnerTest.java | 25 ++++++- .../kafka/ShareGroupIndexTaskTest.java | 24 ++++++- 7 files changed, 108 insertions(+), 22 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java index 89730a6ef1c2..e417fd1cf8b1 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java @@ -224,13 +224,25 @@ public void test_shareGroupIngestion_multiTaskShareGroup_noDuplicatesNoLoss() th final String groupId = "druid-share-group-fanout"; final String taskA = submitShareGroupTask( - topic, groupId, kafkaServer.consumerProperties(), DEFAULT_CSV_FORMAT, defaultTuningConfig() + topic, + groupId, + kafkaServer.consumerProperties(), + DEFAULT_CSV_FORMAT, + defaultTuningConfig() ); final String taskB = submitShareGroupTask( - topic, groupId, kafkaServer.consumerProperties(), DEFAULT_CSV_FORMAT, defaultTuningConfig() + topic, + groupId, + kafkaServer.consumerProperties(), + DEFAULT_CSV_FORMAT, + defaultTuningConfig() ); final String taskC = submitShareGroupTask( - topic, groupId, kafkaServer.consumerProperties(), DEFAULT_CSV_FORMAT, defaultTuningConfig() + topic, + groupId, + kafkaServer.consumerProperties(), + DEFAULT_CSV_FORMAT, + defaultTuningConfig() ); Thread.sleep(SHARE_CONSUMER_READY_DELAY_MS); @@ -295,18 +307,56 @@ private DataSchema buildDataSchema() private static KafkaIndexTaskTuningConfig defaultTuningConfig() { return new KafkaIndexTaskTuningConfig( - null, null, null, null, null, null, null, null, null, null, - null, null, null, null, null, null, null, null, null, null, - null, null + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null ); } private static KafkaIndexTaskTuningConfig tuningConfigWithMaxRowsInMemory(int maxRowsInMemory) { return new KafkaIndexTaskTuningConfig( - null, maxRowsInMemory, null, null, null, null, null, null, null, null, - null, null, null, null, null, null, null, null, null, null, - null, null + null, + maxRowsInMemory, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null ); } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupProbeTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupProbeTest.java index 010f9ca1f788..b0a3277bb84d 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupProbeTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupProbeTest.java @@ -27,9 +27,9 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaShareConsumer; -import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.junit.jupiter.api.AfterEach; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java index a88f593950b0..14a3afa245a9 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java @@ -47,7 +47,6 @@ import java.util.Optional; import java.util.Properties; import java.util.Set; -import java.util.stream.Collectors; /** * Adapts {@link KafkaShareConsumer} to {@link AcknowledgingRecordSupplier}. diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java index a2c45d83209f..b4fe2df8da5a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunner.java @@ -20,7 +20,10 @@ package org.apache.druid.indexing.kafka; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.config.Configs; import org.apache.druid.data.input.Committer; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; @@ -34,10 +37,10 @@ import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.TaskLocks; import org.apache.druid.indexing.common.actions.LockReleaseAction; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; +import org.apache.druid.indexing.common.actions.TaskLocks; import org.apache.druid.indexing.common.task.InputRowFilter; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.input.InputRowSchemas; @@ -47,7 +50,6 @@ import org.apache.druid.indexing.seekablestream.common.AcknowledgeType; import org.apache.druid.indexing.seekablestream.common.AcknowledgingRecordSupplier; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; -import org.apache.druid.common.config.Configs; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.SegmentSchemaMapping; @@ -62,7 +64,6 @@ import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedPartialShardSpec; - import org.apache.kafka.common.errors.WakeupException; import javax.annotation.Nullable; @@ -76,8 +77,6 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Supplier; /** * Single-threaded ingestion loop for {@link ShareGroupIndexTask}. Records are diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskIOConfigTest.java index 4f3ac94bc65a..94e6c518ea77 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskIOConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskIOConfigTest.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.kafka; -import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.collect.ImmutableMap; diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunnerTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunnerTest.java index 35571f9606a8..6ba7690db62e 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunnerTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskRunnerTest.java @@ -64,9 +64,28 @@ public void setUp() null ); final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig( - null, null, null, null, null, null, null, null, null, null, - null, null, null, null, null, null, null, null, null, null, - null, null + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null ); task = new ShareGroupIndexTask( "task_runner_test", diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskTest.java index 74746b3d7f2c..08f56ca079f0 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskTest.java @@ -148,8 +148,28 @@ private ShareGroupIndexTask createTask(String id) ); final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig( - null, null, null, null, null, null, null, null, null, null, - null, null, null, null, null, null, null, null, null, null + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null ); return new ShareGroupIndexTask( From 0b53ae7273cb59bbdc696b4ae1f4463541fcdfa0 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Fri, 8 May 2026 23:40:40 +0530 Subject: [PATCH 36/49] fix: forbidden-API violations in share-group code --- .../indexing/ShareGroupProbeTest.java | 7 +++--- .../kafka/ShareGroupConsumerProperties.java | 4 ++-- .../KafkaShareGroupRecordSupplierTest.java | 22 +++++++++---------- 3 files changed, 17 insertions(+), 16 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupProbeTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupProbeTest.java index b0a3277bb84d..53160f780a81 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupProbeTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/ShareGroupProbeTest.java @@ -37,6 +37,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.HashMap; import java.util.List; @@ -99,7 +100,7 @@ public void probe_mimicDruid_pollLoop100ms_areConsumed() throws Exception final java.util.ArrayList> records = new java.util.ArrayList<>(); for (int i = 0; i < 10; i++) { - records.add(new org.apache.kafka.clients.producer.ProducerRecord<>(topic, i % 2, null, ("v-" + i).getBytes())); + records.add(new org.apache.kafka.clients.producer.ProducerRecord<>(topic, i % 2, null, ("v-" + i).getBytes(StandardCharsets.UTF_8))); } kafkaServer.produceRecordsToTopic(records); } @@ -153,7 +154,7 @@ public void probe_noWarmup_transactionalProducer_areConsumed() throws Exception final java.util.ArrayList> records = new java.util.ArrayList<>(); for (int i = 0; i < 10; i++) { - records.add(new org.apache.kafka.clients.producer.ProducerRecord<>(topic, i % 2, null, ("value-" + i).getBytes())); + records.add(new org.apache.kafka.clients.producer.ProducerRecord<>(topic, i % 2, null, ("value-" + i).getBytes(StandardCharsets.UTF_8))); } kafkaServer.produceRecordsToTopic(records); @@ -225,7 +226,7 @@ public void probe_recordsProducedAfterSubscribe_areConsumed() throws Exception try (KafkaProducer producer = new KafkaProducer<>(producerProps)) { for (int i = 0; i < 10; i++) { - producer.send(new ProducerRecord<>(topic, i % 2, null, ("value-" + i).getBytes())); + producer.send(new ProducerRecord<>(topic, i % 2, null, ("value-" + i).getBytes(StandardCharsets.UTF_8))); } producer.flush(); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupConsumerProperties.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupConsumerProperties.java index 0ab719292ee1..ba9516b0d373 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupConsumerProperties.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupConsumerProperties.java @@ -21,8 +21,8 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.utils.CollectionUtils; -import java.util.LinkedHashMap; import java.util.Map; import java.util.Set; @@ -61,7 +61,7 @@ private ShareGroupConsumerProperties() */ public static Map sanitize(Map consumerProperties) { - final Map sanitized = new LinkedHashMap<>(consumerProperties.size()); + final Map sanitized = CollectionUtils.newLinkedHashMapWithExpectedSize(consumerProperties.size()); for (Map.Entry entry : consumerProperties.entrySet()) { if (UNSUPPORTED_CONFIGS.contains(entry.getKey())) { log.warn( diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java index 68f451cef78d..9bdafc10c513 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java @@ -94,10 +94,10 @@ public void testUnsubscribe() public void testPollWrapsRecords() { final ConsumerRecord record1 = new ConsumerRecord<>( - "test-topic", 0, 100L, "key1".getBytes(), "value1".getBytes() + "test-topic", 0, 100L, "key1".getBytes(StandardCharsets.UTF_8), "value1".getBytes(StandardCharsets.UTF_8) ); final ConsumerRecord record2 = new ConsumerRecord<>( - "test-topic", 1, 200L, "key2".getBytes(), "value2".getBytes() + "test-topic", 1, 200L, "key2".getBytes(StandardCharsets.UTF_8), "value2".getBytes(StandardCharsets.UTF_8) ); final Map>> recordMap = new HashMap<>(); @@ -158,7 +158,7 @@ public void testPollReturnsEmptyOnTimeout() public void testAcknowledgeDefaultAccept() { final ConsumerRecord record = new ConsumerRecord<>( - "test-topic", 0, 42L, "key".getBytes(), "value".getBytes() + "test-topic", 0, 42L, "key".getBytes(StandardCharsets.UTF_8), "value".getBytes(StandardCharsets.UTF_8) ); pollSingleRecord(record); @@ -175,7 +175,7 @@ public void testAcknowledgeDefaultAccept() public void testAcknowledgeWithRelease() { final ConsumerRecord record = new ConsumerRecord<>( - "test-topic", 0, 10L, "key".getBytes(), "value".getBytes() + "test-topic", 0, 10L, "key".getBytes(StandardCharsets.UTF_8), "value".getBytes(StandardCharsets.UTF_8) ); pollSingleRecord(record); @@ -192,7 +192,7 @@ public void testAcknowledgeWithRelease() public void testAcknowledgeWithReject() { final ConsumerRecord record = new ConsumerRecord<>( - "test-topic", 0, 10L, "key".getBytes(), "value".getBytes() + "test-topic", 0, 10L, "key".getBytes(StandardCharsets.UTF_8), "value".getBytes(StandardCharsets.UTF_8) ); pollSingleRecord(record); @@ -209,7 +209,7 @@ public void testAcknowledgeWithReject() public void testAcknowledgeWithRenew() { final ConsumerRecord record = new ConsumerRecord<>( - "test-topic", 0, 99L, "key".getBytes(), "value".getBytes() + "test-topic", 0, 99L, "key".getBytes(StandardCharsets.UTF_8), "value".getBytes(StandardCharsets.UTF_8) ); pollSingleRecord(record); @@ -226,13 +226,13 @@ public void testAcknowledgeWithRenew() public void testAcknowledgeBatch() { final List> recordsP0 = Arrays.asList( - new ConsumerRecord<>("test-topic", 0, 1L, "k".getBytes(), "v".getBytes()), - new ConsumerRecord<>("test-topic", 0, 2L, "k".getBytes(), "v".getBytes()), - new ConsumerRecord<>("test-topic", 0, 3L, "k".getBytes(), "v".getBytes()) + new ConsumerRecord<>("test-topic", 0, 1L, "k".getBytes(StandardCharsets.UTF_8), "v".getBytes(StandardCharsets.UTF_8)), + new ConsumerRecord<>("test-topic", 0, 2L, "k".getBytes(StandardCharsets.UTF_8), "v".getBytes(StandardCharsets.UTF_8)), + new ConsumerRecord<>("test-topic", 0, 3L, "k".getBytes(StandardCharsets.UTF_8), "v".getBytes(StandardCharsets.UTF_8)) ); final List> recordsP1 = Arrays.asList( - new ConsumerRecord<>("test-topic", 1, 10L, "k".getBytes(), "v".getBytes()), - new ConsumerRecord<>("test-topic", 1, 11L, "k".getBytes(), "v".getBytes()) + new ConsumerRecord<>("test-topic", 1, 10L, "k".getBytes(StandardCharsets.UTF_8), "v".getBytes(StandardCharsets.UTF_8)), + new ConsumerRecord<>("test-topic", 1, 11L, "k".getBytes(StandardCharsets.UTF_8), "v".getBytes(StandardCharsets.UTF_8)) ); final Map>> recordMap = new HashMap<>(); recordMap.put(new TopicPartition("test-topic", 0), recordsP0); From d2882c0413f66d97fd0b431406f7cefce7760133 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Sat, 9 May 2026 16:19:16 +0530 Subject: [PATCH 37/49] ShareGroupIndexTask: use realtime task priority (75) --- .../indexing/kafka/ShareGroupIndexTask.java | 6 +++++ .../kafka/ShareGroupIndexTaskTest.java | 25 ++++++++++++++++++- 2 files changed, 30 insertions(+), 1 deletion(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java index 42ce7833785e..ee0f9657cd92 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupIndexTask.java @@ -161,6 +161,12 @@ public String getTaskAllocatorId() return getTaskResource().getAvailabilityGroup(); } + @Override + public int getPriority() + { + return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_REALTIME_TASK_PRIORITY); + } + boolean isStopRequested() { return stopRequested.get(); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskTest.java index 08f56ca079f0..7120742f3855 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/ShareGroupIndexTaskTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.segment.indexing.DataSchema; import org.junit.Assert; @@ -127,7 +128,29 @@ public void testInputSourceResources() Assert.assertFalse(task.getInputSourceResources().isEmpty()); } + @Test + public void testDefaultPriorityIsRealtime() + { + final ShareGroupIndexTask task = createTask("task_priority_default"); + Assert.assertEquals(Tasks.DEFAULT_REALTIME_TASK_PRIORITY, task.getPriority()); + } + + @Test + public void testContextPriorityOverridesDefault() + { + final ShareGroupIndexTask task = createTask( + "task_priority_override", + ImmutableMap.of(Tasks.PRIORITY_KEY, 99) + ); + Assert.assertEquals(99, task.getPriority()); + } + private ShareGroupIndexTask createTask(String id) + { + return createTask(id, null); + } + + private ShareGroupIndexTask createTask(String id, Map context) { final DataSchema dataSchema = DataSchema.builder() .withDataSource("test_datasource") @@ -178,7 +201,7 @@ private ShareGroupIndexTask createTask(String id) dataSchema, tuningConfig, ioConfig, - null, + context, mapper ); } From 1a8bed204dfd345b72af9f16967eb04d8b5f9e7d Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Wed, 13 May 2026 23:31:26 +0530 Subject: [PATCH 38/49] chore(test): use ${apache.kafka.version} for kafka-clients in embedded-tests --- embedded-tests/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/embedded-tests/pom.xml b/embedded-tests/pom.xml index 8747bf9332da..da985f21856f 100644 --- a/embedded-tests/pom.xml +++ b/embedded-tests/pom.xml @@ -47,7 +47,7 @@ org.apache.kafka kafka-clients - 4.2.0 + ${apache.kafka.version} test From 244ffde6d27b7b6d3806544fb07547900fac67b8 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Wed, 13 May 2026 23:46:07 +0530 Subject: [PATCH 39/49] docs: drop Phase 1/Phase 2 phrasing in share-group ingestion doc --- docs/ingestion/kafka-share-group-ingestion.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ingestion/kafka-share-group-ingestion.md b/docs/ingestion/kafka-share-group-ingestion.md index 288f8cc0a29a..88c51fea3d5f 100644 --- a/docs/ingestion/kafka-share-group-ingestion.md +++ b/docs/ingestion/kafka-share-group-ingestion.md @@ -151,7 +151,7 @@ The broker controls the lock via `group.share.record.lock.duration.ms`. The runn Effective broker acquisition lock timeout for share-group[my-group]: 30000 ms ``` -In Phase 1 a single thread does both poll and publish. If a batch exceeds the lock duration, in-flight records may be redelivered (duplicates). Tune `pollTimeout`, `maxRowsInMemory`, and `maxRowsPerSegment` so each cycle stays well under the lock window. Background renewal via `RENEW` is a Phase 2 enhancement. +A single thread does both poll and publish. If a batch exceeds the lock duration, in-flight records may be redelivered (duplicates). Tune `pollTimeout`, `maxRowsInMemory`, and `maxRowsPerSegment` so each cycle stays well under the lock window. ## Scaling @@ -179,7 +179,7 @@ In addition to the standard ingestion metrics (`ingest/events/processed`, `inges ## Limitations (current release) -- Single-threaded ingestion per task. Two-thread architecture with background `RENEW` is a Phase 2 enhancement. +- Single-threaded ingestion per task; a future enhancement may add a background `RENEW` thread to extend the broker lock for long-running batches. - No supervisor integration; tasks are submitted manually via the Overlord API. A `KafkaShareGroupSupervisor` is planned for Phase 2. - No deduplication cache (at-least-once). - Delivery order within a partition is not guaranteed. From 6a25c782eef663f8f0075323bcccdd4c7fbd29e9 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Wed, 13 May 2026 23:47:21 +0530 Subject: [PATCH 40/49] docs: drop remaining Phase 1/Phase 2 phrasing in tuningConfig section --- docs/ingestion/kafka-share-group-ingestion.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ingestion/kafka-share-group-ingestion.md b/docs/ingestion/kafka-share-group-ingestion.md index 88c51fea3d5f..79f80fdd1641 100644 --- a/docs/ingestion/kafka-share-group-ingestion.md +++ b/docs/ingestion/kafka-share-group-ingestion.md @@ -180,7 +180,7 @@ In addition to the standard ingestion metrics (`ingest/events/processed`, `inges ## Limitations (current release) - Single-threaded ingestion per task; a future enhancement may add a background `RENEW` thread to extend the broker lock for long-running batches. -- No supervisor integration; tasks are submitted manually via the Overlord API. A `KafkaShareGroupSupervisor` is planned for Phase 2. +- No supervisor integration; tasks are submitted manually via the Overlord API. A `KafkaShareGroupSupervisor` is planned as a future enhancement. - No deduplication cache (at-least-once). - Delivery order within a partition is not guaranteed. - Mid-batch checkpoint / sequence rollover is not supported. If a batch grossly exceeds `maxRowsPerSegment` the runner still publishes correctly (multiple segments per batch), but the threshold is only checked at end-of-batch boundaries. From 5231b7c60dc0dc05093647968b875d5a9fe4df7b Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Wed, 13 May 2026 23:47:51 +0530 Subject: [PATCH 41/49] docs: drop Phase 1 phrasing in tuningConfig intro --- docs/ingestion/kafka-share-group-ingestion.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ingestion/kafka-share-group-ingestion.md b/docs/ingestion/kafka-share-group-ingestion.md index 79f80fdd1641..8676a1171620 100644 --- a/docs/ingestion/kafka-share-group-ingestion.md +++ b/docs/ingestion/kafka-share-group-ingestion.md @@ -115,12 +115,12 @@ Share consumers (KIP-932) reject some keys that are valid for regular consumer g ### Tuning configuration -`tuningConfig` accepts the standard `KafkaTuningConfig` fields. Phase 1 honors: +`tuningConfig` accepts the standard `KafkaTuningConfig` fields. The runner currently honors: - `maxRowsInMemory` / `maxBytesInMemory`: triggers a mid-batch persist when the appenderator signals `isPersistRequired`. - `maxRowsPerSegment`: when reached during a batch the runner logs the event; over-threshold segments are pushed at the end-of-batch publish boundary. -Mid-batch checkpoint and sequence rollover are deferred to Phase 2. +Mid-batch checkpoint and sequence rollover are not supported. ## How it works From 40d5ed7760b8bb68d793c07f682e21e89320d7c2 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Wed, 13 May 2026 23:50:07 +0530 Subject: [PATCH 42/49] docs(share-group): replace download steps with source build for the demo --- docs/ingestion/kafka-share-group-ingestion.md | 27 ++++++++----------- 1 file changed, 11 insertions(+), 16 deletions(-) diff --git a/docs/ingestion/kafka-share-group-ingestion.md b/docs/ingestion/kafka-share-group-ingestion.md index 8676a1171620..b5857662e24f 100644 --- a/docs/ingestion/kafka-share-group-ingestion.md +++ b/docs/ingestion/kafka-share-group-ingestion.md @@ -191,7 +191,7 @@ In addition to the standard ingestion metrics (`ingest/events/processed`, `inges - Java 17 - Kafka 4.2.0 (with share groups enabled) -- Druid 31.0.0 release (downloaded) +- Druid checked out from this repository (built from source) ### Step 1: Start Kafka with share groups @@ -227,24 +227,19 @@ Paste these JSON records: {"__time":"2025-06-01T04:00:00.000Z","item":"widget_e","value":320,"category":"electronics"} ``` -### Step 3: Build the extension and set up Druid +### Step 3: Build Druid from source + +Build a full Druid distribution from this repository. This packages the share-group code natively, so no JAR overlay is required: ```bash cd /path/to/druid -JAVA_HOME=$(/usr/libexec/java_home -v 17) mvn package \ - -pl extensions-core/kafka-indexing-service -am \ - -Pskip-static-checks -Dmaven.test.skip=true -T1C -q - -cd ~/Downloads -curl -O https://dlcdn.apache.org/druid/31.0.0/apache-druid-31.0.0-bin.tar.gz -tar -xzf apache-druid-31.0.0-bin.tar.gz -cd apache-druid-31.0.0 - -rm extensions/druid-kafka-indexing-service/*.jar -cp /path/to/druid/extensions-core/kafka-indexing-service/target/druid-kafka-indexing-service-*.jar \ - extensions/druid-kafka-indexing-service/ -cp ~/.m2/repository/org/apache/kafka/kafka-clients/4.2.0/kafka-clients-4.2.0.jar \ - extensions/druid-kafka-indexing-service/ +JAVA_HOME=$(/usr/libexec/java_home -v 17) \ + mvn clean install -Pdist -T1C -DskipTests \ + -Dforbiddenapis.skip=true -Dcheckstyle.skip=true \ + -Dpmd.skip=true -Dmaven.javadoc.skip=true -Denforcer.skip=true + +tar -xzf distribution/target/apache-druid-*-bin.tar.gz -C /tmp +cd /tmp/apache-druid-* bin/start-druid ``` From 626b468e0e49b868190beafdfa67b859c4cee543 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Thu, 14 May 2026 11:20:55 +0530 Subject: [PATCH 43/49] docs(share-group): offer source-build (recommended) and JAR-overlay options --- docs/ingestion/kafka-share-group-ingestion.md | 38 ++++++++++++++++++- 1 file changed, 36 insertions(+), 2 deletions(-) diff --git a/docs/ingestion/kafka-share-group-ingestion.md b/docs/ingestion/kafka-share-group-ingestion.md index b5857662e24f..2e44f26774ab 100644 --- a/docs/ingestion/kafka-share-group-ingestion.md +++ b/docs/ingestion/kafka-share-group-ingestion.md @@ -227,9 +227,13 @@ Paste these JSON records: {"__time":"2025-06-01T04:00:00.000Z","item":"widget_e","value":320,"category":"electronics"} ``` -### Step 3: Build Druid from source +### Step 3: Build Druid and run it -Build a full Druid distribution from this repository. This packages the share-group code natively, so no JAR overlay is required: +You can run the demo against either a freshly built Druid distribution or an existing stable Druid binary with the share-group JARs overlaid. Pick the one that matches your environment. + +#### Option A: Build the full Druid distribution from source (recommended) + +Builds the full distribution from this repository so the share-group code is packaged natively, with no JAR overlay required: ```bash cd /path/to/druid @@ -244,6 +248,36 @@ cd /tmp/apache-druid-* bin/start-druid ``` +> Tip: For faster iteration, build only the `kafka-indexing-service` module with `mvn package -pl extensions-core/kafka-indexing-service -am -DskipTests -T1C` and overlay the resulting JAR onto the distribution from a previous full build (Option B steps below). + +#### Option B: Overlay the share-group JAR onto a downloaded Druid binary (faster, best-effort) + +If you already have a Druid release binary and want to avoid a full source build, you can replace the kafka-indexing-service JAR in that distribution with the one built from this branch. + +> Caveat: The branch builds against `38.0.0-SNAPSHOT`. Druid does **not** guarantee extension ABI compatibility across major versions, so the overlay may fail at runtime against an older binary (37.x or earlier). Use the most recent stable Druid release available, and prefer Option A for a reliable demo. + +```bash +cd /path/to/druid +JAVA_HOME=$(/usr/libexec/java_home -v 17) mvn package \ + -pl extensions-core/kafka-indexing-service -am \ + -Pskip-static-checks -DskipTests -T1C -q + +# Use the latest stable Druid release available; 37.0.0 is the example below. +DRUID_VERSION=37.0.0 +cd /tmp +curl -O "https://dlcdn.apache.org/druid/${DRUID_VERSION}/apache-druid-${DRUID_VERSION}-bin.tar.gz" +tar -xzf "apache-druid-${DRUID_VERSION}-bin.tar.gz" +cd "apache-druid-${DRUID_VERSION}" + +rm extensions/druid-kafka-indexing-service/*.jar +cp /path/to/druid/extensions-core/kafka-indexing-service/target/druid-kafka-indexing-service-*.jar \ + extensions/druid-kafka-indexing-service/ +cp ~/.m2/repository/org/apache/kafka/kafka-clients/4.2.0/kafka-clients-4.2.0.jar \ + extensions/druid-kafka-indexing-service/ + +bin/start-druid +``` + ### Step 4: Submit task via Druid console Open `http://localhost:8888`, go to the **Ingestion** tab, click **Submit JSON task**, and paste: From f182aaeaa1d9c5ddd0a4e0cda93fba65ccfbd895 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Thu, 14 May 2026 15:29:16 +0530 Subject: [PATCH 44/49] fix(checkstyle): drop unused mockito imports and move static block to bottom --- .../kafka/KafkaShareGroupRecordSupplierTest.java | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java index 9bdafc10c513..c8ae6ebbbea4 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java @@ -34,24 +34,22 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.mockito.ArgumentCaptor; import org.mockito.Mockito; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + public class KafkaShareGroupRecordSupplierTest { private KafkaShareConsumer mockConsumer; From a25b7e78adb5fc6e0bcebd4cfad934a157352d68 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Thu, 14 May 2026 21:25:42 +0530 Subject: [PATCH 45/49] fix(forbidden-api): pass StandardCharsets.UTF_8 to String.getBytes in share-group test --- .../druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java index c8ae6ebbbea4..58a5214f7d29 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java @@ -130,7 +130,7 @@ public void testPollWrapsRecords() public void testPollWithNullValue() { final ConsumerRecord record = new ConsumerRecord<>( - "test-topic", 0, 50L, "key".getBytes(), null + "test-topic", 0, 50L, "key".getBytes(StandardCharsets.UTF_8), null ); final Map>> recordMap = new HashMap<>(); recordMap.put(new TopicPartition("test-topic", 0), List.of(record)); From 539f3c8751eb43964cbbb03f99d67993a3ad977b Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Thu, 14 May 2026 21:50:06 +0530 Subject: [PATCH 46/49] fix(share-group): re-sanitise consumer props after dynamic-config provider expansion --- .../kafka/KafkaShareGroupRecordSupplier.java | 1 + .../kafka/ShareGroupConsumerProperties.java | 19 +++++++++++++++++++ 2 files changed, 20 insertions(+) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java index 14a3afa245a9..a6054bd5c061 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplier.java @@ -251,6 +251,7 @@ private static KafkaShareConsumer createShareConsumer( final Properties props = new Properties(); KafkaRecordSupplier.addConsumerPropertiesFromConfig(props, sortingMapper, sanitized); + ShareGroupConsumerProperties.sanitize(props); props.setProperty("group.id", groupId); props.setProperty("share.acknowledgement.mode", "explicit"); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupConsumerProperties.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupConsumerProperties.java index ba9516b0d373..cb149171745d 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupConsumerProperties.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/ShareGroupConsumerProperties.java @@ -24,6 +24,7 @@ import org.apache.druid.utils.CollectionUtils; import java.util.Map; +import java.util.Properties; import java.util.Set; /** @@ -75,4 +76,22 @@ public static Map sanitize(Map consumerPropertie } return sanitized; } + + /** + * Removes unsupported keys from a fully-resolved {@link Properties} object in place. + * Use this after expanding {@code druid.dynamic.config.provider} so that values + * pulled from a secrets vault cannot reintroduce a forbidden key. + */ + public static void sanitize(Properties properties) + { + for (String key : UNSUPPORTED_CONFIGS) { + if (properties.remove(key) != null) { + log.warn( + "Stripping unsupported consumer property [%s] for share-group consumer " + + "(see Kafka ShareConsumerConfig).", + key + ); + } + } + } } From 8cbbd708051607fa4f737ef3c175856199383889 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Thu, 14 May 2026 21:50:06 +0530 Subject: [PATCH 47/49] docs(share-group): set share.auto.offset.reset=earliest before producing demo records --- docs/ingestion/kafka-share-group-ingestion.md | 21 +++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/docs/ingestion/kafka-share-group-ingestion.md b/docs/ingestion/kafka-share-group-ingestion.md index 2e44f26774ab..58f1030006a9 100644 --- a/docs/ingestion/kafka-share-group-ingestion.md +++ b/docs/ingestion/kafka-share-group-ingestion.md @@ -207,13 +207,26 @@ echo "group.share.record.lock.duration.ms=30000" >> config/server.properties bin/kafka-server-start.sh config/server.properties ``` -### Step 2: Create topic and produce messages +### Step 2: Create topic and configure the share group ```bash cd kafka_2.13-4.2.0 bin/kafka-topics.sh --create --topic druid-share-test --partitions 4 --bootstrap-server localhost:9092 +# Set share-group reset to earliest so the task picks up records that already exist +# in the topic. The default broker setting is 'latest', which would skip pre-existing +# records and ingest zero rows even though the producer ran successfully. +bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter \ + --entity-type groups --entity-name druid-demo-share-group \ + --add-config share.auto.offset.reset=earliest +``` + +### Step 3: Produce sample messages + +```bash +cd kafka_2.13-4.2.0 + bin/kafka-console-producer.sh --topic druid-share-test --bootstrap-server localhost:9092 ``` @@ -227,7 +240,7 @@ Paste these JSON records: {"__time":"2025-06-01T04:00:00.000Z","item":"widget_e","value":320,"category":"electronics"} ``` -### Step 3: Build Druid and run it +### Step 4: Build Druid and run it You can run the demo against either a freshly built Druid distribution or an existing stable Druid binary with the share-group JARs overlaid. Pick the one that matches your environment. @@ -278,7 +291,7 @@ cp ~/.m2/repository/org/apache/kafka/kafka-clients/4.2.0/kafka-clients-4.2.0.jar bin/start-druid ``` -### Step 4: Submit task via Druid console +### Step 5: Submit task via Druid console Open `http://localhost:8888`, go to the **Ingestion** tab, click **Submit JSON task**, and paste: @@ -303,7 +316,7 @@ Open `http://localhost:8888`, go to the **Ingestion** tab, click **Submit JSON t } ``` -### Step 5: Query data +### Step 6: Query data Go to the **Query** tab and run: From bff13441c94eb93a3e9b4f6f76718c4e163abbdb Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Thu, 14 May 2026 21:57:56 +0530 Subject: [PATCH 48/49] docs(share-group): backtick ABI and 37.x to satisfy mdspell --- docs/ingestion/kafka-share-group-ingestion.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ingestion/kafka-share-group-ingestion.md b/docs/ingestion/kafka-share-group-ingestion.md index 58f1030006a9..4a75a1a59223 100644 --- a/docs/ingestion/kafka-share-group-ingestion.md +++ b/docs/ingestion/kafka-share-group-ingestion.md @@ -267,7 +267,7 @@ bin/start-druid If you already have a Druid release binary and want to avoid a full source build, you can replace the kafka-indexing-service JAR in that distribution with the one built from this branch. -> Caveat: The branch builds against `38.0.0-SNAPSHOT`. Druid does **not** guarantee extension ABI compatibility across major versions, so the overlay may fail at runtime against an older binary (37.x or earlier). Use the most recent stable Druid release available, and prefer Option A for a reliable demo. +> Caveat: The branch builds against `38.0.0-SNAPSHOT`. Druid does **not** guarantee extension `ABI` compatibility across major versions, so the overlay may fail at runtime against an older binary (`37.x` or earlier). Use the most recent stable Druid release available, and prefer Option A for a reliable demo. ```bash cd /path/to/druid From 2d094d5fbe6e11b6d22b1df8bf54c8cf7b7d03d2 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Thu, 14 May 2026 23:41:46 +0530 Subject: [PATCH 49/49] test(share-group): align assertions with Optional topic and non-null OrderedPartitionableRecord data --- .../indexing/kafka/KafkaShareGroupRecordSupplierTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java index 58a5214f7d29..5f1bd6608a22 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaShareGroupRecordSupplierTest.java @@ -140,7 +140,7 @@ public void testPollWithNullValue() supplier.poll(1000); Assert.assertEquals(1, result.size()); - Assert.assertNull(result.get(0).getData()); + Assert.assertTrue(result.get(0).getData().isEmpty()); } @Test @@ -276,7 +276,7 @@ public void testCommitSync() Assert.assertEquals(1, result.size()); final Map.Entry> entry = result.entrySet().iterator().next(); - Assert.assertEquals("test-topic", entry.getKey().topic()); + Assert.assertEquals(Optional.of("test-topic"), entry.getKey().topic()); Assert.assertEquals(0, entry.getKey().partition()); Assert.assertFalse(entry.getValue().isPresent()); }