Skip to content

feat: [Phase 1] Queue Semantics support in Kafka Ingestion#19311

Open
Shekharrajak wants to merge 72 commits into
apache:masterfrom
Shekharrajak:feature/kafka-share-group-ingestion
Open

feat: [Phase 1] Queue Semantics support in Kafka Ingestion#19311
Shekharrajak wants to merge 72 commits into
apache:masterfrom
Shekharrajak:feature/kafka-share-group-ingestion

Conversation

@Shekharrajak
Copy link
Copy Markdown
Contributor

@Shekharrajak Shekharrajak commented Apr 14, 2026

Phase 1 #18439.

Description

Fixed the bug ...

Renamed the class ...

Added a forbidden-apis entry ...

Release note


Key changed/added classes in this PR
  • MyFoo
  • OurBar
  • TheirBaz

This PR has:

  • been self-reviewed.
  • added documentation for new or modified features or behaviors.
  • a release note entry in the PR description.
  • added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
  • added or updated version, license, or notice information in licenses.yaml
  • added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
  • added unit tests or modified existing tests to cover new code paths, ensuring the threshold for code coverage is met.
  • added integration tests.
  • been tested in a test Druid cluster.

Demo

https://youtu.be/K_O1MH-AaE8

cd apache-druid-31.0.0

# Replace the kafka extension with our build
rm extensions/druid-kafka-indexing-service/*.jar
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can build from the source and use the tar

agg -> agg.hasSumAtLeast(numRecords)
);

Assertions.assertEquals(
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Making sure we read all the records.

@Shekharrajak Shekharrajak force-pushed the feature/kafka-share-group-ingestion branch from d31199b to 4a30bee Compare May 8, 2026 06:17
@Shekharrajak
Copy link
Copy Markdown
Contributor Author

Spending time on Integeration testing embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java

@Shekharrajak Shekharrajak changed the title [DRAFT] Queue Semantics support in Kafka Ingestion feat: [Phase 1 - Draft] Queue Semantics support in Kafka Ingestion May 8, 2026
@Shekharrajak
Copy link
Copy Markdown
Contributor Author

For smaller blast radius and clean separation - parking the suggestion #18439 (comment) , Also KafkaShareGroupSupervisor (planned) will be structurally different from KafkaSupervisor — no partition-to-task assignment, no offset bookkeeping, seek/rest .

}

@Test
public void test_shareGroupIngestion_basicEndToEnd() throws InterruptedException
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Subscribe to topic with KafkaShareConsumer (groupId), poll → parse (CSV) → persist → publish → acknowledge(ACCEPT) → commitSync → broker offset persistence. SQL COUNT(*) == 10.

final int numRecords = 5;
final int rowsPerRecord = 4;
kafkaServer.produceRecordsToTopic(
generateMultiObjectJsonRecords(topic, numRecords, rowsPerRecord, DateTimes.of("2025-07-01"))
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

StreamChunkReader multi-row path One Kafka record carrying 4 whitespace-separated JSON objects produces 4 rows. SQL count = numRecords × rowsPerRecord (5 × 4 = 20). Verifies no row is dropped before ACK.

@Shekharrajak
Copy link
Copy Markdown
Contributor Author

To make sure no data loss we are ack only after the publish completed https://github.com/apache/druid/pull/19311/changes#diff-487176908fb14d87486db6f683c9ddb34465f3a1572f5c01b11d7b9c6a482289R381

Although this will be slow and no concurrency but I think for the intial phase this is fine and we can improve the performance step by step.

WDYT @gianm @a2l007 @clintropolis @FrankChen021 ?

@Shekharrajak
Copy link
Copy Markdown
Contributor Author

CI checks are failing due to one flaky test unrelated to this PR : #19435

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));
final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> 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));
*/
Map<PartitionIdType, Optional<Exception>> commitSync();

Set<PartitionIdType> getPartitionIds(String stream);
@Shekharrajak Shekharrajak force-pushed the feature/kafka-share-group-ingestion branch from 234a8ec to e5894ef Compare May 9, 2026 05:36
Copy link
Copy Markdown
Member

@FrankChen021 FrankChen021 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Severity Findings
P0 0
P1 0
P2 2
P3 0
Total 2

Reviewed 22 of 22 changed files. Findings focus on task scheduling semantics and Kafka compatibility test coverage.


This is an automated review by Codex GPT-5

<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<excludes>
<exclude>**/KafkaIndexTaskTest.java</exclude>
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[P2] Kafka compatibility tests are skipped globally

This Surefire exclusion removes the main KafkaIndexTask, KafkaRecordSupplier, KafkaSamplerSpec, and KafkaSupervisor test classes from every normal test run. Since this PR also upgrades kafka-clients to 4.2.0, those are the regression tests most likely to catch compatibility breaks in existing index_kafka ingestion; the new share-group mock and embedded tests do not cover that existing API surface. Please migrate or narrowly isolate the TestBroker-dependent cases instead of excluding the whole classes.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, planning to fix (upgrade to 4.2 version in all places) as part of this #19322 - I feel that we need bunch of changes, API compatibility issues in running tests , kafka docker image for IT tests and in source code - so want to keep it in separate PR.

This PR mainly focus on Share group implementation.

@Shekharrajak
Copy link
Copy Markdown
Contributor Author

Struggling with CI check related to license :


Unsupported license: MPL 1.1
For:org.javassist javassist in: druid-kerberos
org.locationtech.jts jts-core in: parquet-extensions with: Eclipse Public License 2.0 ignoring Eclipse Distribution License - v 1.0
Error:  Command execution failed.
org.apache.commons.exec.ExecuteException: Process exited with an error: 1 (Exit value: 1)
    at org.apache.commons.exec.DefaultExecutor.executeInternal (DefaultExecutor.java:394)
    at org.apache.commons.exec.DefaultExecutor.execute (DefaultExecutor.java:292)
    at org.codehaus.mojo.exec.ExecMojo.executeCommandLine (ExecMojo.java:903)
    at org.codehaus.mojo.exec.ExecMojo.executeCommandLine (ExecMojo.java:863)
    at org.codehaus.mojo.exec.ExecMojo.execute (ExecMojo.java:454)
    at org.apache.maven.plugin.DefaultBuildPluginManager.executeMojo (DefaultBuildPluginManager.java:126)
    at org.apache.maven.lifecycle.internal.MojoExecutor.doExecute2 (MojoExecutor.java:328)
    at org.apache.maven.lifecycle.internal.MojoExecutor.doExecute (MojoExecutor.java:316)
    at org.apache.maven.lifecycle.internal.MojoExecutor.execute (MojoExecutor.java:212)
    at org.apache.maven.lifecycle.internal.MojoExecutor.execute (MojoExecutor.java:174)
    at org.apache.maven.lifecycle.internal.MojoExecutor.access$000 (MojoExecutor.java:75)
    at org.apache.maven.lifecycle.internal.MojoExecutor$1.run (MojoExecutor.java:162)
    at org.apache.maven.plugin.DefaultMojosExecutionStrategy.execute (DefaultMojosExecutionStrategy.java:39)
    at org.apache.maven.lifecycle.internal.MojoExecutor.execute (MojoExecutor.java:159)
    at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject (LifecycleModuleBuilder.java:105)
    at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject (LifecycleModuleBuilder.java:73)
    at org.apache.maven.lifecycle.internal.builder.singlethreaded.SingleThreadedBuilder.build (SingleThreadedBuilder.java:53)
    at org.apache.maven.lifecycle.internal.LifecycleStarter.execute (LifecycleStarter.java:118)
    at org.apache.maven.DefaultMaven.doExecute (DefaultMaven.java:261)
[INFO] simple-client-sslcontext ........................... SUCCESS [  5.195 s]
[INFO] druid-basic-security ............................... SUCCESS [  9.326 s]
[INFO] druid-google-extensions ............................ SUCCESS [  7.697 s]
[INFO] druid-catalog ...................................... SUCCESS [  9.519 s]
[INFO] druid-testing-tools ................................ SUCCESS [  8.122 s]
[INFO] druid-testcontainers ............................... SUCCESS [  2.149 s]
[INFO] druid-influx-extensions ............................ SUCCESS [  4.579 s]
[INFO] druid-cassandra-storage ............................ SUCCESS [  3.987 s]
[INFO] dropwizard-emitter ................................. SUCCESS [  6.528 s]
[INFO] druid-cloudfiles-extensions ........................ SUCCESS [  5.857 s]
[INFO] druid-consul-extensions ............................ SUCCESS [  6.595 s]
[INFO] graphite-emitter ................................... SUCCESS [  6.085 s]
[INFO] druid-distinctcount ................................ SUCCESS [  4.219 s]
[INFO] druid-exact-count-bitmap ........................... SUCCESS [  7.275 s]
[INFO] statsd-emitter ..................................... SUCCESS [  6.282 s]
[INFO] druid-time-min-max ................................. SUCCESS [  5.725 s]
[INFO] druid-virtual-columns .............................. SUCCESS [  4.011 s]
[INFO] druid-thrift-extensions ............................ SUCCESS [  4.915 s]
[INFO] ambari-metrics-emitter ............................. SUCCESS [  6.306 s]
[INFO] sqlserver-metadata-storage ......................... SUCCESS [  6.266 s]
[INFO] kafka-emitter ...................................... SUCCESS [  6.186 s]
[INFO] druid-redis-cache .................................. SUCCESS [  7.346 s]
[INFO] druid-opentsdb-emitter ............................. SUCCESS [  5.524 s]
[INFO] druid-momentsketch ................................. SUCCESS [  6.063 s]
[INFO] druid-moving-average-query ......................... SUCCESS [  7.560 s]
[INFO] tdigestsketch ...................................... SUCCESS [  7.308 s]
[INFO] druid-ddsketch ..................................... SUCCESS [  7.400 s]
[INFO] druid-influxdb-emitter ............................. SUCCESS [  4.017 s]
[INFO] gce-extensions ..................................... SUCCESS [  6.621 s]
[INFO] aliyun-oss-extensions .............................. SUCCESS [  7.980 s]
[INFO] prometheus-emitter ................................. SUCCESS [  5.877 s]
[INFO] opentelemetry-emitter .............................. SUCCESS [  8.167 s]
[INFO] grpc-query ......................................... SUCCESS [ 14.755 s]
[INFO] druid-iceberg-extensions ........................... SUCCESS [  9.575 s]
[INFO] druid-deltalake-extensions ......................... SUCCESS [  6.163 s]
[INFO] druid-spectator-histogram .......................... SUCCESS [  8.500 s]
[INFO] druid-rabbit-indexing-service ...................... SUCCESS [  7.905 s]
[INFO] druid-ranger-security .............................. SUCCESS [  9.363 s]
[INFO] distribution ....................................... FAILURE [05:29 min]
[INFO] druid-quidem-ut .................................... SKIPPED
[INFO] druid-embedded-tests ............................... SKIPPED
[INFO] ------------------------------------------------------------------------
[INFO] BUILD FAILURE
[INFO] ------------------------------------------------------------------------
[INFO] Total time:  22:30 min
[INFO] Finished at: 2026-05-09T05:59:40Z
[INFO] ------------------------------------------------------------------------
Error:  Failed to execute goal org.codehaus.mojo:exec-maven-plugin:3.5.1:exec (check-licenses) on project distribution: Command execution failed. Process exited with an error: 1 (Exit value: 1) -> [Help 1]
Error:  
Error:  To see the full stack trace of the errors, re-run Maven with the -e switch.
Error:  Re-run Maven using the -X switch to enable full debug logging.
Error:  
Error:  For more information about the errors and possible solutions, please read the following articles:
Error:  [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoExecutionException
Error:  
Error:  After correcting the problems, you can resume the build with the command
Error:    mvn <args> -rf :distribution
Error: Process completed with exit code 1.

@Shekharrajak Shekharrajak force-pushed the feature/kafka-share-group-ingestion branch 2 times, most recently from e3a9ad7 to 7865104 Compare May 9, 2026 06:57
Copy link
Copy Markdown
Member

@FrankChen021 FrankChen021 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed 3 of 27 changed files. Follow-up review handled: the Kafka extension POM no longer has the global Surefire exclusions, and ShareGroupIndexTask now uses DEFAULT_REALTIME_TASK_PRIORITY with tests covering the default and context override cases.


This is an automated review by Codex GPT-5.5

@Shekharrajak Shekharrajak force-pushed the feature/kafka-share-group-ingestion branch from ede5072 to e159405 Compare May 13, 2026 17:45
Copy link
Copy Markdown
Member

@FrankChen021 FrankChen021 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Severity Findings
P0 0
P1 0
P2 2
P3 1
Total 3
Severity Findings
P0 0
P1 0
P2 2
P3 1
Total 3

Reviewed 23 of 23 changed files.


This is an automated review by Codex GPT-5.5

TaskToolbox toolbox
) throws Exception
{
recordSupplier.subscribe(Collections.singleton(ioConfig.getTopic()));
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[P2] Close the appenderator when setup fails

After this refactor, appenderator.closeNow() only runs in the try/finally that starts after recordSupplier.subscribe(...) and driver.startJob(...). If supplier creation, subscribe, or startJob throws, run() only calls driver.close(), and BaseAppenderatorDriver.close explicitly does not close the underlying Appenderator, so failed startup can leak realtime appenderator resources. Keep the appenderator close guard in run() or include setup in runLoop's try/finally.

Assertions.assertEquals(task.getId(), payloadResponse.getTask());

final JsonNode payloadJson = mapper.valueToTree(payloadResponse.getPayload());
Assertions.assertEquals("share_group_index", payloadJson.path("type").asText());
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[P2] Fix JSON-submit payload expectations

This new IT serializes a real ShareGroupIndexTask and reads the Overlord payload, but the task is registered as index_kafka_share_group and serializes ioConfig at the top level, not under spec. The round-trip test therefore fails on the type assertion, and still fails on the topic/group assertions after that is fixed; the malformed-json test below also uses the wrong type/shape, so it does not exercise the intended missing-field path.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

);
cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker);

final long deadlineMs = System.currentTimeMillis() + 30_000L;
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[P3] Fail when the row-count wait expires

If verifySqlQuery never observes the expected count, this loop exits when the deadline expires and the test continues to cancel the task without failing. A broken JSON-submit ingestion/query path can pass after 30 seconds. Track the last assertion and fail after the deadline, or reuse the existing assertRowCountEventually pattern.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.


/**
* Verifies that the Kafka broker automatically rebalances new partitions to share-group consumers
* without any Druid-side intervention (no supervisor, no task restart).
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

without any Druid-side intervention (no supervisor, no task restart) - Main advantages of using share group - no stop the world case.


/**
* Verifies at-least-once semantics: uncommitted records are redelivered after Indexer crash.
*/
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Validation of no data loss in crash situation.

Copy link
Copy Markdown
Member

@FrankChen021 FrankChen021 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed all 26 changed files in this follow-up. The JSON-submit payload expectation and row-count timeout threads are addressed by the current changes, and I did not find any new high-confidence PR-caused issues.


This is an automated review by Codex GPT-5.5

kafkaServer.publishRecordsToTopic(topic, csvRecords(batchA, 0, "2025-09-01"));

// Task 1: starts ingesting but Indexer crashes before it completes.
final String taskId1 = submitTask(topic);
@Shekharrajak
Copy link
Copy Markdown
Contributor Author

fixing one unit test :

mbedded-tests/target/test-classes/org/apache/druid/testing/embedded/indexing/ShareGroupIndexTaskJsonSubmitIT.class:96 | org.apache.druid.rpc.HttpResponseException: Server error [400 Bad Request]; body: {"error":"Please make sure to load all the necessary extensions and jars with type 'ShareGroupIndexTask'. Could not resolve type id 'ShareGroupIndexTask' as a subtype of `org.apache.druid.indexing.common.task.Task` known type ids = [archive, compact, index, index_hadoop, index_kafka, index_kafka_share_group, index_parallel, index_sub, kill, move, noop, partial_dimension_cardinality, partial_dimension_distribution, partial_index_generate, partial_index_generic_merge, partial_range_index_generate, query_controller, query_worker, restore, single_phase_sub_task]"} 0.153s
  ℹ️ Posting with conclusion 'failure' to https://github.com/apache/druid/pull/19311 (sha: ad1f11ac54ba82b18a9bee9c1d37e4a853b2e36b)
🚀 Publish results
  ℹ️ - Test Report - test-jdk25-[S*] - 38737 tests run, 38719 passed, 13 skipped, 5 failed.
     🧪 - embedded-tests/target/test-classes/org/apache/druid/testing/embedded/indexing/ShareGroupPartitionRebalancingIT.class | Expected at least [20] rows but got [10] ==> expected: <true> but was: <false>
     🧪 - embedded-tests/target/test-classes/org/apache/druid/testing/embedded/indexing/ShareGroupPartitionRebalancingIT.class | Expected at least [30] rows but got [10] ==> expected: <true> but was: <false>
     🧪 - embedded-tests/target/test-classes/org/apache/druid/testing/embedded/indexing/ShareGroupIndexerCrashIT.class | Timed out waiting for event after [60,000]ms
     🧪 - embedded-tests/target/test-classes/org/apache/druid/testing/embedded/indexing/ShareGroupIndexTaskJsonSubmitIT.class | org.apache.druid.rpc.HttpResponseException: Server error [400 Bad Request]; body: {"error":"Please make sure to load all the necessary extensions and jars with type 'ShareGroupIndexTask'. Could not resolve type id 'ShareGroupIndexTask' as a subtype of `org.apache.druid.indexing.common.task.Task` known type ids = [archive, compact, index, index_hadoop, index_kafka, index_kafka_share_group, index_parallel, index_sub, kill, move, noop, partial_dimension_cardinality, partial_dimension_distribution, partial_index_generate, partial_index_generic_merge, partial_range_index_generate, query_controller, query_worker, restore, single_phase_sub_task]"}

Copy link
Copy Markdown
Member

@FrankChen021 FrankChen021 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Severity Findings
P0 0
P1 0
P2 1
P3 0
Total 1

Reviewed 26 of 26 changed files.


This is an automated review by Codex GPT-5.5

Thread.sleep(SHARE_CONSUMER_READY_DELAY_MS);

// Give Task 2 time to consume redelivered records before cancel.
Thread.sleep(15_000L);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[P2] Crash tests no longer prove redelivery

Replacing the post-restart waitForEventAggregate checks with fixed sleeps means these crash tests no longer assert that Task 2 consumed any redelivered records. Task 1 is allowed to run for 3 seconds before indexer.stop(), and this runner publishes and acks every batch, so it can already make the final SQL row count pass; in the first test the new batch can also satisfy the count. A regression that fails to redeliver unacknowledged records could now slip through. Please wait on a post-restart ingest signal or otherwise assert rows were processed by the restarted task before cancelling it.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated 94cbff5

};

stub.wakeup();
Assert.assertFalse(stub.acquisitionLockTimeoutMs().isPresent());
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just to have test coverage CI check pass.

);
final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> recordMap = new HashMap<>();
recordMap.put(new TopicPartition(testTopic, 0), List.of(record));
when(mockConsumer.poll(any(Duration.class))).thenReturn(new ConsumerRecords<>(recordMap));
Copy link
Copy Markdown
Member

@FrankChen021 FrankChen021 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Handled the ShareGroupIndexerCrashIT follow-up. The latest update replaces fixed post-restart sleeps with processed-event baselines/waits before cancelling Task 2, so I don't have a further inline reply.

Reviewed 1 of 26 changed files.


This is an automated review by Codex GPT-5.5

final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> recordMap = new HashMap<>();
recordMap.put(new TopicPartition(testTopic, 0), List.of(r0));
recordMap.put(new TopicPartition(testTopic, 1), List.of(r1));
when(mockConsumer.poll(any(Duration.class))).thenReturn(new ConsumerRecords<>(recordMap));
Copy link
Copy Markdown
Member

@FrankChen021 FrankChen021 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Severity Findings
P0 0
P1 1
P2 0
P3 0
Total 1
Severity Findings
P0 0
P1 1
P2 0
P3 0
Total 1

Reviewed 26 of 26 changed files.


This is an automated review by Codex GPT-5.5

for (InputRow row : rows) {
final AppenderatorDriverAddResult addResult = driver.add(
row,
SEQUENCE_NAME,
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[P1] Use per-task sequence names for segment allocation

Every ShareGroupIndexTaskRunner uses the same hard-coded sequence name when adding rows. Druid segment allocation is idempotent for the same sequenceName, and this call also skips lineage checks, so concurrent share-group tasks ingesting the same datasource interval can be handed the same pending segment id. If one task publishes that segment first, another task can treat the id as already published and then acknowledge its Kafka records even though its rows were not registered. Make the allocation sequence unique per task or otherwise isolate concurrent producers before supporting multiple tasks in one share group.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants