From fbe0048bb4e6be0ea7252c0374058ca9bfc0522e Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 17 May 2019 09:06:33 +0200 Subject: [PATCH 01/26] Bump Jackson to 2.9.9 --- .../druid/data/input/impl/ParseSpecTest.java | 2 +- .../kafka/KafkaDataSourceMetadata.java | 8 + .../indexing/kafka/KafkaIOConfigTest.java | 3 +- .../indexing/kafka/KafkaIndexTaskTest.java | 2847 ----------------- .../druid/indexer/IndexGeneratorJob.java | 16 +- .../SeekableStreamSequenceNumbers.java | 9 - pom.xml | 3 +- .../IndexerMetadataStorageCoordinator.java | 6 +- .../segment/indexing/DataSchemaTest.java | 2 +- .../server/log/RequestLoggerProviderTest.java | 2 +- 10 files changed, 23 insertions(+), 2875 deletions(-) delete mode 100644 extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java diff --git a/core/src/test/java/org/apache/druid/data/input/impl/ParseSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/ParseSpecTest.java index 1c2e7808e174..eb321cefd59c 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/ParseSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/ParseSpecTest.java @@ -206,7 +206,7 @@ public void testBadTypeSerde() throws IOException expectedException.expect(IllegalArgumentException.class); expectedException.expectCause(CoreMatchers.instanceOf(JsonMappingException.class)); - expectedException.expectMessage("Could not resolve type id 'foo' into a subtype"); + expectedException.expectMessage("Could not resolve type id 'foo' as a subtype of [simple type, class org.apache.druid.data.input.impl.ParseSpec]: known type ids = [csv, javascript, json, jsonLowercase, regex, timeAndDims, tsv]"); mapper.convertValue(mapValue, ParseSpec.class); } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java index 81ea6de396b4..e7809619a805 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java @@ -29,6 +29,14 @@ public class KafkaDataSourceMetadata extends SeekableStreamDataSourceMetadata { + @JsonCreator + public KafkaDataSourceMetadata( + @JsonProperty("partitions") SeekableStreamEndSequenceNumbers kafkaPartitions + ) + { + super(kafkaPartitions); + } + @JsonCreator public KafkaDataSourceMetadata( @JsonProperty("partitions") SeekableStreamSequenceNumbers kafkaPartitions diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java index ac93c1e5b6ac..ab9da01f8fce 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.JsonMappingException; -import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.base.Optional; @@ -51,7 +50,7 @@ public class KafkaIOConfigTest public KafkaIOConfigTest() { mapper = new DefaultObjectMapper(); - mapper.registerModules((Iterable) new KafkaIndexTaskModule().getJacksonModules()); + mapper.registerModules(new KafkaIndexTaskModule().getJacksonModules()); } @Rule diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java deleted file mode 100644 index 783af47ca8bb..000000000000 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ /dev/null @@ -1,2847 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.kafka; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Predicate; -import com.google.common.base.Predicates; -import com.google.common.collect.FluentIterable; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; -import com.google.common.io.Files; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; -import org.apache.curator.test.TestingCluster; -import org.apache.druid.client.cache.CacheConfig; -import org.apache.druid.client.cache.CachePopulatorStats; -import org.apache.druid.client.cache.MapCache; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.FloatDimensionSchema; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.LongDimensionSchema; -import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.discovery.DataNodeService; -import org.apache.druid.discovery.DruidNodeAnnouncer; -import org.apache.druid.discovery.LookupNodeService; -import org.apache.druid.indexer.TaskState; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; -import org.apache.druid.indexing.common.SegmentLoaderFactory; -import org.apache.druid.indexing.common.TaskLock; -import org.apache.druid.indexing.common.TaskReport; -import org.apache.druid.indexing.common.TaskReportFileWriter; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.TaskToolboxFactory; -import org.apache.druid.indexing.common.TestUtils; -import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; -import org.apache.druid.indexing.common.actions.TaskActionClientFactory; -import org.apache.druid.indexing.common.actions.TaskActionToolbox; -import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; -import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.config.TaskStorageConfig; -import org.apache.druid.indexing.common.stats.RowIngestionMeters; -import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; -import org.apache.druid.indexing.common.task.IndexTaskTest; -import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisor; -import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; -import org.apache.druid.indexing.kafka.test.TestBroker; -import org.apache.druid.indexing.overlord.DataSourceMetadata; -import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -import org.apache.druid.indexing.overlord.MetadataTaskStorage; -import org.apache.druid.indexing.overlord.TaskLockbox; -import org.apache.druid.indexing.overlord.TaskStorage; -import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; -import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; -import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; -import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; -import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; -import org.apache.druid.indexing.test.TestDataSegmentKiller; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.concurrent.ListenableFutures; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.common.parsers.JSONPathSpec; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.core.NoopEmitter; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.java.util.metrics.MonitorScheduler; -import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; -import org.apache.druid.metadata.EntryExistsException; -import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; -import org.apache.druid.metadata.TestDerbyConnector; -import org.apache.druid.query.DefaultGenericQueryMetricsFactory; -import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; -import org.apache.druid.query.Druids; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; -import org.apache.druid.query.Query; -import org.apache.druid.query.QueryRunner; -import org.apache.druid.query.QueryRunnerFactory; -import org.apache.druid.query.QueryRunnerFactoryConglomerate; -import org.apache.druid.query.QueryToolChest; -import org.apache.druid.query.QueryWatcher; -import org.apache.druid.query.Result; -import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.CountAggregatorFactory; -import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.query.filter.SelectorDimFilter; -import org.apache.druid.query.scan.ScanQuery; -import org.apache.druid.query.scan.ScanQueryConfig; -import org.apache.druid.query.scan.ScanQueryEngine; -import org.apache.druid.query.scan.ScanQueryQueryToolChest; -import org.apache.druid.query.scan.ScanQueryRunnerFactory; -import org.apache.druid.query.scan.ScanResultValue; -import org.apache.druid.query.spec.QuerySegmentSpec; -import org.apache.druid.query.timeseries.TimeseriesQuery; -import org.apache.druid.query.timeseries.TimeseriesQueryEngine; -import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; -import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; -import org.apache.druid.query.timeseries.TimeseriesResultValue; -import org.apache.druid.segment.DimensionHandlerUtils; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.column.DictionaryEncodedColumn; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.loading.LocalDataSegmentPusher; -import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; -import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; -import org.apache.druid.segment.transform.ExpressionTransform; -import org.apache.druid.segment.transform.TransformSpec; -import org.apache.druid.server.DruidNode; -import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; -import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.utils.CompressionUtils; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.easymock.EasyMock; -import org.joda.time.Interval; -import org.joda.time.Period; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.TreeMap; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executor; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - -import static org.apache.druid.query.QueryPlus.wrap; - -public class KafkaIndexTaskTest -{ - private static final Logger log = new Logger(KafkaIndexTaskTest.class); - private static final ObjectMapper OBJECT_MAPPER = new TestUtils().getTestObjectMapper(); - private static final long POLL_RETRY_MS = 100; - - private static TestingCluster zkServer; - private static TestBroker kafkaServer; - private static ServiceEmitter emitter; - private static ListeningExecutorService taskExec; - private static int topicPostfix; - - static { - new KafkaIndexTaskModule().getJacksonModules().forEach(OBJECT_MAPPER::registerModule); - } - - private final List runningTasks = new ArrayList<>(); - - private long handoffConditionTimeout = 0; - private boolean reportParseExceptions = false; - private boolean logParseExceptions = true; - private Integer maxParseExceptions = null; - private Integer maxSavedParseExceptions = null; - private boolean resetOffsetAutomatically = false; - private boolean doHandoff = true; - private Integer maxRowsPerSegment = null; - private Long maxTotalRows = null; - private Period intermediateHandoffPeriod = null; - - private TaskToolboxFactory toolboxFactory; - private IndexerMetadataStorageCoordinator metadataStorageCoordinator; - private TaskStorage taskStorage; - private TaskLockbox taskLockbox; - private File directory; - private String topic; - private List> records; - private final Set checkpointRequestsHash = new HashSet<>(); - private File reportsFile; - private RowIngestionMetersFactory rowIngestionMetersFactory; - - private static final DataSchema DATA_SCHEMA = new DataSchema( - "test_ds", - OBJECT_MAPPER.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ), - null, - null - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of() - ), - StandardCharsets.UTF_8.name() - ), - Map.class - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - OBJECT_MAPPER - ); - - private static List> generateRecords(String topic) - { - return ImmutableList.of( - new ProducerRecord<>(topic, 0, null, jb("2008", "a", "y", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2009", "b", "y", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2010", "c", "y", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2011", "d", "y", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2011", "e", "y", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable")), - new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable2")), - new ProducerRecord<>(topic, 0, null, null), - new ProducerRecord<>(topic, 0, null, jb("2013", "f", "y", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2049", "f", "y", "notanumber", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2049", "f", "y", "10", "notanumber", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2049", "f", "y", "10", "20.0", "notanumber")), - new ProducerRecord<>(topic, 1, null, jb("2012", "g", "y", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 1, null, jb("2011", "h", "y", "10", "20.0", "1.0")) - ); - } - - private static List> generateSinglePartitionRecords(String topic) - { - return ImmutableList.of( - new ProducerRecord<>(topic, 0, null, jb("2008", "a", "y", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2009", "b", "y", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2010", "c", "y", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2011", "d", "y", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2011", "D", "y", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2012", "e", "y", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2009", "B", "y", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2008", "A", "x", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2009", "B", "x", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2010", "C", "x", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2011", "D", "x", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2011", "d", "x", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2012", "E", "x", "10", "20.0", "1.0")), - new ProducerRecord<>(topic, 0, null, jb("2009", "b", "x", "10", "20.0", "1.0")) - ); - } - - private static String getTopicName() - { - return "topic" + topicPostfix++; - } - - @Rule - public final TemporaryFolder tempFolder = new TemporaryFolder(); - - @Rule - public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); - - @BeforeClass - public static void setupClass() throws Exception - { - emitter = new ServiceEmitter( - "service", - "host", - new NoopEmitter() - ); - emitter.start(); - EmittingLogger.registerEmitter(emitter); - - zkServer = new TestingCluster(1); - zkServer.start(); - - kafkaServer = new TestBroker( - zkServer.getConnectString(), - null, - 1, - ImmutableMap.of("num.partitions", "2") - ); - kafkaServer.start(); - - taskExec = MoreExecutors.listeningDecorator( - Executors.newCachedThreadPool( - Execs.makeThreadFactory("kafka-task-test-%d") - ) - ); - } - - @Before - public void setupTest() throws IOException - { - handoffConditionTimeout = 0; - reportParseExceptions = false; - logParseExceptions = true; - maxParseExceptions = null; - maxSavedParseExceptions = null; - doHandoff = true; - topic = getTopicName(); - records = generateRecords(topic); - reportsFile = File.createTempFile("KafkaIndexTaskTestReports-" + System.currentTimeMillis(), "json"); - makeToolboxFactory(); - } - - @After - public void tearDownTest() - { - synchronized (runningTasks) { - for (Task task : runningTasks) { - task.stopGracefully(toolboxFactory.build(task).getConfig()); - } - - runningTasks.clear(); - } - reportsFile.delete(); - destroyToolboxFactory(); - } - - @AfterClass - public static void tearDownClass() throws Exception - { - taskExec.shutdown(); - taskExec.awaitTermination(9999, TimeUnit.DAYS); - - kafkaServer.close(); - kafkaServer = null; - - zkServer.stop(); - zkServer = null; - - emitter.close(); - } - - @Test(timeout = 60_000L) - public void testRunAfterDataInserted() throws Exception - { - // Insert data - insertData(); - - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture future = runTask(task); - - // Wait for task to exit - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); - SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); - } - - @Test(timeout = 60_000L) - public void testRunBeforeDataInserted() throws Exception - { - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture future = runTask(task); - - // Wait for the task to start reading - while (task.getRunner().getStatus() != Status.READING) { - Thread.sleep(10); - } - - // Insert data - insertData(); - - // Wait for task to exit - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); - SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); - } - - @Test(timeout = 60_000L) - public void testIncrementalHandOff() throws Exception - { - final String baseSequenceName = "sequence0"; - // as soon as any segment has more than one record, incremental publishing should happen - maxRowsPerSegment = 2; - - // Insert data - insertData(); - Map consumerProps = kafkaServer.consumerProperties(); - consumerProps.put("max.poll.records", "1"); - - final SeekableStreamStartSequenceNumbers startPartitions = new SeekableStreamStartSequenceNumbers<>( - topic, - ImmutableMap.of(0, 0L, 1, 0L), - ImmutableSet.of() - ); - // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering - // of events fetched across two partitions from Kafka - final SeekableStreamEndSequenceNumbers checkpoint1 = new SeekableStreamEndSequenceNumbers<>( - topic, - ImmutableMap.of(0, 5L, 1, 0L) - ); - final SeekableStreamEndSequenceNumbers checkpoint2 = new SeekableStreamEndSequenceNumbers<>( - topic, - ImmutableMap.of(0, 4L, 1, 2L) - ); - final SeekableStreamEndSequenceNumbers endPartitions = new SeekableStreamEndSequenceNumbers<>( - topic, - ImmutableMap.of(0, 10L, 1, 2L) - ); - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - baseSequenceName, - startPartitions, - endPartitions, - consumerProps, - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - final ListenableFuture future = runTask(task); - while (task.getRunner().getStatus() != Status.PAUSED) { - Thread.sleep(10); - } - final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); - Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets) - || checkpoint2.getPartitionSequenceNumberMap() - .equals(currentOffsets)); - task.getRunner().setEndOffsets(currentOffsets, false); - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - Assert.assertEquals(1, checkpointRequestsHash.size()); - Assert.assertTrue( - checkpointRequestsHash.contains( - Objects.hash( - DATA_SCHEMA.getDataSource(), - 0, - new KafkaDataSourceMetadata(startPartitions), - new KafkaDataSourceMetadata( - new SeekableStreamEndSequenceNumbers<>(topic, currentOffsets) - ) - ) - ) - ); - - // Check metrics - Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - SegmentDescriptor desc1 = sd(task, "2008/P1D", 0); - SegmentDescriptor desc2 = sd(task, "2009/P1D", 0); - SegmentDescriptor desc3 = sd(task, "2010/P1D", 0); - SegmentDescriptor desc4 = sd(task, "2011/P1D", 0); - SegmentDescriptor desc5 = sd(task, "2011/P1D", 1); - SegmentDescriptor desc6 = sd(task, "2012/P1D", 0); - SegmentDescriptor desc7 = sd(task, "2013/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata( - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L)) - ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); - Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", desc4)) - && ImmutableList.of("h").equals(readSegmentColumn("dim1", desc5))) || - (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", desc4)) - && ImmutableList.of("e").equals(readSegmentColumn("dim1", desc5)))); - Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc6)); - Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7)); - } - - @Test(timeout = 60_000L) - public void testIncrementalHandOffMaxTotalRows() throws Exception - { - final String baseSequenceName = "sequence0"; - // incremental publish should happen every 3 records - maxRowsPerSegment = Integer.MAX_VALUE; - maxTotalRows = 3L; - - // Insert data - int numToAdd = records.size() - 2; - - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (int i = 0; i < numToAdd; i++) { - kafkaProducer.send(records.get(i)).get(); - } - kafkaProducer.commitTransaction(); - } - - Map consumerProps = kafkaServer.consumerProperties(); - consumerProps.put("max.poll.records", "1"); - - final SeekableStreamStartSequenceNumbers startPartitions = new SeekableStreamStartSequenceNumbers<>( - topic, - ImmutableMap.of(0, 0L, 1, 0L), - ImmutableSet.of() - ); - final SeekableStreamEndSequenceNumbers checkpoint1 = new SeekableStreamEndSequenceNumbers<>( - topic, - ImmutableMap.of(0, 3L, 1, 0L) - ); - final SeekableStreamEndSequenceNumbers checkpoint2 = new SeekableStreamEndSequenceNumbers<>( - topic, - ImmutableMap.of(0, 10L, 1, 0L) - ); - - final SeekableStreamEndSequenceNumbers endPartitions = new SeekableStreamEndSequenceNumbers<>( - topic, - ImmutableMap.of(0, 10L, 1, 2L) - ); - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - baseSequenceName, - startPartitions, - endPartitions, - consumerProps, - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - final ListenableFuture future = runTask(task); - while (task.getRunner().getStatus() != Status.PAUSED) { - Thread.sleep(10); - } - final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); - - Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets); - task.getRunner().setEndOffsets(currentOffsets, false); - - while (task.getRunner().getStatus() != Status.PAUSED) { - Thread.sleep(10); - } - - // add remaining records - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (int i = numToAdd; i < records.size(); i++) { - kafkaProducer.send(records.get(i)).get(); - } - kafkaProducer.commitTransaction(); - } - final Map nextOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); - - - Assert.assertTrue(checkpoint2.getPartitionSequenceNumberMap().equals(nextOffsets)); - task.getRunner().setEndOffsets(nextOffsets, false); - - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - Assert.assertEquals(2, checkpointRequestsHash.size()); - Assert.assertTrue( - checkpointRequestsHash.contains( - Objects.hash( - DATA_SCHEMA.getDataSource(), - 0, - new KafkaDataSourceMetadata(startPartitions), - new KafkaDataSourceMetadata( - new SeekableStreamEndSequenceNumbers<>(topic, currentOffsets) - ) - ) - ) - ); - Assert.assertTrue( - checkpointRequestsHash.contains( - Objects.hash( - DATA_SCHEMA.getDataSource(), - 0, - new KafkaDataSourceMetadata( - new SeekableStreamStartSequenceNumbers<>(topic, currentOffsets, ImmutableSet.of()) - ), - new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, nextOffsets)) - ) - ) - ); - - // Check metrics - Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - SegmentDescriptor desc1 = sd(task, "2008/P1D", 0); - SegmentDescriptor desc2 = sd(task, "2009/P1D", 0); - SegmentDescriptor desc3 = sd(task, "2010/P1D", 0); - SegmentDescriptor desc4 = sd(task, "2011/P1D", 0); - SegmentDescriptor desc5 = sd(task, "2011/P1D", 1); - SegmentDescriptor desc6 = sd(task, "2012/P1D", 0); - SegmentDescriptor desc7 = sd(task, "2013/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata( - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L)) - ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata( - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L)) - ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); - Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", desc4)) - && ImmutableList.of("h").equals(readSegmentColumn("dim1", desc5))) || - (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", desc4)) - && ImmutableList.of("e").equals(readSegmentColumn("dim1", desc5)))); - Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc6)); - Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7)); - } - - @Test(timeout = 60_000L) - public void testTimeBasedIncrementalHandOff() throws Exception - { - final String baseSequenceName = "sequence0"; - // as soon as any segment hits maxRowsPerSegment or intermediateHandoffPeriod, incremental publishing should happen - maxRowsPerSegment = Integer.MAX_VALUE; - intermediateHandoffPeriod = new Period().withSeconds(0); - - // Insert data - insertData(); - Map consumerProps = kafkaServer.consumerProperties(); - consumerProps.put("max.poll.records", "1"); - - final SeekableStreamStartSequenceNumbers startPartitions = new SeekableStreamStartSequenceNumbers<>( - topic, - ImmutableMap.of(0, 0L, 1, 0L), - ImmutableSet.of() - ); - // Checkpointing will happen at checkpoint - final SeekableStreamEndSequenceNumbers checkpoint = new SeekableStreamEndSequenceNumbers<>( - topic, - ImmutableMap.of(0, 1L, 1, 0L) - ); - final SeekableStreamEndSequenceNumbers endPartitions = new SeekableStreamEndSequenceNumbers<>( - topic, - ImmutableMap.of(0, 2L, 1, 0L) - ); - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - baseSequenceName, - startPartitions, - endPartitions, - consumerProps, - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - final ListenableFuture future = runTask(task); - - // task will pause for checkpointing - while (task.getRunner().getStatus() != Status.PAUSED) { - Thread.sleep(10); - } - final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); - Assert.assertTrue(checkpoint.getPartitionSequenceNumberMap().equals(currentOffsets)); - task.getRunner().setEndOffsets(currentOffsets, false); - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - Assert.assertEquals(1, checkpointRequestsHash.size()); - Assert.assertTrue( - checkpointRequestsHash.contains( - Objects.hash( - DATA_SCHEMA.getDataSource(), - 0, - new KafkaDataSourceMetadata(startPartitions), - new KafkaDataSourceMetadata( - new SeekableStreamEndSequenceNumbers<>(topic, checkpoint.getPartitionSequenceNumberMap()) - ) - ) - ) - ); - - // Check metrics - Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - SegmentDescriptor desc1 = sd(task, "2008/P1D", 0); - SegmentDescriptor desc2 = sd(task, "2009/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata( - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 2L, 1, 0L)) - ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); - } - - @Test(timeout = 60_000L) - public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception - { - records = generateSinglePartitionRecords(topic); - - final String baseSequenceName = "sequence0"; - // as soon as any segment has more than one record, incremental publishing should happen - maxRowsPerSegment = 2; - - // Insert data - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : records) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } - Map consumerProps = kafkaServer.consumerProperties(); - consumerProps.put("max.poll.records", "1"); - - final SeekableStreamStartSequenceNumbers startPartitions = - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()); - final SeekableStreamEndSequenceNumbers checkpoint1 = - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)); - final SeekableStreamEndSequenceNumbers checkpoint2 = - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 9L)); - final SeekableStreamEndSequenceNumbers endPartitions = - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, Long.MAX_VALUE)); - - final KafkaIndexTask normalReplica = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - baseSequenceName, - startPartitions, - endPartitions, - consumerProps, - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - final KafkaIndexTask staleReplica = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - baseSequenceName, - startPartitions, - endPartitions, - consumerProps, - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture normalReplicaFuture = runTask(normalReplica); - // Simulating one replica is slower than the other - final ListenableFuture staleReplicaFuture = ListenableFutures.transformAsync( - taskExec.submit(() -> { - Thread.sleep(1000); - return staleReplica; - }), - this::runTask - ); - - while (normalReplica.getRunner().getStatus() != Status.PAUSED) { - Thread.sleep(10); - } - staleReplica.getRunner().pause(); - while (staleReplica.getRunner().getStatus() != Status.PAUSED) { - Thread.sleep(10); - } - Map currentOffsets = ImmutableMap.copyOf(normalReplica.getRunner().getCurrentOffsets()); - Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets); - - normalReplica.getRunner().setEndOffsets(currentOffsets, false); - staleReplica.getRunner().setEndOffsets(currentOffsets, false); - - while (normalReplica.getRunner().getStatus() != Status.PAUSED) { - Thread.sleep(10); - } - while (staleReplica.getRunner().getStatus() != Status.PAUSED) { - Thread.sleep(10); - } - currentOffsets = ImmutableMap.copyOf(normalReplica.getRunner().getCurrentOffsets()); - Assert.assertEquals(checkpoint2.getPartitionSequenceNumberMap(), currentOffsets); - currentOffsets = ImmutableMap.copyOf(staleReplica.getRunner().getCurrentOffsets()); - Assert.assertEquals(checkpoint2.getPartitionSequenceNumberMap(), currentOffsets); - - normalReplica.getRunner().setEndOffsets(currentOffsets, true); - staleReplica.getRunner().setEndOffsets(currentOffsets, true); - - Assert.assertEquals(TaskState.SUCCESS, normalReplicaFuture.get().getStatusCode()); - Assert.assertEquals(TaskState.SUCCESS, staleReplicaFuture.get().getStatusCode()); - - Assert.assertEquals(9, normalReplica.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, normalReplica.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, normalReplica.getRunner().getRowIngestionMeters().getThrownAway()); - - Assert.assertEquals(9, staleReplica.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, staleReplica.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, staleReplica.getRunner().getRowIngestionMeters().getThrownAway()); - } - - @Test(timeout = 60_000L) - public void testRunWithMinimumMessageTime() throws Exception - { - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - DateTimes.of("2010"), - null - ) - ); - - final ListenableFuture future = runTask(task); - - // Wait for the task to start reading - while (task.getRunner().getStatus() != Status.READING) { - Thread.sleep(10); - } - - // Insert data - insertData(); - - // Wait for task to exit - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); - SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); - } - - @Test(timeout = 60_000L) - public void testRunWithMaximumMessageTime() throws Exception - { - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - DateTimes.of("2010") - ) - ); - - final ListenableFuture future = runTask(task); - - // Wait for the task to start reading - while (task.getRunner().getStatus() != Status.READING) { - Thread.sleep(10); - } - - // Insert data - insertData(); - - // Wait for task to exit - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - SegmentDescriptor desc1 = sd(task, "2008/P1D", 0); - SegmentDescriptor desc2 = sd(task, "2009/P1D", 0); - SegmentDescriptor desc3 = sd(task, "2010/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); - } - - @Test(timeout = 60_000L) - public void testRunWithTransformSpec() throws Exception - { - final KafkaIndexTask task = createTask( - null, - DATA_SCHEMA.withTransformSpec( - new TransformSpec( - new SelectorDimFilter("dim1", "b", null), - ImmutableList.of( - new ExpressionTransform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) - ) - ) - ), - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture future = runTask(task); - - // Wait for the task to start reading - while (task.getRunner().getStatus() != Status.READING) { - Thread.sleep(10); - } - - // Insert data - insertData(); - - // Wait for task to exit - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - SegmentDescriptor desc1 = sd(task, "2009/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("bb"), readSegmentColumn("dim1t", desc1)); - } - - @Test(timeout = 60_000L) - public void testRunOnNothing() throws Exception - { - // Insert data - insertData(); - - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 2L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture future = runTask(task); - - // Wait for task to exit - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); - } - - @Test(timeout = 60_000L) - public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception - { - handoffConditionTimeout = 5_000; - - // Insert data - insertData(); - - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture future = runTask(task); - - // Wait for task to exit - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); - SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); - } - - @Test(timeout = 60_000L) - public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exception - { - doHandoff = false; - handoffConditionTimeout = 100; - - // Insert data - insertData(); - - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture future = runTask(task); - - // Wait for task to exit - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); - SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata( - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)) - ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); - } - - @Test(timeout = 60_000L) - public void testReportParseExceptions() throws Exception - { - reportParseExceptions = true; - - // these will be ignored because reportParseExceptions is true - maxParseExceptions = 1000; - maxSavedParseExceptions = 2; - - // Insert data - insertData(); - - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 7L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture future = runTask(task); - - // Wait for task to exit - Assert.assertEquals(TaskState.FAILED, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); - Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); - } - - @Test(timeout = 60_000L) - public void testMultipleParseExceptionsSuccess() throws Exception - { - reportParseExceptions = false; - maxParseExceptions = 6; - maxSavedParseExceptions = 6; - - // Insert data - insertData(); - - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture future = runTask(task); - - TaskStatus status = future.get(); - - // Wait for task to exit - Assert.assertEquals(TaskState.SUCCESS, status.getStatusCode()); - Assert.assertEquals(null, status.getErrorMsg()); - - // Check metrics - Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessedWithError()); - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); - SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); - SegmentDescriptor desc3 = sd(task, "2013/P1D", 0); - SegmentDescriptor desc4 = sd(task, "2049/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); - - Map expectedMetrics = ImmutableMap.of( - RowIngestionMeters.BUILD_SEGMENTS, - ImmutableMap.of( - RowIngestionMeters.PROCESSED, 4, - RowIngestionMeters.PROCESSED_WITH_ERROR, 3, - RowIngestionMeters.UNPARSEABLE, 3, - RowIngestionMeters.THROWN_AWAY, 1 - ) - ); - Assert.assertEquals(expectedMetrics, reportData.getRowStats()); - - Map unparseableEvents = ImmutableMap.of( - RowIngestionMeters.BUILD_SEGMENTS, - Arrays.asList( - "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=10, dimFloat=20.0, met1=notanumber}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [Unable to parse value[notanumber] for field[met1],]", - "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=10, dimFloat=notanumber, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [could not convert value [notanumber] to float,]", - "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=notanumber, dimFloat=20.0, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [could not convert value [notanumber] to long,]", - "Unable to parse row [unparseable2]", - "Unable to parse row [unparseable]", - "Encountered row with timestamp that cannot be represented as a long: [MapBasedInputRow{timestamp=246140482-04-24T15:36:27.903Z, event={timestamp=246140482-04-24T15:36:27.903Z, dim1=x, dim2=z, dimLong=10, dimFloat=20.0, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}]" - ) - ); - - Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents()); - } - - @Test(timeout = 60_000L) - public void testMultipleParseExceptionsFailure() throws Exception - { - reportParseExceptions = false; - maxParseExceptions = 2; - maxSavedParseExceptions = 2; - - // Insert data - insertData(); - - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture future = runTask(task); - - TaskStatus status = future.get(); - - // Wait for task to exit - Assert.assertEquals(TaskState.FAILED, status.getStatusCode()); - IndexTaskTest.checkTaskStatusErrorMsgForParseExceptionsExceeded(status); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessedWithError()); - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); - Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); - - IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); - - Map expectedMetrics = ImmutableMap.of( - RowIngestionMeters.BUILD_SEGMENTS, - ImmutableMap.of( - RowIngestionMeters.PROCESSED, 3, - RowIngestionMeters.PROCESSED_WITH_ERROR, 0, - RowIngestionMeters.UNPARSEABLE, 3, - RowIngestionMeters.THROWN_AWAY, 0 - ) - ); - Assert.assertEquals(expectedMetrics, reportData.getRowStats()); - - Map unparseableEvents = ImmutableMap.of( - RowIngestionMeters.BUILD_SEGMENTS, - Arrays.asList( - "Unable to parse row [unparseable2]", - "Unable to parse row [unparseable]" - ) - ); - - Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents()); - } - - @Test(timeout = 60_000L) - public void testRunReplicas() throws Exception - { - final KafkaIndexTask task1 = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - final KafkaIndexTask task2 = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture future1 = runTask(task1); - final ListenableFuture future2 = runTask(task2); - - // Insert data - insertData(); - - // Wait for tasks to exit - Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); - Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published segments & metadata - SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0); - SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); - } - - @Test(timeout = 60_000L) - public void testRunConflicting() throws Exception - { - final KafkaIndexTask task1 = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - final KafkaIndexTask task2 = createTask( - null, - new KafkaIndexTaskIOConfig( - 1, - "sequence1", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 3L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - // Insert data - insertData(); - - // Run first task - final ListenableFuture future1 = runTask(task1); - Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); - - // Run second task - final ListenableFuture future2 = runTask(task2); - Assert.assertEquals(TaskState.FAILED, future2.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published segments & metadata, should all be from the first task - SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0); - SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); - } - - @Test(timeout = 60_000L) - public void testRunConflictingWithoutTransactions() throws Exception - { - final KafkaIndexTask task1 = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - false, - null, - null - ) - ); - final KafkaIndexTask task2 = createTask( - null, - new KafkaIndexTaskIOConfig( - 1, - "sequence1", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 3L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - false, - null, - null - ) - ); - - // Insert data - insertData(); - - // Run first task - final ListenableFuture future1 = runTask(task1); - Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); - - // Check published segments & metadata - SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0); - SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); - Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); - - // Run second task - final ListenableFuture future2 = runTask(task2); - Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published segments & metadata - SegmentDescriptor desc3 = sd(task2, "2011/P1D", 1); - SegmentDescriptor desc4 = sd(task2, "2013/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); - Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc3)); - Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc4)); - } - - @Test(timeout = 60_000L) - public void testRunOneTaskTwoPartitions() throws Exception - { - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L, 1, 0L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 2L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture future = runTask(task); - - // Insert data - insertData(); - - // Wait for tasks to exit - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(5, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published segments & metadata - SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); - SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); - // desc3 will not be created in KafkaIndexTask (0.12.x) as it does not create per Kafka partition Druid segments - SegmentDescriptor desc3 = sd(task, "2011/P1D", 1); - SegmentDescriptor desc4 = sd(task, "2012/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc4), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata( - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 2L)) - ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc4)); - - // Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically - Assert.assertEquals( - ImmutableSet.of(ImmutableList.of("d", "e", "h")), - ImmutableSet.of(readSegmentColumn("dim1", desc2)) - ); - } - - @Test(timeout = 60_000L) - public void testRunTwoTasksTwoPartitions() throws Exception - { - final KafkaIndexTask task1 = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - final KafkaIndexTask task2 = createTask( - null, - new KafkaIndexTaskIOConfig( - 1, - "sequence1", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(1, 0L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(1, 1L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture future1 = runTask(task1); - final ListenableFuture future2 = runTask(task2); - - // Insert data - insertData(); - - // Wait for tasks to exit - Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); - Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published segments & metadata - SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0); - SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0); - SegmentDescriptor desc3 = sd(task2, "2012/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata( - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 1L)) - ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); - Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc3)); - } - - @Test(timeout = 60_000L) - public void testRestore() throws Exception - { - final KafkaIndexTask task1 = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture future1 = runTask(task1); - - // Insert some data, but not enough for the task to finish - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.limit(records, 4)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } - - while (countEvents(task1) != 2) { - Thread.sleep(25); - } - - Assert.assertEquals(2, countEvents(task1)); - - // Stop without publishing segment - task1.stopGracefully(toolboxFactory.build(task1).getConfig()); - unlockAppenderatorBasePersistDirForTask(task1); - - Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); - - // Start a new task - final KafkaIndexTask task2 = createTask( - task1.getId(), - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture future2 = runTask(task2); - - // Insert remaining data - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.skip(records, 4)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } - - // Wait for task to exit - - Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(2, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published segments & metadata - SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0); - SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); - } - - @Test(timeout = 60_000L) - public void testRestoreAfterPersistingSequences() throws Exception - { - records = generateSinglePartitionRecords(topic); - maxRowsPerSegment = 2; - Map consumerProps = kafkaServer.consumerProperties(); - consumerProps.put("max.poll.records", "1"); - - final KafkaIndexTask task1 = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)), - consumerProps, - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final SeekableStreamStartSequenceNumbers checkpoint = new SeekableStreamStartSequenceNumbers<>( - topic, - ImmutableMap.of(0, 5L), - ImmutableSet.of(0) - ); - - final ListenableFuture future1 = runTask(task1); - - // Insert some data, but not enough for the task to finish - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.limit(records, 5)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } - - while (task1.getRunner().getStatus() != Status.PAUSED) { - Thread.sleep(10); - } - final Map currentOffsets = ImmutableMap.copyOf(task1.getRunner().getCurrentOffsets()); - Assert.assertEquals(checkpoint.getPartitionSequenceNumberMap(), currentOffsets); - // Set endOffsets to persist sequences - task1.getRunner().setEndOffsets(ImmutableMap.of(0, 5L), false); - - // Stop without publishing segment - task1.stopGracefully(toolboxFactory.build(task1).getConfig()); - unlockAppenderatorBasePersistDirForTask(task1); - - Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); - - // Start a new task - final KafkaIndexTask task2 = createTask( - task1.getId(), - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)), - consumerProps, - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture future2 = runTask(task2); - // Wait for the task to start reading - - // Insert remaining data - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.skip(records, 5)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } - - // Wait for task to exit - Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(5, task1.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(4, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published segments & metadata - SegmentDescriptor desc1 = sd(task1, "2008/P1D", 0); - SegmentDescriptor desc2 = sd(task1, "2008/P1D", 1); - SegmentDescriptor desc3 = sd(task1, "2009/P1D", 0); - SegmentDescriptor desc4 = sd(task1, "2009/P1D", 1); - SegmentDescriptor desc5 = sd(task1, "2010/P1D", 0); - SegmentDescriptor desc6 = sd(task1, "2011/P1D", 0); - SegmentDescriptor desc7 = sd(task1, "2012/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - } - - @Test(timeout = 60_000L) - public void testRunWithPauseAndResume() throws Exception - { - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture future = runTask(task); - - // Insert some data, but not enough for the task to finish - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.limit(records, 4)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.flush(); - kafkaProducer.commitTransaction(); - } - - while (countEvents(task) != 2) { - Thread.sleep(25); - } - - Assert.assertEquals(2, countEvents(task)); - Assert.assertEquals(Status.READING, task.getRunner().getStatus()); - - Map currentOffsets = OBJECT_MAPPER.readValue( - task.getRunner().pause().getEntity().toString(), - new TypeReference>() - { - } - ); - Assert.assertEquals(Status.PAUSED, task.getRunner().getStatus()); - // Insert remaining data - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.skip(records, 4)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } - - try { - future.get(10, TimeUnit.SECONDS); - Assert.fail("Task completed when it should have been paused"); - } - catch (TimeoutException e) { - // carry on.. - } - - Assert.assertEquals(currentOffsets, task.getRunner().getCurrentOffsets()); - - task.getRunner().resume(); - - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - Assert.assertEquals(task.getRunner().getEndOffsets(), task.getRunner().getCurrentOffsets()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); - SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); - } - - @Test(timeout = 60_000L) - public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception - { - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - runTask(task); - - while (!task.getRunner().getStatus().equals(Status.READING)) { - Thread.sleep(2000); - } - - task.getRunner().pause(); - - while (!task.getRunner().getStatus().equals(Status.PAUSED)) { - Thread.sleep(25); - } - } - - @Test(timeout = 60_000L) - public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAvailable() throws Exception - { - resetOffsetAutomatically = true; - // Insert data - insertData(); - - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 200L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 500L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - runTask(task); - - while (!task.getRunner().getStatus().equals(Status.READING)) { - Thread.sleep(20); - } - - for (int i = 0; i < 5; i++) { - Assert.assertEquals(task.getRunner().getStatus(), Status.READING); - // Offset should not be reset - Assert.assertTrue(task.getRunner().getCurrentOffsets().get(0) == 200L); - } - } - - @Test(timeout = 60_000L) - public void testRunContextSequenceAheadOfStartingOffsets() throws Exception - { - // Insert data - insertData(); - - final TreeMap> sequences = new TreeMap<>(); - // Here the sequence number is 1 meaning that one incremental handoff was done by the failed task - // and this task should start reading from offset 2 for partition 0 - sequences.put(1, ImmutableMap.of(0, 2L)); - final Map context = new HashMap<>(); - context.put( - SeekableStreamSupervisor.CHECKPOINTS_CTX_KEY, - OBJECT_MAPPER.writerFor(KafkaSupervisor.CHECKPOINTS_TYPE_REF).writeValueAsString(sequences) - ); - - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - // task should ignore these and use sequence info sent in the context - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ), - context - ); - - final ListenableFuture future = runTask(task); - - // Wait for task to exit - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); - SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); - } - - @Test(timeout = 60_000L) - public void testRunWithDuplicateRequest() throws Exception - { - // Insert data - insertData(); - - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 200L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 500L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - runTask(task); - - while (!task.getRunner().getStatus().equals(Status.READING)) { - Thread.sleep(20); - } - - // first setEndOffsets request - task.getRunner().pause(); - task.getRunner().setEndOffsets(ImmutableMap.of(0, 500L), true); - Assert.assertEquals(Status.READING, task.getRunner().getStatus()); - - // duplicate setEndOffsets request - task.getRunner().pause(); - task.getRunner().setEndOffsets(ImmutableMap.of(0, 500L), true); - Assert.assertEquals(Status.READING, task.getRunner().getStatus()); - } - - @Test(timeout = 60_000L) - public void testRunTransactionModeRollback() throws Exception - { - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - "sequence0", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L)), - kafkaServer.consumerProperties(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final ListenableFuture future = runTask(task); - - // Insert 2 records initially - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.limit(records, 2)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } - - while (countEvents(task) != 2) { - Thread.sleep(25); - } - - Assert.assertEquals(2, countEvents(task)); - Assert.assertEquals(Status.READING, task.getRunner().getStatus()); - - //verify the 2 indexed records - final QuerySegmentSpec firstInterval = OBJECT_MAPPER.readValue( - "\"2008/2010\"", QuerySegmentSpec.class - ); - Iterable scanResultValues = scanData(task, firstInterval); - Assert.assertEquals(2, Iterables.size(scanResultValues)); - - // Insert 3 more records and rollback - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.limit(Iterables.skip(records, 2), 3)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.flush(); - kafkaProducer.abortTransaction(); - } - - Assert.assertEquals(2, countEvents(task)); - Assert.assertEquals(Status.READING, task.getRunner().getStatus()); - - final QuerySegmentSpec rollbackedInterval = OBJECT_MAPPER.readValue( - "\"2010/2012\"", QuerySegmentSpec.class - ); - scanResultValues = scanData(task, rollbackedInterval); - //verify that there are no records indexed in the rollbacked time period - Assert.assertEquals(0, Iterables.size(scanResultValues)); - - // Insert remaining data - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : Iterables.skip(records, 5)) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } - - final QuerySegmentSpec endInterval = OBJECT_MAPPER.readValue( - "\"2008/2049\"", QuerySegmentSpec.class - ); - Iterable scanResultValues1 = scanData(task, endInterval); - Assert.assertEquals(2, Iterables.size(scanResultValues1)); - - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - Assert.assertEquals(task.getRunner().getEndOffsets(), task.getRunner().getCurrentOffsets()); - - // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - SegmentDescriptor desc1 = sd(task, "2008/P1D", 0); - SegmentDescriptor desc2 = sd(task, "2009/P1D", 0); - SegmentDescriptor desc3 = sd(task, "2013/P1D", 0); - SegmentDescriptor desc4 = sd(task, "2049/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); - Assert.assertEquals( - new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); - Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc3)); - Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc4)); - } - - @Test(timeout = 60_000L) - public void testCanStartFromLaterThanEarliestOffset() throws Exception - { - final String baseSequenceName = "sequence0"; - maxRowsPerSegment = Integer.MAX_VALUE; - maxTotalRows = null; - - // Insert data - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : records) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } - - Map consumerProps = kafkaServer.consumerProperties(); - consumerProps.put("max.poll.records", "1"); - - final SeekableStreamStartSequenceNumbers startPartitions = new SeekableStreamStartSequenceNumbers<>( - topic, - ImmutableMap.of(0, 0L, 1, 1L), - ImmutableSet.of() - ); - - final SeekableStreamEndSequenceNumbers endPartitions = new SeekableStreamEndSequenceNumbers<>( - topic, - ImmutableMap.of(0, 10L, 1, 2L) - ); - - final KafkaIndexTask task = createTask( - null, - new KafkaIndexTaskIOConfig( - 0, - baseSequenceName, - startPartitions, - endPartitions, - consumerProps, - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - final ListenableFuture future = runTask(task); - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - } - - @Test - public void testSerde() throws Exception - { - // This is both a serde test and a regression test for https://github.com/apache/incubator-druid/issues/7724. - - final KafkaIndexTask task = createTask( - "taskid", - DATA_SCHEMA.withTransformSpec( - new TransformSpec( - null, - ImmutableList.of(new ExpressionTransform("beep", "nofunc()", ExprMacroTable.nil())) - ) - ), - new KafkaIndexTaskIOConfig( - 0, - "sequence", - new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(), ImmutableSet.of()), - new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of()), - ImmutableMap.of(), - KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, - true, - null, - null - ) - ); - - final Task task1 = OBJECT_MAPPER.readValue(OBJECT_MAPPER.writeValueAsBytes(task), Task.class); - Assert.assertEquals(task, task1); - } - - private List scanData(final Task task, QuerySegmentSpec spec) - { - ScanQuery query = new Druids.ScanQueryBuilder().dataSource( - DATA_SCHEMA.getDataSource()).intervals(spec).build(); - List results = - task.getQueryRunner(query).run(wrap(query), new HashMap<>()).toList(); - return results; - } - - private void insertData() throws ExecutionException, InterruptedException - { - try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { - kafkaProducer.initTransactions(); - kafkaProducer.beginTransaction(); - for (ProducerRecord record : records) { - kafkaProducer.send(record).get(); - } - kafkaProducer.commitTransaction(); - } - } - - private ListenableFuture runTask(final Task task) - { - try { - taskStorage.insert(task, TaskStatus.running(task.getId())); - } - catch (EntryExistsException e) { - // suppress - } - taskLockbox.syncFromStorage(); - final TaskToolbox toolbox = toolboxFactory.build(task); - synchronized (runningTasks) { - runningTasks.add(task); - } - return taskExec.submit( - () -> { - try { - if (task.isReady(toolbox.getTaskActionClient())) { - return task.run(toolbox); - } else { - throw new ISE("Task is not ready"); - } - } - catch (Exception e) { - log.warn(e, "Task failed"); - return TaskStatus.failure(task.getId()); - } - } - ); - } - - private TaskLock getLock(final Task task, final Interval interval) - { - return Iterables.find( - taskLockbox.findLocksForTask(task), - new Predicate() - { - @Override - public boolean apply(TaskLock lock) - { - return lock.getInterval().contains(interval); - } - } - ); - } - - private KafkaIndexTask createTask( - final String taskId, - final KafkaIndexTaskIOConfig ioConfig - ) throws JsonProcessingException - { - return createTask(taskId, DATA_SCHEMA, ioConfig); - } - - private KafkaIndexTask createTask( - final String taskId, - final KafkaIndexTaskIOConfig ioConfig, - final Map context - ) throws JsonProcessingException - { - return createTask(taskId, DATA_SCHEMA, ioConfig, context); - } - - private KafkaIndexTask createTask( - final String taskId, - final DataSchema dataSchema, - final KafkaIndexTaskIOConfig ioConfig - ) throws JsonProcessingException - { - final Map context = new HashMap<>(); - return createTask(taskId, dataSchema, ioConfig, context); - } - - private KafkaIndexTask createTask( - final String taskId, - final DataSchema dataSchema, - final KafkaIndexTaskIOConfig ioConfig, - final Map context - ) throws JsonProcessingException - { - final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig( - 1000, - null, - maxRowsPerSegment, - maxTotalRows, - new Period("P1Y"), - null, - null, - null, - true, - reportParseExceptions, - handoffConditionTimeout, - resetOffsetAutomatically, - null, - intermediateHandoffPeriod, - logParseExceptions, - maxParseExceptions, - maxSavedParseExceptions - ); - if (!context.containsKey(SeekableStreamSupervisor.CHECKPOINTS_CTX_KEY)) { - final TreeMap> checkpoints = new TreeMap<>(); - checkpoints.put(0, ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap()); - final String checkpointsJson = OBJECT_MAPPER - .writerFor(KafkaSupervisor.CHECKPOINTS_TYPE_REF) - .writeValueAsString(checkpoints); - context.put(SeekableStreamSupervisor.CHECKPOINTS_CTX_KEY, checkpointsJson); - } - - final KafkaIndexTask task = new KafkaIndexTask( - taskId, - null, - cloneDataSchema(dataSchema), - tuningConfig, - ioConfig, - context, - null, - null, - rowIngestionMetersFactory, - OBJECT_MAPPER - ); - task.setPollRetryMs(POLL_RETRY_MS); - return task; - } - - private static DataSchema cloneDataSchema(final DataSchema dataSchema) - { - return new DataSchema( - dataSchema.getDataSource(), - dataSchema.getParserMap(), - dataSchema.getAggregators(), - dataSchema.getGranularitySpec(), - dataSchema.getTransformSpec(), - OBJECT_MAPPER - ); - } - - private QueryRunnerFactoryConglomerate makeTimeseriesAndScanConglomerate() - { - IntervalChunkingQueryRunnerDecorator queryRunnerDecorator = new IntervalChunkingQueryRunnerDecorator( - null, - null, - null - ) - { - @Override - public QueryRunner decorate(QueryRunner delegate, QueryToolChest> toolChest) - { - return delegate; - } - }; - return new DefaultQueryRunnerFactoryConglomerate( - ImmutableMap., QueryRunnerFactory>builder() - .put( - TimeseriesQuery.class, - new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(queryRunnerDecorator), - new TimeseriesQueryEngine(), - new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - // do nothing - } - } - ) - ) - .put( - ScanQuery.class, - new ScanQueryRunnerFactory( - new ScanQueryQueryToolChest( - new ScanQueryConfig(), - new DefaultGenericQueryMetricsFactory(TestHelper.makeJsonMapper()) - ), - new ScanQueryEngine(), - new ScanQueryConfig() - ) - ) - .build() - ); - } - - private void makeToolboxFactory() throws IOException - { - directory = tempFolder.newFolder(); - final TestUtils testUtils = new TestUtils(); - rowIngestionMetersFactory = testUtils.getRowIngestionMetersFactory(); - final ObjectMapper objectMapper = testUtils.getTestObjectMapper(); - for (Module module : new KafkaIndexTaskModule().getJacksonModules()) { - objectMapper.registerModule(module); - } - final TaskConfig taskConfig = new TaskConfig( - new File(directory, "baseDir").getPath(), - new File(directory, "baseTaskDir").getPath(), - null, - 50000, - null, - true, - null, - null - ); - final TestDerbyConnector derbyConnector = derby.getConnector(); - derbyConnector.createDataSourceTable(); - derbyConnector.createPendingSegmentsTable(); - derbyConnector.createSegmentTable(); - derbyConnector.createRulesTable(); - derbyConnector.createConfigTable(); - derbyConnector.createTaskTables(); - derbyConnector.createAuditTable(); - taskStorage = new MetadataTaskStorage( - derbyConnector, - new TaskStorageConfig(null), - new DerbyMetadataStorageActionHandlerFactory( - derbyConnector, - derby.metadataTablesConfigSupplier().get(), - objectMapper - ) - ); - metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( - testUtils.getTestObjectMapper(), - derby.metadataTablesConfigSupplier().get(), - derbyConnector - ); - taskLockbox = new TaskLockbox(taskStorage); - final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( - taskLockbox, - taskStorage, - metadataStorageCoordinator, - emitter, - new SupervisorManager(null) - { - @Override - public boolean checkPointDataSourceMetadata( - String supervisorId, - @Nullable Integer taskGroupId, - String baseSequenceName, - @Nullable DataSourceMetadata previousDataSourceMetadata, - @Nullable DataSourceMetadata currentDataSourceMetadata - ) - { - log.info("Adding checkpoint hash to the set"); - checkpointRequestsHash.add( - Objects.hash( - supervisorId, - taskGroupId, - previousDataSourceMetadata, - currentDataSourceMetadata - ) - ); - return true; - } - } - ); - final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( - taskStorage, - taskActionToolbox, - new TaskAuditLogConfig(false) - ); - final SegmentHandoffNotifierFactory handoffNotifierFactory = dataSource -> new SegmentHandoffNotifier() - { - @Override - public boolean registerSegmentHandoffCallback( - SegmentDescriptor descriptor, - Executor exec, - Runnable handOffRunnable - ) - { - if (doHandoff) { - // Simulate immediate handoff - exec.execute(handOffRunnable); - } - return true; - } - - @Override - public void start() - { - //Noop - } - - @Override - public void close() - { - //Noop - } - }; - final LocalDataSegmentPusherConfig dataSegmentPusherConfig = new LocalDataSegmentPusherConfig(); - dataSegmentPusherConfig.storageDirectory = getSegmentDirectory(); - final DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher(dataSegmentPusherConfig); - toolboxFactory = new TaskToolboxFactory( - taskConfig, - taskActionClientFactory, - emitter, - dataSegmentPusher, - new TestDataSegmentKiller(), - null, // DataSegmentMover - null, // DataSegmentArchiver - new TestDataSegmentAnnouncer(), - EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), - handoffNotifierFactory, - this::makeTimeseriesAndScanConglomerate, - Execs.directExecutor(), // queryExecutorService - EasyMock.createMock(MonitorScheduler.class), - new SegmentLoaderFactory(null, testUtils.getTestObjectMapper()), - testUtils.getTestObjectMapper(), - testUtils.getTestIndexIO(), - MapCache.create(1024), - new CacheConfig(), - new CachePopulatorStats(), - testUtils.getTestIndexMergerV9(), - EasyMock.createNiceMock(DruidNodeAnnouncer.class), - EasyMock.createNiceMock(DruidNode.class), - new LookupNodeService("tier"), - new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0), - new TaskReportFileWriter(reportsFile) - ); - } - - private void destroyToolboxFactory() - { - toolboxFactory = null; - taskStorage = null; - taskLockbox = null; - metadataStorageCoordinator = null; - } - - private Set publishedDescriptors() - { - return FluentIterable.from( - metadataStorageCoordinator.getUsedSegmentsForInterval( - DATA_SCHEMA.getDataSource(), - Intervals.of("0000/3000") - ) - ).transform(DataSegment::toDescriptor).toSet(); - } - - private void unlockAppenderatorBasePersistDirForTask(KafkaIndexTask task) - throws NoSuchMethodException, InvocationTargetException, IllegalAccessException - { - Method unlockBasePersistDir = ((AppenderatorImpl) task.getAppenderator()).getClass() - .getDeclaredMethod( - "unlockBasePersistDirectory"); - unlockBasePersistDir.setAccessible(true); - unlockBasePersistDir.invoke(task.getAppenderator()); - } - - private File getSegmentDirectory() - { - return new File(directory, "segments"); - } - - private List readSegmentColumn(final String column, final SegmentDescriptor descriptor) throws IOException - { - File indexBasePath = new File( - StringUtils.format( - "%s/%s/%s_%s/%s/%d", - getSegmentDirectory(), - DATA_SCHEMA.getDataSource(), - descriptor.getInterval().getStart(), - descriptor.getInterval().getEnd(), - descriptor.getVersion(), - descriptor.getPartitionNumber() - ) - ); - - File outputLocation = new File( - directory, - StringUtils.format( - "%s_%s_%s_%s", - descriptor.getInterval().getStart(), - descriptor.getInterval().getEnd(), - descriptor.getVersion(), - descriptor.getPartitionNumber() - ) - ); - outputLocation.mkdir(); - CompressionUtils.unzip( - Files.asByteSource(new File(indexBasePath.listFiles()[0], "index.zip")), - outputLocation, - Predicates.alwaysFalse(), - false - ); - IndexIO indexIO = new TestUtils().getTestIndexIO(); - QueryableIndex index = indexIO.loadIndex(outputLocation); - DictionaryEncodedColumn theColumn = (DictionaryEncodedColumn) index.getColumnHolder(column) - .getColumn(); - List values = new ArrayList<>(); - for (int i = 0; i < theColumn.length(); i++) { - int id = theColumn.getSingleValueRow(i); - String value = theColumn.lookupName(id); - values.add(value); - } - return values; - } - - public long countEvents(final Task task) - { - // Do a query. - TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() - .dataSource(DATA_SCHEMA.getDataSource()) - .aggregators( - ImmutableList.of( - new LongSumAggregatorFactory("rows", "rows") - ) - ).granularity(Granularities.ALL) - .intervals("0000/3000") - .build(); - - List> results = - task.getQueryRunner(query).run(wrap(query), ImmutableMap.of()).toList(); - - return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows")); - } - - private static byte[] jb(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) - { - try { - return new ObjectMapper().writeValueAsBytes( - ImmutableMap.builder() - .put("timestamp", timestamp) - .put("dim1", dim1) - .put("dim2", dim2) - .put("dimLong", dimLong) - .put("dimFloat", dimFloat) - .put("met1", met1) - .build() - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - private SegmentDescriptor sd(final Task task, final String intervalString, final int partitionNum) - { - final Interval interval = Intervals.of(intervalString); - return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum); - } - - private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOException - { - Map taskReports = OBJECT_MAPPER.readValue( - reportsFile, - new TypeReference>() - { - } - ); - return IngestionStatsAndErrorsTaskReportData.getPayloadFromTaskReports( - taskReports - ); - } -} diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java index 6c032b06e7bb..6ed96e61bbfb 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java @@ -77,6 +77,7 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; +import java.io.InputStream; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Iterator; @@ -113,13 +114,13 @@ public static List getPublishedSegments(HadoopDruidIndexerConfig co final Path descriptorInfoDir = config.makeDescriptorInfoDir(); - try { - FileSystem fs = descriptorInfoDir.getFileSystem(conf); - + try (FileSystem fs = descriptorInfoDir.getFileSystem(conf)) { for (FileStatus status : fs.listStatus(descriptorInfoDir)) { - final DataSegment segment = jsonMapper.readValue(fs.open(status.getPath()), DataSegment.class); - publishedSegmentsBuilder.add(segment); - log.info("Adding segment %s to the list of published segments", segment.getId()); + try (final InputStream input = fs.open(status.getPath())) { + final DataSegment segment = jsonMapper.readValue(input, DataSegment.class); + publishedSegmentsBuilder.add(segment); + log.info("Adding segment %s to the list of published segments", segment.getId()); + } } } catch (FileNotFoundException e) { @@ -133,9 +134,8 @@ public static List getPublishedSegments(HadoopDruidIndexerConfig co catch (IOException e) { throw new RuntimeException(e); } - List publishedSegments = publishedSegmentsBuilder.build(); - return publishedSegments; + return publishedSegmentsBuilder.build(); } private final HadoopDruidIndexerConfig config; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSequenceNumbers.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSequenceNumbers.java index a790974e25f1..3d28bad049c6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSequenceNumbers.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSequenceNumbers.java @@ -19,19 +19,10 @@ package org.apache.druid.indexing.seekablestream; -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonSubTypes.Type; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.fasterxml.jackson.annotation.JsonTypeInfo.Id; import org.apache.druid.indexing.overlord.DataSourceMetadata; import java.util.Map; -@JsonTypeInfo(use = Id.NAME, property = "type", defaultImpl = SeekableStreamEndSequenceNumbers.class) -@JsonSubTypes({ - @Type(name = "start", value = SeekableStreamStartSequenceNumbers.class), - @Type(name = "end", value = SeekableStreamEndSequenceNumbers.class) -}) public interface SeekableStreamSequenceNumbers { /** diff --git a/pom.xml b/pom.xml index daadaaa62443..3b6f6bfdaf18 100644 --- a/pom.xml +++ b/pom.xml @@ -84,8 +84,7 @@ 4.1.0 9.4.10.v20180503 1.19.3 - - 2.6.7 + 2.9.9 2.5 3.10.6.Final diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index 95c1fce15c6e..a6306c08cf4d 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -40,8 +40,6 @@ public interface IndexerMetadataStorageCoordinator * @param interval The interval for which all applicable and used datasources are requested. Start is inclusive, end is exclusive * * @return The DataSegments which include data in the requested interval. These segments may contain data outside the requested interval. - * - * @throws IOException */ default List getUsedSegmentsForInterval(String dataSource, Interval interval) { @@ -65,8 +63,6 @@ default List getUsedSegmentsForInterval(String dataSource, Interval * @param intervals The intervals for which all applicable and used datasources are requested. * * @return The DataSegments which include data in the requested intervals. These segments may contain data outside the requested interval. - * - * @throws IOException */ List getUsedSegmentsForIntervals(String dataSource, List intervals); @@ -77,6 +73,8 @@ default List getUsedSegmentsForInterval(String dataSource, Interval * @param segments set of segments to add * * @return set of segments actually added + * + * @throws IOException */ Set announceHistoricalSegments(Set segments) throws IOException; diff --git a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index c059f29c38d1..89a90380eaf6 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java @@ -280,7 +280,7 @@ public void testSerdeWithInvalidParserMap() throws Exception expectedException.expect(CoreMatchers.instanceOf(IllegalArgumentException.class)); expectedException.expectCause(CoreMatchers.instanceOf(JsonMappingException.class)); expectedException.expectMessage( - "Instantiation of [simple type, class org.apache.druid.data.input.impl.StringInputRowParser] value failed: parseSpec" + "Cannot construct instance of `org.apache.druid.data.input.impl.StringInputRowParser`, problem: parseSpec" ); // Jackson creates a default type parser (StringInputRowParser) for an invalid type. diff --git a/server/src/test/java/org/apache/druid/server/log/RequestLoggerProviderTest.java b/server/src/test/java/org/apache/druid/server/log/RequestLoggerProviderTest.java index 49142550a66a..0f9ece45a118 100644 --- a/server/src/test/java/org/apache/druid/server/log/RequestLoggerProviderTest.java +++ b/server/src/test/java/org/apache/druid/server/log/RequestLoggerProviderTest.java @@ -85,7 +85,7 @@ public void testLoggerPropertiesWithNoType() ); expectedException.expect(ProvisionException.class); - expectedException.expectMessage("missing property 'type'"); + expectedException.expectMessage("missing type id property 'type'"); configurator.configurate( properties, From a119e8f61f088acf164cba5d513ccf05176c7b9e Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 20 May 2019 19:37:50 +0200 Subject: [PATCH 02/26] Add helper for Kinesis --- .../druid/indexing/kinesis/KinesisDataSourceMetadata.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java index d3c1630cf909..f2a4b841289f 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java @@ -28,6 +28,14 @@ public class KinesisDataSourceMetadata extends SeekableStreamDataSourceMetadata { + @JsonCreator + public KinesisDataSourceMetadata( + @JsonProperty("partitions") SeekableStreamEndSequenceNumbers kinesisPartitions + ) + { + super(kinesisPartitions); + } + @JsonCreator public KinesisDataSourceMetadata( @JsonProperty("partitions") SeekableStreamSequenceNumbers kinesisPartitions From 7d0a4650df91976722928266233679783dc4ee08 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 27 May 2019 13:27:55 +0200 Subject: [PATCH 03/26] Restore missing test --- .../indexing/kafka/KafkaIndexTaskTest.java | 2847 +++++++++++++++++ 1 file changed, 2847 insertions(+) create mode 100644 extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java new file mode 100644 index 000000000000..783af47ca8bb --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -0,0 +1,2847 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kafka; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.io.Files; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import org.apache.curator.test.TestingCluster; +import org.apache.druid.client.cache.CacheConfig; +import org.apache.druid.client.cache.CachePopulatorStats; +import org.apache.druid.client.cache.MapCache; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.FloatDimensionSchema; +import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.LongDimensionSchema; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.discovery.DataNodeService; +import org.apache.druid.discovery.DruidNodeAnnouncer; +import org.apache.druid.discovery.LookupNodeService; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; +import org.apache.druid.indexing.common.SegmentLoaderFactory; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.indexing.common.TaskReportFileWriter; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.TaskToolboxFactory; +import org.apache.druid.indexing.common.TestUtils; +import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; +import org.apache.druid.indexing.common.actions.TaskActionClientFactory; +import org.apache.druid.indexing.common.actions.TaskActionToolbox; +import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.config.TaskStorageConfig; +import org.apache.druid.indexing.common.stats.RowIngestionMeters; +import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.IndexTaskTest; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisor; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; +import org.apache.druid.indexing.kafka.test.TestBroker; +import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.MetadataTaskStorage; +import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; +import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; +import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; +import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; +import org.apache.druid.indexing.test.TestDataSegmentKiller; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.concurrent.ListenableFutures; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.core.NoopEmitter; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.metrics.MonitorScheduler; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; +import org.apache.druid.metadata.EntryExistsException; +import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; +import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.query.DefaultGenericQueryMetricsFactory; +import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; +import org.apache.druid.query.Druids; +import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QueryRunnerFactory; +import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.QueryWatcher; +import org.apache.druid.query.Result; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.scan.ScanQueryConfig; +import org.apache.druid.query.scan.ScanQueryEngine; +import org.apache.druid.query.scan.ScanQueryQueryToolChest; +import org.apache.druid.query.scan.ScanQueryRunnerFactory; +import org.apache.druid.query.scan.ScanResultValue; +import org.apache.druid.query.spec.QuerySegmentSpec; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesQueryEngine; +import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; +import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.DictionaryEncodedColumn; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.loading.LocalDataSegmentPusher; +import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; +import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; +import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; +import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; +import org.apache.druid.segment.transform.ExpressionTransform; +import org.apache.druid.segment.transform.TransformSpec; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.utils.CompressionUtils; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.easymock.EasyMock; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.apache.druid.query.QueryPlus.wrap; + +public class KafkaIndexTaskTest +{ + private static final Logger log = new Logger(KafkaIndexTaskTest.class); + private static final ObjectMapper OBJECT_MAPPER = new TestUtils().getTestObjectMapper(); + private static final long POLL_RETRY_MS = 100; + + private static TestingCluster zkServer; + private static TestBroker kafkaServer; + private static ServiceEmitter emitter; + private static ListeningExecutorService taskExec; + private static int topicPostfix; + + static { + new KafkaIndexTaskModule().getJacksonModules().forEach(OBJECT_MAPPER::registerModule); + } + + private final List runningTasks = new ArrayList<>(); + + private long handoffConditionTimeout = 0; + private boolean reportParseExceptions = false; + private boolean logParseExceptions = true; + private Integer maxParseExceptions = null; + private Integer maxSavedParseExceptions = null; + private boolean resetOffsetAutomatically = false; + private boolean doHandoff = true; + private Integer maxRowsPerSegment = null; + private Long maxTotalRows = null; + private Period intermediateHandoffPeriod = null; + + private TaskToolboxFactory toolboxFactory; + private IndexerMetadataStorageCoordinator metadataStorageCoordinator; + private TaskStorage taskStorage; + private TaskLockbox taskLockbox; + private File directory; + private String topic; + private List> records; + private final Set checkpointRequestsHash = new HashSet<>(); + private File reportsFile; + private RowIngestionMetersFactory rowIngestionMetersFactory; + + private static final DataSchema DATA_SCHEMA = new DataSchema( + "test_ds", + OBJECT_MAPPER.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ), + null, + null + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of() + ), + StandardCharsets.UTF_8.name() + ), + Map.class + ), + new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + }, + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), + null, + OBJECT_MAPPER + ); + + private static List> generateRecords(String topic) + { + return ImmutableList.of( + new ProducerRecord<>(topic, 0, null, jb("2008", "a", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2009", "b", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2010", "c", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2011", "d", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2011", "e", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable")), + new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable2")), + new ProducerRecord<>(topic, 0, null, null), + new ProducerRecord<>(topic, 0, null, jb("2013", "f", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2049", "f", "y", "notanumber", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2049", "f", "y", "10", "notanumber", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2049", "f", "y", "10", "20.0", "notanumber")), + new ProducerRecord<>(topic, 1, null, jb("2012", "g", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 1, null, jb("2011", "h", "y", "10", "20.0", "1.0")) + ); + } + + private static List> generateSinglePartitionRecords(String topic) + { + return ImmutableList.of( + new ProducerRecord<>(topic, 0, null, jb("2008", "a", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2009", "b", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2010", "c", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2011", "d", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2011", "D", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2012", "e", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2009", "B", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2008", "A", "x", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2009", "B", "x", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2010", "C", "x", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2011", "D", "x", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2011", "d", "x", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2012", "E", "x", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, jb("2009", "b", "x", "10", "20.0", "1.0")) + ); + } + + private static String getTopicName() + { + return "topic" + topicPostfix++; + } + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + @Rule + public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); + + @BeforeClass + public static void setupClass() throws Exception + { + emitter = new ServiceEmitter( + "service", + "host", + new NoopEmitter() + ); + emitter.start(); + EmittingLogger.registerEmitter(emitter); + + zkServer = new TestingCluster(1); + zkServer.start(); + + kafkaServer = new TestBroker( + zkServer.getConnectString(), + null, + 1, + ImmutableMap.of("num.partitions", "2") + ); + kafkaServer.start(); + + taskExec = MoreExecutors.listeningDecorator( + Executors.newCachedThreadPool( + Execs.makeThreadFactory("kafka-task-test-%d") + ) + ); + } + + @Before + public void setupTest() throws IOException + { + handoffConditionTimeout = 0; + reportParseExceptions = false; + logParseExceptions = true; + maxParseExceptions = null; + maxSavedParseExceptions = null; + doHandoff = true; + topic = getTopicName(); + records = generateRecords(topic); + reportsFile = File.createTempFile("KafkaIndexTaskTestReports-" + System.currentTimeMillis(), "json"); + makeToolboxFactory(); + } + + @After + public void tearDownTest() + { + synchronized (runningTasks) { + for (Task task : runningTasks) { + task.stopGracefully(toolboxFactory.build(task).getConfig()); + } + + runningTasks.clear(); + } + reportsFile.delete(); + destroyToolboxFactory(); + } + + @AfterClass + public static void tearDownClass() throws Exception + { + taskExec.shutdown(); + taskExec.awaitTermination(9999, TimeUnit.DAYS); + + kafkaServer.close(); + kafkaServer = null; + + zkServer.stop(); + zkServer = null; + + emitter.close(); + } + + @Test(timeout = 60_000L) + public void testRunAfterDataInserted() throws Exception + { + // Insert data + insertData(); + + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); + SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + + @Test(timeout = 60_000L) + public void testRunBeforeDataInserted() throws Exception + { + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for the task to start reading + while (task.getRunner().getStatus() != Status.READING) { + Thread.sleep(10); + } + + // Insert data + insertData(); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); + SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + + @Test(timeout = 60_000L) + public void testIncrementalHandOff() throws Exception + { + final String baseSequenceName = "sequence0"; + // as soon as any segment has more than one record, incremental publishing should happen + maxRowsPerSegment = 2; + + // Insert data + insertData(); + Map consumerProps = kafkaServer.consumerProperties(); + consumerProps.put("max.poll.records", "1"); + + final SeekableStreamStartSequenceNumbers startPartitions = new SeekableStreamStartSequenceNumbers<>( + topic, + ImmutableMap.of(0, 0L, 1, 0L), + ImmutableSet.of() + ); + // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering + // of events fetched across two partitions from Kafka + final SeekableStreamEndSequenceNumbers checkpoint1 = new SeekableStreamEndSequenceNumbers<>( + topic, + ImmutableMap.of(0, 5L, 1, 0L) + ); + final SeekableStreamEndSequenceNumbers checkpoint2 = new SeekableStreamEndSequenceNumbers<>( + topic, + ImmutableMap.of(0, 4L, 1, 2L) + ); + final SeekableStreamEndSequenceNumbers endPartitions = new SeekableStreamEndSequenceNumbers<>( + topic, + ImmutableMap.of(0, 10L, 1, 2L) + ); + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + baseSequenceName, + startPartitions, + endPartitions, + consumerProps, + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + final ListenableFuture future = runTask(task); + while (task.getRunner().getStatus() != Status.PAUSED) { + Thread.sleep(10); + } + final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); + Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets) + || checkpoint2.getPartitionSequenceNumberMap() + .equals(currentOffsets)); + task.getRunner().setEndOffsets(currentOffsets, false); + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + Assert.assertEquals(1, checkpointRequestsHash.size()); + Assert.assertTrue( + checkpointRequestsHash.contains( + Objects.hash( + DATA_SCHEMA.getDataSource(), + 0, + new KafkaDataSourceMetadata(startPartitions), + new KafkaDataSourceMetadata( + new SeekableStreamEndSequenceNumbers<>(topic, currentOffsets) + ) + ) + ) + ); + + // Check metrics + Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = sd(task, "2008/P1D", 0); + SegmentDescriptor desc2 = sd(task, "2009/P1D", 0); + SegmentDescriptor desc3 = sd(task, "2010/P1D", 0); + SegmentDescriptor desc4 = sd(task, "2011/P1D", 0); + SegmentDescriptor desc5 = sd(task, "2011/P1D", 1); + SegmentDescriptor desc6 = sd(task, "2012/P1D", 0); + SegmentDescriptor desc7 = sd(task, "2013/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata( + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L)) + ), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); + Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", desc4)) + && ImmutableList.of("h").equals(readSegmentColumn("dim1", desc5))) || + (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", desc4)) + && ImmutableList.of("e").equals(readSegmentColumn("dim1", desc5)))); + Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc6)); + Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7)); + } + + @Test(timeout = 60_000L) + public void testIncrementalHandOffMaxTotalRows() throws Exception + { + final String baseSequenceName = "sequence0"; + // incremental publish should happen every 3 records + maxRowsPerSegment = Integer.MAX_VALUE; + maxTotalRows = 3L; + + // Insert data + int numToAdd = records.size() - 2; + + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + kafkaProducer.initTransactions(); + kafkaProducer.beginTransaction(); + for (int i = 0; i < numToAdd; i++) { + kafkaProducer.send(records.get(i)).get(); + } + kafkaProducer.commitTransaction(); + } + + Map consumerProps = kafkaServer.consumerProperties(); + consumerProps.put("max.poll.records", "1"); + + final SeekableStreamStartSequenceNumbers startPartitions = new SeekableStreamStartSequenceNumbers<>( + topic, + ImmutableMap.of(0, 0L, 1, 0L), + ImmutableSet.of() + ); + final SeekableStreamEndSequenceNumbers checkpoint1 = new SeekableStreamEndSequenceNumbers<>( + topic, + ImmutableMap.of(0, 3L, 1, 0L) + ); + final SeekableStreamEndSequenceNumbers checkpoint2 = new SeekableStreamEndSequenceNumbers<>( + topic, + ImmutableMap.of(0, 10L, 1, 0L) + ); + + final SeekableStreamEndSequenceNumbers endPartitions = new SeekableStreamEndSequenceNumbers<>( + topic, + ImmutableMap.of(0, 10L, 1, 2L) + ); + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + baseSequenceName, + startPartitions, + endPartitions, + consumerProps, + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + final ListenableFuture future = runTask(task); + while (task.getRunner().getStatus() != Status.PAUSED) { + Thread.sleep(10); + } + final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); + + Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets); + task.getRunner().setEndOffsets(currentOffsets, false); + + while (task.getRunner().getStatus() != Status.PAUSED) { + Thread.sleep(10); + } + + // add remaining records + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + kafkaProducer.initTransactions(); + kafkaProducer.beginTransaction(); + for (int i = numToAdd; i < records.size(); i++) { + kafkaProducer.send(records.get(i)).get(); + } + kafkaProducer.commitTransaction(); + } + final Map nextOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); + + + Assert.assertTrue(checkpoint2.getPartitionSequenceNumberMap().equals(nextOffsets)); + task.getRunner().setEndOffsets(nextOffsets, false); + + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + Assert.assertEquals(2, checkpointRequestsHash.size()); + Assert.assertTrue( + checkpointRequestsHash.contains( + Objects.hash( + DATA_SCHEMA.getDataSource(), + 0, + new KafkaDataSourceMetadata(startPartitions), + new KafkaDataSourceMetadata( + new SeekableStreamEndSequenceNumbers<>(topic, currentOffsets) + ) + ) + ) + ); + Assert.assertTrue( + checkpointRequestsHash.contains( + Objects.hash( + DATA_SCHEMA.getDataSource(), + 0, + new KafkaDataSourceMetadata( + new SeekableStreamStartSequenceNumbers<>(topic, currentOffsets, ImmutableSet.of()) + ), + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, nextOffsets)) + ) + ) + ); + + // Check metrics + Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = sd(task, "2008/P1D", 0); + SegmentDescriptor desc2 = sd(task, "2009/P1D", 0); + SegmentDescriptor desc3 = sd(task, "2010/P1D", 0); + SegmentDescriptor desc4 = sd(task, "2011/P1D", 0); + SegmentDescriptor desc5 = sd(task, "2011/P1D", 1); + SegmentDescriptor desc6 = sd(task, "2012/P1D", 0); + SegmentDescriptor desc7 = sd(task, "2013/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata( + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L)) + ), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata( + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L)) + ), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); + Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", desc4)) + && ImmutableList.of("h").equals(readSegmentColumn("dim1", desc5))) || + (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", desc4)) + && ImmutableList.of("e").equals(readSegmentColumn("dim1", desc5)))); + Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc6)); + Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7)); + } + + @Test(timeout = 60_000L) + public void testTimeBasedIncrementalHandOff() throws Exception + { + final String baseSequenceName = "sequence0"; + // as soon as any segment hits maxRowsPerSegment or intermediateHandoffPeriod, incremental publishing should happen + maxRowsPerSegment = Integer.MAX_VALUE; + intermediateHandoffPeriod = new Period().withSeconds(0); + + // Insert data + insertData(); + Map consumerProps = kafkaServer.consumerProperties(); + consumerProps.put("max.poll.records", "1"); + + final SeekableStreamStartSequenceNumbers startPartitions = new SeekableStreamStartSequenceNumbers<>( + topic, + ImmutableMap.of(0, 0L, 1, 0L), + ImmutableSet.of() + ); + // Checkpointing will happen at checkpoint + final SeekableStreamEndSequenceNumbers checkpoint = new SeekableStreamEndSequenceNumbers<>( + topic, + ImmutableMap.of(0, 1L, 1, 0L) + ); + final SeekableStreamEndSequenceNumbers endPartitions = new SeekableStreamEndSequenceNumbers<>( + topic, + ImmutableMap.of(0, 2L, 1, 0L) + ); + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + baseSequenceName, + startPartitions, + endPartitions, + consumerProps, + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + final ListenableFuture future = runTask(task); + + // task will pause for checkpointing + while (task.getRunner().getStatus() != Status.PAUSED) { + Thread.sleep(10); + } + final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); + Assert.assertTrue(checkpoint.getPartitionSequenceNumberMap().equals(currentOffsets)); + task.getRunner().setEndOffsets(currentOffsets, false); + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + Assert.assertEquals(1, checkpointRequestsHash.size()); + Assert.assertTrue( + checkpointRequestsHash.contains( + Objects.hash( + DATA_SCHEMA.getDataSource(), + 0, + new KafkaDataSourceMetadata(startPartitions), + new KafkaDataSourceMetadata( + new SeekableStreamEndSequenceNumbers<>(topic, checkpoint.getPartitionSequenceNumberMap()) + ) + ) + ) + ); + + // Check metrics + Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = sd(task, "2008/P1D", 0); + SegmentDescriptor desc2 = sd(task, "2009/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata( + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 2L, 1, 0L)) + ), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); + } + + @Test(timeout = 60_000L) + public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception + { + records = generateSinglePartitionRecords(topic); + + final String baseSequenceName = "sequence0"; + // as soon as any segment has more than one record, incremental publishing should happen + maxRowsPerSegment = 2; + + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + kafkaProducer.initTransactions(); + kafkaProducer.beginTransaction(); + for (ProducerRecord record : records) { + kafkaProducer.send(record).get(); + } + kafkaProducer.commitTransaction(); + } + Map consumerProps = kafkaServer.consumerProperties(); + consumerProps.put("max.poll.records", "1"); + + final SeekableStreamStartSequenceNumbers startPartitions = + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()); + final SeekableStreamEndSequenceNumbers checkpoint1 = + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)); + final SeekableStreamEndSequenceNumbers checkpoint2 = + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 9L)); + final SeekableStreamEndSequenceNumbers endPartitions = + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, Long.MAX_VALUE)); + + final KafkaIndexTask normalReplica = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + baseSequenceName, + startPartitions, + endPartitions, + consumerProps, + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + final KafkaIndexTask staleReplica = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + baseSequenceName, + startPartitions, + endPartitions, + consumerProps, + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture normalReplicaFuture = runTask(normalReplica); + // Simulating one replica is slower than the other + final ListenableFuture staleReplicaFuture = ListenableFutures.transformAsync( + taskExec.submit(() -> { + Thread.sleep(1000); + return staleReplica; + }), + this::runTask + ); + + while (normalReplica.getRunner().getStatus() != Status.PAUSED) { + Thread.sleep(10); + } + staleReplica.getRunner().pause(); + while (staleReplica.getRunner().getStatus() != Status.PAUSED) { + Thread.sleep(10); + } + Map currentOffsets = ImmutableMap.copyOf(normalReplica.getRunner().getCurrentOffsets()); + Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets); + + normalReplica.getRunner().setEndOffsets(currentOffsets, false); + staleReplica.getRunner().setEndOffsets(currentOffsets, false); + + while (normalReplica.getRunner().getStatus() != Status.PAUSED) { + Thread.sleep(10); + } + while (staleReplica.getRunner().getStatus() != Status.PAUSED) { + Thread.sleep(10); + } + currentOffsets = ImmutableMap.copyOf(normalReplica.getRunner().getCurrentOffsets()); + Assert.assertEquals(checkpoint2.getPartitionSequenceNumberMap(), currentOffsets); + currentOffsets = ImmutableMap.copyOf(staleReplica.getRunner().getCurrentOffsets()); + Assert.assertEquals(checkpoint2.getPartitionSequenceNumberMap(), currentOffsets); + + normalReplica.getRunner().setEndOffsets(currentOffsets, true); + staleReplica.getRunner().setEndOffsets(currentOffsets, true); + + Assert.assertEquals(TaskState.SUCCESS, normalReplicaFuture.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, staleReplicaFuture.get().getStatusCode()); + + Assert.assertEquals(9, normalReplica.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, normalReplica.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, normalReplica.getRunner().getRowIngestionMeters().getThrownAway()); + + Assert.assertEquals(9, staleReplica.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, staleReplica.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, staleReplica.getRunner().getRowIngestionMeters().getThrownAway()); + } + + @Test(timeout = 60_000L) + public void testRunWithMinimumMessageTime() throws Exception + { + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + DateTimes.of("2010"), + null + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for the task to start reading + while (task.getRunner().getStatus() != Status.READING) { + Thread.sleep(10); + } + + // Insert data + insertData(); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); + SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + + @Test(timeout = 60_000L) + public void testRunWithMaximumMessageTime() throws Exception + { + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + DateTimes.of("2010") + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for the task to start reading + while (task.getRunner().getStatus() != Status.READING) { + Thread.sleep(10); + } + + // Insert data + insertData(); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = sd(task, "2008/P1D", 0); + SegmentDescriptor desc2 = sd(task, "2009/P1D", 0); + SegmentDescriptor desc3 = sd(task, "2010/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); + } + + @Test(timeout = 60_000L) + public void testRunWithTransformSpec() throws Exception + { + final KafkaIndexTask task = createTask( + null, + DATA_SCHEMA.withTransformSpec( + new TransformSpec( + new SelectorDimFilter("dim1", "b", null), + ImmutableList.of( + new ExpressionTransform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) + ) + ) + ), + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for the task to start reading + while (task.getRunner().getStatus() != Status.READING) { + Thread.sleep(10); + } + + // Insert data + insertData(); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = sd(task, "2009/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("bb"), readSegmentColumn("dim1t", desc1)); + } + + @Test(timeout = 60_000L) + public void testRunOnNothing() throws Exception + { + // Insert data + insertData(); + + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 2L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); + } + + @Test(timeout = 60_000L) + public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception + { + handoffConditionTimeout = 5_000; + + // Insert data + insertData(); + + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); + SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + + @Test(timeout = 60_000L) + public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exception + { + doHandoff = false; + handoffConditionTimeout = 100; + + // Insert data + insertData(); + + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); + SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata( + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)) + ), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + + @Test(timeout = 60_000L) + public void testReportParseExceptions() throws Exception + { + reportParseExceptions = true; + + // these will be ignored because reportParseExceptions is true + maxParseExceptions = 1000; + maxSavedParseExceptions = 2; + + // Insert data + insertData(); + + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 7L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskState.FAILED, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + } + + @Test(timeout = 60_000L) + public void testMultipleParseExceptionsSuccess() throws Exception + { + reportParseExceptions = false; + maxParseExceptions = 6; + maxSavedParseExceptions = 6; + + // Insert data + insertData(); + + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture future = runTask(task); + + TaskStatus status = future.get(); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, status.getStatusCode()); + Assert.assertEquals(null, status.getErrorMsg()); + + // Check metrics + Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessedWithError()); + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); + SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); + SegmentDescriptor desc3 = sd(task, "2013/P1D", 0); + SegmentDescriptor desc4 = sd(task, "2049/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); + + Map expectedMetrics = ImmutableMap.of( + RowIngestionMeters.BUILD_SEGMENTS, + ImmutableMap.of( + RowIngestionMeters.PROCESSED, 4, + RowIngestionMeters.PROCESSED_WITH_ERROR, 3, + RowIngestionMeters.UNPARSEABLE, 3, + RowIngestionMeters.THROWN_AWAY, 1 + ) + ); + Assert.assertEquals(expectedMetrics, reportData.getRowStats()); + + Map unparseableEvents = ImmutableMap.of( + RowIngestionMeters.BUILD_SEGMENTS, + Arrays.asList( + "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=10, dimFloat=20.0, met1=notanumber}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [Unable to parse value[notanumber] for field[met1],]", + "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=10, dimFloat=notanumber, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [could not convert value [notanumber] to float,]", + "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=notanumber, dimFloat=20.0, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [could not convert value [notanumber] to long,]", + "Unable to parse row [unparseable2]", + "Unable to parse row [unparseable]", + "Encountered row with timestamp that cannot be represented as a long: [MapBasedInputRow{timestamp=246140482-04-24T15:36:27.903Z, event={timestamp=246140482-04-24T15:36:27.903Z, dim1=x, dim2=z, dimLong=10, dimFloat=20.0, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}]" + ) + ); + + Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents()); + } + + @Test(timeout = 60_000L) + public void testMultipleParseExceptionsFailure() throws Exception + { + reportParseExceptions = false; + maxParseExceptions = 2; + maxSavedParseExceptions = 2; + + // Insert data + insertData(); + + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture future = runTask(task); + + TaskStatus status = future.get(); + + // Wait for task to exit + Assert.assertEquals(TaskState.FAILED, status.getStatusCode()); + IndexTaskTest.checkTaskStatusErrorMsgForParseExceptionsExceeded(status); + + // Check metrics + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessedWithError()); + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + + IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); + + Map expectedMetrics = ImmutableMap.of( + RowIngestionMeters.BUILD_SEGMENTS, + ImmutableMap.of( + RowIngestionMeters.PROCESSED, 3, + RowIngestionMeters.PROCESSED_WITH_ERROR, 0, + RowIngestionMeters.UNPARSEABLE, 3, + RowIngestionMeters.THROWN_AWAY, 0 + ) + ); + Assert.assertEquals(expectedMetrics, reportData.getRowStats()); + + Map unparseableEvents = ImmutableMap.of( + RowIngestionMeters.BUILD_SEGMENTS, + Arrays.asList( + "Unable to parse row [unparseable2]", + "Unable to parse row [unparseable]" + ) + ); + + Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents()); + } + + @Test(timeout = 60_000L) + public void testRunReplicas() throws Exception + { + final KafkaIndexTask task1 = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + final KafkaIndexTask task2 = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture future1 = runTask(task1); + final ListenableFuture future2 = runTask(task2); + + // Insert data + insertData(); + + // Wait for tasks to exit + Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published segments & metadata + SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0); + SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + + @Test(timeout = 60_000L) + public void testRunConflicting() throws Exception + { + final KafkaIndexTask task1 = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + final KafkaIndexTask task2 = createTask( + null, + new KafkaIndexTaskIOConfig( + 1, + "sequence1", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 3L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + // Insert data + insertData(); + + // Run first task + final ListenableFuture future1 = runTask(task1); + Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); + + // Run second task + final ListenableFuture future2 = runTask(task2); + Assert.assertEquals(TaskState.FAILED, future2.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published segments & metadata, should all be from the first task + SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0); + SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + + @Test(timeout = 60_000L) + public void testRunConflictingWithoutTransactions() throws Exception + { + final KafkaIndexTask task1 = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + false, + null, + null + ) + ); + final KafkaIndexTask task2 = createTask( + null, + new KafkaIndexTaskIOConfig( + 1, + "sequence1", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 3L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + false, + null, + null + ) + ); + + // Insert data + insertData(); + + // Run first task + final ListenableFuture future1 = runTask(task1); + Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); + + // Check published segments & metadata + SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0); + SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + + // Run second task + final ListenableFuture future2 = runTask(task2); + Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published segments & metadata + SegmentDescriptor desc3 = sd(task2, "2011/P1D", 1); + SegmentDescriptor desc4 = sd(task2, "2013/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc3)); + Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc4)); + } + + @Test(timeout = 60_000L) + public void testRunOneTaskTwoPartitions() throws Exception + { + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L, 1, 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 2L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture future = runTask(task); + + // Insert data + insertData(); + + // Wait for tasks to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(5, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published segments & metadata + SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); + SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); + // desc3 will not be created in KafkaIndexTask (0.12.x) as it does not create per Kafka partition Druid segments + SegmentDescriptor desc3 = sd(task, "2011/P1D", 1); + SegmentDescriptor desc4 = sd(task, "2012/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc4), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata( + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 2L)) + ), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc4)); + + // Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically + Assert.assertEquals( + ImmutableSet.of(ImmutableList.of("d", "e", "h")), + ImmutableSet.of(readSegmentColumn("dim1", desc2)) + ); + } + + @Test(timeout = 60_000L) + public void testRunTwoTasksTwoPartitions() throws Exception + { + final KafkaIndexTask task1 = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + final KafkaIndexTask task2 = createTask( + null, + new KafkaIndexTaskIOConfig( + 1, + "sequence1", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(1, 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(1, 1L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture future1 = runTask(task1); + final ListenableFuture future2 = runTask(task2); + + // Insert data + insertData(); + + // Wait for tasks to exit + Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published segments & metadata + SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0); + SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0); + SegmentDescriptor desc3 = sd(task2, "2012/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata( + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 1L)) + ), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc3)); + } + + @Test(timeout = 60_000L) + public void testRestore() throws Exception + { + final KafkaIndexTask task1 = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture future1 = runTask(task1); + + // Insert some data, but not enough for the task to finish + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + kafkaProducer.initTransactions(); + kafkaProducer.beginTransaction(); + for (ProducerRecord record : Iterables.limit(records, 4)) { + kafkaProducer.send(record).get(); + } + kafkaProducer.commitTransaction(); + } + + while (countEvents(task1) != 2) { + Thread.sleep(25); + } + + Assert.assertEquals(2, countEvents(task1)); + + // Stop without publishing segment + task1.stopGracefully(toolboxFactory.build(task1).getConfig()); + unlockAppenderatorBasePersistDirForTask(task1); + + Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); + + // Start a new task + final KafkaIndexTask task2 = createTask( + task1.getId(), + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture future2 = runTask(task2); + + // Insert remaining data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + kafkaProducer.initTransactions(); + kafkaProducer.beginTransaction(); + for (ProducerRecord record : Iterables.skip(records, 4)) { + kafkaProducer.send(record).get(); + } + kafkaProducer.commitTransaction(); + } + + // Wait for task to exit + + Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(2, task1.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published segments & metadata + SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0); + SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + + @Test(timeout = 60_000L) + public void testRestoreAfterPersistingSequences() throws Exception + { + records = generateSinglePartitionRecords(topic); + maxRowsPerSegment = 2; + Map consumerProps = kafkaServer.consumerProperties(); + consumerProps.put("max.poll.records", "1"); + + final KafkaIndexTask task1 = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)), + consumerProps, + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final SeekableStreamStartSequenceNumbers checkpoint = new SeekableStreamStartSequenceNumbers<>( + topic, + ImmutableMap.of(0, 5L), + ImmutableSet.of(0) + ); + + final ListenableFuture future1 = runTask(task1); + + // Insert some data, but not enough for the task to finish + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + kafkaProducer.initTransactions(); + kafkaProducer.beginTransaction(); + for (ProducerRecord record : Iterables.limit(records, 5)) { + kafkaProducer.send(record).get(); + } + kafkaProducer.commitTransaction(); + } + + while (task1.getRunner().getStatus() != Status.PAUSED) { + Thread.sleep(10); + } + final Map currentOffsets = ImmutableMap.copyOf(task1.getRunner().getCurrentOffsets()); + Assert.assertEquals(checkpoint.getPartitionSequenceNumberMap(), currentOffsets); + // Set endOffsets to persist sequences + task1.getRunner().setEndOffsets(ImmutableMap.of(0, 5L), false); + + // Stop without publishing segment + task1.stopGracefully(toolboxFactory.build(task1).getConfig()); + unlockAppenderatorBasePersistDirForTask(task1); + + Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); + + // Start a new task + final KafkaIndexTask task2 = createTask( + task1.getId(), + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L)), + consumerProps, + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture future2 = runTask(task2); + // Wait for the task to start reading + + // Insert remaining data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + kafkaProducer.initTransactions(); + kafkaProducer.beginTransaction(); + for (ProducerRecord record : Iterables.skip(records, 5)) { + kafkaProducer.send(record).get(); + } + kafkaProducer.commitTransaction(); + } + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(5, task1.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(4, task2.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published segments & metadata + SegmentDescriptor desc1 = sd(task1, "2008/P1D", 0); + SegmentDescriptor desc2 = sd(task1, "2008/P1D", 1); + SegmentDescriptor desc3 = sd(task1, "2009/P1D", 0); + SegmentDescriptor desc4 = sd(task1, "2009/P1D", 1); + SegmentDescriptor desc5 = sd(task1, "2010/P1D", 0); + SegmentDescriptor desc6 = sd(task1, "2011/P1D", 0); + SegmentDescriptor desc7 = sd(task1, "2012/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + } + + @Test(timeout = 60_000L) + public void testRunWithPauseAndResume() throws Exception + { + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture future = runTask(task); + + // Insert some data, but not enough for the task to finish + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + kafkaProducer.initTransactions(); + kafkaProducer.beginTransaction(); + for (ProducerRecord record : Iterables.limit(records, 4)) { + kafkaProducer.send(record).get(); + } + kafkaProducer.flush(); + kafkaProducer.commitTransaction(); + } + + while (countEvents(task) != 2) { + Thread.sleep(25); + } + + Assert.assertEquals(2, countEvents(task)); + Assert.assertEquals(Status.READING, task.getRunner().getStatus()); + + Map currentOffsets = OBJECT_MAPPER.readValue( + task.getRunner().pause().getEntity().toString(), + new TypeReference>() + { + } + ); + Assert.assertEquals(Status.PAUSED, task.getRunner().getStatus()); + // Insert remaining data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + kafkaProducer.initTransactions(); + kafkaProducer.beginTransaction(); + for (ProducerRecord record : Iterables.skip(records, 4)) { + kafkaProducer.send(record).get(); + } + kafkaProducer.commitTransaction(); + } + + try { + future.get(10, TimeUnit.SECONDS); + Assert.fail("Task completed when it should have been paused"); + } + catch (TimeoutException e) { + // carry on.. + } + + Assert.assertEquals(currentOffsets, task.getRunner().getCurrentOffsets()); + + task.getRunner().resume(); + + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + Assert.assertEquals(task.getRunner().getEndOffsets(), task.getRunner().getCurrentOffsets()); + + // Check metrics + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); + SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + + @Test(timeout = 60_000L) + public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception + { + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + runTask(task); + + while (!task.getRunner().getStatus().equals(Status.READING)) { + Thread.sleep(2000); + } + + task.getRunner().pause(); + + while (!task.getRunner().getStatus().equals(Status.PAUSED)) { + Thread.sleep(25); + } + } + + @Test(timeout = 60_000L) + public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAvailable() throws Exception + { + resetOffsetAutomatically = true; + // Insert data + insertData(); + + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 200L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 500L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + runTask(task); + + while (!task.getRunner().getStatus().equals(Status.READING)) { + Thread.sleep(20); + } + + for (int i = 0; i < 5; i++) { + Assert.assertEquals(task.getRunner().getStatus(), Status.READING); + // Offset should not be reset + Assert.assertTrue(task.getRunner().getCurrentOffsets().get(0) == 200L); + } + } + + @Test(timeout = 60_000L) + public void testRunContextSequenceAheadOfStartingOffsets() throws Exception + { + // Insert data + insertData(); + + final TreeMap> sequences = new TreeMap<>(); + // Here the sequence number is 1 meaning that one incremental handoff was done by the failed task + // and this task should start reading from offset 2 for partition 0 + sequences.put(1, ImmutableMap.of(0, 2L)); + final Map context = new HashMap<>(); + context.put( + SeekableStreamSupervisor.CHECKPOINTS_CTX_KEY, + OBJECT_MAPPER.writerFor(KafkaSupervisor.CHECKPOINTS_TYPE_REF).writeValueAsString(sequences) + ); + + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + // task should ignore these and use sequence info sent in the context + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ), + context + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = sd(task, "2010/P1D", 0); + SegmentDescriptor desc2 = sd(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + + @Test(timeout = 60_000L) + public void testRunWithDuplicateRequest() throws Exception + { + // Insert data + insertData(); + + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 200L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 500L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + runTask(task); + + while (!task.getRunner().getStatus().equals(Status.READING)) { + Thread.sleep(20); + } + + // first setEndOffsets request + task.getRunner().pause(); + task.getRunner().setEndOffsets(ImmutableMap.of(0, 500L), true); + Assert.assertEquals(Status.READING, task.getRunner().getStatus()); + + // duplicate setEndOffsets request + task.getRunner().pause(); + task.getRunner().setEndOffsets(ImmutableMap.of(0, 500L), true); + Assert.assertEquals(Status.READING, task.getRunner().getStatus()); + } + + @Test(timeout = 60_000L) + public void testRunTransactionModeRollback() throws Exception + { + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final ListenableFuture future = runTask(task); + + // Insert 2 records initially + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + kafkaProducer.initTransactions(); + kafkaProducer.beginTransaction(); + for (ProducerRecord record : Iterables.limit(records, 2)) { + kafkaProducer.send(record).get(); + } + kafkaProducer.commitTransaction(); + } + + while (countEvents(task) != 2) { + Thread.sleep(25); + } + + Assert.assertEquals(2, countEvents(task)); + Assert.assertEquals(Status.READING, task.getRunner().getStatus()); + + //verify the 2 indexed records + final QuerySegmentSpec firstInterval = OBJECT_MAPPER.readValue( + "\"2008/2010\"", QuerySegmentSpec.class + ); + Iterable scanResultValues = scanData(task, firstInterval); + Assert.assertEquals(2, Iterables.size(scanResultValues)); + + // Insert 3 more records and rollback + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + kafkaProducer.initTransactions(); + kafkaProducer.beginTransaction(); + for (ProducerRecord record : Iterables.limit(Iterables.skip(records, 2), 3)) { + kafkaProducer.send(record).get(); + } + kafkaProducer.flush(); + kafkaProducer.abortTransaction(); + } + + Assert.assertEquals(2, countEvents(task)); + Assert.assertEquals(Status.READING, task.getRunner().getStatus()); + + final QuerySegmentSpec rollbackedInterval = OBJECT_MAPPER.readValue( + "\"2010/2012\"", QuerySegmentSpec.class + ); + scanResultValues = scanData(task, rollbackedInterval); + //verify that there are no records indexed in the rollbacked time period + Assert.assertEquals(0, Iterables.size(scanResultValues)); + + // Insert remaining data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + kafkaProducer.initTransactions(); + kafkaProducer.beginTransaction(); + for (ProducerRecord record : Iterables.skip(records, 5)) { + kafkaProducer.send(record).get(); + } + kafkaProducer.commitTransaction(); + } + + final QuerySegmentSpec endInterval = OBJECT_MAPPER.readValue( + "\"2008/2049\"", QuerySegmentSpec.class + ); + Iterable scanResultValues1 = scanData(task, endInterval); + Assert.assertEquals(2, Iterables.size(scanResultValues1)); + + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + Assert.assertEquals(task.getRunner().getEndOffsets(), task.getRunner().getCurrentOffsets()); + + // Check metrics + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = sd(task, "2008/P1D", 0); + SegmentDescriptor desc2 = sd(task, "2009/P1D", 0); + SegmentDescriptor desc3 = sd(task, "2013/P1D", 0); + SegmentDescriptor desc4 = sd(task, "2049/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); + Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc3)); + Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc4)); + } + + @Test(timeout = 60_000L) + public void testCanStartFromLaterThanEarliestOffset() throws Exception + { + final String baseSequenceName = "sequence0"; + maxRowsPerSegment = Integer.MAX_VALUE; + maxTotalRows = null; + + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + kafkaProducer.initTransactions(); + kafkaProducer.beginTransaction(); + for (ProducerRecord record : records) { + kafkaProducer.send(record).get(); + } + kafkaProducer.commitTransaction(); + } + + Map consumerProps = kafkaServer.consumerProperties(); + consumerProps.put("max.poll.records", "1"); + + final SeekableStreamStartSequenceNumbers startPartitions = new SeekableStreamStartSequenceNumbers<>( + topic, + ImmutableMap.of(0, 0L, 1, 1L), + ImmutableSet.of() + ); + + final SeekableStreamEndSequenceNumbers endPartitions = new SeekableStreamEndSequenceNumbers<>( + topic, + ImmutableMap.of(0, 10L, 1, 2L) + ); + + final KafkaIndexTask task = createTask( + null, + new KafkaIndexTaskIOConfig( + 0, + baseSequenceName, + startPartitions, + endPartitions, + consumerProps, + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + final ListenableFuture future = runTask(task); + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + } + + @Test + public void testSerde() throws Exception + { + // This is both a serde test and a regression test for https://github.com/apache/incubator-druid/issues/7724. + + final KafkaIndexTask task = createTask( + "taskid", + DATA_SCHEMA.withTransformSpec( + new TransformSpec( + null, + ImmutableList.of(new ExpressionTransform("beep", "nofunc()", ExprMacroTable.nil())) + ) + ), + new KafkaIndexTaskIOConfig( + 0, + "sequence", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of()), + ImmutableMap.of(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null + ) + ); + + final Task task1 = OBJECT_MAPPER.readValue(OBJECT_MAPPER.writeValueAsBytes(task), Task.class); + Assert.assertEquals(task, task1); + } + + private List scanData(final Task task, QuerySegmentSpec spec) + { + ScanQuery query = new Druids.ScanQueryBuilder().dataSource( + DATA_SCHEMA.getDataSource()).intervals(spec).build(); + List results = + task.getQueryRunner(query).run(wrap(query), new HashMap<>()).toList(); + return results; + } + + private void insertData() throws ExecutionException, InterruptedException + { + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + kafkaProducer.initTransactions(); + kafkaProducer.beginTransaction(); + for (ProducerRecord record : records) { + kafkaProducer.send(record).get(); + } + kafkaProducer.commitTransaction(); + } + } + + private ListenableFuture runTask(final Task task) + { + try { + taskStorage.insert(task, TaskStatus.running(task.getId())); + } + catch (EntryExistsException e) { + // suppress + } + taskLockbox.syncFromStorage(); + final TaskToolbox toolbox = toolboxFactory.build(task); + synchronized (runningTasks) { + runningTasks.add(task); + } + return taskExec.submit( + () -> { + try { + if (task.isReady(toolbox.getTaskActionClient())) { + return task.run(toolbox); + } else { + throw new ISE("Task is not ready"); + } + } + catch (Exception e) { + log.warn(e, "Task failed"); + return TaskStatus.failure(task.getId()); + } + } + ); + } + + private TaskLock getLock(final Task task, final Interval interval) + { + return Iterables.find( + taskLockbox.findLocksForTask(task), + new Predicate() + { + @Override + public boolean apply(TaskLock lock) + { + return lock.getInterval().contains(interval); + } + } + ); + } + + private KafkaIndexTask createTask( + final String taskId, + final KafkaIndexTaskIOConfig ioConfig + ) throws JsonProcessingException + { + return createTask(taskId, DATA_SCHEMA, ioConfig); + } + + private KafkaIndexTask createTask( + final String taskId, + final KafkaIndexTaskIOConfig ioConfig, + final Map context + ) throws JsonProcessingException + { + return createTask(taskId, DATA_SCHEMA, ioConfig, context); + } + + private KafkaIndexTask createTask( + final String taskId, + final DataSchema dataSchema, + final KafkaIndexTaskIOConfig ioConfig + ) throws JsonProcessingException + { + final Map context = new HashMap<>(); + return createTask(taskId, dataSchema, ioConfig, context); + } + + private KafkaIndexTask createTask( + final String taskId, + final DataSchema dataSchema, + final KafkaIndexTaskIOConfig ioConfig, + final Map context + ) throws JsonProcessingException + { + final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig( + 1000, + null, + maxRowsPerSegment, + maxTotalRows, + new Period("P1Y"), + null, + null, + null, + true, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + null, + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions + ); + if (!context.containsKey(SeekableStreamSupervisor.CHECKPOINTS_CTX_KEY)) { + final TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap()); + final String checkpointsJson = OBJECT_MAPPER + .writerFor(KafkaSupervisor.CHECKPOINTS_TYPE_REF) + .writeValueAsString(checkpoints); + context.put(SeekableStreamSupervisor.CHECKPOINTS_CTX_KEY, checkpointsJson); + } + + final KafkaIndexTask task = new KafkaIndexTask( + taskId, + null, + cloneDataSchema(dataSchema), + tuningConfig, + ioConfig, + context, + null, + null, + rowIngestionMetersFactory, + OBJECT_MAPPER + ); + task.setPollRetryMs(POLL_RETRY_MS); + return task; + } + + private static DataSchema cloneDataSchema(final DataSchema dataSchema) + { + return new DataSchema( + dataSchema.getDataSource(), + dataSchema.getParserMap(), + dataSchema.getAggregators(), + dataSchema.getGranularitySpec(), + dataSchema.getTransformSpec(), + OBJECT_MAPPER + ); + } + + private QueryRunnerFactoryConglomerate makeTimeseriesAndScanConglomerate() + { + IntervalChunkingQueryRunnerDecorator queryRunnerDecorator = new IntervalChunkingQueryRunnerDecorator( + null, + null, + null + ) + { + @Override + public QueryRunner decorate(QueryRunner delegate, QueryToolChest> toolChest) + { + return delegate; + } + }; + return new DefaultQueryRunnerFactoryConglomerate( + ImmutableMap., QueryRunnerFactory>builder() + .put( + TimeseriesQuery.class, + new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest(queryRunnerDecorator), + new TimeseriesQueryEngine(), + new QueryWatcher() + { + @Override + public void registerQuery(Query query, ListenableFuture future) + { + // do nothing + } + } + ) + ) + .put( + ScanQuery.class, + new ScanQueryRunnerFactory( + new ScanQueryQueryToolChest( + new ScanQueryConfig(), + new DefaultGenericQueryMetricsFactory(TestHelper.makeJsonMapper()) + ), + new ScanQueryEngine(), + new ScanQueryConfig() + ) + ) + .build() + ); + } + + private void makeToolboxFactory() throws IOException + { + directory = tempFolder.newFolder(); + final TestUtils testUtils = new TestUtils(); + rowIngestionMetersFactory = testUtils.getRowIngestionMetersFactory(); + final ObjectMapper objectMapper = testUtils.getTestObjectMapper(); + for (Module module : new KafkaIndexTaskModule().getJacksonModules()) { + objectMapper.registerModule(module); + } + final TaskConfig taskConfig = new TaskConfig( + new File(directory, "baseDir").getPath(), + new File(directory, "baseTaskDir").getPath(), + null, + 50000, + null, + true, + null, + null + ); + final TestDerbyConnector derbyConnector = derby.getConnector(); + derbyConnector.createDataSourceTable(); + derbyConnector.createPendingSegmentsTable(); + derbyConnector.createSegmentTable(); + derbyConnector.createRulesTable(); + derbyConnector.createConfigTable(); + derbyConnector.createTaskTables(); + derbyConnector.createAuditTable(); + taskStorage = new MetadataTaskStorage( + derbyConnector, + new TaskStorageConfig(null), + new DerbyMetadataStorageActionHandlerFactory( + derbyConnector, + derby.metadataTablesConfigSupplier().get(), + objectMapper + ) + ); + metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( + testUtils.getTestObjectMapper(), + derby.metadataTablesConfigSupplier().get(), + derbyConnector + ); + taskLockbox = new TaskLockbox(taskStorage); + final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( + taskLockbox, + taskStorage, + metadataStorageCoordinator, + emitter, + new SupervisorManager(null) + { + @Override + public boolean checkPointDataSourceMetadata( + String supervisorId, + @Nullable Integer taskGroupId, + String baseSequenceName, + @Nullable DataSourceMetadata previousDataSourceMetadata, + @Nullable DataSourceMetadata currentDataSourceMetadata + ) + { + log.info("Adding checkpoint hash to the set"); + checkpointRequestsHash.add( + Objects.hash( + supervisorId, + taskGroupId, + previousDataSourceMetadata, + currentDataSourceMetadata + ) + ); + return true; + } + } + ); + final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( + taskStorage, + taskActionToolbox, + new TaskAuditLogConfig(false) + ); + final SegmentHandoffNotifierFactory handoffNotifierFactory = dataSource -> new SegmentHandoffNotifier() + { + @Override + public boolean registerSegmentHandoffCallback( + SegmentDescriptor descriptor, + Executor exec, + Runnable handOffRunnable + ) + { + if (doHandoff) { + // Simulate immediate handoff + exec.execute(handOffRunnable); + } + return true; + } + + @Override + public void start() + { + //Noop + } + + @Override + public void close() + { + //Noop + } + }; + final LocalDataSegmentPusherConfig dataSegmentPusherConfig = new LocalDataSegmentPusherConfig(); + dataSegmentPusherConfig.storageDirectory = getSegmentDirectory(); + final DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher(dataSegmentPusherConfig); + toolboxFactory = new TaskToolboxFactory( + taskConfig, + taskActionClientFactory, + emitter, + dataSegmentPusher, + new TestDataSegmentKiller(), + null, // DataSegmentMover + null, // DataSegmentArchiver + new TestDataSegmentAnnouncer(), + EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), + handoffNotifierFactory, + this::makeTimeseriesAndScanConglomerate, + Execs.directExecutor(), // queryExecutorService + EasyMock.createMock(MonitorScheduler.class), + new SegmentLoaderFactory(null, testUtils.getTestObjectMapper()), + testUtils.getTestObjectMapper(), + testUtils.getTestIndexIO(), + MapCache.create(1024), + new CacheConfig(), + new CachePopulatorStats(), + testUtils.getTestIndexMergerV9(), + EasyMock.createNiceMock(DruidNodeAnnouncer.class), + EasyMock.createNiceMock(DruidNode.class), + new LookupNodeService("tier"), + new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0), + new TaskReportFileWriter(reportsFile) + ); + } + + private void destroyToolboxFactory() + { + toolboxFactory = null; + taskStorage = null; + taskLockbox = null; + metadataStorageCoordinator = null; + } + + private Set publishedDescriptors() + { + return FluentIterable.from( + metadataStorageCoordinator.getUsedSegmentsForInterval( + DATA_SCHEMA.getDataSource(), + Intervals.of("0000/3000") + ) + ).transform(DataSegment::toDescriptor).toSet(); + } + + private void unlockAppenderatorBasePersistDirForTask(KafkaIndexTask task) + throws NoSuchMethodException, InvocationTargetException, IllegalAccessException + { + Method unlockBasePersistDir = ((AppenderatorImpl) task.getAppenderator()).getClass() + .getDeclaredMethod( + "unlockBasePersistDirectory"); + unlockBasePersistDir.setAccessible(true); + unlockBasePersistDir.invoke(task.getAppenderator()); + } + + private File getSegmentDirectory() + { + return new File(directory, "segments"); + } + + private List readSegmentColumn(final String column, final SegmentDescriptor descriptor) throws IOException + { + File indexBasePath = new File( + StringUtils.format( + "%s/%s/%s_%s/%s/%d", + getSegmentDirectory(), + DATA_SCHEMA.getDataSource(), + descriptor.getInterval().getStart(), + descriptor.getInterval().getEnd(), + descriptor.getVersion(), + descriptor.getPartitionNumber() + ) + ); + + File outputLocation = new File( + directory, + StringUtils.format( + "%s_%s_%s_%s", + descriptor.getInterval().getStart(), + descriptor.getInterval().getEnd(), + descriptor.getVersion(), + descriptor.getPartitionNumber() + ) + ); + outputLocation.mkdir(); + CompressionUtils.unzip( + Files.asByteSource(new File(indexBasePath.listFiles()[0], "index.zip")), + outputLocation, + Predicates.alwaysFalse(), + false + ); + IndexIO indexIO = new TestUtils().getTestIndexIO(); + QueryableIndex index = indexIO.loadIndex(outputLocation); + DictionaryEncodedColumn theColumn = (DictionaryEncodedColumn) index.getColumnHolder(column) + .getColumn(); + List values = new ArrayList<>(); + for (int i = 0; i < theColumn.length(); i++) { + int id = theColumn.getSingleValueRow(i); + String value = theColumn.lookupName(id); + values.add(value); + } + return values; + } + + public long countEvents(final Task task) + { + // Do a query. + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(DATA_SCHEMA.getDataSource()) + .aggregators( + ImmutableList.of( + new LongSumAggregatorFactory("rows", "rows") + ) + ).granularity(Granularities.ALL) + .intervals("0000/3000") + .build(); + + List> results = + task.getQueryRunner(query).run(wrap(query), ImmutableMap.of()).toList(); + + return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows")); + } + + private static byte[] jb(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) + { + try { + return new ObjectMapper().writeValueAsBytes( + ImmutableMap.builder() + .put("timestamp", timestamp) + .put("dim1", dim1) + .put("dim2", dim2) + .put("dimLong", dimLong) + .put("dimFloat", dimFloat) + .put("met1", met1) + .build() + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + private SegmentDescriptor sd(final Task task, final String intervalString, final int partitionNum) + { + final Interval interval = Intervals.of(intervalString); + return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum); + } + + private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOException + { + Map taskReports = OBJECT_MAPPER.readValue( + reportsFile, + new TypeReference>() + { + } + ); + return IngestionStatsAndErrorsTaskReportData.getPayloadFromTaskReports( + taskReports + ); + } +} From 4dd064e1b0e45f9e0f8290877b8020baf78ce471 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 30 Jul 2019 23:27:27 +0200 Subject: [PATCH 04/26] Fix deprecated Jackson method --- .../test/java/org/apache/druid/guice/FirehoseModuleTest.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/server/src/test/java/org/apache/druid/guice/FirehoseModuleTest.java b/server/src/test/java/org/apache/druid/guice/FirehoseModuleTest.java index 233eb76a9637..222c036925e0 100644 --- a/server/src/test/java/org/apache/druid/guice/FirehoseModuleTest.java +++ b/server/src/test/java/org/apache/druid/guice/FirehoseModuleTest.java @@ -64,8 +64,7 @@ private static Set getFirehoseFactorySubtypeClasses(ObjectMapper objectMa { Class parentClass = FirehoseFactory.class; MapperConfig config = objectMapper.getDeserializationConfig(); - AnnotationIntrospector annotationIntrospector = config.getAnnotationIntrospector(); - AnnotatedClass ac = AnnotatedClass.constructWithoutSuperTypes(parentClass, annotationIntrospector, config); + AnnotatedClass ac = AnnotatedClass.constructWithoutSuperTypes(parentClass, config); Collection subtypes = objectMapper.getSubtypeResolver().collectAndResolveSubtypesByClass(config, ac); Assert.assertNotNull(subtypes); return subtypes.stream() From 6ef2b31d6f2884cf5f7f30dc9a7023d13056500d Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 27 Sep 2019 07:31:12 +0200 Subject: [PATCH 05/26] Fix licenses --- licenses.yaml | 40 +++++++++++++++++++ .../druid/guice/FirehoseModuleTest.java | 1 - 2 files changed, 40 insertions(+), 1 deletion(-) diff --git a/licenses.yaml b/licenses.yaml index 85fb69645f91..07a6ad87ac93 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -233,6 +233,46 @@ notice: | --- +name: Jackson +license_category: binary +module: java-core +license_name: Apache License version 2.0 +version: 2.9.9 +libraries: + - com.fasterxml.jackson.core: jackson-annotations + - com.fasterxml.jackson.module: jackson-module-jaxb-annotations + - com.fasterxml.jackson.core: jackson-databind + - com.fasterxml.jackson.datatype: jackson-datatype-joda + - com.fasterxml.jackson.core: jackson-core + - com.fasterxml.jackson.dataformat: jackson-dataformat-smile + - com.fasterxml.jackson.datatype: jackson-datatype-guava + - com.fasterxml.jackson.jaxrs: jackson-jaxrs-base + - com.fasterxml.jackson.jaxrs: jackson-jaxrs-json-provider + - com.fasterxml.jackson.jaxrs: jackson-jaxrs-smile-provider +notice: | + # Jackson JSON processor + + Jackson is a high-performance, Free/Open Source JSON processing library. + It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has + been in development since 2007. + It is currently developed by a community of developers, as well as supported + commercially by FasterXML.com. + + ## Licensing + + Jackson core and extension components may licensed under different licenses. + To find the details that apply to this artifact see the accompanying LICENSE file. + For more information, including possible other licensing options, contact + FasterXML.com (http://fasterxml.com). + + ## Credits + + A list of contributors may be found from CREDITS file, which is included + in some artifacts (usually source distributions); but is always available + from the source code management (SCM) system project uses. + +--- + name: Caffeine license_category: binary module: java-core diff --git a/server/src/test/java/org/apache/druid/guice/FirehoseModuleTest.java b/server/src/test/java/org/apache/druid/guice/FirehoseModuleTest.java index 0aba67ef49f6..8ecc93dece2b 100644 --- a/server/src/test/java/org/apache/druid/guice/FirehoseModuleTest.java +++ b/server/src/test/java/org/apache/druid/guice/FirehoseModuleTest.java @@ -19,7 +19,6 @@ package org.apache.druid.guice; -import com.fasterxml.jackson.databind.AnnotationIntrospector; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.cfg.MapperConfig; From a3a8e69aef56c4c6ee539497921755084c9e1e34 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Sat, 28 Sep 2019 08:33:23 +0200 Subject: [PATCH 06/26] Bump to Jackson 2.9.10 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 7f5a50334192..e0656491b0ed 100644 --- a/pom.xml +++ b/pom.xml @@ -88,7 +88,7 @@ 1.3 9.4.10.v20180503 1.19.3 - 2.9.9 + 2.9.10 2.5 3.10.6.Final From 92227050c1d0f82044e0a0eadc87c8fc7c692d8f Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 27 Sep 2019 12:14:37 +0300 Subject: [PATCH 07/26] Bump HttpClient to 4.5.10 (#8404) * Bump HttpClient to 4.5.9 * Remove Licenses file * Revert license * Remove duplicate dependency * Bump HttpClient to 4.5.10 --- licenses.yaml | 31 +++++++++++++++---------------- pom.xml | 2 +- 2 files changed, 16 insertions(+), 17 deletions(-) diff --git a/licenses.yaml b/licenses.yaml index 07a6ad87ac93..5b22de4b013a 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -1309,14 +1309,13 @@ notice: | - Apache License 2.0 - BSD License - --- name: Apache HttpClient license_category: binary module: java-core license_name: Apache License version 2.0 -version: 4.5.3 +version: 4.5.10 libraries: - org.apache.httpcomponents: httpclient notices: @@ -1326,6 +1325,20 @@ notices: --- +name: Apache HttpClient +license_category: binary +module: hadoop-client +license_name: Apache License version 2.0 +version: 4.5.2 +libraries: + - org.apache.httpcomponents: httpclient +notices: + - httpclient: | + Apache HttpClient + Copyright 1999-2016 The Apache Software Foundation + +--- + name: Apache HttpCore license_category: binary module: java-core @@ -2809,20 +2822,6 @@ notices: --- -name: Apache HttpClient -license_category: binary -module: hadoop-client -license_name: Apache License version 2.0 -version: 4.5.2 -libraries: - - org.apache.httpcomponents: httpclient -notices: - - httpclient: | - Apache HttpClient - Copyright 1999-2016 The Apache Software Foundation - ---- - name: Apache Zookeeper license_category: binary module: hadoop-client diff --git a/pom.xml b/pom.xml index e0656491b0ed..d00018611add 100644 --- a/pom.xml +++ b/pom.xml @@ -769,7 +769,7 @@ org.apache.httpcomponents httpclient - 4.5.3 + 4.5.10 org.apache.httpcomponents From 824f94585c16e8ae253b3e444b1b37631b41bc67 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 27 Sep 2019 12:14:58 +0300 Subject: [PATCH 08/26] Remove commons-httpclient (#8407) --- extensions-contrib/cassandra-storage/pom.xml | 4 ---- extensions-core/druid-kerberos/pom.xml | 4 ---- extensions-core/hdfs-storage/pom.xml | 8 -------- pom.xml | 6 ------ 4 files changed, 22 deletions(-) diff --git a/extensions-contrib/cassandra-storage/pom.xml b/extensions-contrib/cassandra-storage/pom.xml index 2d73e897007b..6171a008fc33 100644 --- a/extensions-contrib/cassandra-storage/pom.xml +++ b/extensions-contrib/cassandra-storage/pom.xml @@ -69,10 +69,6 @@ commons-cli commons-cli - - commons-httpclient - commons-httpclient - log4j log4j diff --git a/extensions-core/druid-kerberos/pom.xml b/extensions-core/druid-kerberos/pom.xml index a474bf1edfef..c554f6f7a528 100644 --- a/extensions-core/druid-kerberos/pom.xml +++ b/extensions-core/druid-kerberos/pom.xml @@ -63,10 +63,6 @@ commons-cli commons-cli - - commons-httpclient - commons-httpclient - commons-codec commons-codec diff --git a/extensions-core/hdfs-storage/pom.xml b/extensions-core/hdfs-storage/pom.xml index c4de4c4b5107..49e19650fb08 100644 --- a/extensions-core/hdfs-storage/pom.xml +++ b/extensions-core/hdfs-storage/pom.xml @@ -49,10 +49,6 @@ commons-cli commons-cli - - commons-httpclient - commons-httpclient - log4j log4j @@ -161,10 +157,6 @@ commons-cli commons-cli - - commons-httpclient - commons-httpclient - log4j log4j diff --git a/pom.xml b/pom.xml index d00018611add..95b23edfd58f 100644 --- a/pom.xml +++ b/pom.xml @@ -90,7 +90,6 @@ 1.19.3 2.9.10 2.5 - 3.10.6.Final 4.1.30.Final @@ -201,11 +200,6 @@ commons-codec 1.12 - - commons-httpclient - commons-httpclient - 3.1 - commons-io commons-io From 21ed4e437fe90e17f8a256d75c091917c9e2b2ce Mon Sep 17 00:00:00 2001 From: Faxian Zhao <232775890@qq.com> Date: Fri, 27 Sep 2019 18:55:02 +0800 Subject: [PATCH 09/26] bug fix for lookup leak when we remove the last lookup from lookup tier (#8598) * bug fix for lookup leak when we remove the last lookup from lookup tier * warnings about lookups that will never be loaded * fix unit test --- .../cache/LookupCoordinatorManager.java | 21 ++++++++++++++----- .../cache/LookupCoordinatorManagerTest.java | 4 ++++ 2 files changed, 20 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManager.java b/server/src/main/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManager.java index b6589aae0587..cd3ac2c68bdd 100644 --- a/server/src/main/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManager.java +++ b/server/src/main/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManager.java @@ -557,16 +557,27 @@ void lookupManagementLoop() try { List> futures = new ArrayList<>(); - for (Map.Entry> tierEntry : allLookupTiers.entrySet()) { - LOG.debug("Starting lookup mgmt for tier [%s].", tierEntry.getKey()); + Set discoveredLookupTiers = lookupNodeDiscovery.getAllTiers(); - final Map tierLookups = tierEntry.getValue(); - for (final HostAndPortWithScheme node : lookupNodeDiscovery.getNodesInTier(tierEntry.getKey())) { + // Check and Log warnings about lookups configured by user in DB but no nodes discovered to load those. + for (String tierInDB : allLookupTiers.keySet()) { + if (!discoveredLookupTiers.contains(tierInDB) && + !allLookupTiers.getOrDefault(tierInDB, ImmutableMap.of()).isEmpty()) { + LOG.warn("Found lookups for tier [%s] in DB, but no nodes discovered for it", tierInDB); + } + } + + for (String tier : discoveredLookupTiers) { + + LOG.debug("Starting lookup mgmt for tier [%s].", tier); + + final Map tierLookups = allLookupTiers.getOrDefault(tier, ImmutableMap.of()); + for (final HostAndPortWithScheme node : lookupNodeDiscovery.getNodesInTier(tier)) { LOG.debug( "Starting lookup mgmt for tier [%s] and host [%s:%s:%s].", - tierEntry.getKey(), + tier, node.getScheme(), node.getHostText(), node.getPort() diff --git a/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java b/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java index 772b185240c3..962c950e7ee8 100644 --- a/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java @@ -1138,6 +1138,10 @@ public void testLookupManagementLoop() throws Exception HostAndPortWithScheme host2 = HostAndPortWithScheme.fromParts("http", "host2", 3456); EasyMock.reset(lookupNodeDiscovery); + EasyMock + .expect(lookupNodeDiscovery.getAllTiers()) + .andReturn(ImmutableSet.of("tier1")) + .once(); EasyMock .expect(lookupNodeDiscovery.getNodesInTier("tier1")) .andReturn(ImmutableList.of(host1, host2)) From 12c1ba58b8df8651a8930062fb8ce140753925ce Mon Sep 17 00:00:00 2001 From: Evan Ren Date: Fri, 27 Sep 2019 14:19:04 -0700 Subject: [PATCH 10/26] Increase column size for taskID and createdTime, and decrease Type and Duration (#8594) --- .../views/task-view/__snapshots__/tasks-view.spec.tsx.snap | 6 ++++-- web-console/src/views/task-view/tasks-view.tsx | 6 ++++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/web-console/src/views/task-view/__snapshots__/tasks-view.spec.tsx.snap b/web-console/src/views/task-view/__snapshots__/tasks-view.spec.tsx.snap index 226ad3099576..58e1bf68f86c 100644 --- a/web-console/src/views/task-view/__snapshots__/tasks-view.spec.tsx.snap +++ b/web-console/src/views/task-view/__snapshots__/tasks-view.spec.tsx.snap @@ -537,7 +537,7 @@ exports[`tasks view matches snapshot 1`] = ` "Header": "Task ID", "accessor": "task_id", "show": true, - "width": 300, + "width": 500, }, Object { "Aggregated": [Function], @@ -551,6 +551,7 @@ exports[`tasks view matches snapshot 1`] = ` "Header": "Type", "accessor": "type", "show": true, + "width": 140, }, Object { "Cell": [Function], @@ -570,7 +571,7 @@ exports[`tasks view matches snapshot 1`] = ` "Header": "Created time", "accessor": "created_time", "show": true, - "width": 120, + "width": 190, }, Object { "Cell": [Function], @@ -589,6 +590,7 @@ exports[`tasks view matches snapshot 1`] = ` "accessor": "duration", "filterable": false, "show": true, + "width": 70, }, Object { "Aggregated": [Function], diff --git a/web-console/src/views/task-view/tasks-view.tsx b/web-console/src/views/task-view/tasks-view.tsx index 3316934da794..0962ffc289ab 100644 --- a/web-console/src/views/task-view/tasks-view.tsx +++ b/web-console/src/views/task-view/tasks-view.tsx @@ -741,7 +741,7 @@ ORDER BY "rank" DESC, "created_time" DESC`; { Header: 'Task ID', accessor: 'task_id', - width: 300, + width: 500, Aggregated: () => '', show: hiddenTaskColumns.exists('Task ID'), }, @@ -755,6 +755,7 @@ ORDER BY "rank" DESC, "created_time" DESC`; { Header: 'Type', accessor: 'type', + width: 140, Cell: row => { const value = row.value; return ( @@ -799,7 +800,7 @@ ORDER BY "rank" DESC, "created_time" DESC`; { Header: 'Created time', accessor: 'created_time', - width: 120, + width: 190, Aggregated: () => '', show: hiddenTaskColumns.exists('Created time'), }, @@ -866,6 +867,7 @@ ORDER BY "rank" DESC, "created_time" DESC`; { Header: 'Duration', accessor: 'duration', + width: 70, filterable: false, Cell: row => (row.value > 0 ? formatDuration(row.value) : ''), Aggregated: () => '', From e6703612306437481a861bb3e34abcd81ca4e407 Mon Sep 17 00:00:00 2001 From: Sashidhar Thallam Date: Sat, 28 Sep 2019 11:47:44 +0530 Subject: [PATCH 11/26] Making optimal usage of multiple segment cache locations (#8038) * #7641 - Changing segment distribution algorithm to distribute segments to multiple segment cache locations * Fixing indentation * WIP * Adding interface for location strategy selection, least bytes used strategy impl, round-robin strategy impl, locationSelectorStrategy config with least bytes used strategy as the default strategy * fixing code style * Fixing test * Adding a method visible only for testing, fixing tests * 1. Changing the method contract to return an iterator of locations instead of a single best location. 2. Check style fixes * fixing the conditional statement * Added testSegmentDistributionUsingLeastBytesUsedStrategy, fixed testSegmentDistributionUsingRoundRobinStrategy * to trigger CI build * Add documentation for the selection strategy configuration * to re trigger CI build * updated docs as per review comments, made LeastBytesUsedStorageLocationSelectorStrategy.getLocations a synchronzied method, other minor fixes * In checkLocationConfigForNull method, using getLocations() to check for null instead of directly referring to the locations variable so that tests overriding getLocations() method do not fail * Implementing review comments. Added tests for StorageLocationSelectorStrategy * Checkstyle fixes * Adding java doc comments for StorageLocationSelectorStrategy interface * checkstyle * empty commit to retrigger build * Empty commit * Adding suppressions for words leastBytesUsed and roundRobin of ../docs/configuration/index.md file * Impl review comments including updating docs as suggested * Removing checkLocationConfigForNull(), @NotEmpty annotation serves the purpose * Round robin iterator to keep track of the no. of iterations, impl review comments, added tests for round robin strategy * Fixing the round robin iterator * Removed numLocationsToTry, updated java docs * changing property attribute value from tier to type * Fixing assert messages --- docs/configuration/index.md | 3 + ...esUsedStorageLocationSelectorStrategy.java | 57 ++++ ...dRobinStorageLocationSelectorStrategy.java | 72 +++++ .../segment/loading/SegmentLoaderConfig.java | 27 +- .../SegmentLoaderLocalCacheManager.java | 18 +- .../segment/loading/StorageLocation.java | 5 + .../StorageLocationSelectorStrategy.java | 57 ++++ .../SegmentLoaderLocalCacheManagerTest.java | 265 +++++++++++++++++- .../StorageLocationSelectorStrategyTest.java | 138 +++++++++ website/.spelling | 2 + 10 files changed, 631 insertions(+), 13 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/segment/loading/LeastBytesUsedStorageLocationSelectorStrategy.java create mode 100644 server/src/main/java/org/apache/druid/segment/loading/RoundRobinStorageLocationSelectorStrategy.java create mode 100644 server/src/main/java/org/apache/druid/segment/loading/StorageLocationSelectorStrategy.java create mode 100644 server/src/test/java/org/apache/druid/segment/loading/StorageLocationSelectorStrategyTest.java diff --git a/docs/configuration/index.md b/docs/configuration/index.md index f44d5aef9ac7..b0b60ede7f37 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1286,6 +1286,7 @@ These Historical configurations can be defined in the `historical/runtime.proper |Property|Description|Default| |--------|-----------|-------| |`druid.segmentCache.locations`|Segments assigned to a Historical process are first stored on the local file system (in a disk cache) and then served by the Historical process. These locations define where that local cache resides. This value cannot be NULL or EMPTY. Here is an example `druid.segmentCache.locations=[{"path": "/mnt/druidSegments", "maxSize": 10000, "freeSpacePercent": 1.0}]`. "freeSpacePercent" is optional, if provided then enforces that much of free disk partition space while storing segments. But, it depends on File.getTotalSpace() and File.getFreeSpace() methods, so enable if only if they work for your File System.| none | +|`druid.segmentCache.locationSelectorStrategy`|The strategy used to select a location from the configured `druid.segmentCache.locations` for segment distribution. Possible values are `leastBytesUsed` or `roundRobin`. |leastBytesUsed| |`druid.segmentCache.deleteOnRemove`|Delete segment files from cache once a process is no longer serving a segment.|true| |`druid.segmentCache.dropSegmentDelayMillis`|How long a process delays before completely dropping segment.|30000 (30 seconds)| |`druid.segmentCache.infoDir`|Historical processes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir| @@ -1296,6 +1297,8 @@ These Historical configurations can be defined in the `historical/runtime.proper In `druid.segmentCache.locations`, *freeSpacePercent* was added because *maxSize* setting is only a theoretical limit and assumes that much space will always be available for storing segments. In case of any druid bug leading to unaccounted segment files left alone on disk or some other process writing stuff to disk, This check can start failing segment loading early before filling up the disk completely and leaving the host usable otherwise. +In `druid.segmentCache.locationSelectorStrategy`, one of leastBytesUsed or roundRobin could be specified to represent the strategy to distribute segments across multiple segment cache locations. The leastBytesUsed which is the default strategy always selects a location which has least bytes used in absolute terms. The roundRobin strategy selects a location in a round robin fashion oblivious to the bytes used or the capacity. Note that `if druid.segmentCache.numLoadingThreads` > 1, multiple threads can download different segments at the same time. In this case, with the leastBytesUsed strategy, historicals may select a sub-optimal storage location because each decision is based on a snapshot of the storage location status of when a segment is requested to download. + #### Historical query configs ##### Concurrent Requests diff --git a/server/src/main/java/org/apache/druid/segment/loading/LeastBytesUsedStorageLocationSelectorStrategy.java b/server/src/main/java/org/apache/druid/segment/loading/LeastBytesUsedStorageLocationSelectorStrategy.java new file mode 100644 index 000000000000..e1e94185c50a --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/loading/LeastBytesUsedStorageLocationSelectorStrategy.java @@ -0,0 +1,57 @@ +/* + * 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.segment.loading; + +import com.google.common.collect.Ordering; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; + +/** + * A {@link StorageLocation} selector strategy that selects a segment cache location that is least filled each time + * among the available storage locations. + */ +public class LeastBytesUsedStorageLocationSelectorStrategy implements StorageLocationSelectorStrategy +{ + private static final Ordering ORDERING = Ordering.from(Comparator + .comparingLong(StorageLocation::currSizeBytes)); + + private List storageLocations; + + public LeastBytesUsedStorageLocationSelectorStrategy(List storageLocations) + { + this.storageLocations = storageLocations; + } + + @Override + public Iterator getLocations() + { + return ORDERING.sortedCopy(this.storageLocations).iterator(); + } + + @Override + public String toString() + { + return "LeastBytesUsedStorageLocationSelectorStrategy{" + + "storageLocations=" + storageLocations + + '}'; + } +} diff --git a/server/src/main/java/org/apache/druid/segment/loading/RoundRobinStorageLocationSelectorStrategy.java b/server/src/main/java/org/apache/druid/segment/loading/RoundRobinStorageLocationSelectorStrategy.java new file mode 100644 index 000000000000..362fb8c0eca8 --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/loading/RoundRobinStorageLocationSelectorStrategy.java @@ -0,0 +1,72 @@ +/* + * 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.segment.loading; + +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * A {@link StorageLocation} selector strategy that selects a segment cache location in a round-robin fashion each time + * among the available storage locations. When {@link Iterator#next()} on iterator retuned by + * {@link RoundRobinStorageLocationSelectorStrategy#getLocations()} is called the locations are returned in a round + * robin fashion even when multiple threads are in use. + */ +public class RoundRobinStorageLocationSelectorStrategy implements StorageLocationSelectorStrategy +{ + + private final List storageLocations; + private final AtomicInteger startIndex = new AtomicInteger(0); + + public RoundRobinStorageLocationSelectorStrategy(List storageLocations) + { + this.storageLocations = storageLocations; + } + + @Override + public Iterator getLocations() + { + return new Iterator() { + + private final int numStorageLocations = storageLocations.size(); + private int remainingIterations = numStorageLocations; + + @Override + public boolean hasNext() + { + return remainingIterations > 0; + } + + @Override + public StorageLocation next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + remainingIterations--; + final StorageLocation nextLocation = + storageLocations.get(startIndex.getAndUpdate(n -> (n + 1) % numStorageLocations)); + return nextLocation; + } + }; + } + +} diff --git a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java index 80f0fbc9fdf4..58eb77317e5b 100644 --- a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java +++ b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java @@ -20,8 +20,8 @@ package org.apache.druid.segment.loading; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.utils.JvmUtils; import org.hibernate.validator.constraints.NotEmpty; @@ -52,6 +52,9 @@ public class SegmentLoaderConfig @JsonProperty("numBootstrapThreads") private Integer numBootstrapThreads = null; + @JsonProperty("locationSelectorStrategy") + private StorageLocationSelectorStrategy locationSelectorStrategy; + @JsonProperty private File infoDir = null; @@ -88,16 +91,20 @@ public int getNumBootstrapThreads() return numBootstrapThreads == null ? numLoadingThreads : numBootstrapThreads; } + public StorageLocationSelectorStrategy getStorageLocationSelectorStrategy(List storageLocations) + { + if (locationSelectorStrategy == null) { + // default strategy if no strategy is specified in the config + locationSelectorStrategy = new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations); + } + return locationSelectorStrategy; + } + public File getInfoDir() { if (infoDir == null) { - - if (locations == null || locations.size() == 0) { - throw new ISE("You have no segment cache locations defined. Please configure druid.segmentCache.locations to use one or more locations."); - } infoDir = new File(locations.get(0).getPath(), "info_dir"); } - return infoDir; } @@ -115,6 +122,13 @@ public SegmentLoaderConfig withLocations(List locations) return retVal; } + @VisibleForTesting + SegmentLoaderConfig withStorageLocationSelectorStrategy(StorageLocationSelectorStrategy strategy) + { + this.locationSelectorStrategy = strategy; + return this; + } + @Override public String toString() { @@ -122,6 +136,7 @@ public String toString() "locations=" + locations + ", deleteOnRemove=" + deleteOnRemove + ", dropSegmentDelayMillis=" + dropSegmentDelayMillis + + ", locationSelectorStrategy=" + locationSelectorStrategy + ", infoDir=" + infoDir + '}'; } diff --git a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderLocalCacheManager.java b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderLocalCacheManager.java index e0f0f0b4ccdb..4a8ce98923f0 100644 --- a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderLocalCacheManager.java +++ b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderLocalCacheManager.java @@ -33,7 +33,7 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Comparator; +import java.util.Iterator; import java.util.List; import java.util.concurrent.ConcurrentHashMap; @@ -42,9 +42,6 @@ public class SegmentLoaderLocalCacheManager implements SegmentLoader { private static final EmittingLogger log = new EmittingLogger(SegmentLoaderLocalCacheManager.class); - private static final Comparator COMPARATOR = Comparator - .comparingLong(StorageLocation::availableSizeBytes) - .reversed(); private final IndexIO indexIO; private final SegmentLoaderConfig config; @@ -77,6 +74,8 @@ public class SegmentLoaderLocalCacheManager implements SegmentLoader */ private final ConcurrentHashMap segmentLocks = new ConcurrentHashMap<>(); + private final StorageLocationSelectorStrategy strategy; + // Note that we only create this via injection in historical and realtime nodes. Peons create these // objects via SegmentLoaderFactory objects, so that they can store segments in task-specific // directories rather than statically configured directories. @@ -101,7 +100,7 @@ public SegmentLoaderLocalCacheManager( ) ); } - locations.sort(COMPARATOR); + this.strategy = config.getStorageLocationSelectorStrategy(locations); } @Override @@ -175,10 +174,17 @@ public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException * location may fail because of IO failure, most likely in two cases:

* 1. druid don't have the write access to this location, most likely the administrator doesn't config it correctly

* 2. disk failure, druid can't read/write to this disk anymore + * + * Locations are fetched using {@link StorageLocationSelectorStrategy}. */ private StorageLocation loadSegmentWithRetry(DataSegment segment, String storageDirStr) throws SegmentLoadingException { - for (StorageLocation loc : locations) { + Iterator locationsIterator = strategy.getLocations(); + + while (locationsIterator.hasNext()) { + + StorageLocation loc = locationsIterator.next(); + File storageDir = loc.reserve(storageDirStr, segment); if (storageDir != null) { try { diff --git a/server/src/main/java/org/apache/druid/segment/loading/StorageLocation.java b/server/src/main/java/org/apache/druid/segment/loading/StorageLocation.java index 6ed86a3c2146..40e10ec621df 100644 --- a/server/src/main/java/org/apache/druid/segment/loading/StorageLocation.java +++ b/server/src/main/java/org/apache/druid/segment/loading/StorageLocation.java @@ -177,4 +177,9 @@ public synchronized long availableSizeBytes() { return maxSizeBytes - currSizeBytes; } + + public synchronized long currSizeBytes() + { + return currSizeBytes; + } } diff --git a/server/src/main/java/org/apache/druid/segment/loading/StorageLocationSelectorStrategy.java b/server/src/main/java/org/apache/druid/segment/loading/StorageLocationSelectorStrategy.java new file mode 100644 index 000000000000..6ac12c2222ba --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/loading/StorageLocationSelectorStrategy.java @@ -0,0 +1,57 @@ +/* + * 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.segment.loading; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.timeline.DataSegment; + +import java.util.Iterator; + +/** + * This interface describes the storage location selection strategy which is responsible for ordering the + * available multiple {@link StorageLocation}s for segment distribution. + * + * Only a snapshot of the locations is returned here. The implemntations currently do not handle all kinds of + * concurrency issues and accesses to the underlying storage. Please see + * https://github.com/apache/incubator-druid/pull/8038#discussion_r325520829 of PR https://github + * .com/apache/incubator-druid/pull/8038 for more details. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = + LeastBytesUsedStorageLocationSelectorStrategy.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "leastBytesUsed", value = LeastBytesUsedStorageLocationSelectorStrategy.class), + @JsonSubTypes.Type(name = "roundRobin", value = RoundRobinStorageLocationSelectorStrategy.class) +}) +public interface StorageLocationSelectorStrategy +{ + /** + * Finds the best ordering of the {@link StorageLocation}s to load a {@link DataSegment} according to + * the location selector strategy. This method returns an iterator instead of a single best location. The + * caller is responsible for iterating over the locations and calling {@link StorageLocation#reserve} + * method. This is because a single location may be problematic like failed disk or might become unwritable for + * whatever reasons. + * + * This method can be called by different threads and so should be thread-safe. + * + * @return An iterator of {@link StorageLocation}s from which the callers can iterate and pick a location. + */ + Iterator getLocations(); +} diff --git a/server/src/test/java/org/apache/druid/segment/loading/SegmentLoaderLocalCacheManagerTest.java b/server/src/test/java/org/apache/druid/segment/loading/SegmentLoaderLocalCacheManagerTest.java index cb38d7b1dc9d..5b2b1a5f6024 100644 --- a/server/src/test/java/org/apache/druid/segment/loading/SegmentLoaderLocalCacheManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/loading/SegmentLoaderLocalCacheManagerTest.java @@ -372,6 +372,11 @@ public void testEmptyToFullOrder() throws Exception } private DataSegment dataSegmentWithInterval(String intervalStr) + { + return dataSegmentWithInterval(intervalStr, 10L); + } + + private DataSegment dataSegmentWithInterval(String intervalStr, long size) { return DataSegment.builder() .dataSource("test_segment_loader") @@ -389,7 +394,265 @@ private DataSegment dataSegmentWithInterval(String intervalStr) .metrics(ImmutableList.of()) .shardSpec(NoneShardSpec.instance()) .binaryVersion(9) - .size(10L) + .size(size) .build(); } + + @Test + public void testSegmentDistributionUsingRoundRobinStrategy() throws Exception + { + final List locationConfigs = new ArrayList<>(); + final StorageLocationConfig locationConfig = createStorageLocationConfig("local_storage_folder", 10000000000L, true); + final StorageLocationConfig locationConfig2 = createStorageLocationConfig("local_storage_folder2", 1000000000L, true); + final StorageLocationConfig locationConfig3 = createStorageLocationConfig("local_storage_folder3", 1000000000L, true); + locationConfigs.add(locationConfig); + locationConfigs.add(locationConfig2); + locationConfigs.add(locationConfig3); + + List locations = new ArrayList<>(); + for (StorageLocationConfig locConfig : locationConfigs) { + locations.add( + new StorageLocation( + locConfig.getPath(), + locConfig.getMaxSize(), + locConfig.getFreeSpacePercent() + ) + ); + } + + manager = new SegmentLoaderLocalCacheManager( + TestHelper.getTestIndexIO(), + new SegmentLoaderConfig().withLocations(locationConfigs).withStorageLocationSelectorStrategy( + new RoundRobinStorageLocationSelectorStrategy(locations) + ), + jsonMapper + ); + final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); + + // Segment 1 should be downloaded in local_storage_folder + final DataSegment segmentToDownload1 = dataSegmentWithInterval("2014-10-20T00:00:00Z/P1D").withLoadSpec( + ImmutableMap.of( + "type", + "local", + "path", + segmentSrcFolder.getCanonicalPath() + + "/test_segment_loader" + + "/2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z/2015-05-27T03:38:35.683Z" + + "/0/index.zip" + ) + ); + // manually create a local segment under segmentSrcFolder + createLocalSegmentFile(segmentSrcFolder, "test_segment_loader/2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z/2015-05-27T03:38:35.683Z/0"); + + Assert.assertFalse("Expect cache miss before downloading segment", manager.isSegmentLoaded(segmentToDownload1)); + + File segmentFile = manager.getSegmentFiles(segmentToDownload1); + Assert.assertTrue(segmentFile.getAbsolutePath().contains("/local_storage_folder/")); + Assert.assertTrue("Expect cache hit after downloading segment", manager.isSegmentLoaded(segmentToDownload1)); + + manager.cleanup(segmentToDownload1); + Assert.assertFalse("Expect cache miss after dropping segment", manager.isSegmentLoaded(segmentToDownload1)); + + // Segment 2 should be downloaded in local_storage_folder2 + final DataSegment segmentToDownload2 = dataSegmentWithInterval("2014-11-20T00:00:00Z/P1D").withLoadSpec( + ImmutableMap.of( + "type", + "local", + "path", + segmentSrcFolder.getCanonicalPath() + + "/test_segment_loader" + + "/2014-11-20T00:00:00.000Z_2014-11-21T00:00:00.000Z/2015-05-27T03:38:35.683Z" + + "/0/index.zip" + ) + ); + // manually create a local segment under segmentSrcFolder + createLocalSegmentFile(segmentSrcFolder, "test_segment_loader/2014-11-20T00:00:00.000Z_2014-11-21T00:00:00.000Z/2015-05-27T03:38:35.683Z/0"); + + Assert.assertFalse("Expect cache miss before downloading segment", manager.isSegmentLoaded(segmentToDownload2)); + + File segmentFile2 = manager.getSegmentFiles(segmentToDownload2); + Assert.assertTrue(segmentFile2.getAbsolutePath().contains("/local_storage_folder2/")); + Assert.assertTrue("Expect cache hit after downloading segment", manager.isSegmentLoaded(segmentToDownload2)); + + manager.cleanup(segmentToDownload2); + Assert.assertFalse("Expect cache miss after dropping segment", manager.isSegmentLoaded(segmentToDownload2)); + + // Segment 3 should be downloaded in local_storage_folder3 + final DataSegment segmentToDownload3 = dataSegmentWithInterval("2014-12-20T00:00:00Z/P1D").withLoadSpec( + ImmutableMap.of( + "type", + "local", + "path", + segmentSrcFolder.getCanonicalPath() + + "/test_segment_loader" + + "/2014-12-20T00:00:00.000Z_2014-12-21T00:00:00.000Z/2015-05-27T03:38:35.683Z" + + "/0/index.zip" + ) + ); + // manually create a local segment under segmentSrcFolder + createLocalSegmentFile(segmentSrcFolder, + "test_segment_loader/2014-12-20T00:00:00.000Z_2014-12-21T00:00:00.000Z/2015-05-27T03:38:35.683Z/0"); + + File segmentFile3 = manager.getSegmentFiles(segmentToDownload3); + Assert.assertTrue(segmentFile3.getAbsolutePath().contains("/local_storage_folder3/")); + Assert.assertTrue("Expect cache hit after downloading segment", manager.isSegmentLoaded(segmentToDownload3)); + + manager.cleanup(segmentToDownload3); + Assert.assertFalse("Expect cache miss after dropping segment", manager.isSegmentLoaded(segmentToDownload3)); + + // Segment 4 should be downloaded in local_storage_folder again, asserting round robin distribution of segments + final DataSegment segmentToDownload4 = dataSegmentWithInterval("2014-08-20T00:00:00Z/P1D").withLoadSpec( + ImmutableMap.of( + "type", + "local", + "path", + segmentSrcFolder.getCanonicalPath() + + "/test_segment_loader" + + "/2014-08-20T00:00:00.000Z_2014-08-21T00:00:00.000Z/2015-05-27T03:38:35.683Z" + + "/0/index.zip" + ) + ); + // manually create a local segment under segmentSrcFolder + createLocalSegmentFile(segmentSrcFolder, "test_segment_loader/2014-08-20T00:00:00.000Z_2014-08-21T00:00:00" + + ".000Z/2015-05-27T03:38:35.683Z/0"); + + Assert.assertFalse("Expect cache miss before downloading segment", manager.isSegmentLoaded(segmentToDownload4)); + + File segmentFile1 = manager.getSegmentFiles(segmentToDownload4); + Assert.assertTrue(segmentFile1.getAbsolutePath().contains("/local_storage_folder/")); + Assert.assertTrue("Expect cache hit after downloading segment", manager.isSegmentLoaded(segmentToDownload4)); + manager.cleanup(segmentToDownload4); + Assert.assertFalse("Expect cache miss after dropping segment", manager.isSegmentLoaded(segmentToDownload4)); + } + + private void createLocalSegmentFile(File segmentSrcFolder, String localSegmentPath) throws Exception + { + // manually create a local segment under segmentSrcFolder + final File localSegmentFile = new File(segmentSrcFolder, localSegmentPath); + localSegmentFile.mkdirs(); + final File indexZip = new File(localSegmentFile, "index.zip"); + indexZip.createNewFile(); + } + + private StorageLocationConfig createStorageLocationConfig(String localPath, long maxSize, boolean writable) throws Exception + { + + final File localStorageFolder = tmpFolder.newFolder(localPath); + localStorageFolder.setWritable(writable); + final StorageLocationConfig locationConfig = new StorageLocationConfig(localStorageFolder, maxSize, 1.0); + return locationConfig; + } + + @Test + public void testSegmentDistributionUsingLeastBytesUsedStrategy() throws Exception + { + final List locations = new ArrayList<>(); + final StorageLocationConfig locationConfig = createStorageLocationConfig("local_storage_folder", 10000000000L, + true); + final StorageLocationConfig locationConfig2 = createStorageLocationConfig("local_storage_folder2", 1000000000L, + true); + final StorageLocationConfig locationConfig3 = createStorageLocationConfig("local_storage_folder3", 1000000000L, + true); + locations.add(locationConfig); + locations.add(locationConfig2); + locations.add(locationConfig3); + + manager = new SegmentLoaderLocalCacheManager( + TestHelper.getTestIndexIO(), + new SegmentLoaderConfig().withLocations(locations), + jsonMapper + ); + final File segmentSrcFolder = tmpFolder.newFolder("segmentSrcFolder"); + + // Segment 1 should be downloaded in local_storage_folder, segment1 size 10L + final DataSegment segmentToDownload = dataSegmentWithInterval("2014-10-20T00:00:00Z/P1D", 10L).withLoadSpec( + ImmutableMap.of( + "type", + "local", + "path", + segmentSrcFolder.getCanonicalPath() + + "/test_segment_loader" + + "/2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z/2015-05-27T03:38:35.683Z" + + "/0/index.zip" + ) + ); + // manually create a local segment under segmentSrcFolder + createLocalSegmentFile(segmentSrcFolder, + "test_segment_loader/2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z/2015-05-27T03:38:35.683Z/0"); + + Assert.assertFalse("Expect cache miss before downloading segment", manager.isSegmentLoaded(segmentToDownload)); + + File segmentFile = manager.getSegmentFiles(segmentToDownload); + Assert.assertTrue(segmentFile.getAbsolutePath().contains("/local_storage_folder/")); + Assert.assertTrue("Expect cache hit after downloading segment", manager.isSegmentLoaded(segmentToDownload)); + + // Segment 2 should be downloaded in local_storage_folder2, segment2 size 5L + final DataSegment segmentToDownload2 = dataSegmentWithInterval("2014-11-20T00:00:00Z/P1D", 5L).withLoadSpec( + ImmutableMap.of( + "type", + "local", + "path", + segmentSrcFolder.getCanonicalPath() + + "/test_segment_loader" + + "/2014-11-20T00:00:00.000Z_2014-11-21T00:00:00.000Z/2015-05-27T03:38:35.683Z" + + "/0/index.zip" + ) + ); + // manually create a local segment under segmentSrcFolder + createLocalSegmentFile(segmentSrcFolder, + "test_segment_loader/2014-11-20T00:00:00.000Z_2014-11-21T00:00:00.000Z/2015-05-27T03:38:35.683Z/0"); + + Assert.assertFalse("Expect cache miss before downloading segment", manager.isSegmentLoaded(segmentToDownload2)); + + File segmentFile2 = manager.getSegmentFiles(segmentToDownload2); + Assert.assertTrue(segmentFile2.getAbsolutePath().contains("/local_storage_folder2/")); + Assert.assertTrue("Expect cache hit after downloading segment", manager.isSegmentLoaded(segmentToDownload2)); + + + // Segment 3 should be downloaded in local_storage_folder3, segment3 size 20L + final DataSegment segmentToDownload3 = dataSegmentWithInterval("2014-12-20T00:00:00Z/P1D", 20L).withLoadSpec( + ImmutableMap.of( + "type", + "local", + "path", + segmentSrcFolder.getCanonicalPath() + + "/test_segment_loader" + + "/2014-12-20T00:00:00.000Z_2014-12-21T00:00:00.000Z/2015-05-27T03:38:35.683Z" + + "/0/index.zip" + ) + ); + // manually create a local segment under segmentSrcFolder + createLocalSegmentFile(segmentSrcFolder, + "test_segment_loader/2014-12-20T00:00:00.000Z_2014-12-21T00:00:00.000Z/2015-05-27T03:38:35.683Z/0"); + + File segmentFile3 = manager.getSegmentFiles(segmentToDownload3); + Assert.assertTrue(segmentFile3.getAbsolutePath().contains("/local_storage_folder3/")); + Assert.assertTrue("Expect cache hit after downloading segment", manager.isSegmentLoaded(segmentToDownload3)); + + // Now the storage locations local_storage_folder1, local_storage_folder2 and local_storage_folder3 have 10, 5 and + // 20 bytes occupied respectively. The default strategy should pick location2 (as it has least bytes used) for the + // next segment to be downloaded asserting the least bytes used distribution of segments. + final DataSegment segmentToDownload4 = dataSegmentWithInterval("2014-08-20T00:00:00Z/P1D").withLoadSpec( + ImmutableMap.of( + "type", + "local", + "path", + segmentSrcFolder.getCanonicalPath() + + "/test_segment_loader" + + "/2014-08-20T00:00:00.000Z_2014-08-21T00:00:00.000Z/2015-05-27T03:38:35.683Z" + + "/0/index.zip" + ) + ); + // manually create a local segment under segmentSrcFolder + createLocalSegmentFile(segmentSrcFolder, "test_segment_loader/2014-08-20T00:00:00.000Z_2014-08-21T00:00:00" + + ".000Z/2015-05-27T03:38:35.683Z/0"); + + Assert.assertFalse("Expect cache miss before downloading segment", manager.isSegmentLoaded(segmentToDownload4)); + + File segmentFile1 = manager.getSegmentFiles(segmentToDownload4); + Assert.assertTrue(segmentFile1.getAbsolutePath().contains("/local_storage_folder2/")); + Assert.assertTrue("Expect cache hit after downloading segment", manager.isSegmentLoaded(segmentToDownload4)); + + } + } diff --git a/server/src/test/java/org/apache/druid/segment/loading/StorageLocationSelectorStrategyTest.java b/server/src/test/java/org/apache/druid/segment/loading/StorageLocationSelectorStrategyTest.java new file mode 100644 index 000000000000..bfe79cafb7a0 --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/loading/StorageLocationSelectorStrategyTest.java @@ -0,0 +1,138 @@ +/* + * 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.segment.loading; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +public class StorageLocationSelectorStrategyTest +{ + + @Rule + public final TemporaryFolder tmpFolder = new TemporaryFolder(); + + @Test + public void testLeastBytesUsedLocationSelectorStrategy() throws Exception + { + + List storageLocations = new ArrayList<>(); + + final File localStorageFolder1 = tmpFolder.newFolder("local_storage_folder_1"); + final File localStorageFolder2 = tmpFolder.newFolder("local_storage_folder_2"); + final File localStorageFolder3 = tmpFolder.newFolder("local_storage_folder_3"); + + StorageLocation storageLocation1 = new StorageLocation(localStorageFolder1, 10000000000L, + null); + storageLocations.add(storageLocation1); + storageLocations.add(new StorageLocation(localStorageFolder2, 10000000000L, null)); + storageLocations.add(new StorageLocation(localStorageFolder3, 10000000000L, null)); + + StorageLocationSelectorStrategy leastBytesUsedStrategy = + new LeastBytesUsedStorageLocationSelectorStrategy(storageLocations); + + storageLocation1.reserve("tmp_loc1", "__seg1", 1024L); + + Iterator locations = leastBytesUsedStrategy.getLocations(); + + StorageLocation loc1 = locations.next(); + Assert.assertEquals("The next element of the iterator should point to path local_storage_folder_2", + localStorageFolder2, loc1.getPath()); + + StorageLocation loc2 = locations.next(); + Assert.assertEquals("The next element of the iterator should point to path local_storage_folder_3", + localStorageFolder3, loc2.getPath()); + + StorageLocation loc3 = locations.next(); + Assert.assertEquals("The next element of the iterator should point to path local_storage_folder_1", + localStorageFolder1, loc3.getPath()); + + } + + @Test + public void testRoundRobinLocationSelectorStrategySingleLocation() throws Exception + { + List storageLocations = new ArrayList<>(); + final File localStorageFolder1 = tmpFolder.newFolder("local_storage_folder_1"); + storageLocations.add(new StorageLocation(localStorageFolder1, 10000000000L, null)); + StorageLocationSelectorStrategy roundRobinStrategy = + new RoundRobinStorageLocationSelectorStrategy(storageLocations); + + Iterator locations = roundRobinStrategy.getLocations(); + + StorageLocation loc1 = locations.next(); + Assert.assertEquals("The next element of the iterator should point to path local_storage_folder_1", + localStorageFolder1, loc1.getPath()); + + locations = roundRobinStrategy.getLocations(); + + StorageLocation loc2 = locations.next(); + Assert.assertEquals("The next element of the iterator should point to path local_storage_folder_1", + localStorageFolder1, loc2.getPath()); + } + + + @Test + public void testRoundRobinLocationSelectorStrategy() throws Exception + { + List storageLocations = new ArrayList<>(); + + final File localStorageFolder1 = tmpFolder.newFolder("local_storage_folder_1"); + final File localStorageFolder2 = tmpFolder.newFolder("local_storage_folder_2"); + final File localStorageFolder3 = tmpFolder.newFolder("local_storage_folder_3"); + + storageLocations.add(new StorageLocation(localStorageFolder1, 10000000000L, null)); + storageLocations.add(new StorageLocation(localStorageFolder2, 10000000000L, null)); + storageLocations.add(new StorageLocation(localStorageFolder3, 10000000000L, null)); + + StorageLocationSelectorStrategy roundRobinStrategy = new RoundRobinStorageLocationSelectorStrategy(storageLocations); + + iterateLocs(localStorageFolder1, localStorageFolder2, localStorageFolder3, roundRobinStrategy); + iterateLocs(localStorageFolder1, localStorageFolder2, localStorageFolder3, roundRobinStrategy); + iterateLocs(localStorageFolder1, localStorageFolder2, localStorageFolder3, roundRobinStrategy); + iterateLocs(localStorageFolder1, localStorageFolder2, localStorageFolder3, roundRobinStrategy); + iterateLocs(localStorageFolder1, localStorageFolder2, localStorageFolder3, roundRobinStrategy); + } + + private void iterateLocs(File localStorageFolder1, File localStorageFolder2, File localStorageFolder3, + StorageLocationSelectorStrategy roundRobinStrategy) + { + Iterator locations = roundRobinStrategy.getLocations(); + + StorageLocation loc1 = locations.next(); + Assert.assertEquals("The next element of the iterator should point to path local_storage_folder_1", + localStorageFolder1, loc1.getPath()); + + StorageLocation loc2 = locations.next(); + Assert.assertEquals("The next element of the iterator should point to path local_storage_folder_2", + localStorageFolder2, loc2.getPath()); + + StorageLocation loc3 = locations.next(); + Assert.assertEquals("The next element of the iterator should point to path local_storage_folder_3", + localStorageFolder3, loc3.getPath()); + } + +} diff --git a/website/.spelling b/website/.spelling index e14a2ba6348c..cae3b13b2b64 100644 --- a/website/.spelling +++ b/website/.spelling @@ -1549,6 +1549,7 @@ java.class.path java.io.tmpdir javaOpts javaOptsArray +leastBytesUsed loadList loadqueuepeon loadspec @@ -1572,6 +1573,7 @@ queryType remoteTaskRunnerConfig rendezvousHash resultsets +roundRobin runtime.properties runtime.properties. s3 From f98ea8053709a3c81a5f9a44777ed727fd2aa81e Mon Sep 17 00:00:00 2001 From: Vadim Ogievetsky Date: Sun, 29 Sep 2019 19:32:39 -0700 Subject: [PATCH 12/26] readme gifs (#8608) --- README.md | 27 ++++++++++++++++++++++++--- 1 file changed, 24 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 0567872ac73e..cb496b46c52b 100644 --- a/README.md +++ b/README.md @@ -31,10 +31,31 @@ Apache Druid (incubating) is a high performance analytics data store for event-d *Disclaimer: Apache Druid is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Apache Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.* -### Getting Started +### Getting started You can get started with Druid with our [quickstart](https://druid.apache.org/docs/latest/tutorials/quickstart.html). +Druid provides a rich set of APIs (via HTTP and [JDBC](https://druid.apache.org/docs/latest/querying/sql.html#jdbc)) for loading, managing, and querying your data. +You can also interact with Druid via the [built-in console](https://druid.apache.org/docs/latest/operations/druid-console.html) (shown below). + +#### Load data + +[![data loader Kafka](https://user-images.githubusercontent.com/177816/65819337-054eac80-e1d0-11e9-8842-97b92d8c6159.gif)](https://druid.apache.org/docs/latest/ingestion/index.html) + +Load [streaming](https://druid.apache.org/docs/latest/ingestion/index.html#streaming) and [batch](https://druid.apache.org/docs/latest/ingestion/index.html#batch) data using a point-and-click wizard to guide you through ingestion setup. Monitor one off tasks and ingestion supervisors. + +#### Manage the cluster + +[![management](https://user-images.githubusercontent.com/177816/65819338-08499d00-e1d0-11e9-80fe-faee9e9468cb.gif)](https://druid.apache.org/docs/latest/ingestion/data-management.html) + +Manage your cluster with ease. Get a view of your datasources, [segments](https://druid.apache.org/docs/latest/design/segments.html), [ingestion tasks](https://druid.apache.org/docs/latest/ingestion/tasks.html), and [servers]() from one convenient location. All powered by [SQL systems tables](https://druid.apache.org/docs/latest/querying/sql.html#metadata-tables) allowing you to see the underlying query for each view. + +#### Issue queries + +[![query view combo](https://user-images.githubusercontent.com/177816/65819341-0c75ba80-e1d0-11e9-9730-0f2d084defcc.gif)](https://druid.apache.org/docs/latest/querying/sql.html) + +Use the built-in query workbench to prototype [DruidSQL](https://druid.apache.org/docs/latest/querying/sql.html) and [native](https://druid.apache.org/docs/latest/querying/querying.html) queries or connect one of the [many tools](https://druid.apache.org/libraries.html) that help you make the most out of Druid. + ### Documentation You can find the [documentation for the latest Druid release](https://druid.apache.org/docs/latest/) on @@ -52,9 +73,9 @@ is hosted at Google Groups. Development discussions occur on [dev@druid.apache.org](https://lists.apache.org/list.html?dev@druid.apache.org), which you can subscribe to by emailing [dev-subscribe@druid.apache.org](mailto:dev-subscribe@druid.apache.org). -Some committers and users are present in the channel `#druid` on the Apache Slack team. Please use our [invitation link to join](https://druid.apache.org/community/join-slack), and once you join, add the `#druid` channel. +Chat with Druid committers and users in real-time on the `#druid` channel in the Apache Slack team. Please use [this invitation link to join the ASF Slack](https://s.apache.org/slack-invite), and once joined, go into the `#druid` channel. -### Building From Source +### Building from source Please note that JDK 8 is required to build Druid. From d3f4e52f531e2ace24248deda570e2060685bfe4 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 30 Sep 2019 10:36:56 +0200 Subject: [PATCH 13/26] Remove the Creators --- .../server/coordination/SegmentChangeRequestDrop.java | 8 ++------ .../server/coordination/SegmentChangeRequestLoad.java | 7 ++----- 2 files changed, 4 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestDrop.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestDrop.java index c4229a028806..528eb476feee 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestDrop.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestDrop.java @@ -19,7 +19,6 @@ package org.apache.druid.server.coordination; -import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonUnwrapped; import org.apache.druid.java.util.common.StringUtils; @@ -33,18 +32,15 @@ */ public class SegmentChangeRequestDrop implements DataSegmentChangeRequest { + @JsonUnwrapped private final DataSegment segment; - @JsonCreator - public SegmentChangeRequestDrop( - @JsonUnwrapped DataSegment segment - ) + public SegmentChangeRequestDrop(DataSegment segment) { this.segment = segment; } @JsonProperty - @JsonUnwrapped public DataSegment getSegment() { return segment; diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java index 097e02523032..0b74e9fe3cbb 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java @@ -19,7 +19,6 @@ package org.apache.druid.server.coordination; -import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonUnwrapped; import org.apache.druid.java.util.common.StringUtils; @@ -33,12 +32,10 @@ */ public class SegmentChangeRequestLoad implements DataSegmentChangeRequest { + @JsonUnwrapped private final DataSegment segment; - @JsonCreator - public SegmentChangeRequestLoad( - @JsonUnwrapped DataSegment segment - ) + public SegmentChangeRequestLoad(DataSegment segment) { this.segment = segment; } From cbaad0ddbbed288d7fdbcb78eab252b5a75e8320 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 30 Sep 2019 11:00:21 +0200 Subject: [PATCH 14/26] Fix licenses --- licenses.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/licenses.yaml b/licenses.yaml index 5b22de4b013a..3d29775f5805 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -237,7 +237,7 @@ name: Jackson license_category: binary module: java-core license_name: Apache License version 2.0 -version: 2.9.9 +version: 2.9.10 libraries: - com.fasterxml.jackson.core: jackson-annotations - com.fasterxml.jackson.module: jackson-module-jaxb-annotations From fbe809797126584933da479af741cdb3839537ca Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 30 Sep 2019 12:06:08 +0200 Subject: [PATCH 15/26] Add default constructor --- .../druid/server/coordination/SegmentChangeRequestDrop.java | 6 +++++- .../druid/server/coordination/SegmentChangeRequestLoad.java | 6 +++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestDrop.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestDrop.java index 528eb476feee..8aea75e5f405 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestDrop.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestDrop.java @@ -33,7 +33,11 @@ public class SegmentChangeRequestDrop implements DataSegmentChangeRequest { @JsonUnwrapped - private final DataSegment segment; + private DataSegment segment; + + public SegmentChangeRequestDrop() + { + } public SegmentChangeRequestDrop(DataSegment segment) { diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java index 0b74e9fe3cbb..469ce1d95582 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java @@ -33,7 +33,11 @@ public class SegmentChangeRequestLoad implements DataSegmentChangeRequest { @JsonUnwrapped - private final DataSegment segment; + private DataSegment segment; + + public SegmentChangeRequestLoad() + { + } public SegmentChangeRequestLoad(DataSegment segment) { From 75880cba9296c4a21fb97a2bc45f0a66d489f4cb Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 30 Sep 2019 13:56:20 +0200 Subject: [PATCH 16/26] One missing JsonUnwrapped --- .../druid/server/coordination/SegmentChangeRequestLoad.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java index 469ce1d95582..17251c5ca927 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java @@ -51,7 +51,6 @@ public void go(DataSegmentChangeHandler handler, @Nullable DataSegmentChangeCall } @JsonProperty - @JsonUnwrapped public DataSegment getSegment() { return segment; From 46502acfeb70abe5caa1b2214f296e53eedeea4d Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 30 Sep 2019 14:47:25 +0200 Subject: [PATCH 17/26] Add READONLY annotation --- .../druid/server/coordination/SegmentChangeRequestDrop.java | 3 +++ .../druid/server/coordination/SegmentChangeRequestLoad.java | 3 +++ 2 files changed, 6 insertions(+) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestDrop.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestDrop.java index 8aea75e5f405..b930a824c26f 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestDrop.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestDrop.java @@ -28,11 +28,14 @@ import java.util.Objects; +import static com.fasterxml.jackson.annotation.JsonProperty.Access.READ_ONLY; + /** */ public class SegmentChangeRequestDrop implements DataSegmentChangeRequest { @JsonUnwrapped + @JsonProperty(access = READ_ONLY) private DataSegment segment; public SegmentChangeRequestDrop() diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java index 17251c5ca927..ae636f213c20 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java @@ -28,11 +28,14 @@ import java.util.Objects; +import static com.fasterxml.jackson.annotation.JsonProperty.Access.READ_ONLY; + /** */ public class SegmentChangeRequestLoad implements DataSegmentChangeRequest { @JsonUnwrapped + @JsonProperty(access = READ_ONLY) private DataSegment segment; public SegmentChangeRequestLoad() From dd51bda07f4209720c368c8e4a3a871c77127ded Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 30 Sep 2019 16:18:20 +0200 Subject: [PATCH 18/26] Make the dataSegment optional --- .../timeline/SegmentWithOvershadowedStatus.java | 14 +++++++++++--- .../coordination/SegmentChangeRequestDrop.java | 2 -- .../coordination/SegmentChangeRequestLoad.java | 3 --- 3 files changed, 11 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java b/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java index 3f2972fd07e9..81bab9954958 100644 --- a/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java +++ b/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java @@ -40,16 +40,24 @@ public class SegmentWithOvershadowedStatus implements Comparable Date: Tue, 1 Oct 2019 08:27:42 +0200 Subject: [PATCH 19/26] Fix order --- .../apache/druid/timeline/SegmentWithOvershadowedStatus.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java b/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java index 81bab9954958..d34ae4dd8eb5 100644 --- a/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java +++ b/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java @@ -31,7 +31,6 @@ */ public class SegmentWithOvershadowedStatus implements Comparable { - private final boolean overshadowed; /** * dataSegment is serialized "unwrapped", i.e. it's properties are included as properties of * enclosing class. If in future, if {@Code SegmentWithOvershadowedStatus} were to extend {@link DataSegment}, @@ -40,6 +39,8 @@ public class SegmentWithOvershadowedStatus implements Comparable Date: Tue, 1 Oct 2019 08:56:30 +0200 Subject: [PATCH 20/26] Set the order explicitly --- .../apache/druid/timeline/SegmentWithOvershadowedStatus.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java b/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java index d34ae4dd8eb5..8986350befef 100644 --- a/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java +++ b/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import com.fasterxml.jackson.annotation.JsonUnwrapped; /** @@ -29,6 +30,7 @@ * SegmentWithOvershadowedStatus's {@link #compareTo} method considers only the {@link SegmentId} * of the DataSegment object. */ +@JsonPropertyOrder(alphabetic=true) public class SegmentWithOvershadowedStatus implements Comparable { /** From f3351f4e84c78b9a7bd9ac6931a5b9e4c0b09f4b Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 1 Oct 2019 09:48:36 +0200 Subject: [PATCH 21/26] Fix the order of the keys --- .../test/resources/results/auth_test_sys_schema_segments.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration-tests/src/test/resources/results/auth_test_sys_schema_segments.json b/integration-tests/src/test/resources/results/auth_test_sys_schema_segments.json index 4437e725e28b..a169cfe87361 100644 --- a/integration-tests/src/test/resources/results/auth_test_sys_schema_segments.json +++ b/integration-tests/src/test/resources/results/auth_test_sys_schema_segments.json @@ -13,6 +13,6 @@ "is_available": 1, "is_realtime": 0, "is_overshadowed": 0, - "payload": "{\"dataSource\":\"auth_test\",\"interval\":\"2012-12-29T00:00:00.000Z/2013-01-10T08:00:00.000Z\",\"version\":\"2013-01-10T08:13:47.830Z_v9\",\"loadSpec\":{\"load spec is pruned, because it's not needed on Brokers, but eats a lot of heap space\":\"\"},\"dimensions\":\"anonymous,area_code,city,continent_code,country_name,dma_code,geo,language,namespace,network,newpage,page,postal_code,region_lookup,robot,unpatrolled,user\",\"metrics\":\"added,count,deleted,delta,delta_hist,unique_users,variation\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":446027801,\"identifier\":\"auth_test_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\",\"overshadowed\":false}" + "payload": "{\"overshadowed\":false,\"dataSource\":\"auth_test\",\"interval\":\"2012-12-29T00:00:00.000Z/2013-01-10T08:00:00.000Z\",\"version\":\"2013-01-10T08:13:47.830Z_v9\",\"loadSpec\":{\"load spec is pruned, because it's not needed on Brokers, but eats a lot of heap space\":\"\"},\"dimensions\":\"anonymous,area_code,city,continent_code,country_name,dma_code,geo,language,namespace,network,newpage,page,postal_code,region_lookup,robot,unpatrolled,user\",\"metrics\":\"added,count,deleted,delta,delta_hist,unique_users,variation\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":446027801,\"identifier\":\"auth_test_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\"}" } ] From 385d9c49284b77004049a09709e3cf39d1e14640 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 1 Oct 2019 09:56:26 +0200 Subject: [PATCH 22/26] Fix checkstyle --- .../apache/druid/timeline/SegmentWithOvershadowedStatus.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java b/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java index 8986350befef..d34ae4dd8eb5 100644 --- a/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java +++ b/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonPropertyOrder; import com.fasterxml.jackson.annotation.JsonUnwrapped; /** @@ -30,7 +29,6 @@ * SegmentWithOvershadowedStatus's {@link #compareTo} method considers only the {@link SegmentId} * of the DataSegment object. */ -@JsonPropertyOrder(alphabetic=true) public class SegmentWithOvershadowedStatus implements Comparable { /** From 7db70a15ccf0212688996ba9c47b0aec4f8de00a Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 1 Oct 2019 10:36:48 +0200 Subject: [PATCH 23/26] Remove unnecessary change --- .../apache/druid/timeline/SegmentWithOvershadowedStatus.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java b/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java index d34ae4dd8eb5..81bab9954958 100644 --- a/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java +++ b/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java @@ -31,6 +31,7 @@ */ public class SegmentWithOvershadowedStatus implements Comparable { + private final boolean overshadowed; /** * dataSegment is serialized "unwrapped", i.e. it's properties are included as properties of * enclosing class. If in future, if {@Code SegmentWithOvershadowedStatus} were to extend {@link DataSegment}, @@ -39,8 +40,6 @@ public class SegmentWithOvershadowedStatus implements Comparable Date: Wed, 6 Nov 2019 17:41:02 +0100 Subject: [PATCH 24/26] Bump to Jackson 2.10.0 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index f81d3b06adda..eb51417c0f95 100644 --- a/pom.xml +++ b/pom.xml @@ -88,7 +88,7 @@ 1.3 9.4.10.v20180503 1.19.3 - 2.9.10 + 2.10.0 2.5 3.10.6.Final From 4d0083fab2f5a65b3399a78dde80f478e139decc Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Sun, 5 Jan 2020 20:51:42 +0100 Subject: [PATCH 25/26] Merge conflicts --- pom.xml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/pom.xml b/pom.xml index e29286e08683..49efe01be832 100644 --- a/pom.xml +++ b/pom.xml @@ -89,14 +89,9 @@ 1.3 9.4.12.v20180830 1.19.3 -<<<<<<< HEAD - 2.10.0 - 2.5 -======= 2.10.1 1.9.13 2.8.2 ->>>>>>> 66657012bf50c42b5d847a379f33f8a5bdda2dda 3.10.6.Final 4.1.42.Final From 259e5a000e5d46b024389b9d2ef073303c7536a8 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Sun, 5 Jan 2020 20:53:59 +0100 Subject: [PATCH 26/26] Resolve merge conflicts --- .../druid/timeline/SegmentWithOvershadowedStatus.java | 2 +- .../overlord/IndexerMetadataStorageCoordinator.java | 8 -------- 2 files changed, 1 insertion(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java b/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java index 06bd619e23d3..04c8bf4378b6 100644 --- a/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java +++ b/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java @@ -42,7 +42,7 @@ public class SegmentWithOvershadowedStatus implements Comparable>>>>>> 66657012bf50c42b5d847a379f33f8a5bdda2dda */ default Collection getUsedSegmentsForInterval(String dataSource, Interval interval, Segments visibility) { @@ -95,14 +91,10 @@ default Collection getUsedSegmentsForInterval(String dataSource, In * @return The DataSegments which include data in the requested intervals. These segments may contain data outside the * requested interval. * -<<<<<<< HEAD - * @return The DataSegments which include data in the requested intervals. These segments may contain data outside the requested interval. -======= * @implNote This method doesn't return a {@link Set} because there may be an expectation that {@code Set.contains()} * is O(1) operation, while it's not the case for the returned collection unless it copies all segments into a new * {@link java.util.HashSet} or {@link com.google.common.collect.ImmutableSet} which may in turn be unnecessary in * other use cases. So clients should perform such copy themselves if they need {@link Set} semantics. ->>>>>>> 66657012bf50c42b5d847a379f33f8a5bdda2dda */ Collection getUsedSegmentsForIntervals(String dataSource, List intervals, Segments visibility);