kfaraz commented on code in PR #18383: URL: https://github.com/apache/druid/pull/18383#discussion_r2264468696
########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/SchemaRegistryResource.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.testing.embedded.indexing; + +import org.apache.druid.testing.embedded.TestcontainerResource; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.kafka.KafkaContainer; + +/** + * A resource for managing a Schema Registry instance in embedded tests. + * This class extends TestcontainerResource to provide a Schema Registry container. + * Confluent schema registry is commonly used for managing schemas in Kafka-based applications but not + * natively supported by testcontainers. Review Comment: These lines may be removed once we rename this class. ########## integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/AvroSchemaRegistryEventSerializer.java: ########## @@ -49,6 +48,22 @@ public class AvroSchemaRegistryEventSerializer extends AvroEventSerializer private Schema fromRegistry; + public AvroSchemaRegistryEventSerializer( + String schemaRegistryHost + ) + { + this.config = null; + this.client = new CachedSchemaRegistryClient( + StringUtils.format("http://%s", schemaRegistryHost), + Integer.MAX_VALUE, + ImmutableMap.of( + "basic.auth.credentials.source", "USER_INFO", + "basic.auth.user.info", "druid:diurd" + ), + ImmutableMap.of() + ); Review Comment: This snippet can probably be put in a utility method, in say `KafkaUtil`, as it seems to be used in multiple places. ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaDataFormatsTest.java: ########## @@ -0,0 +1,716 @@ +/* + * 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.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.avro.AvroExtensionsModule; +import org.apache.druid.data.input.avro.AvroStreamInputFormat; +import org.apache.druid.data.input.avro.SchemaRegistryBasedAvroBytesDecoder; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DelimitedInputFormat; +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.data.input.protobuf.FileBasedProtobufBytesDecoder; +import org.apache.druid.data.input.protobuf.ProtobufExtensionsModule; +import org.apache.druid.data.input.protobuf.ProtobufInputFormat; +import org.apache.druid.data.input.protobuf.SchemaRegistryBasedProtobufBytesDecoder; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.TestHelper; +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.druid.testing.tools.AvroSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.EventSerializer; +import org.apache.druid.testing.tools.ProtobufSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.StreamGenerator; +import org.apache.druid.testing.tools.WikipediaStreamEventStreamGenerator; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * Embedded test of kafka indexing for different data formats. + * Formats Included: + * <ul> + * Avro (with and without schema registry) + * CSV + * JSON + * Protobuf (with and without schema registry) + * </ul> + * + * This tests both InputFormat and Parser. Parser is deprecated for Kafka Ingestion, and those tests will be removed in the future. + */ +public class KafkaDataFormatsTest extends EmbeddedClusterTestBase +{ + private static final long CYCLE_PADDING_MS = 100; + private static final int EVENTS_PER_SECOND = 6; + + + private static ObjectMapper jsonMapper; + private final EmbeddedBroker broker = new EmbeddedBroker(); + private final EmbeddedIndexer indexer = new EmbeddedIndexer(); + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedHistorical historical = new EmbeddedHistorical(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + private KafkaResource kafkaServer; + private SchemaRegistryResource schemaRegistry; + + @BeforeAll + public static void beforeAll() + { + jsonMapper = TestHelper.JSON_MAPPER; + } + + @Override + public EmbeddedDruidCluster createCluster() + { + final EmbeddedDruidCluster cluster = EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper().useContainerFriendlyHostname(); + + kafkaServer = new KafkaResource() + { + @Override + public void start() + { + super.start(); + } + + @Override + public void stop() + { + super.stop(); + } + }; + + schemaRegistry = new SchemaRegistryResource() + { + @Override + public void start() + { + this.kafkaContainer = kafkaServer.getContainer(); + super.start(); + } + + @Override + public void stop() + { + super.stop(); + } + }; + + indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s") + .addProperty("druid.worker.capacity", "10"); + overlord.addProperty("druid.indexer.task.default.context", "{\"useConcurrentLocks\": true}") + .addProperty("druid.manager.segments.useIncrementalCache", "ifSynced") + .addProperty("druid.manager.segments.pollDuration", "PT0.1s") + .addProperty("druid.manager.segments.killUnused.enabled", "true") + .addProperty("druid.manager.segments.killUnused.bufferPeriod", "PT0.1s") + .addProperty("druid.manager.segments.killUnused.dutyPeriod", "PT1s"); Review Comment: These can be skipped if these tests are not doing any kill testing. ########## extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/KafkaResource.java: ########## @@ -128,6 +128,17 @@ public void produceRecordsToTopic(List<ProducerRecord<byte[], byte[]>> records) } } + /** + * Produces records to a topic of this embedded Kafka server. + */ + public void produceRecordsToTopic(List<ProducerRecord<byte[], byte[]>> records) + { + produceRecordsToTopicWithExtraProperties( + records, + null + ); Review Comment: Nit: can probably be in a single line ```suggestion produceRecordsToTopicWithExtraProperties(records, null); ``` ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaDataFormatsTest.java: ########## @@ -0,0 +1,716 @@ +/* + * 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.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.avro.AvroExtensionsModule; +import org.apache.druid.data.input.avro.AvroStreamInputFormat; +import org.apache.druid.data.input.avro.SchemaRegistryBasedAvroBytesDecoder; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DelimitedInputFormat; +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.data.input.protobuf.FileBasedProtobufBytesDecoder; +import org.apache.druid.data.input.protobuf.ProtobufExtensionsModule; +import org.apache.druid.data.input.protobuf.ProtobufInputFormat; +import org.apache.druid.data.input.protobuf.SchemaRegistryBasedProtobufBytesDecoder; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.TestHelper; +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.druid.testing.tools.AvroSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.EventSerializer; +import org.apache.druid.testing.tools.ProtobufSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.StreamGenerator; +import org.apache.druid.testing.tools.WikipediaStreamEventStreamGenerator; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * Embedded test of kafka indexing for different data formats. + * Formats Included: + * <ul> + * Avro (with and without schema registry) + * CSV + * JSON + * Protobuf (with and without schema registry) Review Comment: Each of these would need to be put inside `<li></li>` tags. ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/SchemaRegistryResource.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.testing.embedded.indexing; + +import org.apache.druid.testing.embedded.TestcontainerResource; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.kafka.KafkaContainer; + +/** + * A resource for managing a Schema Registry instance in embedded tests. + * This class extends TestcontainerResource to provide a Schema Registry container. + * Confluent schema registry is commonly used for managing schemas in Kafka-based applications but not + * natively supported by testcontainers. + */ +public class SchemaRegistryResource extends TestcontainerResource<GenericContainer<?>> Review Comment: You may name it either `KafkaSchemaRegistryResource` or `ConfluentSchemaRegistryResource` to avoid ambiguity. ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/SchemaRegistryResource.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.testing.embedded.indexing; + +import org.apache.druid.testing.embedded.TestcontainerResource; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.kafka.KafkaContainer; + +/** + * A resource for managing a Schema Registry instance in embedded tests. + * This class extends TestcontainerResource to provide a Schema Registry container. + * Confluent schema registry is commonly used for managing schemas in Kafka-based applications but not + * natively supported by testcontainers. + */ +public class SchemaRegistryResource extends TestcontainerResource<GenericContainer<?>> +{ + private static final String SCHEMA_REGISTRY_IMAGE = "confluentinc/cp-schema-registry:latest"; + + KafkaContainer kafkaContainer; + + @Override + protected GenericContainer<?> createContainer() + { + return new GenericContainer<>(SCHEMA_REGISTRY_IMAGE) + .dependsOn(kafkaContainer) + .withExposedPorts(9081) + .withEnv("SCHEMA_REGISTRY_HOST_NAME", "schema-registry") + .withEnv("SCHEMA_REGISTRY_LISTENERS", "http://0.0.0.0:9081") + .withEnv("SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS", kafkaContainer.getBootstrapServers()); Review Comment: Instead of depending on `KafkaContainer`, we should use `KafkaResource.getBootstrapServerUrl()`. ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaDataFormatsTest.java: ########## @@ -0,0 +1,716 @@ +/* + * 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.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.avro.AvroExtensionsModule; +import org.apache.druid.data.input.avro.AvroStreamInputFormat; +import org.apache.druid.data.input.avro.SchemaRegistryBasedAvroBytesDecoder; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DelimitedInputFormat; +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.data.input.protobuf.FileBasedProtobufBytesDecoder; +import org.apache.druid.data.input.protobuf.ProtobufExtensionsModule; +import org.apache.druid.data.input.protobuf.ProtobufInputFormat; +import org.apache.druid.data.input.protobuf.SchemaRegistryBasedProtobufBytesDecoder; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.TestHelper; +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.druid.testing.tools.AvroSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.EventSerializer; +import org.apache.druid.testing.tools.ProtobufSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.StreamGenerator; +import org.apache.druid.testing.tools.WikipediaStreamEventStreamGenerator; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * Embedded test of kafka indexing for different data formats. + * Formats Included: + * <ul> + * Avro (with and without schema registry) + * CSV + * JSON + * Protobuf (with and without schema registry) + * </ul> + * + * This tests both InputFormat and Parser. Parser is deprecated for Kafka Ingestion, and those tests will be removed in the future. + */ +public class KafkaDataFormatsTest extends EmbeddedClusterTestBase +{ + private static final long CYCLE_PADDING_MS = 100; + private static final int EVENTS_PER_SECOND = 6; + + + private static ObjectMapper jsonMapper; + private final EmbeddedBroker broker = new EmbeddedBroker(); + private final EmbeddedIndexer indexer = new EmbeddedIndexer(); + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedHistorical historical = new EmbeddedHistorical(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + private KafkaResource kafkaServer; + private SchemaRegistryResource schemaRegistry; + + @BeforeAll + public static void beforeAll() + { + jsonMapper = TestHelper.JSON_MAPPER; + } + + @Override + public EmbeddedDruidCluster createCluster() + { + final EmbeddedDruidCluster cluster = EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper().useContainerFriendlyHostname(); + + kafkaServer = new KafkaResource() + { + @Override + public void start() + { + super.start(); + } + + @Override + public void stop() + { + super.stop(); + } Review Comment: Not needed. ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaDataFormatsTest.java: ########## @@ -0,0 +1,716 @@ +/* + * 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.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.avro.AvroExtensionsModule; +import org.apache.druid.data.input.avro.AvroStreamInputFormat; +import org.apache.druid.data.input.avro.SchemaRegistryBasedAvroBytesDecoder; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DelimitedInputFormat; +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.data.input.protobuf.FileBasedProtobufBytesDecoder; +import org.apache.druid.data.input.protobuf.ProtobufExtensionsModule; +import org.apache.druid.data.input.protobuf.ProtobufInputFormat; +import org.apache.druid.data.input.protobuf.SchemaRegistryBasedProtobufBytesDecoder; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.TestHelper; +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.druid.testing.tools.AvroSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.EventSerializer; +import org.apache.druid.testing.tools.ProtobufSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.StreamGenerator; +import org.apache.druid.testing.tools.WikipediaStreamEventStreamGenerator; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * Embedded test of kafka indexing for different data formats. Review Comment: ```suggestion * Embedded test of Kafka indexing for different data formats. ``` ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaDataFormatsTest.java: ########## @@ -0,0 +1,716 @@ +/* + * 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.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.avro.AvroExtensionsModule; +import org.apache.druid.data.input.avro.AvroStreamInputFormat; +import org.apache.druid.data.input.avro.SchemaRegistryBasedAvroBytesDecoder; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DelimitedInputFormat; +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.data.input.protobuf.FileBasedProtobufBytesDecoder; +import org.apache.druid.data.input.protobuf.ProtobufExtensionsModule; +import org.apache.druid.data.input.protobuf.ProtobufInputFormat; +import org.apache.druid.data.input.protobuf.SchemaRegistryBasedProtobufBytesDecoder; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.TestHelper; +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.druid.testing.tools.AvroSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.EventSerializer; +import org.apache.druid.testing.tools.ProtobufSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.StreamGenerator; +import org.apache.druid.testing.tools.WikipediaStreamEventStreamGenerator; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * Embedded test of kafka indexing for different data formats. + * Formats Included: + * <ul> + * Avro (with and without schema registry) + * CSV + * JSON + * Protobuf (with and without schema registry) + * </ul> + * + * This tests both InputFormat and Parser. Parser is deprecated for Kafka Ingestion, and those tests will be removed in the future. + */ +public class KafkaDataFormatsTest extends EmbeddedClusterTestBase +{ + private static final long CYCLE_PADDING_MS = 100; + private static final int EVENTS_PER_SECOND = 6; + + + private static ObjectMapper jsonMapper; + private final EmbeddedBroker broker = new EmbeddedBroker(); + private final EmbeddedIndexer indexer = new EmbeddedIndexer(); + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedHistorical historical = new EmbeddedHistorical(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + private KafkaResource kafkaServer; + private SchemaRegistryResource schemaRegistry; + + @BeforeAll + public static void beforeAll() + { + jsonMapper = TestHelper.JSON_MAPPER; + } + + @Override + public EmbeddedDruidCluster createCluster() + { + final EmbeddedDruidCluster cluster = EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper().useContainerFriendlyHostname(); + + kafkaServer = new KafkaResource() + { + @Override + public void start() + { + super.start(); + } + + @Override + public void stop() + { + super.stop(); + } + }; + + schemaRegistry = new SchemaRegistryResource() + { + @Override + public void start() + { + this.kafkaContainer = kafkaServer.getContainer(); + super.start(); + } + + @Override + public void stop() + { + super.stop(); + } Review Comment: This might not be needed if we pass `KafkaResource` in the constructor. ########## embedded-tests/src/test/resources/test-data/avro/input_format.json: ########## @@ -0,0 +1,81 @@ +{ + "type": "avro_stream", Review Comment: Since we are using inline JSON strings / builders for the other stuff, can we also use a builder for this schema definition? ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaDataFormatsTest.java: ########## @@ -0,0 +1,716 @@ +/* + * 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.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.avro.AvroExtensionsModule; +import org.apache.druid.data.input.avro.AvroStreamInputFormat; +import org.apache.druid.data.input.avro.SchemaRegistryBasedAvroBytesDecoder; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DelimitedInputFormat; +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.data.input.protobuf.FileBasedProtobufBytesDecoder; +import org.apache.druid.data.input.protobuf.ProtobufExtensionsModule; +import org.apache.druid.data.input.protobuf.ProtobufInputFormat; +import org.apache.druid.data.input.protobuf.SchemaRegistryBasedProtobufBytesDecoder; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.TestHelper; +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.druid.testing.tools.AvroSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.EventSerializer; +import org.apache.druid.testing.tools.ProtobufSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.StreamGenerator; +import org.apache.druid.testing.tools.WikipediaStreamEventStreamGenerator; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * Embedded test of kafka indexing for different data formats. + * Formats Included: + * <ul> + * Avro (with and without schema registry) + * CSV + * JSON + * Protobuf (with and without schema registry) + * </ul> + * + * This tests both InputFormat and Parser. Parser is deprecated for Kafka Ingestion, and those tests will be removed in the future. + */ +public class KafkaDataFormatsTest extends EmbeddedClusterTestBase +{ + private static final long CYCLE_PADDING_MS = 100; + private static final int EVENTS_PER_SECOND = 6; + + + private static ObjectMapper jsonMapper; Review Comment: This can be a constant or you can use the mapper bindings from one of the servers, say: ``` overlord.bindings().jsonMapper() ``` ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaDataFormatsTest.java: ########## @@ -0,0 +1,716 @@ +/* + * 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.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.avro.AvroExtensionsModule; +import org.apache.druid.data.input.avro.AvroStreamInputFormat; +import org.apache.druid.data.input.avro.SchemaRegistryBasedAvroBytesDecoder; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DelimitedInputFormat; +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.data.input.protobuf.FileBasedProtobufBytesDecoder; +import org.apache.druid.data.input.protobuf.ProtobufExtensionsModule; +import org.apache.druid.data.input.protobuf.ProtobufInputFormat; +import org.apache.druid.data.input.protobuf.SchemaRegistryBasedProtobufBytesDecoder; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.TestHelper; +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.druid.testing.tools.AvroSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.EventSerializer; +import org.apache.druid.testing.tools.ProtobufSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.StreamGenerator; +import org.apache.druid.testing.tools.WikipediaStreamEventStreamGenerator; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * Embedded test of kafka indexing for different data formats. + * Formats Included: + * <ul> + * Avro (with and without schema registry) + * CSV + * JSON + * Protobuf (with and without schema registry) + * </ul> + * + * This tests both InputFormat and Parser. Parser is deprecated for Kafka Ingestion, and those tests will be removed in the future. + */ +public class KafkaDataFormatsTest extends EmbeddedClusterTestBase +{ + private static final long CYCLE_PADDING_MS = 100; + private static final int EVENTS_PER_SECOND = 6; + + + private static ObjectMapper jsonMapper; + private final EmbeddedBroker broker = new EmbeddedBroker(); + private final EmbeddedIndexer indexer = new EmbeddedIndexer(); + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedHistorical historical = new EmbeddedHistorical(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + private KafkaResource kafkaServer; + private SchemaRegistryResource schemaRegistry; + + @BeforeAll + public static void beforeAll() + { + jsonMapper = TestHelper.JSON_MAPPER; + } + + @Override + public EmbeddedDruidCluster createCluster() + { + final EmbeddedDruidCluster cluster = EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper().useContainerFriendlyHostname(); + + kafkaServer = new KafkaResource() + { + @Override + public void start() + { + super.start(); + } + + @Override + public void stop() + { + super.stop(); + } + }; + + schemaRegistry = new SchemaRegistryResource() + { + @Override + public void start() + { + this.kafkaContainer = kafkaServer.getContainer(); + super.start(); + } + + @Override + public void stop() + { + super.stop(); + } + }; + + indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s") + .addProperty("druid.worker.capacity", "10"); + overlord.addProperty("druid.indexer.task.default.context", "{\"useConcurrentLocks\": true}") + .addProperty("druid.manager.segments.useIncrementalCache", "ifSynced") + .addProperty("druid.manager.segments.pollDuration", "PT0.1s") + .addProperty("druid.manager.segments.killUnused.enabled", "true") + .addProperty("druid.manager.segments.killUnused.bufferPeriod", "PT0.1s") + .addProperty("druid.manager.segments.killUnused.dutyPeriod", "PT1s"); + coordinator.addProperty("druid.manager.segments.useIncrementalCache", "ifSynced"); + cluster.addExtension(KafkaIndexTaskModule.class) + .addExtension(ProtobufExtensionsModule.class) + .addExtension(AvroExtensionsModule.class) + .useLatchableEmitter() + .addCommonProperty("druid.monitoring.emissionPeriod", "PT0.1s") + .addCommonProperty( + "druid.monitoring.monitors", + "[\"org.apache.druid.java.util.metrics.JvmMonitor\"," + + "\"org.apache.druid.server.metrics.TaskCountStatsMonitor\"]" + ) Review Comment: We can remove these if monitoring is not needed. ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaDataFormatsTest.java: ########## @@ -0,0 +1,716 @@ +/* + * 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.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.avro.AvroExtensionsModule; +import org.apache.druid.data.input.avro.AvroStreamInputFormat; +import org.apache.druid.data.input.avro.SchemaRegistryBasedAvroBytesDecoder; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DelimitedInputFormat; +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.data.input.protobuf.FileBasedProtobufBytesDecoder; +import org.apache.druid.data.input.protobuf.ProtobufExtensionsModule; +import org.apache.druid.data.input.protobuf.ProtobufInputFormat; +import org.apache.druid.data.input.protobuf.SchemaRegistryBasedProtobufBytesDecoder; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.TestHelper; +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.druid.testing.tools.AvroSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.EventSerializer; +import org.apache.druid.testing.tools.ProtobufSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.StreamGenerator; +import org.apache.druid.testing.tools.WikipediaStreamEventStreamGenerator; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * Embedded test of kafka indexing for different data formats. + * Formats Included: + * <ul> + * Avro (with and without schema registry) + * CSV + * JSON + * Protobuf (with and without schema registry) + * </ul> + * + * This tests both InputFormat and Parser. Parser is deprecated for Kafka Ingestion, and those tests will be removed in the future. + */ +public class KafkaDataFormatsTest extends EmbeddedClusterTestBase +{ + private static final long CYCLE_PADDING_MS = 100; + private static final int EVENTS_PER_SECOND = 6; + + + private static ObjectMapper jsonMapper; + private final EmbeddedBroker broker = new EmbeddedBroker(); + private final EmbeddedIndexer indexer = new EmbeddedIndexer(); + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedHistorical historical = new EmbeddedHistorical(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + private KafkaResource kafkaServer; + private SchemaRegistryResource schemaRegistry; + + @BeforeAll + public static void beforeAll() + { + jsonMapper = TestHelper.JSON_MAPPER; + } + + @Override + public EmbeddedDruidCluster createCluster() + { + final EmbeddedDruidCluster cluster = EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper().useContainerFriendlyHostname(); + + kafkaServer = new KafkaResource() + { + @Override + public void start() + { + super.start(); + } + + @Override + public void stop() + { + super.stop(); + } + }; + + schemaRegistry = new SchemaRegistryResource() + { + @Override + public void start() + { + this.kafkaContainer = kafkaServer.getContainer(); + super.start(); + } + + @Override + public void stop() + { + super.stop(); + } + }; + + indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s") + .addProperty("druid.worker.capacity", "10"); + overlord.addProperty("druid.indexer.task.default.context", "{\"useConcurrentLocks\": true}") + .addProperty("druid.manager.segments.useIncrementalCache", "ifSynced") + .addProperty("druid.manager.segments.pollDuration", "PT0.1s") + .addProperty("druid.manager.segments.killUnused.enabled", "true") + .addProperty("druid.manager.segments.killUnused.bufferPeriod", "PT0.1s") + .addProperty("druid.manager.segments.killUnused.dutyPeriod", "PT1s"); + coordinator.addProperty("druid.manager.segments.useIncrementalCache", "ifSynced"); + cluster.addExtension(KafkaIndexTaskModule.class) + .addExtension(ProtobufExtensionsModule.class) + .addExtension(AvroExtensionsModule.class) + .useLatchableEmitter() + .addCommonProperty("druid.monitoring.emissionPeriod", "PT0.1s") + .addCommonProperty( + "druid.monitoring.monitors", + "[\"org.apache.druid.java.util.metrics.JvmMonitor\"," + + "\"org.apache.druid.server.metrics.TaskCountStatsMonitor\"]" + ) + .addResource(kafkaServer) + .addResource(schemaRegistry) + .addServer(coordinator) + .addServer(overlord) + .addServer(indexer) + .addServer(broker) + .addServer(historical); + + return cluster; + } + + @Test + @Timeout(value = 30, unit = TimeUnit.SECONDS) + public void test_indexKafka_avroDataFormat_withParser() throws IOException + { + new AvroExtensionsModule().getJacksonModules().forEach(jsonMapper::registerModule); + kafkaServer.createTopicWithPartitions(dataSource, 3); + EventSerializer serializer = jsonMapper.readValue("{\"type\": \"avro\"}", EventSerializer.class); + int recordCount = generateStreamAndPublishToKafka(dataSource, serializer, false); + String jsonString = "{" Review Comment: For all of the JSON strings, please try to use an existing builder or add a new one if necessary. You may add it in `Resources` class or a new class which sits next to `TaskBuilder`. ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/SchemaRegistryResource.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.testing.embedded.indexing; + +import org.apache.druid.testing.embedded.TestcontainerResource; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.kafka.KafkaContainer; + +/** + * A resource for managing a Schema Registry instance in embedded tests. + * This class extends TestcontainerResource to provide a Schema Registry container. + * Confluent schema registry is commonly used for managing schemas in Kafka-based applications but not + * natively supported by testcontainers. + */ +public class SchemaRegistryResource extends TestcontainerResource<GenericContainer<?>> +{ + private static final String SCHEMA_REGISTRY_IMAGE = "confluentinc/cp-schema-registry:latest"; + + KafkaContainer kafkaContainer; Review Comment: We should pass `KafkaResource` in the constructor instead. ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaDataFormatsTest.java: ########## @@ -0,0 +1,716 @@ +/* + * 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.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.avro.AvroExtensionsModule; +import org.apache.druid.data.input.avro.AvroStreamInputFormat; +import org.apache.druid.data.input.avro.SchemaRegistryBasedAvroBytesDecoder; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DelimitedInputFormat; +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.data.input.protobuf.FileBasedProtobufBytesDecoder; +import org.apache.druid.data.input.protobuf.ProtobufExtensionsModule; +import org.apache.druid.data.input.protobuf.ProtobufInputFormat; +import org.apache.druid.data.input.protobuf.SchemaRegistryBasedProtobufBytesDecoder; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.TestHelper; +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.druid.testing.tools.AvroSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.EventSerializer; +import org.apache.druid.testing.tools.ProtobufSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.StreamGenerator; +import org.apache.druid.testing.tools.WikipediaStreamEventStreamGenerator; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * Embedded test of kafka indexing for different data formats. + * Formats Included: + * <ul> + * Avro (with and without schema registry) + * CSV + * JSON + * Protobuf (with and without schema registry) + * </ul> + * + * This tests both InputFormat and Parser. Parser is deprecated for Kafka Ingestion, and those tests will be removed in the future. + */ +public class KafkaDataFormatsTest extends EmbeddedClusterTestBase +{ + private static final long CYCLE_PADDING_MS = 100; + private static final int EVENTS_PER_SECOND = 6; + + + private static ObjectMapper jsonMapper; + private final EmbeddedBroker broker = new EmbeddedBroker(); + private final EmbeddedIndexer indexer = new EmbeddedIndexer(); + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedHistorical historical = new EmbeddedHistorical(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + private KafkaResource kafkaServer; + private SchemaRegistryResource schemaRegistry; + + @BeforeAll + public static void beforeAll() + { + jsonMapper = TestHelper.JSON_MAPPER; + } + + @Override + public EmbeddedDruidCluster createCluster() + { + final EmbeddedDruidCluster cluster = EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper().useContainerFriendlyHostname(); + + kafkaServer = new KafkaResource() + { + @Override + public void start() + { + super.start(); + } + + @Override + public void stop() + { + super.stop(); + } + }; + + schemaRegistry = new SchemaRegistryResource() + { + @Override + public void start() + { + this.kafkaContainer = kafkaServer.getContainer(); + super.start(); + } + + @Override + public void stop() + { + super.stop(); + } + }; + + indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s") + .addProperty("druid.worker.capacity", "10"); + overlord.addProperty("druid.indexer.task.default.context", "{\"useConcurrentLocks\": true}") + .addProperty("druid.manager.segments.useIncrementalCache", "ifSynced") + .addProperty("druid.manager.segments.pollDuration", "PT0.1s") + .addProperty("druid.manager.segments.killUnused.enabled", "true") + .addProperty("druid.manager.segments.killUnused.bufferPeriod", "PT0.1s") + .addProperty("druid.manager.segments.killUnused.dutyPeriod", "PT1s"); + coordinator.addProperty("druid.manager.segments.useIncrementalCache", "ifSynced"); + cluster.addExtension(KafkaIndexTaskModule.class) + .addExtension(ProtobufExtensionsModule.class) + .addExtension(AvroExtensionsModule.class) + .useLatchableEmitter() + .addCommonProperty("druid.monitoring.emissionPeriod", "PT0.1s") + .addCommonProperty( + "druid.monitoring.monitors", + "[\"org.apache.druid.java.util.metrics.JvmMonitor\"," + + "\"org.apache.druid.server.metrics.TaskCountStatsMonitor\"]" + ) + .addResource(kafkaServer) + .addResource(schemaRegistry) + .addServer(coordinator) + .addServer(overlord) + .addServer(indexer) + .addServer(broker) + .addServer(historical); + + return cluster; + } + + @Test + @Timeout(value = 30, unit = TimeUnit.SECONDS) + public void test_indexKafka_avroDataFormat_withParser() throws IOException + { + new AvroExtensionsModule().getJacksonModules().forEach(jsonMapper::registerModule); Review Comment: This won't be needed if we use the mapper bound to one of the servers. ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaDataFormatsTest.java: ########## @@ -0,0 +1,716 @@ +/* + * 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.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.avro.AvroExtensionsModule; +import org.apache.druid.data.input.avro.AvroStreamInputFormat; +import org.apache.druid.data.input.avro.SchemaRegistryBasedAvroBytesDecoder; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DelimitedInputFormat; +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.data.input.protobuf.FileBasedProtobufBytesDecoder; +import org.apache.druid.data.input.protobuf.ProtobufExtensionsModule; +import org.apache.druid.data.input.protobuf.ProtobufInputFormat; +import org.apache.druid.data.input.protobuf.SchemaRegistryBasedProtobufBytesDecoder; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.TestHelper; +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.druid.testing.tools.AvroSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.EventSerializer; +import org.apache.druid.testing.tools.ProtobufSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.StreamGenerator; +import org.apache.druid.testing.tools.WikipediaStreamEventStreamGenerator; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * Embedded test of kafka indexing for different data formats. + * Formats Included: + * <ul> + * Avro (with and without schema registry) + * CSV + * JSON + * Protobuf (with and without schema registry) + * </ul> + * + * This tests both InputFormat and Parser. Parser is deprecated for Kafka Ingestion, and those tests will be removed in the future. + */ +public class KafkaDataFormatsTest extends EmbeddedClusterTestBase +{ + private static final long CYCLE_PADDING_MS = 100; + private static final int EVENTS_PER_SECOND = 6; + + + private static ObjectMapper jsonMapper; + private final EmbeddedBroker broker = new EmbeddedBroker(); + private final EmbeddedIndexer indexer = new EmbeddedIndexer(); + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedHistorical historical = new EmbeddedHistorical(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + private KafkaResource kafkaServer; + private SchemaRegistryResource schemaRegistry; + + @BeforeAll + public static void beforeAll() + { + jsonMapper = TestHelper.JSON_MAPPER; + } + + @Override + public EmbeddedDruidCluster createCluster() + { + final EmbeddedDruidCluster cluster = EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper().useContainerFriendlyHostname(); + + kafkaServer = new KafkaResource() + { + @Override + public void start() + { + super.start(); + } + + @Override + public void stop() + { + super.stop(); + } + }; + + schemaRegistry = new SchemaRegistryResource() + { + @Override + public void start() + { + this.kafkaContainer = kafkaServer.getContainer(); + super.start(); + } + + @Override + public void stop() + { + super.stop(); + } + }; + + indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s") + .addProperty("druid.worker.capacity", "10"); + overlord.addProperty("druid.indexer.task.default.context", "{\"useConcurrentLocks\": true}") Review Comment: This can be skipped if we don't plan to run any compaction in these tests. ########## embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaDataFormatsTest.java: ########## @@ -0,0 +1,716 @@ +/* + * 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.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.avro.AvroExtensionsModule; +import org.apache.druid.data.input.avro.AvroStreamInputFormat; +import org.apache.druid.data.input.avro.SchemaRegistryBasedAvroBytesDecoder; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DelimitedInputFormat; +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.data.input.protobuf.FileBasedProtobufBytesDecoder; +import org.apache.druid.data.input.protobuf.ProtobufExtensionsModule; +import org.apache.druid.data.input.protobuf.ProtobufInputFormat; +import org.apache.druid.data.input.protobuf.SchemaRegistryBasedProtobufBytesDecoder; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.TestHelper; +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.druid.testing.tools.AvroSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.EventSerializer; +import org.apache.druid.testing.tools.ProtobufSchemaRegistryEventSerializer; +import org.apache.druid.testing.tools.StreamGenerator; +import org.apache.druid.testing.tools.WikipediaStreamEventStreamGenerator; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * Embedded test of kafka indexing for different data formats. + * Formats Included: + * <ul> + * Avro (with and without schema registry) + * CSV + * JSON + * Protobuf (with and without schema registry) + * </ul> + * + * This tests both InputFormat and Parser. Parser is deprecated for Kafka Ingestion, and those tests will be removed in the future. + */ +public class KafkaDataFormatsTest extends EmbeddedClusterTestBase +{ + private static final long CYCLE_PADDING_MS = 100; + private static final int EVENTS_PER_SECOND = 6; + + + private static ObjectMapper jsonMapper; + private final EmbeddedBroker broker = new EmbeddedBroker(); + private final EmbeddedIndexer indexer = new EmbeddedIndexer(); + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedHistorical historical = new EmbeddedHistorical(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + private KafkaResource kafkaServer; + private SchemaRegistryResource schemaRegistry; + + @BeforeAll + public static void beforeAll() + { + jsonMapper = TestHelper.JSON_MAPPER; + } + + @Override + public EmbeddedDruidCluster createCluster() + { + final EmbeddedDruidCluster cluster = EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper().useContainerFriendlyHostname(); + + kafkaServer = new KafkaResource() + { + @Override + public void start() + { + super.start(); + } + + @Override + public void stop() + { + super.stop(); + } + }; + + schemaRegistry = new SchemaRegistryResource() + { + @Override + public void start() + { + this.kafkaContainer = kafkaServer.getContainer(); + super.start(); + } + + @Override + public void stop() + { + super.stop(); + } + }; + + indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s") + .addProperty("druid.worker.capacity", "10"); + overlord.addProperty("druid.indexer.task.default.context", "{\"useConcurrentLocks\": true}") + .addProperty("druid.manager.segments.useIncrementalCache", "ifSynced") + .addProperty("druid.manager.segments.pollDuration", "PT0.1s") + .addProperty("druid.manager.segments.killUnused.enabled", "true") + .addProperty("druid.manager.segments.killUnused.bufferPeriod", "PT0.1s") + .addProperty("druid.manager.segments.killUnused.dutyPeriod", "PT1s"); + coordinator.addProperty("druid.manager.segments.useIncrementalCache", "ifSynced"); + cluster.addExtension(KafkaIndexTaskModule.class) + .addExtension(ProtobufExtensionsModule.class) + .addExtension(AvroExtensionsModule.class) + .useLatchableEmitter() + .addCommonProperty("druid.monitoring.emissionPeriod", "PT0.1s") + .addCommonProperty( + "druid.monitoring.monitors", + "[\"org.apache.druid.java.util.metrics.JvmMonitor\"," + + "\"org.apache.druid.server.metrics.TaskCountStatsMonitor\"]" + ) + .addResource(kafkaServer) + .addResource(schemaRegistry) + .addServer(coordinator) + .addServer(overlord) + .addServer(indexer) + .addServer(broker) + .addServer(historical); + + return cluster; + } + + @Test + @Timeout(value = 30, unit = TimeUnit.SECONDS) Review Comment: Nit: shorthand since default unit is seconds. ```suggestion @Timeout(30) ``` -- 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]
