abhishekrb19 commented on code in PR #18870:
URL: https://github.com/apache/druid/pull/18870#discussion_r2658373270
##########
extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java:
##########
@@ -30,29 +32,30 @@
public class KafkaSupervisorReportPayload extends
SeekableStreamSupervisorReportPayload<KafkaTopicPartition, Long>
{
+ @JsonCreator
public KafkaSupervisorReportPayload(
- String id,
- String dataSource,
- String topic,
- int partitions,
- int replicas,
- long durationSeconds,
- @Nullable Map<KafkaTopicPartition, Long> latestOffsets,
- @Nullable Map<KafkaTopicPartition, Long> minimumLag,
- @Nullable Map<KafkaTopicPartition, Long> minimumLagMillis,
- @Nullable Long aggregateLag,
- @Nullable DateTime offsetsLastUpdated,
- boolean suspended,
- boolean healthy,
- SupervisorStateManager.State state,
- SupervisorStateManager.State detailedState,
- List<SupervisorStateManager.ExceptionEvent> recentErrors
+ @JsonProperty("id") String id,
+ @JsonProperty("dataSource") String dataSource,
+ @JsonProperty("stream") String stream,
+ @JsonProperty("partitions") int partitions,
+ @JsonProperty("replicas") int replicas,
+ @JsonProperty("durationSeconds") long durationSeconds,
Review Comment:
I suppose these fields will always be present, but I had thought that
Jackson serde required Boxed types and doesn't work well with primitive types.
I couldn't find examples of it in our code base either...
##########
embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaFaultToleranceTest.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.core.type.TypeReference;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.DruidMetrics;
+import org.apache.druid.rpc.RequestBuilder;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.List;
+import java.util.Map;
+
+public class KafkaFaultToleranceTest extends KafkaTestBase
+{
+ private SupervisorSpec supervisorSpec = null;
+ private String topic = null;
+ private int totalRecords = 0;
+
+ @BeforeEach
+ public void setupTopicAndSupervisor()
+ {
+ totalRecords = 0;
+ topic = "topic_" + dataSource;
+ kafkaServer.createTopicWithPartitions(topic, 2);
+
+ supervisorSpec = createSupervisor().withId("supe_" +
dataSource).build(dataSource, topic);
+ cluster.callApi().postSupervisor(supervisorSpec);
+ }
+
+ @AfterEach
+ public void verifyAndTearDown()
+ {
+ waitUntilPublishedRecordsAreIngested(totalRecords);
+ verifySupervisorIsRunningHealthy(supervisorSpec.getId());
+ cluster.callApi().postSupervisor(supervisorSpec.createSuspendedSpec());
+ kafkaServer.deleteTopic(topic);
+ verifyRowCount(totalRecords);
+ }
+
+ @ParameterizedTest
+ @ValueSource(booleans = {true, false})
+ public void test_supervisorRecovers_afterOverlordRestart(boolean
useTransactions) throws Exception
+ {
+ totalRecords = publish1kRecords(topic, useTransactions);
+ waitUntilPublishedRecordsAreIngested(totalRecords);
+
+ overlord.stop();
+ totalRecords += publish1kRecords(topic, useTransactions);
+
+ overlord.start();
+ totalRecords += publish1kRecords(topic, useTransactions);
+ }
+
+ @Test
+ public void test_supervisorRecovers_afterCoordinatorRestart() throws
Exception
+ {
+ final boolean useTransactions = true;
+ totalRecords = publish1kRecords(topic, useTransactions);
+ waitUntilPublishedRecordsAreIngested(totalRecords);
+
+ coordinator.stop();
+ totalRecords += publish1kRecords(topic, useTransactions);
+
+ coordinator.start();
+ totalRecords += publish1kRecords(topic, useTransactions);
+ }
+
+ @Test
+ public void test_supervisorRecovers_afterHistoricalRestart() throws Exception
+ {
+ final boolean useTransactions = false;
+ totalRecords = publish1kRecords(topic, useTransactions);
+ waitUntilPublishedRecordsAreIngested(totalRecords);
+
+ historical.stop();
+ totalRecords += publish1kRecords(topic, useTransactions);
+
+ historical.start();
+ totalRecords += publish1kRecords(topic, useTransactions);
+ }
+
+ @ParameterizedTest
+ @ValueSource(booleans = {true, false})
+ public void test_supervisorRecovers_afterSuspendResume(boolean
useTransactions)
+ {
+ totalRecords = publish1kRecords(topic, useTransactions);
+ waitUntilPublishedRecordsAreIngested(totalRecords);
+
+ cluster.callApi().postSupervisor(supervisorSpec.createSuspendedSpec());
+ totalRecords += publish1kRecords(topic, useTransactions);
+
+ cluster.callApi().postSupervisor(supervisorSpec.createRunningSpec());
+ totalRecords += publish1kRecords(topic, useTransactions);
+ }
+
+ @ParameterizedTest
+ @ValueSource(booleans = {true, false})
+ public void test_supervisorRecovers_afterChangeInTopicPartitions(boolean
useTransactions)
+ {
+ totalRecords = publish1kRecords(topic, useTransactions);
+
+ kafkaServer.increasePartitionsInTopic(topic, 4);
+ totalRecords += publish1kRecords(topic, useTransactions);
+ }
+
+ @Test
+ public void test_supervisorLaunchesNewTask_ifEarlyHandoff()
+ {
+ final boolean useTransactions = true;
+ totalRecords = publish1kRecords(topic, useTransactions);
+
+ final String path = StringUtils.format(
+ "/druid/indexer/v1/supervisor/%s/taskGroups/handoff",
+ supervisorSpec.getId()
+ );
+ cluster.callApi().serviceClient().onLeaderOverlord(
+ mapper -> new RequestBuilder(HttpMethod.POST, path)
+ .jsonContent(mapper, Map.of("taskGroupIds", List.of(0, 1))),
+ new TypeReference<>() {}
+ );
+
+ // Wait for the handoff notice to be processed
+ overlord.latchableEmitter().waitForEvent(
+ event -> event.hasMetricName("ingest/notices/time")
+ .hasDimension(DruidMetrics.SUPERVISOR_ID,
supervisorSpec.getId())
+ .hasDimension("noticeType", "handoff_task_group_notice")
+ );
+
+ totalRecords += publish1kRecords(topic, useTransactions);
Review Comment:
Should this test actually verify that the old task(s) have completed
successfully on early handoff and new ones have been spun?
##########
indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java:
##########
@@ -432,7 +432,7 @@ public Response handoffTaskGroups(
manager -> {
try {
if (manager.handoffTaskGroupsEarly(id, taskGroupIds)) {
- return Response.ok().build();
+ return Response.ok(Map.of("success", true)).build();
Review Comment:
Curious why this is needed. Also, “success” and `Response.ok()` feels a bit
misleading here since this is an asynchronous handoff execution and the API
will return immediately. Would an `Accepted` response status might be more
appropriate for this?
It'll probably be
[good](https://druid.apache.org/docs/latest/api-reference/supervisor-api/#sample-response-18)
to clarify this in the docs too based on what we do here.
##########
embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaFaultToleranceTest.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.core.type.TypeReference;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.DruidMetrics;
+import org.apache.druid.rpc.RequestBuilder;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.List;
+import java.util.Map;
+
+public class KafkaFaultToleranceTest extends KafkaTestBase
+{
+ private SupervisorSpec supervisorSpec = null;
+ private String topic = null;
+ private int totalRecords = 0;
+
+ @BeforeEach
+ public void setupTopicAndSupervisor()
+ {
+ totalRecords = 0;
+ topic = "topic_" + dataSource;
+ kafkaServer.createTopicWithPartitions(topic, 2);
+
+ supervisorSpec = createSupervisor().withId("supe_" +
dataSource).build(dataSource, topic);
+ cluster.callApi().postSupervisor(supervisorSpec);
+ }
+
+ @AfterEach
+ public void verifyAndTearDown()
+ {
+ waitUntilPublishedRecordsAreIngested(totalRecords);
+ verifySupervisorIsRunningHealthy(supervisorSpec.getId());
+ cluster.callApi().postSupervisor(supervisorSpec.createSuspendedSpec());
+ kafkaServer.deleteTopic(topic);
+ verifyRowCount(totalRecords);
+ }
+
+ @ParameterizedTest
+ @ValueSource(booleans = {true, false})
+ public void test_supervisorRecovers_afterOverlordRestart(boolean
useTransactions) throws Exception
+ {
+ totalRecords = publish1kRecords(topic, useTransactions);
+ waitUntilPublishedRecordsAreIngested(totalRecords);
+
+ overlord.stop();
+ totalRecords += publish1kRecords(topic, useTransactions);
+
+ overlord.start();
+ totalRecords += publish1kRecords(topic, useTransactions);
+ }
+
+ @Test
+ public void test_supervisorRecovers_afterCoordinatorRestart() throws
Exception
+ {
+ final boolean useTransactions = true;
+ totalRecords = publish1kRecords(topic, useTransactions);
+ waitUntilPublishedRecordsAreIngested(totalRecords);
+
+ coordinator.stop();
+ totalRecords += publish1kRecords(topic, useTransactions);
+
+ coordinator.start();
+ totalRecords += publish1kRecords(topic, useTransactions);
+ }
+
+ @Test
+ public void test_supervisorRecovers_afterHistoricalRestart() throws Exception
+ {
+ final boolean useTransactions = false;
+ totalRecords = publish1kRecords(topic, useTransactions);
+ waitUntilPublishedRecordsAreIngested(totalRecords);
+
+ historical.stop();
+ totalRecords += publish1kRecords(topic, useTransactions);
+
+ historical.start();
+ totalRecords += publish1kRecords(topic, useTransactions);
+ }
+
+ @ParameterizedTest
+ @ValueSource(booleans = {true, false})
+ public void test_supervisorRecovers_afterSuspendResume(boolean
useTransactions)
+ {
+ totalRecords = publish1kRecords(topic, useTransactions);
+ waitUntilPublishedRecordsAreIngested(totalRecords);
+
+ cluster.callApi().postSupervisor(supervisorSpec.createSuspendedSpec());
+ totalRecords += publish1kRecords(topic, useTransactions);
+
+ cluster.callApi().postSupervisor(supervisorSpec.createRunningSpec());
+ totalRecords += publish1kRecords(topic, useTransactions);
+ }
+
+ @ParameterizedTest
+ @ValueSource(booleans = {true, false})
+ public void test_supervisorRecovers_afterChangeInTopicPartitions(boolean
useTransactions)
+ {
+ totalRecords = publish1kRecords(topic, useTransactions);
+
+ kafkaServer.increasePartitionsInTopic(topic, 4);
+ totalRecords += publish1kRecords(topic, useTransactions);
Review Comment:
This test failed locally for me with transactions disabled:
`org.apache.druid.java.util.common.ISE: Timed out waiting for event after
[60,000]ms`
It seems like it's stuck in some loop from these logs:
```
2026-01-02T20:18:32,119 INFO
[org.apache.druid.metadata.SqlSegmentsMetadataManager-Exec--0]
org.apache.druid.metadata.SqlSegmentsMetadataManager - Polled and found [57]
segments in the database in [0]ms.
2026-01-02T20:18:32,225 INFO
[org.apache.druid.metadata.SqlSegmentsMetadataManager-Exec--0]
org.apache.druid.metadata.SqlSegmentsMetadataManager - Polled and found [57]
segments in the database in [1]ms.
2026-01-02T20:18:32,331 INFO
[org.apache.druid.metadata.SqlSegmentsMetadataManager-Exec--0]
org.apache.druid.metadata.SqlSegmentsMetadataManager - Polled and found [57]
segments in the database in [1]ms.
2026-01-02T20:18:32,387 WARN [KafkaSupervisor-supe_datasource_nhjeoeib]
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor -
Configured task count[2] for supervisor[supe_datasource_nhjeoeib] is greater
than the number of partitions[1].
2026-01-02T20:18:32,388 INFO [KafkaSupervisor-supe_datasource_nhjeoeib]
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor -
Early stop requested for supervisor[supe_datasource_nhjeoeib], signalling tasks
to complete.
2026-01-02T20:18:32,438 INFO
[org.apache.druid.metadata.SqlSegmentsMetadataManager-Exec--0]
org.apache.druid.metadata.SqlSegmentsMetadataManager - Polled and found [57]
segments in the database in [0]ms.
2026-01-02T20:18:32,509 WARN [KafkaSupervisor-supe_datasource_nfgacpoa]
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor -
Configured task count[2] for supervisor[supe_datasource_nfgacpoa] is greater
than the number of partitions[1].
2026-01-02T20:18:32,510 INFO [KafkaSupervisor-supe_datasource_nfgacpoa]
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor -
Early stop requested for supervisor[supe_datasource_nfgacpoa], signalling tasks
to complete.
2026-01-02T20:18:32,542 INFO
[org.apache.druid.metadata.SqlSegmentsMetadataManager-Exec--0]
org.apache.druid.metadata.SqlSegmentsMetadataManager - Polled and found [57]
segments in the database in [1]ms.
2026-01-02T20:18:32,648 INFO
[org.apache.druid.metadata.SqlSegmentsMetadataManager-Exec--0]
org.apache.druid.metadata.SqlSegmentsMetadataManager - Polled and found [57]
segments in the database in [1]ms.
2026-01-02T20:18:32,754 INFO
[org.apache.druid.metadata.SqlSegmentsMetadataManager-Exec--0]
org.apache.druid.metadata.SqlSegmentsMetadataManager - Polled and found [57]
segments in the database in [1]ms.
2026-01-02T20:18:32,861 INFO
[org.apache.druid.metadata.SqlSegmentsMetadataManager-Exec--0]
org.apache.druid.metadata.SqlSegmentsMetadataManager - Polled and found [57]
segments in the database in [0]ms.
2026-01-02T20:18:32,962 INFO
[org.apache.druid.metadata.SqlSegmentsMetadataManager-Exec--0]
org.apache.druid.metadata.SqlSegmentsMetadataManager - Polled and found [57]
segments in the database in [0]ms.
2026-01-02T20:18:33,035 WARN [KafkaSupervisor-supe_datasource_lnedibpp]
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor -
Configured task count[2] for supervisor[supe_datasource_lnedibpp] is greater
than the number of partitions[1].
2026-01-02T20:18:33,035 INFO [KafkaSupervisor-supe_datasource_lnedibpp]
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor -
Early stop requested for supervisor[supe_datasource_lnedibpp], signalling tasks
to complete.
2026-01-02T20:18:33,066 INFO
[org.apache.druid.metadata.SqlSegmentsMetadataManager-Exec--0]
org.apache.druid.metadata.SqlSegmentsMetadataManager - Polled and found [57]
segments in the database in [0]ms.
2026-01-02T20:18:33,169 INFO
[org.apache.druid.metadata.SqlSegmentsMetadataManager-Exec--0]
org.apache.druid.metadata.SqlSegmentsMetadataManager - Polled and found [57]
segments in the database in [0]ms.
2026-01-02T20:18:33,275 INFO
[org.apache.druid.metadata.SqlSegmentsMetadataManager-Exec--0]
org.apache.druid.metadata.SqlSegmentsMetadataManager - Polled and found [57]
segments in the database in [0]ms.
2026-01-02T20:18:33,378 INFO
[org.apache.druid.metadata.SqlSegmentsMetadataManager-Exec--0]
org.apache.druid.metadata.SqlSegmentsMetadataManager - Polled and found [57]
segments in the database in [1]ms.
2026-01-02T20:18:33,413 WARN [KafkaSupervisor-supe_datasource_abegdknn]
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor -
Configured task count[2] for supervisor[supe_datasource_abegdknn] is greater
than the number of partitions[1].
2026-01-02T20:18:33,414 INFO [KafkaSupervisor-supe_datasource_abegdknn]
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor -
Early stop requested for supervisor[supe_datasource_abegdknn], signalling tasks
to complete.
2026-01-02T20:18:33,481 INFO
[org.apache.druid.metadata.SqlSegmentsMetadataManager-Exec--0]
org.apache.druid.metadata.SqlSegmentsMetadataManager - Polled and found [57]
segments in the database in [0]ms.
2026-01-02T20:18:33,577 WARN [KafkaSupervisor-supe_datasource_dpcjklpc]
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor -
Configured task count[2] for supervisor[supe_datasource_dpcjklpc] is greater
than the number of partitions[1].
2026-01-02T20:18:33,578 INFO [KafkaSupervisor-supe_datasource_dpcjklpc]
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor -
Early stop requested for supervisor[supe_datasource_dpcjklpc], signalling tasks
to complete.
2026-01-02T20:18:33,584 INFO
[org.apache.druid.metadata.SqlSegmentsMetadataManager-Exec--0]
org.apache.druid.metadata.SqlSegmentsMetadataManager - Polled and found [57]
segments in the database in [1]ms.
org.apache.druid.java.util.common.ISE: Timed out waiting for event after
[60,000]ms
```
##########
embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTestBase.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.data.input.impl.TimestampSpec;
+import org.apache.druid.indexing.kafka.KafkaIndexTaskModule;
+import org.apache.druid.indexing.kafka.simulate.KafkaResource;
+import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpecBuilder;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus;
+import org.apache.druid.query.DruidMetrics;
+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.EmbeddedRouter;
+import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase;
+import org.apache.druid.testing.tools.EventSerializer;
+import org.apache.druid.testing.tools.JsonEventSerializer;
+import org.apache.druid.testing.tools.StreamGenerator;
+import org.apache.druid.testing.tools.WikipediaStreamEventStreamGenerator;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.junit.jupiter.api.Assertions;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public abstract class KafkaTestBase extends EmbeddedClusterTestBase
+{
+ protected final KafkaResource kafkaServer = new KafkaResource();
+ protected final EmbeddedOverlord overlord = new EmbeddedOverlord();
+ protected final EmbeddedIndexer indexer = new EmbeddedIndexer();
+ protected final EmbeddedBroker broker = new EmbeddedBroker();
+ protected final EmbeddedHistorical historical = new EmbeddedHistorical();
+ protected final EmbeddedCoordinator coordinator = new EmbeddedCoordinator();
+
+ @Override
+ protected EmbeddedDruidCluster createCluster()
+ {
+ indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s");
+
+ return EmbeddedDruidCluster
+ .withEmbeddedDerbyAndZookeeper()
+ .useContainerFriendlyHostname()
+ .addExtension(KafkaIndexTaskModule.class)
+ .useLatchableEmitter()
+ .useDefaultTimeoutForLatchableEmitter(60)
+ .addResource(kafkaServer)
+ .addServer(indexer)
+ .addServer(coordinator)
+ .addServer(overlord)
+ .addServer(broker)
+ .addServer(historical)
+ .addServer(new EmbeddedRouter());
+ }
+
+ protected KafkaSupervisorSpecBuilder createSupervisor()
+ {
+ return MoreResources.Supervisor.KAFKA_JSON
+ .get()
+ .withDataSchema(schema -> schema.withTimestamp(new
TimestampSpec("timestamp", "iso", null)))
+ .withTuningConfig(tuningConfig ->
tuningConfig.withMaxRowsPerSegment(1))
+ .withIoConfig(
+ ioConfig -> ioConfig
+ .withConsumerProperties(kafkaServer.consumerProperties())
+ .withTaskCount(2)
+ );
+ }
+
+ /**
+ * Waits until number of processed events matches {@code expectedRowCount}.
+ */
+ protected void waitUntilPublishedRecordsAreIngested(int expectedRowCount)
+ {
+ indexer.latchableEmitter().waitForEventAggregate(
+ event -> event.hasMetricName("ingest/events/processed")
+ .hasDimension(DruidMetrics.DATASOURCE, dataSource),
+ agg -> agg.hasSumAtLeast(expectedRowCount)
Review Comment:
nit: I see this matcher doesn't support an exact count. Should we consider
adding a stricter matcher `hasSum` rather than an `hasSumAtLeast`?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]