diff --git a/.github/workflows/revised-its.yml b/.github/workflows/revised-its.yml index 62aac48dc99..069562bf7bd 100644 --- a/.github/workflows/revised-its.yml +++ b/.github/workflows/revised-its.yml @@ -50,7 +50,7 @@ jobs: matrix: #jdk: [8, 11, 17] jdk: [8] - it: [HighAvailability, MultiStageQuery, Catalog, BatchIndex, MultiStageQueryWithMM, InputSource, InputFormat, Security] + it: [HighAvailability, MultiStageQuery, Catalog, BatchIndex, MultiStageQueryWithMM, InputSource, InputFormat, Security, Query] #indexer: [indexer, middleManager] indexer: [middleManager] uses: ./.github/workflows/reusable-revised-its.yml diff --git a/distribution/bin/tag-missing-milestones.py b/distribution/bin/tag-missing-milestones.py index e25ca743749..cf41afd63b6 100755 --- a/distribution/bin/tag-missing-milestones.py +++ b/distribution/bin/tag-missing-milestones.py @@ -16,10 +16,9 @@ # limitations under the License. import os +import requests import subprocess import sys -import requests - if len(sys.argv) != 5: sys.stderr.write('usage: program \n') diff --git a/docs/ingestion/tranquility.md b/docs/ingestion/tranquility.md index 9cc0636fd6c..9124ff04b86 100644 --- a/docs/ingestion/tranquility.md +++ b/docs/ingestion/tranquility.md @@ -22,15 +22,9 @@ title: "Tranquility" ~ under the License. --> -[Tranquility](https://github.com/druid-io/tranquility/) is a separately distributed package for pushing -streams to Druid in real-time. - -Tranquility has not been built against a version of Druid later than Druid 0.9.2 -release. It may still work with the latest Druid servers, but not all features and functionality will be available -due to limitations of older Druid APIs on the Tranquility side. +[Tranquility](https://github.com/druid-io/tranquility/) was a separately distributed package for pushing +streams to Druid in real-time. It is not compatible with recent versions of Druid. For new projects that require streaming ingestion, we recommend using Druid's native support for [Apache Kafka](../ingestion/kafka-ingestion.md) or [Amazon Kinesis](../ingestion/kinesis-ingestion.md). - -For more details, check out the [Tranquility GitHub page](https://github.com/druid-io/tranquility/). diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java index 80a5408cfe3..38f15a840dc 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java @@ -64,7 +64,6 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; -import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -145,7 +144,7 @@ public class MaterializedViewSupervisorTest } @Test - public void testCheckSegments() throws IOException + public void testCheckSegments() { List baseSegments = createBaseSegments(); Set derivativeSegments = Sets.newHashSet(createDerivativeSegments()); @@ -171,7 +170,7 @@ public class MaterializedViewSupervisorTest } @Test - public void testSubmitTasksDoesNotFailIfTaskAlreadyExists() throws IOException + public void testSubmitTasksDoesNotFailIfTaskAlreadyExists() { Set baseSegments = Sets.newHashSet(createBaseSegments()); Set derivativeSegments = Sets.newHashSet(createDerivativeSegments()); @@ -193,7 +192,7 @@ public class MaterializedViewSupervisorTest } @Test - public void testSubmitTasksFailsIfTaskCannotBeAdded() throws IOException + public void testSubmitTasksFailsIfTaskCannotBeAdded() { Set baseSegments = Sets.newHashSet(createBaseSegments()); Set derivativeSegments = Sets.newHashSet(createDerivativeSegments()); @@ -219,7 +218,7 @@ public class MaterializedViewSupervisorTest } @Test - public void testCheckSegmentsAndSubmitTasks() throws IOException + public void testCheckSegmentsAndSubmitTasks() { Set baseSegments = Collections.singleton(createBaseSegments().get(0)); indexerMetadataStorageCoordinator.commitSegments(baseSegments, null); diff --git a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java index bb6d649f70e..98ab56022db 100644 --- a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java +++ b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java @@ -178,13 +178,8 @@ public class DatasourceOptimizerTest extends CuratorTestBase Lists.newArrayList("dim1", "dim2", "dim3", "dim4"), 1024 * 1024 ); - try { - metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment), null); - announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper); - } - catch (IOException e) { - return false; - } + metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment), null); + announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper); return true; } ); @@ -203,13 +198,8 @@ public class DatasourceOptimizerTest extends CuratorTestBase Lists.newArrayList("dim1", "dim2", "dim3"), 1024 ); - try { - metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment), null); - announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper); - } - catch (IOException e) { - return false; - } + metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment), null); + announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper); return true; } ); diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorTest.java index d680abcadf3..5888b7d13dd 100644 --- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorTest.java +++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorTest.java @@ -49,7 +49,6 @@ import org.junit.Assert; import org.junit.Test; import javax.annotation.Nullable; - import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/extensions-core/druid-pac4j/src/main/java/org/apache/druid/security/pac4j/Pac4jFilter.java b/extensions-core/druid-pac4j/src/main/java/org/apache/druid/security/pac4j/Pac4jFilter.java index db7d5affe90..05a4ce78b44 100644 --- a/extensions-core/druid-pac4j/src/main/java/org/apache/druid/security/pac4j/Pac4jFilter.java +++ b/extensions-core/druid-pac4j/src/main/java/org/apache/druid/security/pac4j/Pac4jFilter.java @@ -41,7 +41,6 @@ import javax.servlet.ServletRequest; import javax.servlet.ServletResponse; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; - import java.io.IOException; import java.util.Collection; 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 index 5ee288d2622..1309ed0dde0 100644 --- 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 @@ -367,7 +367,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3)); - Assert.assertTrue(task.getRunner().getFireDepartmentMetrics().isProcessingDone()); + Assert.assertTrue(task.getRunner().getSegmentGenerationMetrics().isProcessingDone()); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -505,7 +505,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3)); - Assert.assertTrue(task.getRunner().getFireDepartmentMetrics().isProcessingDone()); + Assert.assertTrue(task.getRunner().getSegmentGenerationMetrics().isProcessingDone()); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -554,7 +554,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3)); - Assert.assertTrue(task.getRunner().getFireDepartmentMetrics().isProcessingDone()); + Assert.assertTrue(task.getRunner().getSegmentGenerationMetrics().isProcessingDone()); // Check published metadata and segments in deep storage assertEqualsExceptVersion( diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index a7d47788325..b18c1749125 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -40,7 +40,7 @@ import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.TestUtils; -import org.apache.druid.indexing.common.task.RealtimeIndexTask; +import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.kafka.KafkaConsumerConfigs; import org.apache.druid.indexing.kafka.KafkaDataSourceMetadata; @@ -89,9 +89,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.realtime.FireDepartment; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; @@ -1194,15 +1192,13 @@ public class KafkaSupervisorTest extends EasyMockSupport addSomeEvents(1); // non KafkaIndexTask (don't kill) - Task id2 = new RealtimeIndexTask( + Task id2 = new NoopTask( "id2", null, - new FireDepartment( - dataSchema, - new RealtimeIOConfig(null, null), - null - ), - null + dataSchema.getDataSource(), + 100, + 100, + ImmutableMap.of() ); List existingTasks = ImmutableList.of(id2); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index a4ab6a30a80..9001f148e99 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -37,7 +37,7 @@ import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.TestUtils; -import org.apache.druid.indexing.common.task.RealtimeIndexTask; +import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata; import org.apache.druid.indexing.kinesis.KinesisIndexTask; @@ -84,9 +84,7 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.realtime.FireDepartment; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; @@ -940,15 +938,13 @@ public class KinesisSupervisorTest extends EasyMockSupport EasyMock.expectLastCall().anyTimes(); // non KinesisIndexTask (don't kill) - Task id2 = new RealtimeIndexTask( + Task id2 = new NoopTask( "id2", null, - new FireDepartment( - dataSchema, - new RealtimeIOConfig(null, null), - null - ), - null + dataSchema.getDataSource(), + 100, + 100, + ImmutableMap.of() ); List existingTasks = ImmutableList.of(id2); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/SegmentGeneratorMetricsWrapper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/SegmentGeneratorMetricsWrapper.java index 8f3e2d79aae..81f67a9ce05 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/SegmentGeneratorMetricsWrapper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/counters/SegmentGeneratorMetricsWrapper.java @@ -20,15 +20,15 @@ package org.apache.druid.msq.counters; import org.apache.druid.msq.indexing.processor.SegmentGeneratorFrameProcessor; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; /** - * Wrapper around {@link FireDepartmentMetrics} which updates the progress counters while updating its metrics. This + * Wrapper around {@link SegmentGenerationMetrics} which updates the progress counters while updating its metrics. This * is necessary as the {@link org.apache.druid.segment.realtime.appenderator.BatchAppenderator} used by the * {@link SegmentGeneratorFrameProcessor} is not part of the MSQ extension, and hence, * cannot update the counters used in MSQ reports as it persists and pushes segments to deep storage. */ -public class SegmentGeneratorMetricsWrapper extends FireDepartmentMetrics +public class SegmentGeneratorMetricsWrapper extends SegmentGenerationMetrics { private final SegmentGenerationProgressCounter segmentGenerationProgressCounter; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index 888a1f3f547..a144a0b3594 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -88,7 +88,6 @@ import org.joda.time.Interval; import org.mockito.Mockito; import javax.annotation.Nullable; - import java.io.File; import java.util.List; import java.util.Set; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorBuilder.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorBuilder.java index abd95169073..8129b16a531 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorBuilder.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorBuilder.java @@ -19,14 +19,12 @@ package org.apache.druid.indexing.common; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.incremental.RowIngestionMeters; -import org.apache.druid.segment.realtime.FireDepartment; -import org.apache.druid.segment.realtime.RealtimeMetricsMonitor; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; public class TaskRealtimeMetricsMonitorBuilder { @@ -34,31 +32,21 @@ public class TaskRealtimeMetricsMonitorBuilder { } - public static RealtimeMetricsMonitor build(Task task, FireDepartment fireDepartment) - { - return new RealtimeMetricsMonitor( - ImmutableList.of(fireDepartment), - ImmutableMap.of( - DruidMetrics.TASK_ID, new String[]{task.getId()}, - DruidMetrics.TASK_TYPE, new String[]{task.getType()} - ) - ); - } - public static TaskRealtimeMetricsMonitor build( Task task, - FireDepartment fireDepartment, + SegmentGenerationMetrics metrics, RowIngestionMeters meters ) { return new TaskRealtimeMetricsMonitor( - fireDepartment, + metrics, meters, ImmutableMap.of( + DruidMetrics.DATASOURCE, new String[]{task.getDataSource()}, DruidMetrics.TASK_ID, new String[]{task.getId()}, DruidMetrics.TASK_TYPE, new String[]{task.getType()}, DruidMetrics.GROUP_ID, new String[]{task.getGroupId()} - ), + ), task.getContextValue(DruidMetrics.TAGS) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorIngestionSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorIngestionSpec.java deleted file mode 100644 index c244a0c509d..00000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorIngestionSpec.java +++ /dev/null @@ -1,40 +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.common.index; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.IngestionSpec; -import org.apache.druid.segment.indexing.RealtimeIOConfig; - -public class RealtimeAppenderatorIngestionSpec extends IngestionSpec -{ - - @JsonCreator - public RealtimeAppenderatorIngestionSpec( - @JsonProperty("dataSchema") DataSchema dataSchema, - @JsonProperty("ioConfig") RealtimeIOConfig ioConfig, - @JsonProperty("tuningConfig") RealtimeAppenderatorTuningConfig tuningConfig - ) - { - super(dataSchema, ioConfig, tuningConfig); - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java deleted file mode 100644 index 4a117976e18..00000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java +++ /dev/null @@ -1,357 +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.common.index; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import com.google.common.base.Preconditions; -import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; -import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.incremental.AppendableIndexSpec; -import org.apache.druid.segment.indexing.TuningConfig; -import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; -import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; -import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.apache.druid.timeline.partition.ShardSpec; -import org.joda.time.Period; - -import javax.annotation.Nullable; -import java.io.File; - -@JsonTypeName("realtime_appenderator") -public class RealtimeAppenderatorTuningConfig implements AppenderatorConfig -{ - private static final Period DEFAULT_INTERMEDIATE_PERSIST_PERIOD = new Period("PT10M"); - private static final int DEFAULT_MAX_PENDING_PERSISTS = 0; - private static final ShardSpec DEFAULT_SHARD_SPEC = new NumberedShardSpec(0, 1); - private static final IndexSpec DEFAULT_INDEX_SPEC = IndexSpec.DEFAULT; - private static final Boolean DEFAULT_REPORT_PARSE_EXCEPTIONS = Boolean.FALSE; - private static final long DEFAULT_HANDOFF_CONDITION_TIMEOUT = 0; - private static final long DEFAULT_ALERT_TIMEOUT = 0; - - private final AppendableIndexSpec appendableIndexSpec; - private final int maxRowsInMemory; - private final long maxBytesInMemory; - private final boolean skipBytesInMemoryOverheadCheck; - private final DynamicPartitionsSpec partitionsSpec; - private final Period intermediatePersistPeriod; - private final File basePersistDirectory; - private final int maxPendingPersists; - private final ShardSpec shardSpec; - private final IndexSpec indexSpec; - private final IndexSpec indexSpecForIntermediatePersists; - private final boolean reportParseExceptions; - private final long publishAndHandoffTimeout; - private final long alertTimeout; - @Nullable - private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; - - private final boolean logParseExceptions; - private final int maxParseExceptions; - private final int maxSavedParseExceptions; - - private final int numPersistThreads; - - public RealtimeAppenderatorTuningConfig( - @Nullable AppendableIndexSpec appendableIndexSpec, - Integer maxRowsInMemory, - @Nullable Long maxBytesInMemory, - @Nullable Boolean skipBytesInMemoryOverheadCheck, - @Nullable Integer maxRowsPerSegment, - @Nullable Long maxTotalRows, - Period intermediatePersistPeriod, - File basePersistDirectory, - Integer maxPendingPersists, - ShardSpec shardSpec, - IndexSpec indexSpec, - @Nullable IndexSpec indexSpecForIntermediatePersists, - Boolean reportParseExceptions, - Long publishAndHandoffTimeout, - Long alertTimeout, - @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, - @Nullable Boolean logParseExceptions, - @Nullable Integer maxParseExceptions, - @Nullable Integer maxSavedParseExceptions, - @Nullable Integer numPersistThreads - ) - { - this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; - this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY_REALTIME : maxRowsInMemory; - // initializing this to 0, it will be lazily initialized to a value - // @see #getMaxBytesInMemoryOrDefault() - this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; - this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck == null ? - DEFAULT_SKIP_BYTES_IN_MEMORY_OVERHEAD_CHECK : skipBytesInMemoryOverheadCheck; - this.partitionsSpec = new DynamicPartitionsSpec(maxRowsPerSegment, maxTotalRows); - this.intermediatePersistPeriod = intermediatePersistPeriod == null - ? DEFAULT_INTERMEDIATE_PERSIST_PERIOD - : intermediatePersistPeriod; - this.basePersistDirectory = basePersistDirectory; - this.maxPendingPersists = maxPendingPersists == null ? DEFAULT_MAX_PENDING_PERSISTS : maxPendingPersists; - this.shardSpec = shardSpec == null ? DEFAULT_SHARD_SPEC : shardSpec; - this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec; - this.indexSpecForIntermediatePersists = indexSpecForIntermediatePersists == null ? - this.indexSpec : indexSpecForIntermediatePersists; - this.reportParseExceptions = reportParseExceptions == null - ? DEFAULT_REPORT_PARSE_EXCEPTIONS - : reportParseExceptions; - this.publishAndHandoffTimeout = publishAndHandoffTimeout == null - ? DEFAULT_HANDOFF_CONDITION_TIMEOUT - : publishAndHandoffTimeout; - Preconditions.checkArgument(this.publishAndHandoffTimeout >= 0, "publishAndHandoffTimeout must be >= 0"); - - this.alertTimeout = alertTimeout == null ? DEFAULT_ALERT_TIMEOUT : alertTimeout; - Preconditions.checkArgument(this.alertTimeout >= 0, "alertTimeout must be >= 0"); - this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; - - if (this.reportParseExceptions) { - this.maxParseExceptions = 0; - this.maxSavedParseExceptions = maxSavedParseExceptions == null ? 0 : Math.min(1, maxSavedParseExceptions); - } else { - this.maxParseExceptions = maxParseExceptions == null - ? TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS - : maxParseExceptions; - this.maxSavedParseExceptions = maxSavedParseExceptions == null - ? TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS - : maxSavedParseExceptions; - } - this.logParseExceptions = logParseExceptions == null - ? TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS - : logParseExceptions; - this.numPersistThreads = numPersistThreads == null ? - DEFAULT_NUM_PERSIST_THREADS : Math.max(numPersistThreads, DEFAULT_NUM_PERSIST_THREADS); - } - - @JsonCreator - private RealtimeAppenderatorTuningConfig( - @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, - @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, - @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, - @JsonProperty("skipBytesInMemoryOverheadCheck") @Nullable Boolean skipBytesInMemoryOverheadCheck, - @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, - @JsonProperty("maxTotalRows") @Nullable Long maxTotalRows, - @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, - @JsonProperty("maxPendingPersists") Integer maxPendingPersists, - @JsonProperty("shardSpec") ShardSpec shardSpec, - @JsonProperty("indexSpec") IndexSpec indexSpec, - @JsonProperty("indexSpecForIntermediatePersists") @Nullable IndexSpec indexSpecForIntermediatePersists, - @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, - @JsonProperty("publishAndHandoffTimeout") Long publishAndHandoffTimeout, - @JsonProperty("alertTimeout") Long alertTimeout, - @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, - @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, - @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, - @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, - @JsonProperty("numPersistThreads") @Nullable Integer numPersistThreads - ) - { - this( - appendableIndexSpec, - maxRowsInMemory, - maxBytesInMemory, - skipBytesInMemoryOverheadCheck, - maxRowsPerSegment, - maxTotalRows, - intermediatePersistPeriod, - null, - maxPendingPersists, - shardSpec, - indexSpec, - indexSpecForIntermediatePersists, - reportParseExceptions, - publishAndHandoffTimeout, - alertTimeout, - segmentWriteOutMediumFactory, - logParseExceptions, - maxParseExceptions, - maxSavedParseExceptions, - numPersistThreads - ); - } - - @Override - @JsonProperty - public AppendableIndexSpec getAppendableIndexSpec() - { - return appendableIndexSpec; - } - - @Override - @JsonProperty - public int getMaxRowsInMemory() - { - return maxRowsInMemory; - } - - @Override - @JsonProperty - public long getMaxBytesInMemory() - { - return maxBytesInMemory; - } - - @JsonProperty - @Override - public boolean isSkipBytesInMemoryOverheadCheck() - { - return skipBytesInMemoryOverheadCheck; - } - - @Override - @JsonProperty - public Integer getMaxRowsPerSegment() - { - return partitionsSpec.getMaxRowsPerSegment(); - } - - @Override - @JsonProperty - @Nullable - public Long getMaxTotalRows() - { - return partitionsSpec.getMaxTotalRows(); - } - - @Override - public DynamicPartitionsSpec getPartitionsSpec() - { - return partitionsSpec; - } - - @Override - @JsonProperty - public Period getIntermediatePersistPeriod() - { - return intermediatePersistPeriod; - } - - @Override - public File getBasePersistDirectory() - { - return Preconditions.checkNotNull(basePersistDirectory, "basePersistDirectory not set"); - } - - @Override - @JsonProperty - public int getMaxPendingPersists() - { - return maxPendingPersists; - } - - @JsonProperty - public ShardSpec getShardSpec() - { - return shardSpec; - } - - @Override - @JsonProperty - public IndexSpec getIndexSpec() - { - return indexSpec; - } - - @JsonProperty - @Override - public IndexSpec getIndexSpecForIntermediatePersists() - { - return indexSpecForIntermediatePersists; - } - - @Override - @JsonProperty - public boolean isReportParseExceptions() - { - return reportParseExceptions; - } - - @JsonProperty - public long getPublishAndHandoffTimeout() - { - return publishAndHandoffTimeout; - } - - @JsonProperty - public long getAlertTimeout() - { - return alertTimeout; - } - - @Override - @JsonProperty - @Nullable - public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() - { - return segmentWriteOutMediumFactory; - } - - @JsonProperty - public boolean isLogParseExceptions() - { - return logParseExceptions; - } - - @JsonProperty - public int getMaxParseExceptions() - { - return maxParseExceptions; - } - - @JsonProperty - public int getMaxSavedParseExceptions() - { - return maxSavedParseExceptions; - } - - @Override - @JsonProperty - public int getNumPersistThreads() - { - return numPersistThreads; - } - - @Override - public RealtimeAppenderatorTuningConfig withBasePersistDirectory(File dir) - { - return new RealtimeAppenderatorTuningConfig( - appendableIndexSpec, - maxRowsInMemory, - maxBytesInMemory, - skipBytesInMemoryOverheadCheck, - partitionsSpec.getMaxRowsPerSegment(), - partitionsSpec.getMaxTotalRows(), - intermediatePersistPeriod, - dir, - maxPendingPersists, - shardSpec, - indexSpec, - indexSpecForIntermediatePersists, - reportParseExceptions, - publishAndHandoffTimeout, - alertTimeout, - segmentWriteOutMediumFactory, - logParseExceptions, - maxParseExceptions, - maxSavedParseExceptions, - numPersistThreads - ); - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java index 393035b03ef..2fb11cd7a12 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java @@ -28,54 +28,49 @@ import org.apache.druid.java.util.metrics.MonitorUtils; import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; -import org.apache.druid.segment.realtime.FireDepartment; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import javax.annotation.Nullable; import java.util.Map; /** - * Replaces the old RealtimeMetricsMonitor for indexing tasks that use a single FireDepartment, with changes to - * read row ingestion stats from RowIngestionMeters (which supports moving averages) instead of FireDepartmentMetrics. - * See comment on RowIngestionMeters for more information regarding relationship between RowIngestionMeters and - * FireDepartmentMetrics. + * Emits metrics from {@link SegmentGenerationMetrics} and {@link RowIngestionMeters}. */ public class TaskRealtimeMetricsMonitor extends AbstractMonitor { private static final EmittingLogger log = new EmittingLogger(TaskRealtimeMetricsMonitor.class); - private final FireDepartment fireDepartment; + private final SegmentGenerationMetrics segmentGenerationMetrics; private final RowIngestionMeters rowIngestionMeters; private final Map dimensions; @Nullable private final Map metricTags; - private FireDepartmentMetrics previousFireDepartmentMetrics; + private SegmentGenerationMetrics previousSegmentGenerationMetrics; private RowIngestionMetersTotals previousRowIngestionMetersTotals; public TaskRealtimeMetricsMonitor( - FireDepartment fireDepartment, + SegmentGenerationMetrics segmentGenerationMetrics, RowIngestionMeters rowIngestionMeters, Map dimensions, @Nullable Map metricTags ) { - this.fireDepartment = fireDepartment; + this.segmentGenerationMetrics = segmentGenerationMetrics; this.rowIngestionMeters = rowIngestionMeters; this.dimensions = ImmutableMap.copyOf(dimensions); this.metricTags = metricTags; - previousFireDepartmentMetrics = new FireDepartmentMetrics(); + previousSegmentGenerationMetrics = new SegmentGenerationMetrics(); previousRowIngestionMetersTotals = new RowIngestionMetersTotals(0, 0, 0, 0, 0); } @Override public boolean doMonitor(ServiceEmitter emitter) { - FireDepartmentMetrics metrics = fireDepartment.getMetrics().snapshot(); + SegmentGenerationMetrics metrics = segmentGenerationMetrics.snapshot(); RowIngestionMetersTotals rowIngestionMetersTotals = rowIngestionMeters.getTotals(); - final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DATASOURCE, fireDepartment.getDataSchema().getDataSource()); + final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); MonitorUtils.addDimensionsToBuilder(builder, dimensions); final long thrownAway = rowIngestionMetersTotals.getThrownAway() - previousRowIngestionMetersTotals.getThrownAway(); @@ -103,7 +98,7 @@ public class TaskRealtimeMetricsMonitor extends AbstractMonitor emitter.emit(builder.setMetric("ingest/events/processed", rowIngestionMetersTotals.getProcessed() - previousRowIngestionMetersTotals.getProcessed())); - final long dedup = metrics.dedup() - previousFireDepartmentMetrics.dedup(); + final long dedup = metrics.dedup() - previousSegmentGenerationMetrics.dedup(); if (dedup > 0) { log.warn("[%,d] duplicate events!", dedup); } @@ -115,21 +110,21 @@ public class TaskRealtimeMetricsMonitor extends AbstractMonitor ) ); - emitter.emit(builder.setMetric("ingest/rows/output", metrics.rowOutput() - previousFireDepartmentMetrics.rowOutput())); - emitter.emit(builder.setMetric("ingest/persists/count", metrics.numPersists() - previousFireDepartmentMetrics.numPersists())); - emitter.emit(builder.setMetric("ingest/persists/time", metrics.persistTimeMillis() - previousFireDepartmentMetrics.persistTimeMillis())); - emitter.emit(builder.setMetric("ingest/persists/cpu", metrics.persistCpuTime() - previousFireDepartmentMetrics.persistCpuTime())); + emitter.emit(builder.setMetric("ingest/rows/output", metrics.rowOutput() - previousSegmentGenerationMetrics.rowOutput())); + emitter.emit(builder.setMetric("ingest/persists/count", metrics.numPersists() - previousSegmentGenerationMetrics.numPersists())); + emitter.emit(builder.setMetric("ingest/persists/time", metrics.persistTimeMillis() - previousSegmentGenerationMetrics.persistTimeMillis())); + emitter.emit(builder.setMetric("ingest/persists/cpu", metrics.persistCpuTime() - previousSegmentGenerationMetrics.persistCpuTime())); emitter.emit( builder.setMetric( "ingest/persists/backPressure", - metrics.persistBackPressureMillis() - previousFireDepartmentMetrics.persistBackPressureMillis() + metrics.persistBackPressureMillis() - previousSegmentGenerationMetrics.persistBackPressureMillis() ) ); - emitter.emit(builder.setMetric("ingest/persists/failed", metrics.failedPersists() - previousFireDepartmentMetrics.failedPersists())); - emitter.emit(builder.setMetric("ingest/handoff/failed", metrics.failedHandoffs() - previousFireDepartmentMetrics.failedHandoffs())); - emitter.emit(builder.setMetric("ingest/merge/time", metrics.mergeTimeMillis() - previousFireDepartmentMetrics.mergeTimeMillis())); - emitter.emit(builder.setMetric("ingest/merge/cpu", metrics.mergeCpuTime() - previousFireDepartmentMetrics.mergeCpuTime())); - emitter.emit(builder.setMetric("ingest/handoff/count", metrics.handOffCount() - previousFireDepartmentMetrics.handOffCount())); + emitter.emit(builder.setMetric("ingest/persists/failed", metrics.failedPersists() - previousSegmentGenerationMetrics.failedPersists())); + emitter.emit(builder.setMetric("ingest/handoff/failed", metrics.failedHandoffs() - previousSegmentGenerationMetrics.failedHandoffs())); + emitter.emit(builder.setMetric("ingest/merge/time", metrics.mergeTimeMillis() - previousSegmentGenerationMetrics.mergeTimeMillis())); + emitter.emit(builder.setMetric("ingest/merge/cpu", metrics.mergeCpuTime() - previousSegmentGenerationMetrics.mergeCpuTime())); + emitter.emit(builder.setMetric("ingest/handoff/count", metrics.handOffCount() - previousSegmentGenerationMetrics.handOffCount())); emitter.emit(builder.setMetric("ingest/sink/count", metrics.sinkCount())); long messageGap = metrics.messageGap(); @@ -143,7 +138,7 @@ public class TaskRealtimeMetricsMonitor extends AbstractMonitor } previousRowIngestionMetersTotals = rowIngestionMetersTotals; - previousFireDepartmentMetrics = metrics; + previousSegmentGenerationMetrics = metrics; return true; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java deleted file mode 100644 index d97caaec874..00000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ /dev/null @@ -1,853 +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.common.task; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.base.Supplier; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.MoreExecutors; -import org.apache.druid.data.input.Committer; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.discovery.DiscoveryDruidNode; -import org.apache.druid.discovery.LookupNodeService; -import org.apache.druid.discovery.NodeRole; -import org.apache.druid.indexer.IngestionState; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; -import org.apache.druid.indexer.report.IngestionStatsAndErrors; -import org.apache.druid.indexer.report.IngestionStatsAndErrorsTaskReport; -import org.apache.druid.indexer.report.TaskContextReport; -import org.apache.druid.indexer.report.TaskReport; -import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; -import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; -import org.apache.druid.indexing.common.LockGranularity; -import org.apache.druid.indexing.common.TaskLock; -import org.apache.druid.indexing.common.TaskLockType; -import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.SegmentAllocateAction; -import org.apache.druid.indexing.common.actions.SegmentLockAcquireAction; -import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; -import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.actions.TaskLocks; -import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction; -import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.index.RealtimeAppenderatorIngestionSpec; -import org.apache.druid.indexing.common.index.RealtimeAppenderatorTuningConfig; -import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.UOE; -import org.apache.druid.java.util.common.parsers.ParseException; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.query.NoopQueryRunner; -import org.apache.druid.query.Query; -import org.apache.druid.query.QueryRunner; -import org.apache.druid.segment.SegmentUtils; -import org.apache.druid.segment.incremental.ParseExceptionHandler; -import org.apache.druid.segment.incremental.ParseExceptionReport; -import org.apache.druid.segment.incremental.RowIngestionMeters; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; -import org.apache.druid.segment.realtime.FireDepartment; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.segment.realtime.appenderator.Appenderator; -import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; -import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; -import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; -import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; -import org.apache.druid.segment.realtime.firehose.ChatHandler; -import org.apache.druid.segment.realtime.firehose.ClippedFirehoseFactory; -import org.apache.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; -import org.apache.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory; -import org.apache.druid.segment.realtime.plumber.Committers; -import org.apache.druid.server.security.Action; -import org.apache.druid.server.security.AuthorizerMapper; -import org.apache.druid.server.security.ResourceAction; -import org.apache.druid.timeline.partition.NumberedPartialShardSpec; -import org.apache.druid.utils.CloseableUtils; -import org.checkerframework.checker.nullness.qual.MonotonicNonNull; - -import javax.annotation.Nonnull; -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.GET; -import javax.ws.rs.Path; -import javax.ws.rs.Produces; -import javax.ws.rs.core.Context; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; -import java.io.IOException; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Queue; -import java.util.Set; -import java.util.Timer; -import java.util.TimerTask; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - -@Deprecated -public class AppenderatorDriverRealtimeIndexTask extends AbstractTask - implements ChatHandler, PendingSegmentAllocatingTask -{ - public static final String TYPE = "index_realtime_appenderator"; - private static final String CTX_KEY_LOOKUP_TIER = "lookupTier"; - - private static final EmittingLogger log = new EmittingLogger(RealtimeIndexTask.class); - - private static String makeTaskId(RealtimeAppenderatorIngestionSpec spec) - { - return StringUtils.format( - "index_realtime_%s_%d_%s_%s", - spec.getDataSchema().getDataSource(), - spec.getTuningConfig().getShardSpec().getPartitionNum(), - DateTimes.nowUtc(), - RealtimeIndexTask.makeRandomId() - ); - } - - @JsonIgnore - private final RealtimeAppenderatorIngestionSpec spec; - - @JsonIgnore - private final Queue> pendingHandoffs; - - @JsonIgnore - private volatile Appenderator appenderator = null; - - @JsonIgnore - private volatile Firehose firehose = null; - - @JsonIgnore - private volatile FireDepartmentMetrics metrics = null; - - @JsonIgnore - private volatile boolean gracefullyStopped = false; - - @JsonIgnore - private volatile boolean finishingJob = false; - - @JsonIgnore - private volatile Thread runThread = null; - - @JsonIgnore - private final LockGranularity lockGranularity; - - @JsonIgnore - @MonotonicNonNull - private ParseExceptionHandler parseExceptionHandler; - - @JsonIgnore - @MonotonicNonNull - private IngestionState ingestionState; - - @JsonIgnore - @MonotonicNonNull - private AuthorizerMapper authorizerMapper; - - @JsonIgnore - @MonotonicNonNull - private RowIngestionMeters rowIngestionMeters; - - @JsonIgnore - @MonotonicNonNull - private String errorMsg; - - @JsonCreator - public AppenderatorDriverRealtimeIndexTask( - @JsonProperty("id") String id, - @JsonProperty("resource") TaskResource taskResource, - @JsonProperty("spec") RealtimeAppenderatorIngestionSpec spec, - @JsonProperty("context") Map context - ) - { - super( - id == null ? makeTaskId(spec) : id, - StringUtils.format("index_realtime_appenderator_%s", spec.getDataSchema().getDataSource()), - taskResource, - spec.getDataSchema().getDataSource(), - context - ); - this.spec = spec; - this.pendingHandoffs = new ConcurrentLinkedQueue<>(); - - this.ingestionState = IngestionState.NOT_STARTED; - this.lockGranularity = getContextValue(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK) - ? LockGranularity.TIME_CHUNK - : LockGranularity.SEGMENT; - } - - @Override - public int getPriority() - { - return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_REALTIME_TASK_PRIORITY); - } - - @Override - public String getType() - { - return TYPE; - } - - @Override - @JsonIgnore - @Nonnull - public Set getInputSourceResources() throws UOE - { - throw new UOE(StringUtils.format( - "Task type [%s], does not support input source based security", - getType() - )); - } - - @Override - public String getNodeType() - { - return "realtime"; - } - - @Override - public QueryRunner getQueryRunner(Query query) - { - if (appenderator == null) { - // Not yet initialized, no data yet, just return a noop runner. - return new NoopQueryRunner<>(); - } - - return (queryPlus, responseContext) -> queryPlus.run(appenderator, responseContext); - } - - @Override - public boolean supportsQueries() - { - return true; - } - - @Override - public boolean isReady(TaskActionClient taskActionClient) - { - return true; - } - - @Override - public String getTaskAllocatorId() - { - return getGroupId(); - } - - @Override - public TaskStatus runTask(final TaskToolbox toolbox) - { - runThread = Thread.currentThread(); - authorizerMapper = toolbox.getAuthorizerMapper(); - rowIngestionMeters = toolbox.getRowIngestionMetersFactory().createRowIngestionMeters(); - parseExceptionHandler = new ParseExceptionHandler( - rowIngestionMeters, - spec.getTuningConfig().isLogParseExceptions(), - spec.getTuningConfig().getMaxParseExceptions(), - spec.getTuningConfig().getMaxSavedParseExceptions() - ); - - setupTimeoutAlert(); - - DataSchema dataSchema = spec.getDataSchema(); - RealtimeAppenderatorTuningConfig tuningConfig = spec.getTuningConfig() - .withBasePersistDirectory(toolbox.getPersistDir()); - - final FireDepartment fireDepartmentForMetrics = - new FireDepartment(dataSchema, new RealtimeIOConfig(null, null), null); - - final TaskRealtimeMetricsMonitor metricsMonitor = TaskRealtimeMetricsMonitorBuilder.build( - this, - fireDepartmentForMetrics, - rowIngestionMeters - ); - - this.metrics = fireDepartmentForMetrics.getMetrics(); - - final Supplier committerSupplier = Committers.nilSupplier(); - - DiscoveryDruidNode discoveryDruidNode = createDiscoveryDruidNode(toolbox); - - appenderator = newAppenderator(dataSchema, tuningConfig, metrics, toolbox); - final TaskLockType lockType = TaskLocks.determineLockTypeForAppend(getContext()); - StreamAppenderatorDriver driver = newDriver(dataSchema, appenderator, toolbox, metrics, lockType); - - try { - log.debug("Found chat handler of class[%s]", toolbox.getChatHandlerProvider().getClass().getName()); - toolbox.getChatHandlerProvider().register(getId(), this, false); - - if (toolbox.getAppenderatorsManager().shouldTaskMakeNodeAnnouncements()) { - toolbox.getDataSegmentServerAnnouncer().announce(); - toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); - } - - driver.startJob( - segmentId -> { - try { - if (lockGranularity == LockGranularity.SEGMENT) { - return toolbox.getTaskActionClient().submit( - new SegmentLockAcquireAction( - TaskLockType.EXCLUSIVE, - segmentId.getInterval(), - segmentId.getVersion(), - segmentId.getShardSpec().getPartitionNum(), - 1000L - ) - ).isOk(); - } else { - final TaskLock lock = toolbox.getTaskActionClient().submit( - new TimeChunkLockAcquireAction( - TaskLockType.EXCLUSIVE, - segmentId.getInterval(), - 1000L - ) - ); - if (lock == null) { - return false; - } - lock.assertNotRevoked(); - return true; - } - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - ); - - // Set up metrics emission - toolbox.addMonitor(metricsMonitor); - - // Delay firehose connection to avoid claiming input resources while the plumber is starting up. - final FirehoseFactory firehoseFactory = spec.getIOConfig().getFirehoseFactory(); - final boolean firehoseDrainableByClosing = isFirehoseDrainableByClosing(firehoseFactory); - - int sequenceNumber = 0; - String sequenceName = makeSequenceName(getId(), sequenceNumber); - - final TransactionalSegmentPublisher publisher = (mustBeNullOrEmptyOverwriteSegments, segments, commitMetadata, map) -> { - if (mustBeNullOrEmptyOverwriteSegments != null && !mustBeNullOrEmptyOverwriteSegments.isEmpty()) { - throw new ISE( - "Stream ingestion task unexpectedly attempted to overwrite segments: %s", - SegmentUtils.commaSeparatedIdentifiers(mustBeNullOrEmptyOverwriteSegments) - ); - } - final SegmentTransactionalInsertAction action = SegmentTransactionalInsertAction.appendAction( - segments, - null, - null, - null - ); - return toolbox.getTaskActionClient().submit(action); - }; - - // Skip connecting firehose if we've been stopped before we got started. - synchronized (this) { - if (!gracefullyStopped) { - firehose = firehoseFactory.connect( - Preconditions.checkNotNull(spec.getDataSchema().getParser(), "inputRowParser"), - toolbox.getIndexingTmpDir() - ); - } - } - - ingestionState = IngestionState.BUILD_SEGMENTS; - - // Time to read data! - while (!gracefullyStopped && firehoseDrainableByClosing && firehose.hasMore()) { - try { - InputRow inputRow = firehose.nextRow(); - - if (inputRow == null) { - log.debug("Discarded null row, considering thrownAway."); - rowIngestionMeters.incrementThrownAway(); - } else { - AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName, committerSupplier); - - if (addResult.isOk()) { - final boolean isPushRequired = addResult.isPushRequired( - tuningConfig.getPartitionsSpec().getMaxRowsPerSegment(), - tuningConfig.getPartitionsSpec().getMaxTotalRowsOr(DynamicPartitionsSpec.DEFAULT_MAX_TOTAL_ROWS) - ); - if (isPushRequired) { - publishSegments(driver, publisher, committerSupplier, sequenceName); - sequenceNumber++; - sequenceName = makeSequenceName(getId(), sequenceNumber); - } - } else { - // Failure to allocate segment puts determinism at risk, bail out to be safe. - // May want configurable behavior here at some point. - // If we allow continuing, then consider blacklisting the interval for a while to avoid constant checks. - throw new ISE("Could not allocate segment for row with timestamp[%s]", inputRow.getTimestamp()); - } - } - } - catch (ParseException e) { - handleParseException(e); - } - } - - ingestionState = IngestionState.COMPLETED; - - if (!gracefullyStopped) { - synchronized (this) { - if (gracefullyStopped) { - // Someone called stopGracefully after we checked the flag. That's okay, just stop now. - log.info("Gracefully stopping."); - } else { - finishingJob = true; - } - } - - if (finishingJob) { - log.info("Finishing job..."); - // Publish any remaining segments - publishSegments(driver, publisher, committerSupplier, sequenceName); - - waitForSegmentPublishAndHandoff(tuningConfig.getPublishAndHandoffTimeout()); - } - } else if (firehose != null) { - log.info("Task was gracefully stopped, will persist data before exiting"); - - persistAndWait(driver, committerSupplier.get()); - } - } - catch (Throwable e) { - log.makeAlert(e, "Exception aborted realtime processing[%s]", dataSchema.getDataSource()) - .emit(); - errorMsg = Throwables.getStackTraceAsString(e); - toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); - return TaskStatus.failure( - getId(), - errorMsg - ); - } - finally { - toolbox.getChatHandlerProvider().unregister(getId()); - - CloseableUtils.closeAndSuppressExceptions(firehose, e -> log.warn("Failed to close Firehose")); - appenderator.close(); - CloseableUtils.closeAndSuppressExceptions(driver, e -> log.warn("Failed to close AppenderatorDriver")); - - toolbox.removeMonitor(metricsMonitor); - - if (toolbox.getAppenderatorsManager().shouldTaskMakeNodeAnnouncements()) { - toolbox.getDataSegmentServerAnnouncer().unannounce(); - toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); - } - } - - log.info("Job done!"); - toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); - return TaskStatus.success(getId()); - } - - @Override - public boolean canRestore() - { - return true; - } - - @Override - public void stopGracefully(TaskConfig taskConfig) - { - if (taskConfig.isRestoreTasksOnRestart()) { - try { - synchronized (this) { - if (!gracefullyStopped) { - gracefullyStopped = true; - if (firehose == null) { - log.info("stopGracefully: Firehose not started yet, so nothing to stop."); - } else if (finishingJob) { - log.info("stopGracefully: Interrupting finishJob."); - runThread.interrupt(); - } else if (isFirehoseDrainableByClosing(spec.getIOConfig().getFirehoseFactory())) { - log.info("stopGracefully: Draining firehose."); - firehose.close(); - } else { - log.info("stopGracefully: Cannot drain firehose by closing, interrupting run thread."); - runThread.interrupt(); - } - } - } - } - catch (Exception e) { - throw new RuntimeException(e); - } - } else { - synchronized (this) { - if (!gracefullyStopped) { - // If task restore is not enabled, just interrupt immediately. - gracefullyStopped = true; - runThread.interrupt(); - } - } - } - } - - /** - * Public for tests. - */ - @JsonIgnore - @VisibleForTesting - public Firehose getFirehose() - { - return firehose; - } - - /** - * Public for tests. - */ - @JsonIgnore - @VisibleForTesting - public FireDepartmentMetrics getMetrics() - { - return metrics; - } - - @JsonIgnore - @VisibleForTesting - public RowIngestionMeters getRowIngestionMeters() - { - return rowIngestionMeters; - } - - @JsonProperty("spec") - public RealtimeAppenderatorIngestionSpec getSpec() - { - return spec; - } - - - @GET - @Path("/rowStats") - @Produces(MediaType.APPLICATION_JSON) - public Response getRowStats( - @Context final HttpServletRequest req - ) - { - IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - Map returnMap = new HashMap<>(); - Map totalsMap = new HashMap<>(); - Map averagesMap = new HashMap<>(); - - totalsMap.put( - RowIngestionMeters.BUILD_SEGMENTS, - rowIngestionMeters.getTotals() - ); - averagesMap.put( - RowIngestionMeters.BUILD_SEGMENTS, - rowIngestionMeters.getMovingAverages() - ); - - returnMap.put("movingAverages", averagesMap); - returnMap.put("totals", totalsMap); - return Response.ok(returnMap).build(); - } - - @GET - @Path("/unparseableEvents") - @Produces(MediaType.APPLICATION_JSON) - public Response getUnparseableEvents( - @Context final HttpServletRequest req - ) - { - IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - List events = IndexTaskUtils.getReportListFromSavedParseExceptions( - parseExceptionHandler.getSavedParseExceptionReports() - ); - return Response.ok(events).build(); - } - - /** - * Is a firehose from this factory drainable by closing it? If so, we should drain on stopGracefully rather than - * abruptly stopping. - *

- * This is a hack to get around the fact that the Firehose and FirehoseFactory interfaces do not help us do this. - *

- * Protected for tests. - */ - protected boolean isFirehoseDrainableByClosing(FirehoseFactory firehoseFactory) - { - return firehoseFactory instanceof EventReceiverFirehoseFactory - || (firehoseFactory instanceof TimedShutoffFirehoseFactory - && isFirehoseDrainableByClosing(((TimedShutoffFirehoseFactory) firehoseFactory).getDelegateFactory())) - || (firehoseFactory instanceof ClippedFirehoseFactory - && isFirehoseDrainableByClosing(((ClippedFirehoseFactory) firehoseFactory).getDelegate())); - } - - /** - * Return a map of reports for the task. - * - * A successfull task should always have a null errorMsg. A falied task should always have a non-null - * errorMsg. - * - * @return Map of reports for the task. - */ - private TaskReport.ReportMap getTaskCompletionReports() - { - return TaskReport.buildTaskReports( - new IngestionStatsAndErrorsTaskReport( - getId(), - new IngestionStatsAndErrors( - ingestionState, - getTaskCompletionUnparseableEvents(), - getTaskCompletionRowStats(), - errorMsg, - errorMsg == null, - 0L, - Collections.emptyMap(), - null, - null - ) - ), - new TaskContextReport(getId(), getContext()) - ); - } - - private Map getTaskCompletionUnparseableEvents() - { - Map unparseableEventsMap = new HashMap<>(); - List buildSegmentsParseExceptionMessages = IndexTaskUtils.getReportListFromSavedParseExceptions( - parseExceptionHandler.getSavedParseExceptionReports() - ); - if (buildSegmentsParseExceptionMessages != null) { - unparseableEventsMap.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegmentsParseExceptionMessages); - } - return unparseableEventsMap; - } - - private Map getTaskCompletionRowStats() - { - Map metricsMap = new HashMap<>(); - metricsMap.put( - RowIngestionMeters.BUILD_SEGMENTS, - rowIngestionMeters.getTotals() - ); - return metricsMap; - } - - private void handleParseException(ParseException pe) - { - parseExceptionHandler.handle(pe); - - if (rowIngestionMeters.getUnparseable() + rowIngestionMeters.getProcessedWithError() - > spec.getTuningConfig().getMaxParseExceptions()) { - log.error("Max parse exceptions exceeded, terminating task..."); - throw new RuntimeException("Max parse exceptions exceeded, terminating task..."); - } - } - - private void setupTimeoutAlert() - { - if (spec.getTuningConfig().getAlertTimeout() > 0) { - Timer timer = new Timer("RealtimeIndexTask-Timer", true); - timer.schedule( - new TimerTask() - { - @Override - public void run() - { - log.makeAlert( - "RealtimeIndexTask for dataSource [%s] hasn't finished in configured time [%d] ms.", - spec.getDataSchema().getDataSource(), - spec.getTuningConfig().getAlertTimeout() - ).emit(); - } - }, - spec.getTuningConfig().getAlertTimeout() - ); - } - } - - private void publishSegments( - StreamAppenderatorDriver driver, - TransactionalSegmentPublisher publisher, - Supplier committerSupplier, - String sequenceName - ) - { - final ListenableFuture publishFuture = driver.publish( - publisher, - committerSupplier.get(), - Collections.singletonList(sequenceName) - ); - pendingHandoffs.add(Futures.transformAsync( - publishFuture, - driver::registerHandoff, - MoreExecutors.directExecutor() - )); - } - - private void waitForSegmentPublishAndHandoff(long timeout) throws InterruptedException, ExecutionException, - TimeoutException - { - if (!pendingHandoffs.isEmpty()) { - ListenableFuture allHandoffs = Futures.allAsList(pendingHandoffs); - log.info("Waiting for handoffs"); - - - if (timeout > 0) { - allHandoffs.get(timeout, TimeUnit.MILLISECONDS); - } else { - allHandoffs.get(); - } - } - } - - private void persistAndWait(StreamAppenderatorDriver driver, Committer committer) - { - try { - final CountDownLatch persistLatch = new CountDownLatch(1); - driver.persist( - new Committer() - { - @Override - public Object getMetadata() - { - return committer.getMetadata(); - } - - @Override - public void run() - { - try { - committer.run(); - } - finally { - persistLatch.countDown(); - } - } - } - ); - persistLatch.await(); - } - catch (InterruptedException e) { - log.debug(e, "Interrupted while finishing the job"); - } - catch (Exception e) { - log.makeAlert(e, "Failed to finish realtime task").emit(); - throw e; - } - } - - private DiscoveryDruidNode createDiscoveryDruidNode(TaskToolbox toolbox) - { - LookupNodeService lookupNodeService = getContextValue(CTX_KEY_LOOKUP_TIER) == null ? - toolbox.getLookupNodeService() : - new LookupNodeService(getContextValue(CTX_KEY_LOOKUP_TIER)); - return new DiscoveryDruidNode( - toolbox.getDruidNode(), - NodeRole.PEON, - ImmutableMap.of( - toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), - lookupNodeService.getName(), lookupNodeService - ) - ); - } - - private Appenderator newAppenderator( - final DataSchema dataSchema, - final RealtimeAppenderatorTuningConfig tuningConfig, - final FireDepartmentMetrics metrics, - final TaskToolbox toolbox - ) - { - return toolbox.getAppenderatorsManager().createRealtimeAppenderatorForTask( - null, - getId(), - dataSchema, - tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), - metrics, - toolbox.getSegmentPusher(), - toolbox.getJsonMapper(), - toolbox.getIndexIO(), - toolbox.getIndexMergerV9(), - toolbox.getQueryRunnerFactoryConglomerate(), - toolbox.getSegmentAnnouncer(), - toolbox.getEmitter(), - toolbox.getQueryProcessingPool(), - toolbox.getJoinableFactory(), - toolbox.getCache(), - toolbox.getCacheConfig(), - toolbox.getCachePopulatorStats(), - rowIngestionMeters, - parseExceptionHandler, - isUseMaxMemoryEstimates(), - toolbox.getCentralizedTableSchemaConfig() - ); - } - - private static StreamAppenderatorDriver newDriver( - final DataSchema dataSchema, - final Appenderator appenderator, - final TaskToolbox toolbox, - final FireDepartmentMetrics metrics, - final TaskLockType lockType - ) - { - return new StreamAppenderatorDriver( - appenderator, - new ActionBasedSegmentAllocator( - toolbox.getTaskActionClient(), - dataSchema, - (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction( - schema.getDataSource(), - row.getTimestamp(), - schema.getGranularitySpec().getQueryGranularity(), - schema.getGranularitySpec().getSegmentGranularity(), - sequenceName, - previousSegmentId, - skipSegmentLineageCheck, - NumberedPartialShardSpec.instance(), - LockGranularity.TIME_CHUNK, - lockType - ) - ), - toolbox.getSegmentHandoffNotifierFactory(), - new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), - toolbox.getDataSegmentKiller(), - toolbox.getJsonMapper(), - metrics - ); - } - - private static String makeSequenceName(String taskId, int sequenceNumber) - { - return taskId + "_" + sequenceNumber; - } -} - diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java index e47bc0bc1c8..3a8f70d6746 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java @@ -27,7 +27,7 @@ import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; @@ -39,7 +39,7 @@ public final class BatchAppenderators public static Appenderator newAppenderator( String taskId, AppenderatorsManager appenderatorsManager, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, TaskToolbox toolbox, DataSchema dataSchema, AppenderatorConfig appenderatorConfig, @@ -65,7 +65,7 @@ public final class BatchAppenderators public static Appenderator newAppenderator( String taskId, AppenderatorsManager appenderatorsManager, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, TaskToolbox toolbox, DataSchema dataSchema, AppenderatorConfig appenderatorConfig, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 14e37cc81c5..cc253f46a52 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -84,12 +84,10 @@ import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.BatchIOConfig; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.IngestionSpec; -import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.realtime.FireDepartment; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver; @@ -832,15 +830,9 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler, Pe final PartitionAnalysis partitionAnalysis ) throws IOException, InterruptedException { - final FireDepartment fireDepartmentForMetrics = - new FireDepartment(dataSchema, new RealtimeIOConfig(null, null), null); - FireDepartmentMetrics buildSegmentsFireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); - - final TaskRealtimeMetricsMonitor metricsMonitor = TaskRealtimeMetricsMonitorBuilder.build( - this, - fireDepartmentForMetrics, - buildSegmentsMeters - ); + final SegmentGenerationMetrics buildSegmentsSegmentGenerationMetrics = new SegmentGenerationMetrics(); + final TaskRealtimeMetricsMonitor metricsMonitor = + TaskRealtimeMetricsMonitorBuilder.build(this, buildSegmentsSegmentGenerationMetrics, buildSegmentsMeters); toolbox.addMonitor(metricsMonitor); final PartitionsSpec partitionsSpec = partitionAnalysis.getPartitionsSpec(); @@ -894,7 +886,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler, Pe final Appenderator appenderator = BatchAppenderators.newAppenderator( effectiveId, toolbox.getAppenderatorsManager(), - buildSegmentsFireDepartmentMetrics, + buildSegmentsSegmentGenerationMetrics, toolbox, dataSchema, tuningConfig, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java deleted file mode 100644 index 443b72ec21a..00000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java +++ /dev/null @@ -1,643 +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.common.task; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import com.google.common.base.Supplier; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.apache.druid.data.input.Committer; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.discovery.DiscoveryDruidNode; -import org.apache.druid.discovery.LookupNodeService; -import org.apache.druid.discovery.NodeRole; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.common.TaskLock; -import org.apache.druid.indexing.common.TaskLockType; -import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.LockReleaseAction; -import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction; -import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.UOE; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.query.NoopQueryRunner; -import org.apache.druid.query.Query; -import org.apache.druid.query.QueryRunner; -import org.apache.druid.query.QueryRunnerFactoryConglomerate; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.realtime.FireDepartment; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.segment.realtime.RealtimeMetricsMonitor; -import org.apache.druid.segment.realtime.SegmentPublisher; -import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; -import org.apache.druid.segment.realtime.firehose.ClippedFirehoseFactory; -import org.apache.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; -import org.apache.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory; -import org.apache.druid.segment.realtime.plumber.Committers; -import org.apache.druid.segment.realtime.plumber.Plumber; -import org.apache.druid.segment.realtime.plumber.PlumberSchool; -import org.apache.druid.segment.realtime.plumber.Plumbers; -import org.apache.druid.segment.realtime.plumber.RealtimePlumberSchool; -import org.apache.druid.segment.realtime.plumber.VersioningPolicy; -import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.apache.druid.server.security.ResourceAction; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.utils.CloseableUtils; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -import javax.annotation.Nonnull; -import java.io.IOException; -import java.util.Map; -import java.util.Set; -import java.util.Timer; -import java.util.TimerTask; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ThreadLocalRandom; - -@Deprecated -public class RealtimeIndexTask extends AbstractTask -{ - public static final String TYPE = "index_realtime"; - public static final String CTX_KEY_LOOKUP_TIER = "lookupTier"; - - private static final EmittingLogger log = new EmittingLogger(RealtimeIndexTask.class); - - private static final int TASK_ID_BITS_PER_SYMBOL = 4; - private static final int TASK_ID_SYMBOL_MASK = (1 << TASK_ID_BITS_PER_SYMBOL) - 1; - private static final int TASK_ID_LENGTH = Integer.SIZE / TASK_ID_BITS_PER_SYMBOL; - - public static String makeRandomId() - { - final StringBuilder suffix = new StringBuilder(TASK_ID_LENGTH); - int randomBits = ThreadLocalRandom.current().nextInt(); - for (int i = 0; i < TASK_ID_LENGTH; i++) { - suffix.append((char) ('a' + ((randomBits >>> (i * TASK_ID_BITS_PER_SYMBOL)) & TASK_ID_SYMBOL_MASK))); - } - return suffix.toString(); - } - - private static String makeTaskId(FireDepartment fireDepartment) - { - return makeTaskId( - fireDepartment.getDataSchema().getDataSource(), - fireDepartment.getTuningConfig().getShardSpec().getPartitionNum(), - DateTimes.nowUtc(), - makeRandomId() - ); - } - - static String makeTaskId(String dataSource, int partitionNumber, DateTime timestamp, String suffix) - { - return StringUtils.format( - "index_realtime_%s_%d_%s_%s", - dataSource, - partitionNumber, - timestamp, - suffix - ); - } - - private static String makeDatasource(FireDepartment fireDepartment) - { - return fireDepartment.getDataSchema().getDataSource(); - } - - @JsonIgnore - private final FireDepartment spec; - - @JsonIgnore - private volatile Plumber plumber = null; - - @JsonIgnore - private volatile Firehose firehose = null; - - @JsonIgnore - private volatile FireDepartmentMetrics metrics = null; - - @JsonIgnore - private volatile boolean gracefullyStopped = false; - - @JsonIgnore - private volatile boolean finishingJob = false; - - @JsonIgnore - private volatile Thread runThread = null; - - @JsonIgnore - private volatile QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate = null; - - @JsonCreator - public RealtimeIndexTask( - @JsonProperty("id") String id, - @JsonProperty("resource") TaskResource taskResource, - @JsonProperty("spec") FireDepartment fireDepartment, - @JsonProperty("context") Map context - ) - { - super( - id == null ? makeTaskId(fireDepartment) : id, - StringUtils.format("index_realtime_%s", makeDatasource(fireDepartment)), - taskResource, - makeDatasource(fireDepartment), - context - ); - this.spec = fireDepartment; - } - - @Override - public int getPriority() - { - return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_REALTIME_TASK_PRIORITY); - } - - @Override - public String getType() - { - return TYPE; - } - - @Override - @JsonIgnore - @Nonnull - public Set getInputSourceResources() throws UOE - { - throw new UOE(StringUtils.format( - "Task type [%s], does not support input source based security", - getType() - )); - } - - @Override - public String getNodeType() - { - return "realtime"; - } - - @Override - public QueryRunner getQueryRunner(Query query) - { - if (plumber == null) { - // Not yet initialized, no data yet, just return a noop runner. - return new NoopQueryRunner<>(); - } - - return plumber.getQueryRunner(query); - } - - @Override - public boolean supportsQueries() - { - return true; - } - - @Override - public boolean isReady(TaskActionClient taskActionClient) - { - return true; - } - - @Override - public TaskStatus runTask(final TaskToolbox toolbox) throws Exception - { - runThread = Thread.currentThread(); - - if (this.plumber != null) { - throw new IllegalStateException("Plumber must be null"); - } - - setupTimeoutAlert(); - - boolean normalExit = true; - - // It would be nice to get the PlumberSchool in the constructor. Although that will need jackson injectables for - // stuff like the ServerView, which seems kind of odd? Perhaps revisit this when Guice has been introduced. - - final SegmentPublisher segmentPublisher = new TaskActionSegmentPublisher(toolbox); - - // NOTE: We talk to the coordinator in various places in the plumber and we could be more robust to issues - // with the coordinator. Right now, we'll block/throw in whatever thread triggered the coordinator behavior, - // which will typically be either the main data processing loop or the persist thread. - - // Wrap default DataSegmentAnnouncer such that we unlock intervals as we unannounce segments - final long lockTimeoutMs = getContextValue(Tasks.LOCK_TIMEOUT_KEY, Tasks.DEFAULT_LOCK_TIMEOUT_MILLIS); - // Note: if lockTimeoutMs is larger than ServerConfig.maxIdleTime, http timeout error can occur while waiting for a - // lock to be acquired. - final DataSegmentAnnouncer lockingSegmentAnnouncer = new DataSegmentAnnouncer() - { - @Override - public void announceSegment(final DataSegment segment) throws IOException - { - // Side effect: Calling announceSegment causes a lock to be acquired - final TaskLock lock = Preconditions.checkNotNull( - toolbox.getTaskActionClient().submit( - new TimeChunkLockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) - ), - "Cannot acquire a lock for interval[%s]", - segment.getInterval() - ); - lock.assertNotRevoked(); - toolbox.getSegmentAnnouncer().announceSegment(segment); - } - - @Override - public void unannounceSegment(final DataSegment segment) throws IOException - { - try { - toolbox.getSegmentAnnouncer().unannounceSegment(segment); - } - finally { - toolbox.getTaskActionClient().submit(new LockReleaseAction(segment.getInterval())); - } - } - - @Override - public void announceSegments(Iterable segments) throws IOException - { - // Side effect: Calling announceSegments causes locks to be acquired - for (DataSegment segment : segments) { - final TaskLock lock = Preconditions.checkNotNull( - toolbox.getTaskActionClient().submit( - new TimeChunkLockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) - ), - "Cannot acquire a lock for interval[%s]", - segment.getInterval() - ); - lock.assertNotRevoked(); - } - toolbox.getSegmentAnnouncer().announceSegments(segments); - } - - @Override - public void unannounceSegments(Iterable segments) throws IOException - { - try { - toolbox.getSegmentAnnouncer().unannounceSegments(segments); - } - finally { - for (DataSegment segment : segments) { - toolbox.getTaskActionClient().submit(new LockReleaseAction(segment.getInterval())); - } - } - } - - @Override - public void announceSegmentSchemas(String taskId, SegmentSchemas sinksSchema, SegmentSchemas sinksSchemaChange) - { - } - - @Override - public void removeSegmentSchemasForTask(String taskId) - { - } - }; - - // NOTE: getVersion will block if there is lock contention, which will block plumber.getSink - // NOTE: (and thus the firehose) - - // Shouldn't usually happen, since we don't expect people to submit tasks that intersect with the - // realtime window, but if they do it can be problematic. If we decide to care, we can use more threads in - // the plumber such that waiting for the coordinator doesn't block data processing. - final VersioningPolicy versioningPolicy = new VersioningPolicy() - { - @Override - public String getVersion(final Interval interval) - { - try { - // Side effect: Calling getVersion causes a lock to be acquired - final TimeChunkLockAcquireAction action = new TimeChunkLockAcquireAction( - TaskLockType.EXCLUSIVE, - interval, - lockTimeoutMs - ); - final TaskLock lock = Preconditions.checkNotNull( - toolbox.getTaskActionClient().submit(action), - "Cannot acquire a lock for interval[%s]", - interval - ); - lock.assertNotRevoked(); - return lock.getVersion(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - }; - - DataSchema dataSchema = spec.getDataSchema(); - RealtimeIOConfig realtimeIOConfig = spec.getIOConfig(); - RealtimeTuningConfig tuningConfig = spec.getTuningConfig() - .withBasePersistDirectory(toolbox.getPersistDir()) - .withVersioningPolicy(versioningPolicy); - - final FireDepartment fireDepartment = new FireDepartment( - dataSchema, - realtimeIOConfig, - tuningConfig - ); - this.metrics = fireDepartment.getMetrics(); - final RealtimeMetricsMonitor metricsMonitor = TaskRealtimeMetricsMonitorBuilder.build(this, fireDepartment); - - this.queryRunnerFactoryConglomerate = toolbox.getQueryRunnerFactoryConglomerate(); - - // NOTE: This pusher selects path based purely on global configuration and the DataSegment, which means - // NOTE: that redundant realtime tasks will upload to the same location. This can cause index.zip - // NOTE: (partitionNum_index.zip for HDFS data storage) to mismatch, or it can cause historical nodes to load - // NOTE: different instances of the "same" segment. - final PlumberSchool plumberSchool = new RealtimePlumberSchool( - toolbox.getEmitter(), - toolbox.getQueryRunnerFactoryConglomerate(), - toolbox.getSegmentPusher(), - lockingSegmentAnnouncer, - segmentPublisher, - toolbox.getSegmentHandoffNotifierFactory(), - toolbox.getQueryProcessingPool(), - toolbox.getJoinableFactory(), - toolbox.getIndexMergerV9(), - toolbox.getIndexIO(), - toolbox.getCache(), - toolbox.getCacheConfig(), - toolbox.getCachePopulatorStats(), - toolbox.getJsonMapper() - ); - - this.plumber = plumberSchool.findPlumber(dataSchema, tuningConfig, metrics); - - final Supplier committerSupplier = Committers.nilSupplier(); - - LookupNodeService lookupNodeService = getContextValue(CTX_KEY_LOOKUP_TIER) == null ? - toolbox.getLookupNodeService() : - new LookupNodeService((String) getContextValue(CTX_KEY_LOOKUP_TIER)); - DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( - toolbox.getDruidNode(), - NodeRole.PEON, - ImmutableMap.of( - toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), - lookupNodeService.getName(), lookupNodeService - ) - ); - - try { - toolbox.getDataSegmentServerAnnouncer().announce(); - toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); - - - plumber.startJob(); - - // Set up metrics emission - toolbox.addMonitor(metricsMonitor); - - // Delay firehose connection to avoid claiming input resources while the plumber is starting up. - final FirehoseFactory firehoseFactory = spec.getIOConfig().getFirehoseFactory(); - final boolean firehoseDrainableByClosing = isFirehoseDrainableByClosing(firehoseFactory); - - // Skip connecting firehose if we've been stopped before we got started. - synchronized (this) { - if (!gracefullyStopped) { - firehose = firehoseFactory.connect( - Preconditions.checkNotNull(spec.getDataSchema().getParser(), "inputRowParser"), - toolbox.getIndexingTmpDir() - ); - } - } - - // Time to read data! - while (firehose != null && (!gracefullyStopped || firehoseDrainableByClosing) && firehose.hasMore()) { - Plumbers.addNextRow( - committerSupplier, - firehose, - plumber, - tuningConfig.isReportParseExceptions(), - metrics - ); - } - } - catch (Throwable e) { - normalExit = false; - log.makeAlert(e, "Exception aborted realtime processing[%s]", dataSchema.getDataSource()) - .emit(); - throw e; - } - finally { - if (normalExit) { - try { - // Persist if we had actually started. - if (firehose != null) { - log.info("Persisting remaining data."); - - final Committer committer = committerSupplier.get(); - final CountDownLatch persistLatch = new CountDownLatch(1); - plumber.persist( - new Committer() - { - @Override - public Object getMetadata() - { - return committer.getMetadata(); - } - - @Override - public void run() - { - try { - committer.run(); - } - finally { - persistLatch.countDown(); - } - } - } - ); - persistLatch.await(); - } - - if (gracefullyStopped) { - log.info("Gracefully stopping."); - } else { - log.info("Finishing the job."); - synchronized (this) { - if (gracefullyStopped) { - // Someone called stopGracefully after we checked the flag. That's okay, just stop now. - log.info("Gracefully stopping."); - } else { - finishingJob = true; - } - } - - if (finishingJob) { - plumber.finishJob(); - } - } - } - catch (InterruptedException e) { - log.debug(e, "Interrupted while finishing the job"); - } - catch (Exception e) { - log.makeAlert(e, "Failed to finish realtime task").emit(); - throw e; - } - finally { - if (firehose != null) { - CloseableUtils.closeAndSuppressExceptions(firehose, e -> log.warn("Failed to close Firehose")); - } - toolbox.removeMonitor(metricsMonitor); - } - } - - toolbox.getDataSegmentServerAnnouncer().unannounce(); - toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); - } - - log.info("Job done!"); - return TaskStatus.success(getId()); - } - - @Override - public boolean canRestore() - { - return true; - } - - @Override - public void stopGracefully(TaskConfig taskConfig) - { - if (taskConfig.isRestoreTasksOnRestart()) { - try { - synchronized (this) { - if (!gracefullyStopped) { - gracefullyStopped = true; - if (firehose == null) { - log.info("stopGracefully: Firehose not started yet, so nothing to stop."); - } else if (finishingJob) { - log.info("stopGracefully: Interrupting finishJob."); - runThread.interrupt(); - } else if (isFirehoseDrainableByClosing(spec.getIOConfig().getFirehoseFactory())) { - log.info("stopGracefully: Draining firehose."); - firehose.close(); - } else { - log.info("stopGracefully: Cannot drain firehose by closing, interrupting run thread."); - runThread.interrupt(); - } - } - } - } - catch (Exception e) { - throw new RuntimeException(e); - } - } else { - synchronized (this) { - if (!gracefullyStopped) { - // If task restore is not enabled, just interrupt immediately. - gracefullyStopped = true; - runThread.interrupt(); - } - } - } - } - - /** - * Public for tests. - */ - @JsonIgnore - public Firehose getFirehose() - { - return firehose; - } - - /** - * Public for tests. - */ - @JsonIgnore - public FireDepartmentMetrics getMetrics() - { - return metrics; - } - - @JsonProperty("spec") - public FireDepartment getRealtimeIngestionSchema() - { - return spec; - } - - /** - * Is a firehose from this factory drainable by closing it? If so, we should drain on stopGracefully rather than - * abruptly stopping. - * - * This is a hack to get around the fact that the Firehose and FirehoseFactory interfaces do not help us do this. - * - * Protected for tests. - */ - protected boolean isFirehoseDrainableByClosing(FirehoseFactory firehoseFactory) - { - return firehoseFactory instanceof EventReceiverFirehoseFactory - || (firehoseFactory instanceof TimedShutoffFirehoseFactory - && isFirehoseDrainableByClosing(((TimedShutoffFirehoseFactory) firehoseFactory).getDelegateFactory())) - || (firehoseFactory instanceof ClippedFirehoseFactory - && isFirehoseDrainableByClosing(((ClippedFirehoseFactory) firehoseFactory).getDelegate())); - } - - public static class TaskActionSegmentPublisher implements SegmentPublisher - { - final TaskToolbox taskToolbox; - - public TaskActionSegmentPublisher(TaskToolbox taskToolbox) - { - this.taskToolbox = taskToolbox; - } - - @Override - public void publishSegment(DataSegment segment) throws IOException - { - taskToolbox.publishSegments(ImmutableList.of(segment)); - } - } - - private void setupTimeoutAlert() - { - if (spec.getTuningConfig().getAlertTimeout() > 0) { - Timer timer = new Timer("RealtimeIndexTask-Timer", true); - timer.schedule( - new TimerTask() - { - @Override - public void run() - { - log.makeAlert( - "RealtimeIndexTask for dataSource [%s] hasn't finished in configured time [%d] ms.", - spec.getDataSchema().getDataSource(), - spec.getTuningConfig().getAlertTimeout() - ).emit(); - } - }, - spec.getTuningConfig().getAlertTimeout() - ); - } - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index 18d2ac7d696..4fb4bb7852c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -81,8 +81,6 @@ import java.util.Set; @Type(name = PartialDimensionDistributionTask.TYPE, value = PartialDimensionDistributionTask.class), @Type(name = PartialGenericSegmentMergeTask.TYPE, value = PartialGenericSegmentMergeTask.class), @Type(name = HadoopIndexTask.TYPE, value = HadoopIndexTask.class), - @Type(name = RealtimeIndexTask.TYPE, value = RealtimeIndexTask.class), - @Type(name = AppenderatorDriverRealtimeIndexTask.TYPE, value = AppenderatorDriverRealtimeIndexTask.class), @Type(name = NoopTask.TYPE, value = NoopTask.class), @Type(name = CompactionTask.TYPE, value = CompactionTask.class) }) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java index d051038634c..c53557c6655 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java @@ -46,9 +46,7 @@ import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; -import org.apache.druid.segment.realtime.FireDepartment; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; @@ -171,19 +169,10 @@ abstract class PartialSegmentGenerateTask e ) throws IOException, InterruptedException, ExecutionException, TimeoutException { final DataSchema dataSchema = ingestionSchema.getDataSchema(); - final FireDepartment fireDepartmentForMetrics = new FireDepartment( - dataSchema, - new RealtimeIOConfig(null, null), - null - ); - final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); + final SegmentGenerationMetrics segmentGenerationMetrics = new SegmentGenerationMetrics(); buildSegmentsMeters = toolbox.getRowIngestionMetersFactory().createRowIngestionMeters(); - - TaskRealtimeMetricsMonitor metricsMonitor = TaskRealtimeMetricsMonitorBuilder.build( - this, - fireDepartmentForMetrics, - buildSegmentsMeters - ); + final TaskRealtimeMetricsMonitor metricsMonitor = + TaskRealtimeMetricsMonitorBuilder.build(this, segmentGenerationMetrics, buildSegmentsMeters); toolbox.addMonitor(metricsMonitor); final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); @@ -206,7 +195,7 @@ abstract class PartialSegmentGenerateTask e final Appenderator appenderator = BatchAppenderators.newAppenderator( getId(), toolbox.getAppenderatorsManager(), - fireDepartmentMetrics, + segmentGenerationMetrics, toolbox, dataSchema, tuningConfig, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index b8027fcc5ea..cd05e50ff1c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -57,12 +57,10 @@ import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.FireDepartment; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver; @@ -368,15 +366,9 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand { final DataSchema dataSchema = ingestionSchema.getDataSchema(); final GranularitySpec granularitySpec = dataSchema.getGranularitySpec(); - final FireDepartment fireDepartmentForMetrics = - new FireDepartment(dataSchema, new RealtimeIOConfig(null, null), null); - final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); - - TaskRealtimeMetricsMonitor metricsMonitor = TaskRealtimeMetricsMonitorBuilder.build( - this, - fireDepartmentForMetrics, - rowIngestionMeters - ); + final SegmentGenerationMetrics segmentGenerationMetrics = new SegmentGenerationMetrics(); + final TaskRealtimeMetricsMonitor metricsMonitor = + TaskRealtimeMetricsMonitorBuilder.build(this, segmentGenerationMetrics, rowIngestionMeters); toolbox.addMonitor(metricsMonitor); final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); @@ -409,7 +401,7 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand final Appenderator appenderator = BatchAppenderators.newAppenderator( getId(), toolbox.getAppenderatorsManager(), - fireDepartmentMetrics, + segmentGenerationMetrics, toolbox, dataSchema, tuningConfig, @@ -461,8 +453,6 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand } else { throw new ISE("Failed to add a row with timestamp[%s]", inputRow.getTimestamp()); } - - fireDepartmentMetrics.incrementProcessed(); } final SegmentsAndCommitMetadata pushed = driver.pushAllAndClear(pushTimeout); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 2fab1ed9bd1..1267bb645a6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -38,7 +38,6 @@ import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.incremental.ParseExceptionReport; import javax.annotation.Nullable; - import java.util.List; import java.util.Map; import java.util.Set; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 0ec9a67e8c1..f8e78bb711d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -49,7 +49,7 @@ import org.apache.druid.query.QueryRunner; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; import org.apache.druid.segment.realtime.firehose.ChatHandler; @@ -182,7 +182,7 @@ public abstract class SeekableStreamIndexTask implements ChatHandler { + private static final String CTX_KEY_LOOKUP_TIER = "lookupTier"; + public enum Status { NOT_STARTED, @@ -226,7 +225,7 @@ public abstract class SeekableStreamIndexTaskRunner nextCheckpointTime) { @@ -784,7 +779,7 @@ public abstract class SeekableStreamIndexTaskRunner TAGS = ImmutableMap.of("author", "Author Name", "version", 10); @Mock(answer = Answers.RETURNS_MOCKS) - private FireDepartment fireDepartment; + private SegmentGenerationMetrics segmentGenerationMetrics; @Mock(answer = Answers.RETURNS_MOCKS) private RowIngestionMeters rowIngestionMeters; @Mock @@ -74,7 +74,7 @@ public class TaskRealtimeMetricsMonitorTest return null; }) .when(emitter).emit(ArgumentMatchers.any(Event.class)); - target = new TaskRealtimeMetricsMonitor(fireDepartment, rowIngestionMeters, DIMENSIONS, TAGS); + target = new TaskRealtimeMetricsMonitor(segmentGenerationMetrics, rowIngestionMeters, DIMENSIONS, TAGS); } @Test @@ -89,7 +89,7 @@ public class TaskRealtimeMetricsMonitorTest @Test public void testdoMonitorWithoutTagsShouldNotEmitTags() { - target = new TaskRealtimeMetricsMonitor(fireDepartment, rowIngestionMeters, DIMENSIONS, null); + target = new TaskRealtimeMetricsMonitor(segmentGenerationMetrics, rowIngestionMeters, DIMENSIONS, null); for (ServiceMetricEvent sme : emittedEvents.values()) { Assert.assertFalse(sme.getUserDims().containsKey(DruidMetrics.TAGS)); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java index 915385e811e..fec0c8e59fd 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java @@ -33,7 +33,6 @@ import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; -import java.io.IOException; import java.util.HashSet; import java.util.Set; @@ -51,7 +50,7 @@ public class RetrieveSegmentsActionsTest private static Set expectedUsedSegments; @BeforeClass - public static void setup() throws IOException + public static void setup() { task = NoopTask.create(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index 1857f6d67f7..02a3da0e1d0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -60,7 +60,6 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.io.IOException; import java.time.Duration; import java.util.Collections; import java.util.HashMap; @@ -404,7 +403,7 @@ public class SegmentAllocateActionTest } @Test - public void testSegmentIsAllocatedForLatestUsedSegmentVersion() throws IOException + public void testSegmentIsAllocatedForLatestUsedSegmentVersion() { final Task task = NoopTask.create(); taskActionTestKit.getTaskLockbox().add(task); @@ -636,7 +635,7 @@ public class SegmentAllocateActionTest } @Test - public void testAddToExistingLinearShardSpecsSameGranularity() throws Exception + public void testAddToExistingLinearShardSpecsSameGranularity() { final Task task = NoopTask.create(); @@ -702,7 +701,7 @@ public class SegmentAllocateActionTest } @Test - public void testAddToExistingNumberedShardSpecsSameGranularity() throws Exception + public void testAddToExistingNumberedShardSpecsSameGranularity() { final Task task = NoopTask.create(); @@ -766,7 +765,7 @@ public class SegmentAllocateActionTest } @Test - public void testAddToExistingNumberedShardSpecsCoarserPreferredGranularity() throws Exception + public void testAddToExistingNumberedShardSpecsCoarserPreferredGranularity() { final Task task = NoopTask.create(); @@ -806,7 +805,7 @@ public class SegmentAllocateActionTest } @Test - public void testAddToExistingNumberedShardSpecsFinerPreferredGranularity() throws Exception + public void testAddToExistingNumberedShardSpecsFinerPreferredGranularity() { final Task task = NoopTask.create(); @@ -846,7 +845,7 @@ public class SegmentAllocateActionTest } @Test - public void testCannotAddToExistingNumberedShardSpecsWithCoarserQueryGranularity() throws Exception + public void testCannotAddToExistingNumberedShardSpecsWithCoarserQueryGranularity() { final Task task = NoopTask.create(); @@ -889,7 +888,7 @@ public class SegmentAllocateActionTest } @Test - public void testWithPartialShardSpecAndOvershadowingSegments() throws IOException + public void testWithPartialShardSpecAndOvershadowingSegments() { final Task task = NoopTask.create(); taskActionTestKit.getTaskLockbox().add(task); @@ -1065,7 +1064,7 @@ public class SegmentAllocateActionTest } @Test - public void testSegmentIdMustNotBeReused() throws IOException + public void testSegmentIdMustNotBeReused() { final IndexerMetadataStorageCoordinator coordinator = taskActionTestKit.getMetadataStorageCoordinator(); final TaskLockbox lockbox = taskActionTestKit.getTaskLockbox(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java deleted file mode 100644 index b6383cb1659..00000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ /dev/null @@ -1,1706 +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.common.task; - -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; -import org.apache.commons.io.FileUtils; -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.client.coordinator.NoopCoordinatorClient; -import org.apache.druid.client.indexing.NoopOverlordClient; -import org.apache.druid.common.config.NullHandling; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.FloatDimensionSchema; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.LongDimensionSchema; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.TimeAndDimsParseSpec; -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.error.DruidException; -import org.apache.druid.indexer.IngestionState; -import org.apache.druid.indexer.TaskState; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexer.report.IngestionStatsAndErrors; -import org.apache.druid.indexer.report.SingleFileTaskReportFileWriter; -import org.apache.druid.indexer.report.TaskReport; -import org.apache.druid.indexing.common.SegmentCacheManagerFactory; -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.TaskConfigBuilder; -import org.apache.druid.indexing.common.config.TaskStorageConfig; -import org.apache.druid.indexing.common.index.RealtimeAppenderatorIngestionSpec; -import org.apache.druid.indexing.common.index.RealtimeAppenderatorTuningConfig; -import org.apache.druid.indexing.overlord.DataSourceMetadata; -import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; -import org.apache.druid.indexing.overlord.SegmentPublishResult; -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.test.TestDataSegmentAnnouncer; -import org.apache.druid.indexing.test.TestDataSegmentKiller; -import org.apache.druid.indexing.test.TestDataSegmentPusher; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.UOE; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.jackson.JacksonUtils; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.common.parsers.ParseException; -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.IndexerSQLMetadataStorageCoordinator; -import org.apache.druid.metadata.MetadataStorageTablesConfig; -import org.apache.druid.metadata.TestDerbyConnector; -import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; -import org.apache.druid.query.DirectQueryProcessingPool; -import org.apache.druid.query.Druids; -import org.apache.druid.query.QueryPlus; -import org.apache.druid.query.QueryRunnerFactoryConglomerate; -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.LongSumAggregatorFactory; -import org.apache.druid.query.filter.DimFilter; -import org.apache.druid.query.filter.SelectorDimFilter; -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.SegmentSchemaMapping; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.handoff.SegmentHandoffNotifier; -import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; -import org.apache.druid.segment.incremental.RowIngestionMeters; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.join.NoopJoinableFactory; -import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.SegmentSchemaManager; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; -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.server.security.AuthTestUtils; -import org.apache.druid.testing.InitializedNullHandlingTest; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.LinearShardSpec; -import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.easymock.EasyMock; -import org.joda.time.DateTime; -import org.joda.time.Period; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.rules.TemporaryFolder; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.util.ArrayDeque; -import java.util.Arrays; -import java.util.Collection; -import java.util.Deque; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executor; -import java.util.concurrent.TimeUnit; -import java.util.regex.Pattern; - -public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHandlingTest -{ - private static final Logger log = new Logger(AppenderatorDriverRealtimeIndexTaskTest.class); - private static final ServiceEmitter EMITTER = new ServiceEmitter( - "service", - "host", - new NoopEmitter() - ); - - private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper(); - - private static final String FAIL_DIM = "__fail__"; - - private static class TestFirehose implements Firehose - { - private final InputRowParser> parser; - private final Deque>> queue = new ArrayDeque<>(); - private boolean closed = false; - - public TestFirehose(final InputRowParser> parser) - { - this.parser = parser; - } - - public void addRows(List> rows) - { - synchronized (this) { - rows.stream().map(Optional::ofNullable).forEach(queue::add); - notifyAll(); - } - } - - @Override - public boolean hasMore() - { - try { - synchronized (this) { - while (queue.isEmpty() && !closed) { - wait(); - } - return !queue.isEmpty(); - } - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } - - @Override - public InputRow nextRow() - { - synchronized (this) { - final InputRow row = parser.parseBatch(queue.removeFirst().orElse(null)).get(0); - if (row != null && row.getRaw(FAIL_DIM) != null) { - throw new ParseException(null, FAIL_DIM); - } - return row; - } - } - - @Override - public void close() - { - synchronized (this) { - closed = true; - notifyAll(); - } - } - } - - private static class TestFirehoseFactory implements FirehoseFactory - { - public TestFirehoseFactory() - { - } - - @Override - @SuppressWarnings("unchecked") - public Firehose connect(InputRowParser parser, File temporaryDirectory) throws ParseException - { - return new TestFirehose(parser); - } - } - - @Rule - public final ExpectedException expectedException = ExpectedException.none(); - - @Rule - public final TemporaryFolder tempFolder = new TemporaryFolder(); - - @Rule - public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - private final ObjectMapper mapper = TestHelper.makeJsonMapper(); - - private DateTime now; - private ListeningExecutorService taskExec; - private Map> handOffCallbacks; - private Collection publishedSegments; - private CountDownLatch segmentLatch; - private CountDownLatch handoffLatch; - private TaskStorage taskStorage; - private TaskLockbox taskLockbox; - private TaskToolboxFactory taskToolboxFactory; - private File baseDir; - private File reportsFile; - private SegmentSchemaManager segmentSchemaManager; - - @Before - public void setUp() throws IOException - { - EmittingLogger.registerEmitter(EMITTER); - EMITTER.start(); - taskExec = MoreExecutors.listeningDecorator(Execs.singleThreaded("realtime-index-task-test-%d")); - now = DateTimes.nowUtc(); - - TestDerbyConnector derbyConnector = derbyConnectorRule.getConnector(); - derbyConnector.createDataSourceTable(); - derbyConnector.createTaskTables(); - derbyConnector.createSegmentSchemasTable(); - derbyConnector.createSegmentTable(); - derbyConnector.createPendingSegmentsTable(); - - baseDir = tempFolder.newFolder(); - reportsFile = File.createTempFile("KafkaIndexTaskTestReports-" + System.currentTimeMillis(), "json"); - makeToolboxFactory(baseDir); - segmentSchemaManager = new SegmentSchemaManager(MetadataStorageTablesConfig.fromBase(null), mapper, derbyConnector); - } - - @After - public void tearDown() - { - taskExec.shutdownNow(); - reportsFile.delete(); - } - - @Test(timeout = 60_000L) - public void testDefaultResource() - { - final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null); - Assert.assertEquals(task.getId(), task.getTaskResource().getAvailabilityGroup()); - } - - - @Test(timeout = 60_000L) - public void testHandoffTimeout() throws Exception - { - expectPublishedSegments(1); - final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, TransformSpec.NONE, true, 100L, true, 0, 1); - final ListenableFuture statusFuture = runTask(task); - - // Wait for firehose to show up, it starts off null. - while (task.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1") - ) - ); - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - // handoff would timeout, resulting in exception - TaskStatus status = statusFuture.get(); - Assert.assertTrue(status.getErrorMsg() - .contains("java.util.concurrent.TimeoutException: Waited 100 milliseconds")); - } - - @Test(timeout = 60_000L) - public void testBasics() throws Exception - { - expectPublishedSegments(1); - final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null); - Assert.assertTrue(task.supportsQueries()); - final ListenableFuture statusFuture = runTask(task); - - // Wait for firehose to show up, it starts off null. - while (task.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), - ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) - ) - ); - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - // Wait for publish. - Collection publishedSegments = awaitSegments(); - - // Check metrics. - Assert.assertEquals(2, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); - - // Do some queries. - Assert.assertEquals(2, sumMetric(task, null, "rows").longValue()); - Assert.assertEquals(3, sumMetric(task, null, "met1").longValue()); - - awaitHandoffs(); - - for (DataSegment publishedSegment : publishedSegments) { - Pair executorRunnablePair = handOffCallbacks.get( - new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ) - ); - Assert.assertNotNull( - publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, - executorRunnablePair - ); - - // Simulate handoff. - executorRunnablePair.lhs.execute(executorRunnablePair.rhs); - } - handOffCallbacks.clear(); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - } - - @Test(timeout = 60_000L) - public void testLateData() throws Exception - { - expectPublishedSegments(1); - final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null); - final ListenableFuture statusFuture = runTask(task); - - // Wait for firehose to show up, it starts off null. - while (task.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), - // Data is from 2 days ago, should still be processed - ImmutableMap.of("t", now.minus(new Period("P2D")).getMillis(), "dim2", "bar", "met1", 2.0) - ) - ); - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - // Wait for publish. - Collection publishedSegments = awaitSegments(); - - // Check metrics. - Assert.assertEquals(2, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); - - // Do some queries. - Assert.assertEquals(2, sumMetric(task, null, "rows").longValue()); - Assert.assertEquals(3, sumMetric(task, null, "met1").longValue()); - - awaitHandoffs(); - - for (DataSegment publishedSegment : publishedSegments) { - Pair executorRunnablePair = handOffCallbacks.get( - new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ) - ); - Assert.assertNotNull( - publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, - executorRunnablePair - ); - - // Simulate handoff. - executorRunnablePair.lhs.execute(executorRunnablePair.rhs); - } - handOffCallbacks.clear(); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - } - - @Test(timeout = 60_000L) - public void testMaxRowsPerSegment() throws Exception - { - // Expect 2 segments as we will hit maxRowsPerSegment - expectPublishedSegments(2); - - final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null); - final ListenableFuture statusFuture = runTask(task); - - // Wait for firehose to show up, it starts off null. - while (task.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task.getFirehose(); - - // maxRowsPerSegment is 1000 as configured in #makeRealtimeTask - for (int i = 0; i < 2000; i++) { - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo-" + i, "met1", "1") - ) - ); - } - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - // Wait for publish. - Collection publishedSegments = awaitSegments(); - - // Check metrics. - Assert.assertEquals(2000, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); - - // Do some queries. - Assert.assertEquals(2000, sumMetric(task, null, "rows").longValue()); - Assert.assertEquals(2000, sumMetric(task, null, "met1").longValue()); - - awaitHandoffs(); - - for (DataSegment publishedSegment : publishedSegments) { - Pair executorRunnablePair = handOffCallbacks.get( - new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ) - ); - Assert.assertNotNull( - publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, - executorRunnablePair - ); - - // Simulate handoff. - executorRunnablePair.lhs.execute(executorRunnablePair.rhs); - } - handOffCallbacks.clear(); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - } - - @Test(timeout = 60_000L) - public void testMaxTotalRows() throws Exception - { - // Expect 2 segments as we will hit maxTotalRows - expectPublishedSegments(2); - - final AppenderatorDriverRealtimeIndexTask task = - makeRealtimeTask(null, Integer.MAX_VALUE, 1500L); - final ListenableFuture statusFuture = runTask(task); - - // Wait for firehose to show up, it starts off null. - while (task.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task.getFirehose(); - - // maxTotalRows is 1500 - for (int i = 0; i < 2000; i++) { - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo-" + i, "met1", "1") - ) - ); - } - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - // Wait for publish. - Collection publishedSegments = awaitSegments(); - - // Check metrics. - Assert.assertEquals(2000, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); - - // Do some queries. - Assert.assertEquals(2000, sumMetric(task, null, "rows").longValue()); - Assert.assertEquals(2000, sumMetric(task, null, "met1").longValue()); - - awaitHandoffs(); - - Assert.assertEquals(2, publishedSegments.size()); - for (DataSegment publishedSegment : publishedSegments) { - Pair executorRunnablePair = handOffCallbacks.get( - new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ) - ); - Assert.assertNotNull( - publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, - executorRunnablePair - ); - - // Simulate handoff. - executorRunnablePair.lhs.execute(executorRunnablePair.rhs); - } - handOffCallbacks.clear(); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - } - - @Test(timeout = 60_000L) - public void testTransformSpec() throws Exception - { - expectPublishedSegments(2); - - final TransformSpec transformSpec = new TransformSpec( - new SelectorDimFilter("dim1", "foo", null), - ImmutableList.of( - new ExpressionTransform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) - ) - ); - final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, transformSpec, true, 0, true, 0, 1); - final ListenableFuture statusFuture = runTask(task); - - // Wait for firehose to show up, it starts off null. - while (task.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), - ImmutableMap.of("t", now.minus(new Period("P1D")).getMillis(), "dim1", "foo", "met1", 2.0), - ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) - ) - ); - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - Collection publishedSegments = awaitSegments(); - - // Check metrics. - Assert.assertEquals(2, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(1, task.getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); - - // Do some queries. - Assert.assertEquals(2, sumMetric(task, null, "rows").longValue()); - Assert.assertEquals(2, sumMetric(task, new SelectorDimFilter("dim1t", "foofoo", null), "rows").longValue()); - if (NullHandling.replaceWithDefault()) { - Assert.assertEquals(0, sumMetric(task, new SelectorDimFilter("dim1t", "barbar", null), "metric1").longValue()); - } else { - Assert.assertNull(sumMetric(task, new SelectorDimFilter("dim1t", "barbar", null), "metric1")); - } - Assert.assertEquals(3, sumMetric(task, null, "met1").longValue()); - - awaitHandoffs(); - - for (DataSegment publishedSegment : publishedSegments) { - Pair executorRunnablePair = handOffCallbacks.get( - new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ) - ); - Assert.assertNotNull( - publishedSegment + " missing from handoff callbacks: " + handOffCallbacks, - executorRunnablePair - ); - - // Simulate handoff. - executorRunnablePair.lhs.execute(executorRunnablePair.rhs); - } - handOffCallbacks.clear(); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - } - - @Test(timeout = 60_000L) - public void testReportParseExceptionsOnBadMetric() throws Exception - { - expectPublishedSegments(0); - final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, true); - final ListenableFuture statusFuture = runTask(task); - - // Wait for firehose to show up, it starts off null. - while (task.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", 2000000L, "dim1", "foo", "met1", "1"), - ImmutableMap.of("t", 3000000L, "dim1", "foo", "met1", "foo"), - ImmutableMap.of("t", now.minus(new Period("P1D")).getMillis(), "dim1", "foo", "met1", "foo"), - ImmutableMap.of("t", 4000000L, "dim2", "bar", "met1", 2.0) - ) - ); - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - // Wait for the task to finish. - TaskStatus status = statusFuture.get(); - Assert.assertTrue(status.getErrorMsg().contains("org.apache.druid.java.util.common.RE: Max parse exceptions[0] exceeded")); - - IngestionStatsAndErrors reportData = getTaskReportData(); - - ParseExceptionReport parseExceptionReport = - ParseExceptionReport.forPhase(reportData, RowIngestionMeters.BUILD_SEGMENTS); - List expectedMessages = ImmutableList.of( - "Unable to parse value[foo] for field[met1]" - ); - Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); - - List expectedInputs = ImmutableList.of( - "{t=3000000, dim1=foo, met1=foo}" - ); - Assert.assertEquals(expectedInputs, parseExceptionReport.getInputs()); - } - - @Test(timeout = 60_000L) - public void testNoReportParseExceptions() throws Exception - { - expectPublishedSegments(1); - - final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask( - null, - TransformSpec.NONE, - false, - 0, - true, - null, - 1 - ); - final ListenableFuture statusFuture = runTask(task); - - // Wait for firehose to show up, it starts off null. - while (task.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task.getFirehose(); - - firehose.addRows( - Arrays.asList( - // Good row- will be processed. - ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), - - // Null row- will be thrown away. - null, - - // Bad metric- will count as processed, but that particular metric won't update. - ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "foo"), - - // Bad row- will be unparseable. - ImmutableMap.of("dim1", "foo", "met1", 2.0, FAIL_DIM, "x"), - - // Good row- will be processed. - ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) - ) - ); - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - // Wait for publish. - Collection publishedSegments = awaitSegments(); - - DataSegment publishedSegment = Iterables.getOnlyElement(publishedSegments); - - // Check metrics. - Assert.assertEquals(2, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(1, task.getRowIngestionMeters().getProcessedWithError()); - Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(2, task.getRowIngestionMeters().getUnparseable()); - - // Do some queries. - Assert.assertEquals(3, sumMetric(task, null, "rows").longValue()); - Assert.assertEquals(3, sumMetric(task, null, "met1").longValue()); - - awaitHandoffs(); - - // Simulate handoff. - for (Map.Entry> entry : handOffCallbacks.entrySet()) { - final Pair executorRunnablePair = entry.getValue(); - Assert.assertEquals( - new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ), - entry.getKey() - ); - executorRunnablePair.lhs.execute(executorRunnablePair.rhs); - } - handOffCallbacks.clear(); - - Map expectedMetrics = ImmutableMap.of( - RowIngestionMeters.BUILD_SEGMENTS, - ImmutableMap.of( - RowIngestionMeters.PROCESSED, 2, - RowIngestionMeters.PROCESSED_BYTES, 0, - RowIngestionMeters.PROCESSED_WITH_ERROR, 1, - RowIngestionMeters.UNPARSEABLE, 2, - RowIngestionMeters.THROWN_AWAY, 0 - ) - ); - - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - - IngestionStatsAndErrors reportData = getTaskReportData(); - - Assert.assertEquals(expectedMetrics, reportData.getRowStats()); - } - - @Test(timeout = 60_000L) - public void testMultipleParseExceptionsSuccess() throws Exception - { - expectPublishedSegments(1); - - final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, TransformSpec.NONE, false, 0, true, 10, 10); - final ListenableFuture statusFuture = runTask(task); - - // Wait for firehose to show up, it starts off null. - while (task.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task.getFirehose(); - - firehose.addRows( - Arrays.asList( - // Good row- will be processed. - ImmutableMap.of("t", 1521251960729L, "dim1", "foo", "met1", "1"), - - // Null row- will be thrown away. - null, - - // Bad metric- will count as processed, but that particular metric won't update. - ImmutableMap.of("t", 1521251960729L, "dim1", "foo", "met1", "foo"), - - // Bad long dim- will count as processed, but bad dims will get default values - ImmutableMap.of( - "t", - 1521251960729L, - "dim1", - "foo", - "dimLong", - "notnumber", - "dimFloat", - "notnumber", - "met1", - "foo" - ), - - // Bad row- will be unparseable. - ImmutableMap.of("dim1", "foo", "met1", 2.0, FAIL_DIM, "x"), - - // Good row- will be processed. - ImmutableMap.of("t", 1521251960729L, "dim2", "bar", "met1", 2.0) - ) - ); - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - // Wait for publish. - Collection publishedSegments = awaitSegments(); - - DataSegment publishedSegment = Iterables.getOnlyElement(publishedSegments); - - // Check metrics. - Assert.assertEquals(2, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(2, task.getRowIngestionMeters().getProcessedWithError()); - Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(2, task.getRowIngestionMeters().getUnparseable()); - - // Do some queries. - Assert.assertEquals(4, sumMetric(task, null, "rows").longValue()); - Assert.assertEquals(3, sumMetric(task, null, "met1").longValue()); - - awaitHandoffs(); - - // Simulate handoff. - for (Map.Entry> entry : handOffCallbacks.entrySet()) { - final Pair executorRunnablePair = entry.getValue(); - Assert.assertEquals( - new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ), - entry.getKey() - ); - executorRunnablePair.lhs.execute(executorRunnablePair.rhs); - } - handOffCallbacks.clear(); - - Map expectedMetrics = ImmutableMap.of( - RowIngestionMeters.BUILD_SEGMENTS, - ImmutableMap.of( - RowIngestionMeters.PROCESSED, 2, - RowIngestionMeters.PROCESSED_BYTES, 0, - RowIngestionMeters.PROCESSED_WITH_ERROR, 2, - RowIngestionMeters.UNPARSEABLE, 2, - RowIngestionMeters.THROWN_AWAY, 0 - ) - ); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - - IngestionStatsAndErrors reportData = getTaskReportData(); - Assert.assertEquals(expectedMetrics, reportData.getRowStats()); - - ParseExceptionReport parseExceptionReport = - ParseExceptionReport.forPhase(reportData, RowIngestionMeters.BUILD_SEGMENTS); - - List expectedMessages = Arrays.asList( - "Timestamp[null] is unparseable! Event: {dim1=foo, met1=2.0, __fail__=x}", - "could not convert value [notnumber] to long", - "Unable to parse value[foo] for field[met1]", - "Timestamp[null] is unparseable! Event: null" - ); - Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); - - List expectedInputs = Arrays.asList( - "{dim1=foo, met1=2.0, __fail__=x}", - "{t=1521251960729, dim1=foo, dimLong=notnumber, dimFloat=notnumber, met1=foo}", - "{t=1521251960729, dim1=foo, met1=foo}", - null - ); - Assert.assertEquals(expectedInputs, parseExceptionReport.getInputs()); - Assert.assertEquals(IngestionState.COMPLETED, reportData.getIngestionState()); - } - - @Test(timeout = 60_000L) - public void testMultipleParseExceptionsFailure() throws Exception - { - expectPublishedSegments(1); - - final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, TransformSpec.NONE, false, 0, true, 3, 10); - final ListenableFuture statusFuture = runTask(task); - - // Wait for firehose to show up, it starts off null. - while (task.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task.getFirehose(); - - firehose.addRows( - Arrays.asList( - // Good row- will be processed. - ImmutableMap.of("t", 1521251960729L, "dim1", "foo", "met1", "1"), - - // Null row- will be thrown away. - null, - - // Bad metric- will count as processed, but that particular metric won't update. - ImmutableMap.of("t", 1521251960729L, "dim1", "foo", "met1", "foo"), - - // Bad long dim- will count as processed, but bad dims will get default values - ImmutableMap.of( - "t", - 1521251960729L, - "dim1", - "foo", - "dimLong", - "notnumber", - "dimFloat", - "notnumber", - "met1", - "foo" - ), - - // Bad row- will be unparseable. - ImmutableMap.of("dim1", "foo", "met1", 2.0, FAIL_DIM, "x"), - - // Good row- will be processed. - ImmutableMap.of("t", 1521251960729L, "dim2", "bar", "met1", 2.0) - ) - ); - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.FAILED, taskStatus.getStatusCode()); - Assert.assertTrue(taskStatus.getErrorMsg().contains("Max parse exceptions[3] exceeded")); - - IngestionStatsAndErrors reportData = getTaskReportData(); - - Map expectedMetrics = ImmutableMap.of( - RowIngestionMeters.BUILD_SEGMENTS, - ImmutableMap.of( - RowIngestionMeters.PROCESSED, 1, - RowIngestionMeters.PROCESSED_BYTES, 0, - RowIngestionMeters.PROCESSED_WITH_ERROR, 2, - RowIngestionMeters.UNPARSEABLE, 2, - RowIngestionMeters.THROWN_AWAY, 0 - ) - ); - Assert.assertEquals(expectedMetrics, reportData.getRowStats()); - - ParseExceptionReport parseExceptionReport = - ParseExceptionReport.forPhase(reportData, RowIngestionMeters.BUILD_SEGMENTS); - - List expectedMessages = ImmutableList.of( - "Timestamp[null] is unparseable! Event: {dim1=foo, met1=2.0, __fail__=x}", - "could not convert value [notnumber] to long", - "Unable to parse value[foo] for field[met1]", - "Timestamp[null] is unparseable! Event: null" - ); - Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); - - List expectedInputs = Arrays.asList( - "{dim1=foo, met1=2.0, __fail__=x}", - "{t=1521251960729, dim1=foo, dimLong=notnumber, dimFloat=notnumber, met1=foo}", - "{t=1521251960729, dim1=foo, met1=foo}", - null - ); - Assert.assertEquals(expectedInputs, parseExceptionReport.getInputs()); - Assert.assertEquals(IngestionState.BUILD_SEGMENTS, reportData.getIngestionState()); - } - - @Test(timeout = 60_000L) - public void testRestore() throws Exception - { - expectPublishedSegments(0); - - final AppenderatorDriverRealtimeIndexTask task1 = makeRealtimeTask(null); - final DataSegment publishedSegment; - - // First run: - { - final ListenableFuture statusFuture = runTask(task1); - - // Wait for firehose to show up, it starts off null. - while (task1.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task1.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo") - ) - ); - - // Trigger graceful shutdown. - task1.stopGracefully(taskToolboxFactory.build(task1).getConfig()); - - // Wait for the task to finish. The status doesn't really matter, but we'll check it anyway. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - - // Nothing should be published. - Assert.assertTrue(publishedSegments.isEmpty()); - } - - // Second run: - { - expectPublishedSegments(1); - final AppenderatorDriverRealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); - final ListenableFuture statusFuture = runTask(task2); - - // Wait for firehose to show up, it starts off null. - while (task2.getFirehose() == null) { - Thread.sleep(50); - } - - // Do a query, at this point the previous data should be loaded. - Assert.assertEquals(1, sumMetric(task2, null, "rows").longValue()); - - final TestFirehose firehose = (TestFirehose) task2.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim2", "bar") - ) - ); - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - Collection publishedSegments = awaitSegments(); - - publishedSegment = Iterables.getOnlyElement(publishedSegments); - - // Do a query. - Assert.assertEquals(2, sumMetric(task2, null, "rows").longValue()); - - awaitHandoffs(); - - // Simulate handoff. - for (Map.Entry> entry : handOffCallbacks.entrySet()) { - final Pair executorRunnablePair = entry.getValue(); - Assert.assertEquals( - new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ), - entry.getKey() - ); - executorRunnablePair.lhs.execute(executorRunnablePair.rhs); - } - handOffCallbacks.clear(); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - } - } - - @Test(timeout = 60_000L) - public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception - { - final AppenderatorDriverRealtimeIndexTask task1 = makeRealtimeTask(null); - final DataSegment publishedSegment; - - // First run: - { - expectPublishedSegments(1); - final ListenableFuture statusFuture = runTask(task1); - - // Wait for firehose to show up, it starts off null. - while (task1.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task1.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo") - ) - ); - - // Stop the firehose, this will trigger a finishJob. - firehose.close(); - - Collection publishedSegments = awaitSegments(); - - publishedSegment = Iterables.getOnlyElement(publishedSegments); - - // Do a query. - Assert.assertEquals(1, sumMetric(task1, null, "rows").longValue()); - - // Trigger graceful shutdown. - task1.stopGracefully(taskToolboxFactory.build(task1).getConfig()); - - // Wait for the task to finish. The status doesn't really matter. - while (!statusFuture.isDone()) { - Thread.sleep(50); - } - } - - // Second run: - { - expectPublishedSegments(1); - final AppenderatorDriverRealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); - final ListenableFuture statusFuture = runTask(task2); - - // Wait for firehose to show up, it starts off null. - while (task2.getFirehose() == null) { - Thread.sleep(50); - } - - // Stop the firehose again, this will start another handoff. - final TestFirehose firehose = (TestFirehose) task2.getFirehose(); - - // Stop the firehose, this will trigger a finishJob. - firehose.close(); - - awaitHandoffs(); - - // Simulate handoff. - for (Map.Entry> entry : handOffCallbacks.entrySet()) { - final Pair executorRunnablePair = entry.getValue(); - Assert.assertEquals( - new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ), - entry.getKey() - ); - executorRunnablePair.lhs.execute(executorRunnablePair.rhs); - } - handOffCallbacks.clear(); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - } - } - - @Test(timeout = 60_000L) - public void testRestoreCorruptData() throws Exception - { - final AppenderatorDriverRealtimeIndexTask task1 = makeRealtimeTask(null); - - // First run: - { - expectPublishedSegments(0); - - final ListenableFuture statusFuture = runTask(task1); - - // Wait for firehose to show up, it starts off null. - while (task1.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task1.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo") - ) - ); - - // Trigger graceful shutdown. - task1.stopGracefully(taskToolboxFactory.build(task1).getConfig()); - - // Wait for the task to finish. The status doesn't really matter, but we'll check it anyway. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - - // Nothing should be published. - Assert.assertTrue(publishedSegments.isEmpty()); - } - - Optional optional = FileUtils.listFiles(baseDir, null, true).stream() - .filter(f -> f.getName().equals("00000.smoosh")) - .findFirst(); - - Assert.assertTrue("Could not find smoosh file", optional.isPresent()); - - // Corrupt the data: - final File smooshFile = optional.get(); - - Files.write(smooshFile.toPath(), StringUtils.toUtf8("oops!")); - - // Second run: - { - expectPublishedSegments(0); - - final AppenderatorDriverRealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); - final ListenableFuture statusFuture = runTask(task2); - - // Wait for the task to finish. - TaskStatus status = statusFuture.get(); - - Map expectedMetrics = ImmutableMap.of( - RowIngestionMeters.BUILD_SEGMENTS, - ImmutableMap.of( - RowIngestionMeters.PROCESSED_WITH_ERROR, 0, - RowIngestionMeters.PROCESSED, 0, - RowIngestionMeters.PROCESSED_BYTES, 0, - RowIngestionMeters.UNPARSEABLE, 0, - RowIngestionMeters.THROWN_AWAY, 0 - ) - ); - - IngestionStatsAndErrors reportData = getTaskReportData(); - Assert.assertEquals(expectedMetrics, reportData.getRowStats()); - - Pattern errorPattern = Pattern.compile( - "(?s)java\\.lang\\.IllegalArgumentException.*\n" - + "\tat (java\\.base/)?java\\.nio\\.Buffer\\..*" - ); - Assert.assertTrue(errorPattern.matcher(status.getErrorMsg()).matches()); - } - } - - @Test(timeout = 60_000L) - public void testStopBeforeStarting() throws Exception - { - expectPublishedSegments(0); - - final AppenderatorDriverRealtimeIndexTask task1 = makeRealtimeTask(null); - - task1.stopGracefully(taskToolboxFactory.build(task1).getConfig()); - final ListenableFuture statusFuture = runTask(task1); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - } - - @Test(timeout = 60_000L) - public void testInputSourceResourcesThrowException() - { - // Expect 2 segments as we will hit maxTotalRows - expectPublishedSegments(2); - - final AppenderatorDriverRealtimeIndexTask task = - makeRealtimeTask(null, Integer.MAX_VALUE, 1500L); - Assert.assertThrows( - UOE.class, - task::getInputSourceResources - ); - } - - private ListenableFuture runTask(final Task task) - { - try { - taskStorage.insert(task, TaskStatus.running(task.getId())); - } - catch (DruidException e) { - log.noStackTrace().info(e, "Suppressing exception while inserting task [%s]", task.getId()); - } - taskLockbox.syncFromStorage(); - final TaskToolbox toolbox = taskToolboxFactory.build(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"); - throw e; - } - } - ); - } - - private AppenderatorDriverRealtimeIndexTask makeRealtimeTask(final String taskId) - { - return makeRealtimeTask( - taskId, - TransformSpec.NONE, - true, - 0, - true, - 0, - 1 - ); - } - - private AppenderatorDriverRealtimeIndexTask makeRealtimeTask( - final String taskId, - final Integer maxRowsPerSegment, - final Long maxTotalRows - ) - { - return makeRealtimeTask( - taskId, - TransformSpec.NONE, - true, - 0, - true, - 0, - 1, - maxRowsPerSegment, - maxTotalRows - ); - } - - private AppenderatorDriverRealtimeIndexTask makeRealtimeTask(final String taskId, boolean reportParseExceptions) - { - return makeRealtimeTask( - taskId, - TransformSpec.NONE, - reportParseExceptions, - 0, - true, - null, - 1 - ); - } - - private AppenderatorDriverRealtimeIndexTask makeRealtimeTask( - final String taskId, - final TransformSpec transformSpec, - final boolean reportParseExceptions, - final long handoffTimeout, - final Boolean logParseExceptions, - final Integer maxParseExceptions, - final Integer maxSavedParseExceptions - ) - { - - return makeRealtimeTask( - taskId, - transformSpec, - reportParseExceptions, - handoffTimeout, - logParseExceptions, - maxParseExceptions, - maxSavedParseExceptions, - 1000, - null - ); - } - - private AppenderatorDriverRealtimeIndexTask makeRealtimeTask( - final String taskId, - final TransformSpec transformSpec, - final boolean reportParseExceptions, - final long handoffTimeout, - final Boolean logParseExceptions, - final Integer maxParseExceptions, - final Integer maxSavedParseExceptions, - final Integer maxRowsPerSegment, - final Long maxTotalRows - ) - { - DataSchema dataSchema = new DataSchema( - "test_ds", - TestHelper.makeJsonMapper().convertValue( - new MapInputRowParser( - new TimeAndDimsParseSpec( - new TimestampSpec("t", "auto", null), - new DimensionsSpec( - ImmutableList.of( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim2"), - new StringDimensionSchema("dim1t"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ) - ) - ) - ), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ), - new AggregatorFactory[]{new CountAggregatorFactory("rows"), new LongSumAggregatorFactory("met1", "met1")}, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - transformSpec, - OBJECT_MAPPER - ); - RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig( - new TestFirehoseFactory(), - null - ); - RealtimeAppenderatorTuningConfig tuningConfig = new RealtimeAppenderatorTuningConfig( - null, - 1000, - null, - null, - maxRowsPerSegment, - maxTotalRows, - null, - null, - null, - null, - null, - null, - reportParseExceptions, - handoffTimeout, - null, - null, - logParseExceptions, - maxParseExceptions, - maxSavedParseExceptions, - null - ); - return new AppenderatorDriverRealtimeIndexTask( - taskId, - null, - new RealtimeAppenderatorIngestionSpec(dataSchema, realtimeIOConfig, tuningConfig), - null - ) - { - @Override - protected boolean isFirehoseDrainableByClosing(FirehoseFactory firehoseFactory) - { - return true; - } - }; - } - - private void expectPublishedSegments(int count) - { - segmentLatch = new CountDownLatch(count); - handoffLatch = new CountDownLatch(count); - } - - private Collection awaitSegments() throws InterruptedException - { - Assert.assertTrue( - "Timed out waiting for segments to be published", - segmentLatch.await(1, TimeUnit.MINUTES) - ); - - return publishedSegments; - } - - private void awaitHandoffs() throws InterruptedException - { - Assert.assertTrue( - "Timed out waiting for segments to be handed off", - handoffLatch.await(1, TimeUnit.MINUTES) - ); - } - - private void makeToolboxFactory(final File directory) - { - taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); - publishedSegments = new CopyOnWriteArrayList<>(); - - ObjectMapper mapper = new DefaultObjectMapper(); - mapper.registerSubtypes(LinearShardSpec.class); - mapper.registerSubtypes(NumberedShardSpec.class); - IndexerSQLMetadataStorageCoordinator mdc = new IndexerSQLMetadataStorageCoordinator( - mapper, - derbyConnectorRule.metadataTablesConfigSupplier().get(), - derbyConnectorRule.getConnector(), - segmentSchemaManager, - CentralizedDatasourceSchemaConfig.create() - ) - { - @Override - public Set commitSegments(Set segments, SegmentSchemaMapping segmentSchemaMapping) throws IOException - { - Set result = super.commitSegments(segments, segmentSchemaMapping); - - Assert.assertFalse( - "Segment latch not initialized, did you forget to call expectPublishSegments?", - segmentLatch == null - ); - - publishedSegments.addAll(result); - segments.forEach(s -> segmentLatch.countDown()); - - return result; - } - - @Override - public SegmentPublishResult commitSegmentsAndMetadata( - Set segments, - DataSourceMetadata startMetadata, - DataSourceMetadata endMetadata, - SegmentSchemaMapping segmentSchemaMapping - ) throws IOException - { - SegmentPublishResult result = super.commitSegmentsAndMetadata(segments, startMetadata, endMetadata, segmentSchemaMapping); - - Assert.assertNotNull( - "Segment latch not initialized, did you forget to call expectPublishSegments?", - segmentLatch - ); - - publishedSegments.addAll(result.getSegments()); - result.getSegments().forEach(s -> segmentLatch.countDown()); - - return result; - } - }; - - taskLockbox = new TaskLockbox(taskStorage, mdc); - final TaskConfig taskConfig = new TaskConfigBuilder() - .setBaseDir(directory.getPath()) - .setDefaultRowFlushBoundary(50000) - .setRestoreTasksOnRestart(true) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) - .build(); - - final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( - taskLockbox, - taskStorage, - mdc, - EMITTER, - EasyMock.createMock(SupervisorManager.class), - OBJECT_MAPPER - ); - - final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( - taskStorage, - taskActionToolbox, - new TaskAuditLogConfig(false) - ); - - final QueryRunnerFactoryConglomerate conglomerate = new DefaultQueryRunnerFactoryConglomerate( - ImmutableMap.of( - TimeseriesQuery.class, - new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(), - new TimeseriesQueryEngine(), - (query, future) -> { - // do nothing - } - ) - ) - ); - - handOffCallbacks = new ConcurrentHashMap<>(); - final SegmentHandoffNotifierFactory handoffNotifierFactory = dataSource -> new SegmentHandoffNotifier() - { - @Override - public boolean registerSegmentHandoffCallback( - SegmentDescriptor descriptor, - Executor exec, - Runnable handOffRunnable - ) - { - handOffCallbacks.put(descriptor, new Pair<>(exec, handOffRunnable)); - handoffLatch.countDown(); - return true; - } - - @Override - public void start() - { - //Noop - } - - @Override - public void close() - { - //Noop - } - - }; - final TestUtils testUtils = new TestUtils(); - taskToolboxFactory = new TaskToolboxFactory( - null, - taskConfig, - new DruidNode("druid/middlemanager", "localhost", false, 8091, null, true, false), - taskActionClientFactory, - EMITTER, - new TestDataSegmentPusher(), - new TestDataSegmentKiller(), - null, // DataSegmentMover - null, // DataSegmentArchiver - new TestDataSegmentAnnouncer(), - EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), - handoffNotifierFactory, - () -> conglomerate, - DirectQueryProcessingPool.INSTANCE, // queryExecutorService - NoopJoinableFactory.INSTANCE, - () -> EasyMock.createMock(MonitorScheduler.class), - new SegmentCacheManagerFactory(TestIndex.INDEX_IO, testUtils.getTestObjectMapper()), - testUtils.getTestObjectMapper(), - testUtils.getTestIndexIO(), - MapCache.create(1024), - new CacheConfig(), - new CachePopulatorStats(), - testUtils.getIndexMergerV9Factory(), - EasyMock.createNiceMock(DruidNodeAnnouncer.class), - EasyMock.createNiceMock(DruidNode.class), - new LookupNodeService("tier"), - new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0), - new SingleFileTaskReportFileWriter(reportsFile), - null, - AuthTestUtils.TEST_AUTHORIZER_MAPPER, - new NoopChatHandlerProvider(), - testUtils.getRowIngestionMetersFactory(), - new TestAppenderatorsManager(), - new NoopOverlordClient(), - new NoopCoordinatorClient(), - null, - null, - null, - "1", - CentralizedDatasourceSchemaConfig.create() - ); - } - - @Nullable - public Long sumMetric(final Task task, final DimFilter filter, final String metric) - { - // Do a query. - TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() - .dataSource("test_ds") - .filters(filter) - .aggregators( - ImmutableList.of( - new LongSumAggregatorFactory(metric, metric) - ) - ).granularity(Granularities.ALL) - .intervals("2000/3000") - .build(); - - List> results = - task.getQueryRunner(query).run(QueryPlus.wrap(query)).toList(); - - if (results.isEmpty()) { - return 0L; - } else { - return results.get(0).getValue().getLongMetric(metric); - } - } - - private IngestionStatsAndErrors getTaskReportData() throws IOException - { - TaskReport.ReportMap taskReports = OBJECT_MAPPER.readValue( - reportsFile, - new TypeReference() - { - } - ); - return IngestionStatsAndErrors.getPayloadFromTaskReports( - taskReports - ); - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java deleted file mode 100644 index 12f82b84a2a..00000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java +++ /dev/null @@ -1,553 +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.common.task; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.granularity.Granularities; -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.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.CountAggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.IndexMerger; -import org.apache.druid.segment.IndexMergerV9; -import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.column.ColumnConfig; -import org.apache.druid.segment.incremental.AppendableIndexSpec; -import org.apache.druid.segment.incremental.ParseExceptionHandler; -import org.apache.druid.segment.incremental.RowIngestionMeters; -import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.TuningConfig; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.segment.realtime.appenderator.Appenderator; -import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; -import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; -import org.apache.druid.segment.realtime.appenderator.Appenderators; -import org.apache.druid.segment.realtime.appenderator.BatchAppenderator; -import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; -import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.LinearShardSpec; -import org.joda.time.Period; -import org.junit.Assert; -import org.junit.Test; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.concurrent.CopyOnWriteArrayList; - - -public class AppenderatorsTest -{ - @Test - public void testOpenSegmentsOfflineAppenderator() throws Exception - { - try (final AppenderatorTester tester = new AppenderatorTester("OPEN_SEGMENTS")) { - Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl); - AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator; - Assert.assertTrue(appenderator.isOpenSegments()); - } - } - - @Test - public void testClosedSegmentsOfflineAppenderator() throws Exception - { - try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS")) { - Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl); - AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator; - Assert.assertFalse(appenderator.isOpenSegments()); - } - } - - @Test - public void testClosedSegmentsSinksOfflineAppenderator() throws Exception - { - try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS_SINKS")) { - Assert.assertTrue(tester.appenderator instanceof BatchAppenderator); - } - } - - - private static class AppenderatorTester implements AutoCloseable - { - public static final String DATASOURCE = "foo"; - - private final DataSchema schema; - private final AppenderatorConfig tuningConfig; - private final FireDepartmentMetrics metrics; - private final ObjectMapper objectMapper; - private final Appenderator appenderator; - private final ServiceEmitter emitter; - - private final List pushedSegments = new CopyOnWriteArrayList<>(); - - - public AppenderatorTester( - final String batchMode - ) - { - this(100, 100, null, false, new SimpleRowIngestionMeters(), - false, batchMode - ); - } - - public AppenderatorTester( - final int maxRowsInMemory, - final long maxSizeInBytes, - @Nullable final File basePersistDirectory, - final boolean enablePushFailure, - final RowIngestionMeters rowIngestionMeters, - final boolean skipBytesInMemoryOverheadCheck, - String batchMode - ) - { - objectMapper = new DefaultObjectMapper(); - objectMapper.registerSubtypes(LinearShardSpec.class); - - final Map parserMap = objectMapper.convertValue( - new MapInputRowParser( - new JSONParseSpec( - new TimestampSpec("ts", "auto", null), - DimensionsSpec.EMPTY, - null, - null, - null - ) - ), - Map.class - ); - - schema = new DataSchema( - DATASOURCE, - null, - null, - new AggregatorFactory[]{ - new CountAggregatorFactory("count"), - new LongSumAggregatorFactory("met", "met") - }, - new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), - null, - parserMap, - objectMapper - ); - - tuningConfig = new TestIndexTuningConfig( - TuningConfig.DEFAULT_APPENDABLE_INDEX, - maxRowsInMemory, - maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, - skipBytesInMemoryOverheadCheck, - IndexSpec.DEFAULT, - 0, - false, - 0L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE, - basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory, - null - ); - metrics = new FireDepartmentMetrics(); - - IndexIO indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT); - IndexMergerV9 indexMerger = new IndexMergerV9( - objectMapper, - indexIO, - OffHeapMemorySegmentWriteOutMediumFactory.instance() - ); - - emitter = new ServiceEmitter( - "test", - "test", - new NoopEmitter() - ); - emitter.start(); - EmittingLogger.registerEmitter(emitter); - DataSegmentPusher dataSegmentPusher = new DataSegmentPusher() - { - private boolean mustFail = true; - - @Deprecated - @Override - public String getPathForHadoop(String dataSource) - { - return getPathForHadoop(); - } - - @Override - public String getPathForHadoop() - { - throw new UnsupportedOperationException(); - } - - @Override - public DataSegment push(File file, DataSegment segment, boolean useUniquePath) throws IOException - { - if (enablePushFailure && mustFail) { - mustFail = false; - throw new IOException("Push failure test"); - } else if (enablePushFailure) { - mustFail = true; - } - pushedSegments.add(segment); - return segment; - } - - @Override - public Map makeLoadSpec(URI uri) - { - throw new UnsupportedOperationException(); - } - }; - - switch (batchMode) { - case "OPEN_SEGMENTS": - appenderator = Appenderators.createOpenSegmentsOffline( - schema.getDataSource(), - schema, - tuningConfig, - metrics, - dataSegmentPusher, - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - false, - CentralizedDatasourceSchemaConfig.create() - ); - break; - case "CLOSED_SEGMENTS": - appenderator = Appenderators.createClosedSegmentsOffline( - schema.getDataSource(), - schema, - tuningConfig, - metrics, - dataSegmentPusher, - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - false, - CentralizedDatasourceSchemaConfig.create() - ); - - break; - case "CLOSED_SEGMENTS_SINKS": - appenderator = Appenderators.createOffline( - schema.getDataSource(), - schema, - tuningConfig, - metrics, - dataSegmentPusher, - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), - false, - CentralizedDatasourceSchemaConfig.create() - ); - break; - default: - throw new IllegalArgumentException("Unrecognized batchMode: " + batchMode); - } - } - - private long getDefaultMaxBytesInMemory() - { - return (Runtime.getRuntime().totalMemory()) / 3; - } - - public DataSchema getSchema() - { - return schema; - } - - public AppenderatorConfig getTuningConfig() - { - return tuningConfig; - } - - public FireDepartmentMetrics getMetrics() - { - return metrics; - } - - public ObjectMapper getObjectMapper() - { - return objectMapper; - } - - public Appenderator getAppenderator() - { - return appenderator; - } - - public List getPushedSegments() - { - return pushedSegments; - } - - @Override - public void close() throws Exception - { - appenderator.close(); - emitter.close(); - FileUtils.deleteDirectory(tuningConfig.getBasePersistDirectory()); - } - - private static File createNewBasePersistDirectory() - { - return FileUtils.createTempDir("druid-batch-persist"); - } - - - static class TestIndexTuningConfig implements AppenderatorConfig - { - private final AppendableIndexSpec appendableIndexSpec; - private final int maxRowsInMemory; - private final long maxBytesInMemory; - private final boolean skipBytesInMemoryOverheadCheck; - private final int maxColumnsToMerge; - private final PartitionsSpec partitionsSpec; - private final IndexSpec indexSpec; - private final File basePersistDirectory; - private final int maxPendingPersists; - private final boolean reportParseExceptions; - private final long pushTimeout; - private final IndexSpec indexSpecForIntermediatePersists; - @Nullable - private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; - private final int numPersistThreads; - - public TestIndexTuningConfig( - AppendableIndexSpec appendableIndexSpec, - Integer maxRowsInMemory, - Long maxBytesInMemory, - Boolean skipBytesInMemoryOverheadCheck, - IndexSpec indexSpec, - Integer maxPendingPersists, - Boolean reportParseExceptions, - Long pushTimeout, - @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, - Integer maxColumnsToMerge, - File basePersistDirectory, - Integer numPersistThreads - ) - { - this.appendableIndexSpec = appendableIndexSpec; - this.maxRowsInMemory = maxRowsInMemory; - this.maxBytesInMemory = maxBytesInMemory; - this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck; - this.indexSpec = indexSpec; - this.maxPendingPersists = maxPendingPersists; - this.reportParseExceptions = reportParseExceptions; - this.pushTimeout = pushTimeout; - this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; - this.maxColumnsToMerge = maxColumnsToMerge; - this.basePersistDirectory = basePersistDirectory; - - this.partitionsSpec = null; - this.indexSpecForIntermediatePersists = this.indexSpec; - - this.numPersistThreads = numPersistThreads == null ? DEFAULT_NUM_PERSIST_THREADS : numPersistThreads; - } - - @Override - public TestIndexTuningConfig withBasePersistDirectory(File dir) - { - throw new UnsupportedOperationException(); - } - - @Override - public AppendableIndexSpec getAppendableIndexSpec() - { - return appendableIndexSpec; - } - - @Override - public int getMaxRowsInMemory() - { - return maxRowsInMemory; - } - - @Override - public long getMaxBytesInMemory() - { - return maxBytesInMemory; - } - - @Override - public boolean isSkipBytesInMemoryOverheadCheck() - { - return skipBytesInMemoryOverheadCheck; - } - - @Nullable - @Override - public PartitionsSpec getPartitionsSpec() - { - return partitionsSpec; - } - - @Override - public IndexSpec getIndexSpec() - { - return indexSpec; - } - - @Override - public IndexSpec getIndexSpecForIntermediatePersists() - { - return indexSpecForIntermediatePersists; - } - - @Override - public int getMaxPendingPersists() - { - return maxPendingPersists; - } - - @Override - public boolean isReportParseExceptions() - { - return reportParseExceptions; - } - - @Nullable - @Override - public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() - { - return segmentWriteOutMediumFactory; - } - - @Override - public int getMaxColumnsToMerge() - { - return maxColumnsToMerge; - } - - @Override - public File getBasePersistDirectory() - { - return basePersistDirectory; - } - - @Override - public Period getIntermediatePersistPeriod() - { - return new Period(Integer.MAX_VALUE); // intermediate persist doesn't make much sense for batch jobs - } - - @Override - public int getNumPersistThreads() - { - return numPersistThreads; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - TestIndexTuningConfig that = (TestIndexTuningConfig) o; - return Objects.equals(appendableIndexSpec, that.appendableIndexSpec) && - maxRowsInMemory == that.maxRowsInMemory && - maxBytesInMemory == that.maxBytesInMemory && - skipBytesInMemoryOverheadCheck == that.skipBytesInMemoryOverheadCheck && - maxColumnsToMerge == that.maxColumnsToMerge && - maxPendingPersists == that.maxPendingPersists && - reportParseExceptions == that.reportParseExceptions && - pushTimeout == that.pushTimeout && - numPersistThreads == that.numPersistThreads && - Objects.equals(partitionsSpec, that.partitionsSpec) && - Objects.equals(indexSpec, that.indexSpec) && - Objects.equals(indexSpecForIntermediatePersists, that.indexSpecForIntermediatePersists) && - Objects.equals(basePersistDirectory, that.basePersistDirectory) && - Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory); - } - - @Override - public int hashCode() - { - return Objects.hash( - appendableIndexSpec, - maxRowsInMemory, - maxBytesInMemory, - skipBytesInMemoryOverheadCheck, - maxColumnsToMerge, - partitionsSpec, - indexSpec, - indexSpecForIntermediatePersists, - basePersistDirectory, - maxPendingPersists, - reportParseExceptions, - pushTimeout, - segmentWriteOutMediumFactory, - numPersistThreads - ); - } - - @Override - public String toString() - { - return "IndexTuningConfig{" + - "maxRowsInMemory=" + maxRowsInMemory + - ", maxBytesInMemory=" + maxBytesInMemory + - ", skipBytesInMemoryOverheadCheck=" + skipBytesInMemoryOverheadCheck + - ", maxColumnsToMerge=" + maxColumnsToMerge + - ", partitionsSpec=" + partitionsSpec + - ", indexSpec=" + indexSpec + - ", indexSpecForIntermediatePersists=" + indexSpecForIntermediatePersists + - ", basePersistDirectory=" + basePersistDirectory + - ", maxPendingPersists=" + maxPendingPersists + - ", reportParseExceptions=" + reportParseExceptions + - ", pushTimeout=" + pushTimeout + - ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + - ", numPersistThreads=" + numPersistThreads + - '}'; - } - } - - } - -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java index bbc87327048..f789c1e641f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java @@ -52,7 +52,7 @@ import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; @@ -114,7 +114,7 @@ public class BatchAppenderatorsTest private final DataSchema schema; private final AppenderatorConfig tuningConfig; - private final FireDepartmentMetrics metrics; + private final SegmentGenerationMetrics metrics; private final ObjectMapper objectMapper; private final Appenderator appenderator; private final ServiceEmitter emitter; @@ -185,7 +185,7 @@ public class BatchAppenderatorsTest basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory, null ); - metrics = new FireDepartmentMetrics(); + metrics = new SegmentGenerationMetrics(); IndexIO indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT); IndexMergerV9 indexMerger = new IndexMergerV9( @@ -329,7 +329,7 @@ public class BatchAppenderatorsTest return tuningConfig; } - public FireDepartmentMetrics getMetrics() + public SegmentGenerationMetrics getMetrics() { return metrics; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index ce685c03cf2..134f5305169 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.exc.ValueInstantiationException; import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; @@ -122,7 +121,6 @@ import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.indexing.BatchIOConfig; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.NoopJoinableFactory; @@ -884,48 +882,6 @@ public class CompactionTaskTest Assert.assertEquals(compactionTuningConfig, CompactionTask.getTuningConfig(parallelIndexTuningConfig)); } - @Test - public void testSerdeWithUnknownTuningConfigThrowingError() throws IOException - { - final OldCompactionTaskWithAnyTuningConfigType taskWithUnknownTuningConfig = - new OldCompactionTaskWithAnyTuningConfigType( - null, - null, - DATA_SOURCE, - null, - SEGMENTS, - null, - null, - null, - null, - null, - RealtimeTuningConfig.makeDefaultTuningConfig(null), - null, - OBJECT_MAPPER, - AuthTestUtils.TEST_AUTHORIZER_MAPPER, - null, - toolbox.getRowIngestionMetersFactory(), - COORDINATOR_CLIENT, - segmentCacheManagerFactory, - RETRY_POLICY_FACTORY, - toolbox.getAppenderatorsManager() - ); - - final ObjectMapper mapper = new DefaultObjectMapper((DefaultObjectMapper) OBJECT_MAPPER); - mapper.registerSubtypes( - new NamedType(OldCompactionTaskWithAnyTuningConfigType.class, "compact"), - new NamedType(RealtimeTuningConfig.class, "realtime") - ); - final byte[] bytes = mapper.writeValueAsBytes(taskWithUnknownTuningConfig); - - expectedException.expect(ValueInstantiationException.class); - expectedException.expectCause(CoreMatchers.instanceOf(IllegalStateException.class)); - expectedException.expectMessage( - "Unknown tuningConfig type: [org.apache.druid.segment.indexing.RealtimeTuningConfig]" - ); - mapper.readValue(bytes, CompactionTask.class); - } - private static void assertEquals(CompactionTask expected, CompactionTask actual) { Assert.assertEquals(expected.getType(), actual.getType()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 2a364197b7b..f0e0198e35b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -79,6 +79,7 @@ import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.handoff.NoopSegmentHandoffNotifierFactory; import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.incremental.RowIngestionMeters; @@ -93,7 +94,6 @@ import org.apache.druid.segment.loading.SegmentLocalCacheManager; import org.apache.druid.segment.loading.StorageLocation; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; -import org.apache.druid.segment.realtime.plumber.NoopSegmentHandoffNotifierFactory; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.metrics.NoopServiceEmitter; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java deleted file mode 100644 index b41848ea57b..00000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ /dev/null @@ -1,1052 +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.common.task; - -import com.fasterxml.jackson.databind.ObjectMapper; -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.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; -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.client.coordinator.NoopCoordinatorClient; -import org.apache.druid.client.indexing.NoopOverlordClient; -import org.apache.druid.common.config.NullHandling; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.TimeAndDimsParseSpec; -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.error.DruidException; -import org.apache.druid.indexer.TaskState; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.common.SegmentCacheManagerFactory; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.TaskToolboxFactory; -import org.apache.druid.indexing.common.TestFirehose; -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.TaskConfigBuilder; -import org.apache.druid.indexing.common.config.TaskStorageConfig; -import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; -import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -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.test.TestDataSegmentAnnouncer; -import org.apache.druid.indexing.test.TestDataSegmentKiller; -import org.apache.druid.indexing.test.TestDataSegmentPusher; -import org.apache.druid.indexing.test.TestIndexerMetadataStorageCoordinator; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.UOE; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.jackson.JacksonUtils; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.common.parsers.ParseException; -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.query.DefaultQueryRunnerFactoryConglomerate; -import org.apache.druid.query.DirectQueryProcessingPool; -import org.apache.druid.query.Druids; -import org.apache.druid.query.Query; -import org.apache.druid.query.QueryPlus; -import org.apache.druid.query.QueryRunnerFactoryConglomerate; -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.LongSumAggregatorFactory; -import org.apache.druid.query.filter.DimFilter; -import org.apache.druid.query.filter.SelectorDimFilter; -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.TestHelper; -import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.handoff.SegmentHandoffNotifier; -import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.join.NoopJoinableFactory; -import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.FireDepartment; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; -import org.apache.druid.segment.realtime.plumber.ServerTimeRejectionPolicyFactory; -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.server.security.AuthTestUtils; -import org.apache.druid.testing.InitializedNullHandlingTest; -import org.apache.druid.timeline.DataSegment; -import org.easymock.EasyMock; -import org.hamcrest.CoreMatchers; -import org.joda.time.DateTime; -import org.joda.time.Period; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.internal.matchers.ThrowableMessageMatcher; -import org.junit.rules.ExpectedException; -import org.junit.rules.TemporaryFolder; - -import javax.annotation.Nullable; -import java.io.File; -import java.nio.file.Files; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executor; - -public class RealtimeIndexTaskTest extends InitializedNullHandlingTest -{ - private static final Logger log = new Logger(RealtimeIndexTaskTest.class); - private static final ServiceEmitter EMITTER = new ServiceEmitter( - "service", - "host", - new NoopEmitter() - ); - - @Rule - public final ExpectedException expectedException = ExpectedException.none(); - - @Rule - public final TemporaryFolder tempFolder = new TemporaryFolder(); - - private DateTime now; - private ListeningExecutorService taskExec; - private Map> handOffCallbacks; - - @Before - public void setUp() - { - EmittingLogger.registerEmitter(EMITTER); - EMITTER.start(); - taskExec = MoreExecutors.listeningDecorator(Execs.singleThreaded("realtime-index-task-test-%d")); - now = DateTimes.nowUtc(); - } - - @After - public void tearDown() - { - taskExec.shutdownNow(); - } - - @Test - public void testMakeTaskId() - { - Assert.assertEquals( - "index_realtime_test_0_2015-01-02T00:00:00.000Z_abcdefgh", - RealtimeIndexTask.makeTaskId("test", 0, DateTimes.of("2015-01-02"), "abcdefgh") - ); - } - - @Test(timeout = 60_000L) - public void testDefaultResource() - { - final RealtimeIndexTask task = makeRealtimeTask(null); - Assert.assertEquals(task.getId(), task.getTaskResource().getAvailabilityGroup()); - } - - @Test(timeout = 60_000L) - public void testSupportsQueries() - { - final RealtimeIndexTask task = makeRealtimeTask(null); - Assert.assertTrue(task.supportsQueries()); - } - - @Test(timeout = 60_000L) - public void testInputSourceResources() - { - final RealtimeIndexTask task = makeRealtimeTask(null); - Assert.assertThrows( - UOE.class, - task::getInputSourceResources - ); - } - - @Test(timeout = 60_000L, expected = ExecutionException.class) - public void testHandoffTimeout() throws Exception - { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); - final RealtimeIndexTask task = makeRealtimeTask(null, TransformSpec.NONE, true, 100L); - final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); - final ListenableFuture statusFuture = runTask(task, taskToolbox); - - // Wait for firehose to show up, it starts off null. - while (task.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1") - ) - ); - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - // Wait for publish. - while (mdc.getPublished().isEmpty()) { - Thread.sleep(50); - } - - Assert.assertEquals(1, task.getMetrics().processed()); - Assert.assertNotNull(Iterables.getOnlyElement(mdc.getPublished())); - - - // handoff would timeout, resulting in exception - statusFuture.get(); - } - - @Test(timeout = 60_000L) - public void testBasics() throws Exception - { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); - final RealtimeIndexTask task = makeRealtimeTask(null); - final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); - final ListenableFuture statusFuture = runTask(task, taskToolbox); - final DataSegment publishedSegment; - - // Wait for firehose to show up, it starts off null. - while (task.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), - ImmutableMap.of("t", now.minus(new Period("P1D")).getMillis(), "dim1", "foo", "met1", 2.0), - ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) - ) - ); - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - // Wait for publish. - while (mdc.getPublished().isEmpty()) { - Thread.sleep(50); - } - - publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); - - // Check metrics. - Assert.assertEquals(2, task.getMetrics().processed()); - Assert.assertEquals(1, task.getMetrics().thrownAway()); - Assert.assertEquals(0, task.getMetrics().unparseable()); - - // Do some queries. - Assert.assertEquals(2, sumMetric(task, null, "rows").longValue()); - Assert.assertEquals(3, sumMetric(task, null, "met1").longValue()); - - // Simulate handoff. - for (Map.Entry> entry : handOffCallbacks.entrySet()) { - final Pair executorRunnablePair = entry.getValue(); - Assert.assertEquals( - new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ), - entry.getKey() - ); - executorRunnablePair.lhs.execute(executorRunnablePair.rhs); - } - handOffCallbacks.clear(); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - } - - @Test(timeout = 60_000L) - public void testTransformSpec() throws Exception - { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); - final TransformSpec transformSpec = new TransformSpec( - new SelectorDimFilter("dim1", "foo", null), - ImmutableList.of( - new ExpressionTransform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) - ) - ); - final RealtimeIndexTask task = makeRealtimeTask(null, transformSpec, true, 0); - final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); - final ListenableFuture statusFuture = runTask(task, taskToolbox); - final DataSegment publishedSegment; - - // Wait for firehose to show up, it starts off null. - while (task.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), - ImmutableMap.of("t", now.minus(new Period("P1D")).getMillis(), "dim1", "foo", "met1", 2.0), - ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) - ) - ); - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - // Wait for publish. - while (mdc.getPublished().isEmpty()) { - Thread.sleep(50); - } - - publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); - - // Check metrics. - Assert.assertEquals(1, task.getMetrics().processed()); - Assert.assertEquals(2, task.getMetrics().thrownAway()); - Assert.assertEquals(0, task.getMetrics().unparseable()); - - // Do some queries. - Assert.assertEquals(1, sumMetric(task, null, "rows").longValue()); - Assert.assertEquals(1, sumMetric(task, new SelectorDimFilter("dim1t", "foofoo", null), "rows").longValue()); - if (NullHandling.replaceWithDefault()) { - Assert.assertEquals(0, sumMetric(task, new SelectorDimFilter("dim1t", "barbar", null), "rows").longValue()); - } else { - Assert.assertNull(sumMetric(task, new SelectorDimFilter("dim1t", "barbar", null), "rows")); - - } - Assert.assertEquals(1, sumMetric(task, null, "met1").longValue()); - - // Simulate handoff. - for (Map.Entry> entry : handOffCallbacks.entrySet()) { - final Pair executorRunnablePair = entry.getValue(); - Assert.assertEquals( - new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ), - entry.getKey() - ); - executorRunnablePair.lhs.execute(executorRunnablePair.rhs); - } - handOffCallbacks.clear(); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - } - - @Test(timeout = 60_000L) - public void testReportParseExceptionsOnBadMetric() throws Exception - { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); - final RealtimeIndexTask task = makeRealtimeTask(null, true); - final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); - final ListenableFuture statusFuture = runTask(task, taskToolbox); - - // Wait for firehose to show up, it starts off null. - while (task.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), - ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "foo"), - ImmutableMap.of("t", now.minus(new Period("P1D")).getMillis(), "dim1", "foo", "met1", "foo"), - ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) - ) - ); - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - // Wait for the task to finish. - expectedException.expect(ExecutionException.class); - expectedException.expectCause(CoreMatchers.instanceOf(ParseException.class)); - expectedException.expectCause( - ThrowableMessageMatcher.hasMessage( - CoreMatchers.containsString("[Unable to parse value[foo] for field[met1]") - ) - ); - - statusFuture.get(); - } - - @Test(timeout = 60_000L) - public void testNoReportParseExceptions() throws Exception - { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); - final RealtimeIndexTask task = makeRealtimeTask(null, false); - final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); - final ListenableFuture statusFuture = runTask(task, taskToolbox); - final DataSegment publishedSegment; - - // Wait for firehose to show up, it starts off null. - while (task.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task.getFirehose(); - - firehose.addRows( - Arrays.asList( - // Good row- will be processed. - ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), - - // Null row- will be thrown away. - null, - - // Bad metric- will count as processed, but that particular metric won't update. - ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "foo"), - - // Bad row- will be unparseable. - ImmutableMap.of("dim1", "foo", "met1", 2.0, TestFirehose.FAIL_DIM, "x"), - - // Old row- will be thrownAway. - ImmutableMap.of("t", now.minus(Period.days(1)).getMillis(), "dim1", "foo", "met1", 2.0), - - // Good row- will be processed. - ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) - ) - ); - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - // Wait for publish. - while (mdc.getPublished().isEmpty()) { - Thread.sleep(50); - } - - publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); - - // Check metrics. - Assert.assertEquals(3, task.getMetrics().processed()); - Assert.assertEquals(1, task.getMetrics().thrownAway()); - Assert.assertEquals(2, task.getMetrics().unparseable()); - - // Do some queries. - Assert.assertEquals(3, sumMetric(task, null, "rows").longValue()); - Assert.assertEquals(3, sumMetric(task, null, "met1").longValue()); - - // Simulate handoff. - for (Map.Entry> entry : handOffCallbacks.entrySet()) { - final Pair executorRunnablePair = entry.getValue(); - Assert.assertEquals( - new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ), - entry.getKey() - ); - executorRunnablePair.lhs.execute(executorRunnablePair.rhs); - } - handOffCallbacks.clear(); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - } - - @Test(timeout = 60_000L) - public void testRestore() throws Exception - { - final File directory = tempFolder.newFolder(); - final RealtimeIndexTask task1 = makeRealtimeTask(null); - final DataSegment publishedSegment; - - // First run: - { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); - final TaskToolbox taskToolbox = makeToolbox(task1, mdc, directory); - final ListenableFuture statusFuture = runTask(task1, taskToolbox); - - // Wait for firehose to show up, it starts off null. - while (task1.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task1.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo") - ) - ); - - // Trigger graceful shutdown. - task1.stopGracefully(taskToolbox.getConfig()); - - // Wait for the task to finish. The status doesn't really matter, but we'll check it anyway. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - - // Nothing should be published. - Assert.assertEquals(new HashSet<>(), mdc.getPublished()); - } - - // Second run: - { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); - final RealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); - final TaskToolbox taskToolbox = makeToolbox(task2, mdc, directory); - final ListenableFuture statusFuture = runTask(task2, taskToolbox); - - // Wait for firehose to show up, it starts off null. - while (task2.getFirehose() == null) { - Thread.sleep(50); - } - - // Do a query, at this point the previous data should be loaded. - Assert.assertEquals(1, sumMetric(task2, null, "rows").longValue()); - - final TestFirehose firehose = (TestFirehose) task2.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim2", "bar") - ) - ); - - // Stop the firehose, this will drain out existing events. - firehose.close(); - - // Wait for publish. - while (mdc.getPublished().isEmpty()) { - Thread.sleep(50); - } - - publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); - - // Do a query. - Assert.assertEquals(2, sumMetric(task2, null, "rows").longValue()); - - // Simulate handoff. - for (Map.Entry> entry : handOffCallbacks.entrySet()) { - final Pair executorRunnablePair = entry.getValue(); - Assert.assertEquals( - new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ), - entry.getKey() - ); - executorRunnablePair.lhs.execute(executorRunnablePair.rhs); - } - handOffCallbacks.clear(); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - } - } - - @Test(timeout = 60_000L) - public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception - { - final TaskStorage taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); - final File directory = tempFolder.newFolder(); - final RealtimeIndexTask task1 = makeRealtimeTask(null); - final DataSegment publishedSegment; - - // First run: - { - final TaskToolbox taskToolbox = makeToolbox(task1, taskStorage, mdc, directory); - final ListenableFuture statusFuture = runTask(task1, taskToolbox); - - // Wait for firehose to show up, it starts off null. - while (task1.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task1.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo") - ) - ); - - // Stop the firehose, this will trigger a finishJob. - firehose.close(); - - // Wait for publish. - while (mdc.getPublished().isEmpty()) { - Thread.sleep(50); - } - - publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); - - // Do a query. - Assert.assertEquals(1, sumMetric(task1, null, "rows").longValue()); - - // Trigger graceful shutdown. - task1.stopGracefully(taskToolbox.getConfig()); - - // Wait for the task to finish. The status doesn't really matter. - while (!statusFuture.isDone()) { - Thread.sleep(50); - } - } - - // Second run: - { - final RealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); - final TaskToolbox taskToolbox = makeToolbox(task2, taskStorage, mdc, directory); - final ListenableFuture statusFuture = runTask(task2, taskToolbox); - - // Wait for firehose to show up, it starts off null. - while (task2.getFirehose() == null) { - Thread.sleep(50); - } - - // Stop the firehose again, this will start another handoff. - final TestFirehose firehose = (TestFirehose) task2.getFirehose(); - - // Stop the firehose, this will trigger a finishJob. - firehose.close(); - - // publishedSegment is still published. No reason it shouldn't be. - Assert.assertEquals(ImmutableSet.of(publishedSegment), mdc.getPublished()); - - // Wait for a handoffCallback to show up. - while (handOffCallbacks.isEmpty()) { - Thread.sleep(50); - } - - // Simulate handoff. - for (Map.Entry> entry : handOffCallbacks.entrySet()) { - final Pair executorRunnablePair = entry.getValue(); - Assert.assertEquals( - new SegmentDescriptor( - publishedSegment.getInterval(), - publishedSegment.getVersion(), - publishedSegment.getShardSpec().getPartitionNum() - ), - entry.getKey() - ); - executorRunnablePair.lhs.execute(executorRunnablePair.rhs); - } - handOffCallbacks.clear(); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - } - } - - @Test(timeout = 60_000L) - public void testRestoreCorruptData() throws Exception - { - final File directory = tempFolder.newFolder(); - final RealtimeIndexTask task1 = makeRealtimeTask(null); - - // First run: - { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); - final TaskToolbox taskToolbox = makeToolbox(task1, mdc, directory); - final ListenableFuture statusFuture = runTask(task1, taskToolbox); - - // Wait for firehose to show up, it starts off null. - while (task1.getFirehose() == null) { - Thread.sleep(50); - } - - final TestFirehose firehose = (TestFirehose) task1.getFirehose(); - - firehose.addRows( - ImmutableList.of( - ImmutableMap.of("t", now.getMillis(), "dim1", "foo") - ) - ); - - // Trigger graceful shutdown. - task1.stopGracefully(taskToolbox.getConfig()); - - // Wait for the task to finish. The status doesn't really matter, but we'll check it anyway. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - - // Nothing should be published. - Assert.assertEquals(new HashSet<>(), mdc.getPublished()); - } - - // Corrupt the data: - final File smooshFile = new File( - StringUtils.format( - "%s/persistent/task/%s/work/persist/%s/%s_%s/0/00000.smoosh", - directory, - task1.getId(), - task1.getDataSource(), - Granularities.DAY.bucketStart(now), - Granularities.DAY.bucketEnd(now) - ) - ); - - Files.write(smooshFile.toPath(), StringUtils.toUtf8("oops!")); - - // Second run: - { - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); - final RealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); - final TaskToolbox taskToolbox = makeToolbox(task2, mdc, directory); - final ListenableFuture statusFuture = runTask(task2, taskToolbox); - - // Wait for the task to finish. - boolean caught = false; - try { - statusFuture.get(); - } - catch (Exception e) { - caught = true; - } - Assert.assertTrue("expected exception", caught); - } - } - - @Test(timeout = 60_000L) - public void testStopBeforeStarting() throws Exception - { - final File directory = tempFolder.newFolder(); - final RealtimeIndexTask task1 = makeRealtimeTask(null); - - final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); - final TaskToolbox taskToolbox = makeToolbox(task1, mdc, directory); - task1.stopGracefully(taskToolbox.getConfig()); - final ListenableFuture statusFuture = runTask(task1, taskToolbox); - - // Wait for the task to finish. - final TaskStatus taskStatus = statusFuture.get(); - Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); - } - - private ListenableFuture runTask(final Task task, final TaskToolbox toolbox) - { - return taskExec.submit( - new Callable() - { - @Override - public TaskStatus call() throws Exception - { - 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"); - throw e; - } - } - } - ); - } - - private RealtimeIndexTask makeRealtimeTask(final String taskId) - { - return makeRealtimeTask(taskId, TransformSpec.NONE, true, 0); - } - - private RealtimeIndexTask makeRealtimeTask(final String taskId, boolean reportParseExceptions) - { - return makeRealtimeTask(taskId, TransformSpec.NONE, reportParseExceptions, 0); - } - - private RealtimeIndexTask makeRealtimeTask( - final String taskId, - final TransformSpec transformSpec, - final boolean reportParseExceptions, - final long handoffTimeout - ) - { - ObjectMapper objectMapper = new DefaultObjectMapper(); - DataSchema dataSchema = new DataSchema( - "test_ds", - TestHelper.makeJsonMapper().convertValue( - new MapInputRowParser( - new TimeAndDimsParseSpec( - new TimestampSpec("t", "auto", null), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim1t")) - ) - ) - ), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ), - new AggregatorFactory[]{new CountAggregatorFactory("rows"), new LongSumAggregatorFactory("met1", "met1")}, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - transformSpec, - objectMapper - ); - RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig( - new TestFirehose.TestFirehoseFactory(), - null - ); - RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig( - null, - 1000, - null, - null, - new Period("P1Y"), - new Period("PT10M"), - null, - null, - new ServerTimeRejectionPolicyFactory(), - null, - null, - null, - null, - 0, - 0, - reportParseExceptions, - handoffTimeout, - null, - null, - null, - null - ); - return new RealtimeIndexTask( - taskId, - null, - new FireDepartment(dataSchema, realtimeIOConfig, realtimeTuningConfig), - null - ) - { - @Override - protected boolean isFirehoseDrainableByClosing(FirehoseFactory firehoseFactory) - { - return true; - } - }; - } - - private TaskToolbox makeToolbox( - final Task task, - final IndexerMetadataStorageCoordinator mdc, - final File directory - ) - { - return makeToolbox( - task, - new HeapMemoryTaskStorage(new TaskStorageConfig(null)), - mdc, - directory - ); - } - - private TaskToolbox makeToolbox( - final Task task, - final TaskStorage taskStorage, - final IndexerMetadataStorageCoordinator mdc, - final File directory - ) - { - final TaskConfig taskConfig = new TaskConfigBuilder() - .setBaseDir(directory.getPath()) - .setDefaultRowFlushBoundary(50000) - .setRestoreTasksOnRestart(true) - .setBatchProcessingMode(TaskConfig.BATCH_PROCESSING_MODE_DEFAULT.name()) - .build(); - final TaskLockbox taskLockbox = new TaskLockbox(taskStorage, mdc); - try { - taskStorage.insert(task, TaskStatus.running(task.getId())); - } - catch (DruidException e) { - log.noStackTrace().info(e, "Suppressing exception while inserting task [%s]", task.getId()); - } - taskLockbox.syncFromStorage(); - final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( - taskLockbox, - taskStorage, - mdc, - EMITTER, - EasyMock.createMock(SupervisorManager.class), - new DefaultObjectMapper() - ); - final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( - taskStorage, - taskActionToolbox, - new TaskAuditLogConfig(false) - ); - final QueryRunnerFactoryConglomerate conglomerate = new DefaultQueryRunnerFactoryConglomerate( - ImmutableMap.of( - TimeseriesQuery.class, - new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(), - new TimeseriesQueryEngine(), - new QueryWatcher() - { - @Override - public void registerQueryFuture(Query query, ListenableFuture future) - { - // do nothing - } - } - ) - ) - ); - handOffCallbacks = new ConcurrentHashMap<>(); - final SegmentHandoffNotifierFactory handoffNotifierFactory = new SegmentHandoffNotifierFactory() - { - @Override - public SegmentHandoffNotifier createSegmentHandoffNotifier(String dataSource) - { - return new SegmentHandoffNotifier() - { - @Override - public boolean registerSegmentHandoffCallback( - SegmentDescriptor descriptor, - Executor exec, - Runnable handOffRunnable - ) - { - handOffCallbacks.put(descriptor, new Pair<>(exec, handOffRunnable)); - return true; - } - - @Override - public void start() - { - //Noop - } - - @Override - public void close() - { - //Noop - } - - }; - } - }; - final TestUtils testUtils = new TestUtils(); - final TaskToolboxFactory toolboxFactory = new TaskToolboxFactory( - null, - taskConfig, - null, // taskExecutorNode - taskActionClientFactory, - EMITTER, - new TestDataSegmentPusher(), - new TestDataSegmentKiller(), - null, // DataSegmentMover - null, // DataSegmentArchiver - new TestDataSegmentAnnouncer(), - EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), - handoffNotifierFactory, - () -> conglomerate, - DirectQueryProcessingPool.INSTANCE, - NoopJoinableFactory.INSTANCE, - () -> EasyMock.createMock(MonitorScheduler.class), - new SegmentCacheManagerFactory(TestIndex.INDEX_IO, testUtils.getTestObjectMapper()), - testUtils.getTestObjectMapper(), - testUtils.getTestIndexIO(), - MapCache.create(1024), - new CacheConfig(), - new CachePopulatorStats(), - testUtils.getIndexMergerV9Factory(), - EasyMock.createNiceMock(DruidNodeAnnouncer.class), - EasyMock.createNiceMock(DruidNode.class), - new LookupNodeService("tier"), - new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0), - new NoopTestTaskReportFileWriter(), - null, - AuthTestUtils.TEST_AUTHORIZER_MAPPER, - new NoopChatHandlerProvider(), - testUtils.getRowIngestionMetersFactory(), - new TestAppenderatorsManager(), - new NoopOverlordClient(), - new NoopCoordinatorClient(), - null, - null, - null, - "1", - CentralizedDatasourceSchemaConfig.create() - ); - - return toolboxFactory.build(task); - } - - @Nullable - public Long sumMetric(final Task task, final DimFilter filter, final String metric) - { - // Do a query. - TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() - .dataSource("test_ds") - .filters(filter) - .aggregators( - ImmutableList.of( - new LongSumAggregatorFactory(metric, metric) - ) - ).granularity(Granularities.ALL) - .intervals("2000/3000") - .build(); - - List> results = task.getQueryRunner(query).run(QueryPlus.wrap(query)).toList(); - if (results.isEmpty()) { - return 0L; - } else { - return results.get(0).getValue().getLongMetric(metric); - } - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java index 8c0cbd92562..b217f8d5bd8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java @@ -47,12 +47,8 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.realtime.FireDepartment; import org.apache.druid.server.security.AuthTestUtils; -import org.apache.druid.timeline.partition.NoneShardSpec; import org.hamcrest.CoreMatchers; import org.joda.time.Period; import org.junit.Assert; @@ -387,92 +383,6 @@ public class TaskSerdeTest Assert.assertTrue(task2.getIngestionSchema().getIOConfig().getInputSource() instanceof LocalInputSource); } - @Test - public void testRealtimeIndexTaskSerde() throws Exception - { - - final RealtimeIndexTask task = new RealtimeIndexTask( - null, - new TaskResource("rofl", 2), - new FireDepartment( - new DataSchema( - "foo", - null, - new AggregatorFactory[0], - new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null), - null, - jsonMapper - ), - new RealtimeIOConfig( - new MockFirehoseFactory(), - (schema, config, metrics) -> null - ), - - new RealtimeTuningConfig( - null, - 1, - 10L, - null, - new Period("PT10M"), - null, - null, - null, - null, - 1, - NoneShardSpec.instance(), - indexSpec, - null, - 0, - 0, - true, - null, - null, - null, - null, - null - ) - ), - null - ); - - final String json = jsonMapper.writeValueAsString(task); - - Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change - final RealtimeIndexTask task2 = (RealtimeIndexTask) jsonMapper.readValue(json, Task.class); - - Assert.assertEquals("foo", task.getDataSource()); - Assert.assertEquals(2, task.getTaskResource().getRequiredCapacity()); - Assert.assertEquals("rofl", task.getTaskResource().getAvailabilityGroup()); - Assert.assertEquals( - new Period("PT10M"), - task.getRealtimeIngestionSchema() - .getTuningConfig().getWindowPeriod() - ); - Assert.assertEquals( - Granularities.HOUR, - task.getRealtimeIngestionSchema().getDataSchema().getGranularitySpec().getSegmentGranularity() - ); - Assert.assertTrue(task.getRealtimeIngestionSchema().getTuningConfig().isReportParseExceptions()); - - Assert.assertEquals(task.getId(), task2.getId()); - Assert.assertEquals(task.getGroupId(), task2.getGroupId()); - Assert.assertEquals(task.getDataSource(), task2.getDataSource()); - Assert.assertEquals(task.getTaskResource().getRequiredCapacity(), task2.getTaskResource().getRequiredCapacity()); - Assert.assertEquals(task.getTaskResource().getAvailabilityGroup(), task2.getTaskResource().getAvailabilityGroup()); - Assert.assertEquals( - task.getRealtimeIngestionSchema().getTuningConfig().getWindowPeriod(), - task2.getRealtimeIngestionSchema().getTuningConfig().getWindowPeriod() - ); - Assert.assertEquals( - task.getRealtimeIngestionSchema().getTuningConfig().getMaxBytesInMemory(), - task2.getRealtimeIngestionSchema().getTuningConfig().getMaxBytesInMemory() - ); - Assert.assertEquals( - task.getRealtimeIngestionSchema().getDataSchema().getGranularitySpec().getSegmentGranularity(), - task2.getRealtimeIngestionSchema().getDataSchema().getGranularitySpec().getSegmentGranularity() - ); - } - @Test public void testArchiveTaskSerde() throws Exception { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java index 515b9f350ef..e6186329126 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java @@ -38,7 +38,7 @@ import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; import org.apache.druid.segment.realtime.appenderator.Appenderators; @@ -56,7 +56,7 @@ public class TestAppenderatorsManager implements AppenderatorsManager String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -105,7 +105,7 @@ public class TestAppenderatorsManager implements AppenderatorsManager String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -137,7 +137,7 @@ public class TestAppenderatorsManager implements AppenderatorsManager String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -169,7 +169,7 @@ public class TestAppenderatorsManager implements AppenderatorsManager String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index f8f9806abde..91d74a2bd67 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -47,9 +47,7 @@ import org.apache.druid.data.input.InputStats; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.MapInputRowParser; import org.apache.druid.data.input.impl.NoopInputFormat; -import org.apache.druid.data.input.impl.TimeAndDimsParseSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.DruidNodeAnnouncer; @@ -83,7 +81,6 @@ import org.apache.druid.indexing.common.task.KillUnusedSegmentsTask; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.NoopTaskContextEnricher; import org.apache.druid.indexing.common.task.NoopTestTaskReportFileWriter; -import org.apache.druid.indexing.common.task.RealtimeIndexTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.TestAppenderatorsManager; @@ -103,15 +100,12 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.Stopwatch; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Comparators; -import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.java.util.metrics.Monitor; import org.apache.druid.java.util.metrics.MonitorScheduler; import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import org.apache.druid.metadata.TestDerbyConnector; @@ -121,18 +115,14 @@ import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9Factory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.SegmentSchemaMapping; -import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.JoinableFactoryWrapperTest; import org.apache.druid.segment.join.NoopJoinableFactory; @@ -141,8 +131,6 @@ import org.apache.druid.segment.loading.LocalDataSegmentKiller; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.NoopDataSegmentArchiver; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.FireDepartment; -import org.apache.druid.segment.realtime.FireDepartmentTest; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; @@ -651,7 +639,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest MAPPER, INDEX_IO, MapCache.create(0), - FireDepartmentTest.NO_CACHE_CONFIG, + new CacheConfig(), new CachePopulatorStats(), INDEX_MERGER_V9_FACTORY, EasyMock.createNiceMock(DruidNodeAnnouncer.class), @@ -1251,114 +1239,6 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest Assert.assertEquals("segments nuked", 0, mdc.getNuked().size()); } - @Test(timeout = 60_000L) - public void testRealtimeIndexTask() throws Exception - { - publishCountDown = new CountDownLatch(1); - monitorScheduler.addMonitor(EasyMock.anyObject(Monitor.class)); - EasyMock.expectLastCall().times(1); - monitorScheduler.removeMonitor(EasyMock.anyObject(Monitor.class)); - EasyMock.expectLastCall().times(1); - EasyMock.replay(monitorScheduler, queryRunnerFactoryConglomerate); - - RealtimeIndexTask realtimeIndexTask = newRealtimeIndexTask(); - final String taskId = realtimeIndexTask.getId(); - - taskQueue.start(); - taskQueue.add(realtimeIndexTask); - //wait for task to process events and publish segment - publishCountDown.await(); - - // Realtime Task has published the segment, simulate loading of segment to a historical node so that task finishes with SUCCESS status - Assert.assertEquals(1, handOffCallbacks.size()); - Pair executorRunnablePair = Iterables.getOnlyElement(handOffCallbacks.values()); - executorRunnablePair.lhs.execute(executorRunnablePair.rhs); - handOffCallbacks.clear(); - - // Wait for realtime index task to handle callback in plumber and succeed - while (tsqa.getStatus(taskId).get().isRunnable()) { - Thread.sleep(10); - } - - TaskStatus status = tsqa.getStatus(taskId).get(); - Assert.assertTrue("Task should be in Success state", status.isSuccess()); - Assert.assertEquals(taskLocation, status.getLocation()); - - Assert.assertEquals(1, announcedSinks); - Assert.assertEquals(1, pushedSegments); - Assert.assertEquals(1, mdc.getPublished().size()); - DataSegment segment = mdc.getPublished().iterator().next(); - Assert.assertEquals("test_ds", segment.getDataSource()); - Assert.assertEquals(ImmutableList.of("dim1", "dim2"), segment.getDimensions()); - Assert.assertEquals( - Intervals.of(now.toString("YYYY-MM-dd") + "/" + now.plusDays(1).toString("YYYY-MM-dd")), - segment.getInterval() - ); - Assert.assertEquals(ImmutableList.of("count"), segment.getMetrics()); - EasyMock.verify(monitorScheduler, queryRunnerFactoryConglomerate); - } - - @Test(timeout = 60_000L) - public void testRealtimeIndexTaskFailure() throws Exception - { - dataSegmentPusher = new DataSegmentPusher() - { - @Deprecated - @Override - public String getPathForHadoop(String s) - { - return getPathForHadoop(); - } - - @Override - public String getPathForHadoop() - { - throw new UnsupportedOperationException(); - } - - @Override - public DataSegment push(File file, DataSegment dataSegment, boolean useUniquePath) - { - throw new RuntimeException("FAILURE"); - } - - @Override - public Map makeLoadSpec(URI uri) - { - throw new UnsupportedOperationException(); - } - }; - - tb = setUpTaskToolboxFactory(dataSegmentPusher, handoffNotifierFactory, mdc); - - taskRunner = setUpThreadPoolTaskRunner(tb); - - taskQueue = setUpTaskQueue(taskStorage, taskRunner); - - monitorScheduler.addMonitor(EasyMock.anyObject(Monitor.class)); - EasyMock.expectLastCall().times(1); - monitorScheduler.removeMonitor(EasyMock.anyObject(Monitor.class)); - EasyMock.expectLastCall().times(1); - EasyMock.replay(monitorScheduler, queryRunnerFactoryConglomerate); - - RealtimeIndexTask realtimeIndexTask = newRealtimeIndexTask(); - final String taskId = realtimeIndexTask.getId(); - - taskQueue.start(); - taskQueue.add(realtimeIndexTask); - - // Wait for realtime index task to fail - while (tsqa.getStatus(taskId).get().isRunnable()) { - Thread.sleep(10); - } - - TaskStatus status = tsqa.getStatus(taskId).get(); - Assert.assertTrue("Task should be in Failure state", status.isFailure()); - Assert.assertEquals(taskLocation, status.getLocation()); - - EasyMock.verify(monitorScheduler, queryRunnerFactoryConglomerate); - } - @Test public void testResumeTasks() throws Exception { @@ -1629,60 +1509,4 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest return retVal; } - - private RealtimeIndexTask newRealtimeIndexTask() - { - String taskId = StringUtils.format("rt_task_%s", System.currentTimeMillis()); - DataSchema dataSchema = new DataSchema( - "test_ds", - TestHelper.makeJsonMapper().convertValue( - new MapInputRowParser( - new TimeAndDimsParseSpec( - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY - ) - ), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ), - new AggregatorFactory[]{new LongSumAggregatorFactory("count", "rows")}, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - mapper - ); - RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig( - new MockFirehoseFactory(), - null - // PlumberSchool - Realtime Index Task always uses RealtimePlumber which is hardcoded in RealtimeIndexTask class - ); - RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig( - null, - 1000, - null, - null, - new Period("P1Y"), - null, //default window period of 10 minutes - null, // base persist dir ignored by Realtime Index task - null, - null, - null, - null, - null, - null, - 0, - 0, - null, - null, - null, - null, - null, - null - ); - FireDepartment fireDepartment = new FireDepartment(dataSchema, realtimeIOConfig, realtimeTuningConfig); - return new RealtimeIndexTask( - taskId, - new TaskResource(taskId, 1), - fireDepartment, - null - ); - } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java index c61bb40bfd7..ca3cd6edf8b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java @@ -20,18 +20,16 @@ package org.apache.druid.indexing.worker; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.impl.NoopInputSource; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TestUtils; -import org.apache.druid.indexing.common.task.RealtimeIndexTask; +import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; -import org.apache.druid.segment.realtime.FireDepartment; -import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; @@ -48,13 +46,13 @@ public class TaskAnnouncementTest @Test public void testBackwardsCompatibleSerde() throws Exception { - final RealtimeIOConfig realtimeIOConfig = EasyMock.createNiceMock(RealtimeIOConfig.class); - final Task task = new RealtimeIndexTask( + final IndexTask.IndexIOConfig ioConfig = new IndexTask.IndexIOConfig(null, new NoopInputSource(), null, null, null); + final Task task = new IndexTask( "theid", new TaskResource("rofl", 2), - new FireDepartment( + new IndexTask.IndexIngestionSpec( new DataSchema("foo", null, new AggregatorFactory[0], null, null, new DefaultObjectMapper()), - realtimeIOConfig, + ioConfig, null ), null diff --git a/integration-tests-ex/cases/cluster.sh b/integration-tests-ex/cases/cluster.sh index 1ff1cdf1307..bc626a559c4 100755 --- a/integration-tests-ex/cases/cluster.sh +++ b/integration-tests-ex/cases/cluster.sh @@ -252,7 +252,7 @@ case $CMD in cd $COMPOSE_DIR $DOCKER_COMPOSE $DOCKER_ARGS up -d # Enable the following for debugging - #show_status + # show_status ;; "status" ) check_env_file diff --git a/integration-tests-ex/cases/cluster/Common/dependencies.yaml b/integration-tests-ex/cases/cluster/Common/dependencies.yaml index 10b01c46896..0409c30bf53 100644 --- a/integration-tests-ex/cases/cluster/Common/dependencies.yaml +++ b/integration-tests-ex/cases/cluster/Common/dependencies.yaml @@ -46,28 +46,27 @@ services: kafka: image: bitnami/kafka:${KAFKA_VERSION} container_name: kafka + # platform: linux/x86_64 labels: druid-int-test: "true" ports: - 9092:9092 - - 9093:9093 + - 9094:9094 networks: druid-it-net: ipv4_address: 172.172.172.2 volumes: - ${SHARED_DIR}/kafka:/bitnami/kafka environment: - # This is the default: making it explicit - KAFKA_CFG_ZOOKEEPER_CONNECT: zookeeper:2181 - # Plaintext is disabled by default - ALLOW_PLAINTEXT_LISTENER: "yes" - # Adapted from base-setup.sh and Bitnami docs - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP: "INTERNAL:PLAINTEXT,EXTERNAL:PLAINTEXT" - KAFKA_CFG_LISTENERS: "INTERNAL://:9092,EXTERNAL://:9093" - KAFKA_CFG_ADVERTISED_LISTENERS: "INTERNAL://kafka:9092,EXTERNAL://localhost:9093" - KAFKA_CFG_INTER_BROKER_LISTENER_NAME: INTERNAL - depends_on: - - zookeeper + - KAFKA_CFG_NODE_ID=1001 + - KAFKA_CFG_PROCESS_ROLES=controller,broker + - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093 + - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT + - KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=1001@kafka:9093 + - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093,EXTERNAL://:9094 + - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://kafka:9092,EXTERNAL://localhost:9094 + - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT + - KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER # Uses the official MySQL image # See https://hub.docker.com/_/mysql diff --git a/integration-tests-ex/cases/cluster/Query/docker-compose.yaml b/integration-tests-ex/cases/cluster/Query/docker-compose.yaml new file mode 100644 index 00000000000..0b7c461e126 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Query/docker-compose.yaml @@ -0,0 +1,105 @@ +# 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. + +networks: + druid-it-net: + name: druid-it-net + ipam: + config: + - subnet: 172.172.172.0/24 + +services: + zookeeper: + extends: + file: ../Common/dependencies.yaml + service: zookeeper + + metadata: + extends: + file: ../Common/dependencies.yaml + service: metadata + + coordinator: + extends: + file: ../Common/druid.yaml + service: coordinator + container_name: coordinator + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + # The frequency with which the coordinator polls the database + # for changes. The DB population code has to wait at least this + # long for the coordinator to notice changes. + - druid_manager_segments_pollDuration=PT5S + - druid_coordinator_period=PT10S + depends_on: + - zookeeper + - metadata + + overlord: + extends: + file: ../Common/druid.yaml + service: overlord + container_name: overlord + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - zookeeper + - metadata + + broker: + extends: + file: ../Common/druid.yaml + service: broker + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - zookeeper + + router: + extends: + file: ../Common/druid.yaml + service: router + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - zookeeper + + historical: + extends: + file: ../Common/druid.yaml + service: historical + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - zookeeper + + middlemanager: + extends: + file: ../Common/druid.yaml + service: middlemanager + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + volumes: + # Test data + - ../../resources:/resources + depends_on: + - zookeeper + + kafka: + extends: + file: ../Common/dependencies.yaml + service: kafka + depends_on: + - zookeeper diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml index 2ec2eabc3b3..5e38ac407f8 100644 --- a/integration-tests-ex/cases/pom.xml +++ b/integration-tests-ex/cases/pom.xml @@ -74,6 +74,11 @@ druid-indexing-service ${project.parent.version} + + org.apache.druid.extensions + druid-kafka-indexing-service + ${project.parent.version} + com.google.inject guice @@ -399,6 +404,15 @@ AzureDeepStorage + + IT-Query + + false + + + Query + + IT-MultiStageQuery diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentPublisher.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/Query.java similarity index 83% rename from server/src/main/java/org/apache/druid/metadata/MetadataSegmentPublisher.java rename to integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/Query.java index 34f98e8e381..252d1325f03 100644 --- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentPublisher.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/Query.java @@ -17,10 +17,8 @@ * under the License. */ -package org.apache.druid.metadata; +package org.apache.druid.testsEx.categories; -import org.apache.druid.segment.realtime.SegmentPublisher; - -public interface MetadataSegmentPublisher extends SegmentPublisher +public class Query { } diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java index e23c996c884..0725a09fd91 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java @@ -62,13 +62,13 @@ class IntegrationTestingConfigEx implements IntegrationTestingConfig @Override public String getKafkaHost() { - return config.requireKafka().instance().clientHost(); + return config.requireKafka().instance().clientHost() + ":" + config.requireKafka().instance().clientPort(); } @Override public String getKafkaInternalHost() { - return config.requireKafka().instance().host(); + return config.requireKafka().instance().host() + ":" + config.requireKafka().instance().port(); } @Override diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/query/ITUnionQueryTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/query/ITUnionQueryTest.java new file mode 100644 index 00000000000..2d0d8e4f50e --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/query/ITUnionQueryTest.java @@ -0,0 +1,213 @@ +/* + * 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.testsEx.query; + +import org.apache.druid.indexing.kafka.KafkaConsumerConfigs; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.utils.ITRetryUtil; +import org.apache.druid.testing.utils.KafkaAdminClient; +import org.apache.druid.testing.utils.KafkaEventWriter; +import org.apache.druid.testing.utils.KafkaUtil; +import org.apache.druid.testing.utils.StreamEventWriter; +import org.apache.druid.testsEx.categories.Query; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.apache.druid.testsEx.indexer.AbstractIndexerTest; +import org.joda.time.Interval; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.BufferedReader; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.function.Function; + +@RunWith(DruidTestRunner.class) +@Category(Query.class) +public class ITUnionQueryTest extends AbstractIndexerTest +{ + private static final Logger LOG = new Logger(ITUnionQueryTest.class); + private static final String UNION_SUPERVISOR_TEMPLATE = "/query/union_kafka_supervisor_template.json"; + private static final String UNION_DATA_FILE = "/query/union_data.json"; + private static final String UNION_QUERIES_RESOURCE = "/query/union_queries.json"; + private static final String UNION_DATASOURCE = "wikipedia_index_test"; + private String fullDatasourceName; + + @Test + public void testUnionQuery() throws Exception + { + fullDatasourceName = UNION_DATASOURCE + config.getExtraDatasourceNameSuffix(); + final String baseName = fullDatasourceName + UUID.randomUUID(); + KafkaAdminClient streamAdminClient = new KafkaAdminClient(config); + List supervisors = new ArrayList<>(); + + final int numDatasources = 3; + for (int i = 0; i < numDatasources; i++) { + String datasource = baseName + "-" + i; + streamAdminClient.createStream(datasource, 1, Collections.emptyMap()); + ITRetryUtil.retryUntil( + () -> streamAdminClient.isStreamActive(datasource), + true, + 10000, + 30, + "Wait for stream active" + ); + String supervisorSpec = generateStreamIngestionPropsTransform( + datasource, + datasource, + config + ).apply(getResourceAsString(UNION_SUPERVISOR_TEMPLATE)); + LOG.info("supervisorSpec: [%s]\n", supervisorSpec); + // Start supervisor + String specResponse = indexer.submitSupervisor(supervisorSpec); + LOG.info("Submitted supervisor [%s]", specResponse); + supervisors.add(specResponse); + + int ctr = 0; + try ( + StreamEventWriter streamEventWriter = new KafkaEventWriter(config, false); + BufferedReader reader = new BufferedReader( + new InputStreamReader(getResourceAsStream(UNION_DATA_FILE), StandardCharsets.UTF_8) + ) + ) { + String line; + while ((line = reader.readLine()) != null) { + streamEventWriter.write(datasource, StringUtils.toUtf8(line)); + ctr++; + } + } + final int numWritten = ctr; + + LOG.info("Waiting for stream indexing tasks to consume events"); + + ITRetryUtil.retryUntilTrue( + () -> + numWritten == this.queryHelper.countRows( + datasource, + Intervals.ETERNITY, + name -> new LongSumAggregatorFactory(name, "count") + ), + StringUtils.format( + "dataSource[%s] consumed [%,d] events, expected [%,d]", + datasource, + this.queryHelper.countRows( + datasource, + Intervals.ETERNITY, + name -> new LongSumAggregatorFactory(name, "count") + ), + numWritten + ) + ); + } + + String queryResponseTemplate = StringUtils.replace( + getResourceAsString(UNION_QUERIES_RESOURCE), + "%%DATASOURCE%%", + baseName + ); + + queryHelper.testQueriesFromString(queryResponseTemplate); + + + for (int i = 0; i < numDatasources; i++) { + indexer.terminateSupervisor(supervisors.get(i)); + streamAdminClient.deleteStream(baseName + "-" + i); + } + + for (int i = 0; i < numDatasources; i++) { + final int datasourceNumber = i; + ITRetryUtil.retryUntil( + () -> coordinator.areSegmentsLoaded(baseName + "-" + datasourceNumber), + true, + 10000, + 10, + "Kafka segments loaded" + ); + } + + queryHelper.testQueriesFromString(queryResponseTemplate); + + for (int i = 0; i < numDatasources; i++) { + final String datasource = baseName + "-" + i; + List intervals = coordinator.getSegmentIntervals(datasource); + + Collections.sort(intervals); + String first = intervals.get(0).split("/")[0]; + String last = intervals.get(intervals.size() - 1).split("/")[1]; + Interval interval = Intervals.of(first + "/" + last); + coordinator.unloadSegmentsForDataSource(baseName + "-" + i); + ITRetryUtil.retryUntilFalse( + () -> coordinator.areSegmentsLoaded(datasource), + "Segment Unloading" + ); + coordinator.deleteSegmentsDataSource(baseName + "-" + i, interval); + } + } + + + /** + * sad version of + * {@link org.apache.druid.tests.indexer.AbstractKafkaIndexingServiceTest#generateStreamIngestionPropsTransform} + */ + private Function generateStreamIngestionPropsTransform( + String streamName, + String fullDatasourceName, + IntegrationTestingConfig config + ) + { + final Map consumerConfigs = KafkaConsumerConfigs.getConsumerProperties(); + final Properties consumerProperties = new Properties(); + consumerProperties.putAll(consumerConfigs); + consumerProperties.setProperty("bootstrap.servers", config.getKafkaInternalHost()); + KafkaUtil.addPropertiesFromTestConfig(config, consumerProperties); + return spec -> { + try { + spec = StringUtils.replace( + spec, + "%%DATASOURCE%%", + fullDatasourceName + ); + spec = StringUtils.replace( + spec, + "%%TOPIC_VALUE%%", + streamName + ); + return StringUtils.replace( + spec, + "%%STREAM_PROPERTIES_VALUE%%", + jsonMapper.writeValueAsString(consumerProperties) + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + }; + } +} diff --git a/integration-tests-ex/cases/src/test/resources/cluster/Common/kafka.yaml b/integration-tests-ex/cases/src/test/resources/cluster/Common/kafka.yaml index 769c80989a4..5722cd72072 100644 --- a/integration-tests-ex/cases/src/test/resources/cluster/Common/kafka.yaml +++ b/integration-tests-ex/cases/src/test/resources/cluster/Common/kafka.yaml @@ -19,4 +19,4 @@ kafka: instances: - port: 9092 - proxyPort: 9093 + proxyPort: 9094 diff --git a/integration-tests-ex/cases/src/test/resources/cluster/Query/docker.yaml b/integration-tests-ex/cases/src/test/resources/cluster/Query/docker.yaml new file mode 100644 index 00000000000..e217490df5d --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/cluster/Query/docker.yaml @@ -0,0 +1,199 @@ +# 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. +#------------------------------------------------------------------------- + +# Definition of the query test cluster. +# See https://yaml.org/spec/1.2.2 for more about YAML +include: + - /cluster/Common/zk-metastore.yaml + - /cluster/Common/kafka.yaml + +druid: + coordinator: + instances: + - port: 8081 + overlord: + instances: + - port: 8090 + broker: + instances: + - port: 8082 + router: + instances: + - port: 8888 + historical: + instances: + - port: 8083 + middlemanager: + instances: + - port: 8091 + +# Metastore initialization queries. +# REPLACE is used so that the statements are idempotent +# The fancy formatting is for human consumption, it is compacted internally +metastoreInit: + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9', + 'twitterstream', + '2013-05-13T01:08:18.192Z', + '2013-01-01T00:00:00.000Z', + '2013-01-02T00:00:00.000Z', + 0, + '2013-01-02T04:13:41.980Z_v9', + 1, + '{"dataSource": "twitterstream", + "interval": "2013-01-01T00:00:00.000Z/2013-01-02T00:00:00.000Z", + "version": "2013-01-02T04:13:41.980Z_v9", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/twitterstream/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/2013-01-02T04:13:41.980Z_v9/0/index.zip" + }, + "dimensions": "has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang, + rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name", + "metrics": "count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets", + "shardSpec": {"type": "none"}, + "binaryVersion": 9, + "size": 445235220, + "identifier": "twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9" + }' + ) + + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9', + 'twitterstream', + '2013-05-13T00:03:28.640Z', + '2013-01-02T00:00:00.000Z', + '2013-01-03T00:00:00.000Z', + 0, + '2013-01-03T03:44:58.791Z_v9', + 1, + '{"dataSource": "twitterstream", + "interval": "2013-01-02T00:00:00.000Z/2013-01-03T00:00:00.000Z", + "version": "2013-01-03T03:44:58.791Z_v9", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/twitterstream/2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z/2013-01-03T03:44:58.791Z_v9/0/index.zip" + }, + "dimensions": "has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang, + rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name", + "metrics": "count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets", + "shardSpec": {"type": "none"}, + "binaryVersion": 9, + "size": 435325540, + "identifier": "twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9" + }' + ) + + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9', + 'twitterstream', + '2013-05-13T00:03:48.807Z', + '2013-01-03T00:00:00.000Z', + '2013-01-04T00:00:00.000Z', + 0, + '2013-01-04T04:09:13.590Z_v9', + 1, + '{"dataSource": "twitterstream", + "interval": "2013-01-03T00:00:00.000Z/2013-01-04T00:00:00.000Z", + "version": "2013-01-04T04:09:13.590Z_v9", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/twitterstream/2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z/2013-01-04T04:09:13.590Z_v9/0/index.zip" + }, + "dimensions": "has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang, + rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name", + "metrics": "count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets", + "shardSpec": {"type": "none"}, + "binaryVersion": 9, + "size": 411651320, + "identifier": "twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9" + }' + ) + + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9', + 'wikipedia_editstream', + '2013-03-15T20:49:52.348Z', + '2012-12-29T00:00:00.000Z', + '2013-01-10T08:00:00.000Z', + 0, + '2013-01-10T08:13:47.830Z_v9', + 1, + '{"dataSource": "wikipedia_editstream", + "interval": "2012-12-29T00:00:00.000Z/2013-01-10T08:00:00.000Z", + "version": "2013-01-10T08:13:47.830Z_v9", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/wikipedia_editstream/2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z/2013-01-10T08:13:47.830Z_v9/0/index.zip"}, + "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": "wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9" + }' + ) + + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z', + 'wikipedia', + '2013-08-08T21:26:23.799Z', + '2013-08-01T00:00:00.000Z', + '2013-08-02T00:00:00.000Z', + 0, + '2013-08-08T21:22:48.989Z', + 1, + '{"dataSource": "wikipedia", + "interval": "2013-08-01T00:00:00.000Z/2013-08-02T00:00:00.000Z", + "version": "2013-08-08T21:22:48.989Z", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/wikipedia/20130801T000000.000Z_20130802T000000.000Z/2013-08-08T21_22_48.989Z/0/index.zip" + }, + "dimensions": "dma_code,continent_code,geo,area_code,robot,country_name,network,city,namespace, + anonymous,unpatrolled,page,postal_code,language,newpage,user,region_lookup", + "metrics": "count,delta,variation,added,deleted", + "shardSpec": {"type": "none"}, + "binaryVersion": 9, + "size": 24664730, + "identifier": "wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z" + }' + ) diff --git a/integration-tests-ex/cases/src/test/resources/indexer/broadcast_join_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/broadcast_join_index_task.json index 20c3b162ea6..699ca547ff4 100644 --- a/integration-tests-ex/cases/src/test/resources/indexer/broadcast_join_index_task.json +++ b/integration-tests-ex/cases/src/test/resources/indexer/broadcast_join_index_task.json @@ -61,7 +61,7 @@ "type": "index_parallel", "inputSource": { "type": "local", - "baseDir": "/resources/data/union_query/", + "baseDir": "/resources/data/broadcast/", "filter": "wikipedia_index_data*" }, "appendToExisting": false, diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_task.json deleted file mode 100644 index 9e773609cb7..00000000000 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_task.json +++ /dev/null @@ -1,94 +0,0 @@ -{ - "type": "index_realtime_appenderator", - "spec": { - "dataSchema": { - "dataSource": "%%DATASOURCE%%", - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - } - ], - "granularitySpec": { - "segmentGranularity": "minute", - "queryGranularity": "second" - }, - "parser": { - "type": "map", - "parseSpec": { - "format": "tsv", - "columns": [ - "timestamp", - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city", - "added", - "deleted", - "delta" - ], - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } - } - }, - "ioConfig": { - "type": "realtime", - "firehose": { - "type": "timed", - "shutoffTime": "#SHUTOFFTIME", - "delegate": { - "type": "receiver", - "serviceName": "eventReceiverServiceName", - "bufferSize": 100000 - } - } - }, - "tuningConfig": { - "type": "realtime_appenderator", - "maxRowsInMemory": 1, - "intermediatePersistPeriod": "PT1M" - } - } -} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_task.json deleted file mode 100644 index 5f48162c488..00000000000 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_task.json +++ /dev/null @@ -1,98 +0,0 @@ -{ - "type": "index_realtime", - "spec": { - "dataSchema": { - "dataSource": "%%DATASOURCE%%", - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - } - ], - "granularitySpec": { - "segmentGranularity": "minute", - "queryGranularity": "second" - }, - "parser": { - "type": "map", - "parseSpec": { - "format": "tsv", - "columns": [ - "timestamp", - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city", - "added", - "deleted", - "delta" - ], - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } - } - }, - "ioConfig": { - "type": "realtime", - "firehose": { - "type": "timed", - "shutoffTime": "#SHUTOFFTIME", - "delegate": { - "type": "receiver", - "serviceName": "eventReceiverServiceName", - "bufferSize": 100000 - } - } - }, - "tuningConfig": { - "type": "realtime", - "maxRowsInMemory": 1, - "intermediatePersistPeriod": "PT1M", - "windowPeriod": "PT1M", - "rejectionPolicy": { - "type": "serverTime" - } - } - } -} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_union_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_union_index_task.json deleted file mode 100644 index 75c1281fcd2..00000000000 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_union_index_task.json +++ /dev/null @@ -1,98 +0,0 @@ -{ - "type": "index_realtime", - "spec": { - "dataSchema": { - "dataSource": "%%DATASOURCE%%", - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - } - ], - "granularitySpec": { - "segmentGranularity": "DAY", - "queryGranularity": "second" - }, - "parser": { - "type": "map", - "parseSpec": { - "format": "tsv", - "columns": [ - "timestamp", - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city", - "added", - "deleted", - "delta" - ], - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } - } - }, - "ioConfig": { - "type": "realtime", - "firehose": { - "type": "timed", - "shutoffTime": "#SHUTOFFTIME", - "delegate": { - "type": "receiver", - "serviceName": "eventReceiverServiceName", - "bufferSize": 100000 - } - } - }, - "tuningConfig": { - "type": "realtime", - "maxRowsInMemory": 1, - "intermediatePersistPeriod": "PT1M", - "windowPeriod": "PT1M", - "rejectionPolicy": { - "type": "none" - } - } - } -} diff --git a/integration-tests/src/test/resources/data/union_query/wikipedia_index_data.json b/integration-tests-ex/cases/src/test/resources/query/union_data.json similarity index 100% rename from integration-tests/src/test/resources/data/union_query/wikipedia_index_data.json rename to integration-tests-ex/cases/src/test/resources/query/union_data.json diff --git a/integration-tests-ex/cases/src/test/resources/query/union_kafka_supervisor_template.json b/integration-tests-ex/cases/src/test/resources/query/union_kafka_supervisor_template.json new file mode 100644 index 00000000000..a37340fe858 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/query/union_kafka_supervisor_template.json @@ -0,0 +1,69 @@ +{ + "type": "kafka", + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ], + "dimensionExclusions": [], + "spatialDimensions": [] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "DAY", + "queryGranularity": "second" + } + }, + "tuningConfig": { + "type": "kafka", + "intermediatePersistPeriod": "PT1H", + "maxRowsPerSegment": 5000000, + "maxRowsInMemory": 500000 + }, + "ioConfig": { + "topic": "%%TOPIC_VALUE%%", + "consumerProperties": %%STREAM_PROPERTIES_VALUE%%, + "taskCount": 2, + "replicas": 1, + "taskDuration": "PT120S", + "useEarliestOffset": true, + "inputFormat" : {"type": "json"} + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/query/union_queries.json b/integration-tests-ex/cases/src/test/resources/query/union_queries.json new file mode 100644 index 00000000000..74dd6db299d --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/query/union_queries.json @@ -0,0 +1,566 @@ +[ + { + "description": "timeseries, filtered, all aggs, all", + "query": { + "queryType": "timeseries", + "dataSource": { + "type": "union", + "dataSources": [ + "%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3", + "%%DATASOURCE%%-0" + ] + }, + "intervals": ["2013-08-31/2013-09-01"], + "granularity": "all", + "filter": { + "type": "selector", + "dimension": "language", + "value": "en" + }, + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + }, + { + "type": "doubleSum", + "fieldName": "added", + "name": "added" + }, + { + "type": "doubleSum", + "fieldName": "deleted", + "name": "deleted" + }, + { + "type": "doubleSum", + "fieldName": "delta", + "name": "delta" + } + ], + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": { + "added": 1548.0, + "count": 6, + "delta": 561.0, + "deleted": 987.0, + "rows": 6 + } + } + ] + }, + { + "description": "topN, all aggs, page dim, uniques metric", + "query": { + "queryType": "topN", + "dataSource": { + "type": "union", + "dataSources": [ + "%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3", + "%%DATASOURCE%%-0" + ] + }, + "intervals": ["2013-08-31/2013-09-01"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + }, + { + "type": "doubleSum", + "fieldName": "added", + "name": "added" + }, + { + "type": "doubleSum", + "fieldName": "deleted", + "name": "deleted" + }, + { + "type": "doubleSum", + "fieldName": "delta", + "name": "delta" + } + ], + "dimension": "page", + "metric": "added", + "threshold": 3, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": [ + { + "added": 2715.0, + "count": 3, + "page": "Crimson Typhoon", + "delta": 2700.0, + "deleted": 15.0, + "rows": 3 + }, + { + "added": 1377.0, + "count": 3, + "page": "Striker Eureka", + "delta": 990.0, + "deleted": 387.0, + "rows": 3 + }, + { + "added": 369.0, + "count": 3, + "page": "Cherno Alpha", + "delta": 333.0, + "deleted": 36.0, + "rows": 3 + } + ] + } + ] + }, + { + "description": "topN, all aggs, page dim, count metric, postAggs", + "query": { + "queryType": "topN", + "dataSource": { + "type": "union", + "dataSources": [ + "%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3", + "%%DATASOURCE%%-0" + ] + }, + "intervals": ["2013-08-31/2013-09-01"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + }, + { + "type": "doubleSum", + "fieldName": "added", + "name": "added" + }, + { + "type": "doubleSum", + "fieldName": "deleted", + "name": "deleted" + }, + { + "type": "doubleSum", + "fieldName": "delta", + "name": "delta" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "sumOfAddedDeletedConst", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "added", + "fieldName": "added" + }, + { + "type": "arithmetic", + "name": "", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "constant", + "name": "constant", + "value": 1000 + } + ] + } + ] + } + ], + "dimension": "page", + "metric": "added", + "threshold": 3, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": [ + { + "added": 2715.0, + "count": 3, + "page": "Crimson Typhoon", + "delta": 2700.0, + "deleted": 15.0, + "sumOfAddedDeletedConst": 3730.0, + "rows": 3 + }, + { + "added": 1377.0, + "count": 3, + "page": "Striker Eureka", + "delta": 990.0, + "deleted": 387.0, + "sumOfAddedDeletedConst": 2764.0, + "rows": 3 + }, + { + "added": 369.0, + "count": 3, + "page": "Cherno Alpha", + "delta": 333.0, + "deleted": 36.0, + "sumOfAddedDeletedConst": 1405.0, + "rows": 3 + } + ] + } + ] + }, + { + "description": "topN, lexicographic, two aggs, language dim, postAggs", + "query": { + "queryType": "topN", + "dataSource": { + "type": "union", + "dataSources": [ + "%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3", + "%%DATASOURCE%%-0" + ] + }, + "intervals": ["2013-08-31/2013-09-01"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "sumOfRowsAndCount", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "rows", + "fieldName": "rows" + }, + { + "type": "fieldAccess", + "name": "count", + "fieldName": "count" + } + ] + } + ], + "dimension": "language", + "metric": { + "type": "lexicographic", + "previousStop": "a" + }, + "threshold": 3, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": [ + { + "sumOfRowsAndCount": 12.0, + "count": 6, + "language": "en", + "rows": 6 + }, + { + "sumOfRowsAndCount": 6.0, + "count": 3, + "language": "ja", + "rows": 3 + }, + { + "sumOfRowsAndCount": 6.0, + "count": 3, + "language": "ru", + "rows": 3 + } + ] + } + ] + }, + { + "description": "groupBy, two aggs, namespace dim, postAggs", + "query": { + "queryType": "groupBy", + "dataSource": { + "type": "union", + "dataSources": [ + "%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3", + "%%DATASOURCE%%-0" + ] + }, + "intervals": ["2013-08-31/2013-09-01"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "sumOfRowsAndCount", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "rows", + "fieldName": "rows" + }, + { + "type": "fieldAccess", + "name": "count", + "fieldName": "count" + } + ] + } + ], + "dimensions": ["namespace"], + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "version": "v1", + "timestamp": "2013-08-31T00:00:00.000Z", + "event": { + "sumOfRowsAndCount": 12.0, + "count": 6, + "rows": 6, + "namespace": "article" + } + }, + { + "version": "v1", + "timestamp": "2013-08-31T00:00:00.000Z", + "event": { + "sumOfRowsAndCount": 18.0, + "count": 9, + "rows": 9, + "namespace": "wikipedia" + } + } + ] + }, + { + "description": "groupBy, two aggs, namespace + robot dim, postAggs", + "query": { + "queryType": "groupBy", + "dataSource": { + "type": "union", + "dataSources": [ + "%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3", + "%%DATASOURCE%%-0" + ] + }, + "intervals": ["2013-08-31/2013-09-01"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "count", + "name": "count" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "sumOfRowsAndCount", + "fn": "+", + "fields": [ + { + "type": "fieldAccess", + "name": "rows", + "fieldName": "rows" + }, + { + "type": "fieldAccess", + "name": "count", + "fieldName": "count" + } + ] + } + ], + "dimensions": ["namespace", "robot"], + "limitSpec": { + "type": "default", + "limit": 3, + "orderBy": ["robot", "namespace"] + }, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "version": "v1", + "timestamp": "2013-08-31T00:00:00.000Z", + "event": { + "sumOfRowsAndCount": 6.0, + "count": 3, + "robot": "false", + "rows": 3, + "namespace": "article" + } + }, + { + "version": "v1", + "timestamp": "2013-08-31T00:00:00.000Z", + "event": { + "sumOfRowsAndCount": 6.0, + "count": 3, + "robot": "true", + "rows": 3, + "namespace": "article" + } + }, + { + "version": "v1", + "timestamp": "2013-08-31T00:00:00.000Z", + "event": { + "sumOfRowsAndCount": 18.0, + "count": 9, + "robot": "true", + "rows": 9, + "namespace": "wikipedia" + } + } + ] + }, + { + "query": { + "queryType": "search", + "intervals": ["2013-08-31/2013-09-01"], + "dataSource": { + "type": "union", + "dataSources": [ + "%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3", + "%%DATASOURCE%%-0" + ] + }, + "granularity": "all", + "query": { + "type": "insensitive_contains", + "value": "ip" + }, + "context": { + "useCache": "true", + "populateCache": "true", + "timeout": 60000 + } + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T00:00:00.000Z", + "result": [ + { + "dimension": "user", + "value": "triplets", + "count":3 + }, + { + "dimension": "namespace", + "value": "wikipedia", + "count":9 + } + ] + } + ] + }, + { + "description": "timeboundary, 1 agg, union", + "query": { + "queryType": "timeBoundary", + "dataSource": { + "type": "union", + "dataSources": [ + "%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3", + "%%DATASOURCE%%-0" + ] + } + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": { + "minTime": "2013-08-31T01:02:33.000Z", + "maxTime": "2013-09-01T12:41:27.000Z" + } + } + ] + } +] diff --git a/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java b/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java index b3417902cef..9ba40a8d1d8 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/TestNGGroup.java @@ -58,10 +58,6 @@ public class TestNGGroup public static final String QUERY_ERROR = "query-error"; - public static final String CLI_INDEXER = "cli-indexer"; - - public static final String REALTIME_INDEX = "realtime-index"; - /** * This group can only be run individually using -Dgroups=security since it requires specific test data setup. */ diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITRealtimeIndexTaskTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITRealtimeIndexTaskTest.java deleted file mode 100644 index fb614216c42..00000000000 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITRealtimeIndexTaskTest.java +++ /dev/null @@ -1,198 +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.tests.indexer; - -import com.google.inject.Inject; -import org.apache.commons.io.IOUtils; -import org.apache.druid.curator.discovery.ServerDiscoveryFactory; -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.logger.Logger; -import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.testing.IntegrationTestingConfig; -import org.apache.druid.testing.guice.TestClient; -import org.apache.druid.testing.utils.ITRetryUtil; -import org.joda.time.DateTime; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; - -import java.io.Closeable; -import java.io.InputStream; -import java.nio.charset.StandardCharsets; -import java.util.concurrent.TimeUnit; - -/** - * Steps - * 1) Submit a realtime index task - * 2) Load data using EventReceiverFirehose - * 3) Run queries to verify that the ingested data is available for queries - * 4) Wait for handover of the segment to historical node - * 5) Query data (will be from historical node) - * 6) Disable and delete the created data segment - */ -public abstract class AbstractITRealtimeIndexTaskTest extends AbstractIndexerTest -{ - static final String EVENT_RECEIVER_SERVICE_NAME = "eventReceiverServiceName"; - static final String EVENT_DATA_FILE = "/indexer/wikipedia_realtime_index_data.json"; - - private static final Logger LOG = new Logger(AbstractITRealtimeIndexTaskTest.class); - private static final String INDEX_DATASOURCE = "wikipedia_index_test"; - - static final int DELAY_BETWEEN_EVENTS_SECS = 4; - final String TIME_PLACEHOLDER = "YYYY-MM-DDTHH:MM:SS"; - // format for putting timestamp into events - static final DateTimeFormatter EVENT_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss"); - // format for the querying interval - private static final DateTimeFormatter INTERVAL_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:'00Z'"); - // format for the expected timestamp in a query response - private static final DateTimeFormatter TIMESTAMP_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'.000Z'"); - DateTime dtFirst; // timestamp of 1st event - DateTime dtLast; // timestamp of last event - DateTime dtGroupBy; // timestamp for expected response for groupBy query - - static final int NUM_RETRIES = 60; - static final long DELAY_FOR_RETRIES_MS = 10000; - - @Inject - ServerDiscoveryFactory factory; - @Inject - @TestClient - HttpClient httpClient; - - @Inject - IntegrationTestingConfig config; - - private String fullDatasourceName; - - void doTest() - { - fullDatasourceName = INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix(); - - LOG.info("Starting test: %s", this.getClass().getSimpleName()); - try (final Closeable ignored = unloader(fullDatasourceName)) { - // the task will run for 5 minutes and then shutdown itself - String task = setShutOffTime( - getResourceAsString(getTaskResource()), - DateTimes.utc(System.currentTimeMillis() + TimeUnit.MINUTES.toMillis(5)) - ); - task = StringUtils.replace(task, "%%DATASOURCE%%", fullDatasourceName); - - LOG.info("indexerSpec: [%s]\n", task); - String taskID = indexer.submitTask(task); - - // sleep for a while to let peons finish starting up - TimeUnit.SECONDS.sleep(60); - - // this posts 22 events, one every 4 seconds - // each event contains the current time as its timestamp except - // the timestamp for the 14th event is early enough that the event should be ignored - // the timestamp for the 18th event is 2 seconds earlier than the 17th - postEvents(); - - // wait for a while to let the events be ingested - ITRetryUtil.retryUntil( - () -> { - final int countRows = queryHelper.countRows( - fullDatasourceName, - Intervals.ETERNITY, - name -> new LongSumAggregatorFactory(name, "count") - ); - return countRows == getNumExpectedRowsIngested(); - }, - true, - DELAY_FOR_RETRIES_MS, - NUM_RETRIES, - "Waiting all events are ingested" - ); - - // put the timestamps into the query structure - String query_response_template; - InputStream is = ITRealtimeIndexTaskTest.class.getResourceAsStream(getQueriesResource()); - if (null == is) { - throw new ISE("could not open query file: %s", getQueriesResource()); - } - query_response_template = IOUtils.toString(is, StandardCharsets.UTF_8); - - String queryStr = query_response_template; - queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst)); - queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", TIMESTAMP_FMT.print(dtLast)); - queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_MINTIME%%", TIMESTAMP_FMT.print(dtFirst)); - queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_QUERY_START%%", INTERVAL_FMT.print(dtFirst)); - queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_QUERY_END%%", INTERVAL_FMT.print(dtLast.plusMinutes(2))); - queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst)); - queryStr = StringUtils.replace(queryStr, "%%POST_AG_REQUEST_START%%", INTERVAL_FMT.print(dtFirst)); - queryStr = StringUtils.replace(queryStr, "%%POST_AG_REQUEST_END%%", INTERVAL_FMT.print(dtLast.plusMinutes(2))); - String postAgResponseTimestamp = TIMESTAMP_FMT.print(dtGroupBy.withSecondOfMinute(0)); - queryStr = StringUtils.replace(queryStr, "%%POST_AG_RESPONSE_TIMESTAMP%%", postAgResponseTimestamp); - queryStr = StringUtils.replace(queryStr, "%%DATASOURCE%%", fullDatasourceName); - - // should hit the queries all on realtime task or some on realtime task - // and some on historical. Which it is depends on where in the minute we were - // when we started posting events. - try { - this.queryHelper.testQueriesFromString(getRouterURL(), queryStr); - } - catch (Exception e) { - throw new RuntimeException(e); - } - - // wait for the task to complete - indexer.waitUntilTaskCompletes(taskID); - - // task should complete only after the segments are loaded by historical node - ITRetryUtil.retryUntil( - () -> coordinator.areSegmentsLoaded(fullDatasourceName), - true, - DELAY_FOR_RETRIES_MS, - NUM_RETRIES, - "Real-time generated segments loaded" - ); - - // queries should be answered by historical - this.queryHelper.testQueriesFromString(getRouterURL(), queryStr); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - private String setShutOffTime(String taskAsString, DateTime time) - { - return StringUtils.replace(taskAsString, "#SHUTOFFTIME", time.toString()); - } - - private String getRouterURL() - { - return StringUtils.format( - "%s/druid/v2?pretty", - config.getRouterUrl() - ); - } - - abstract String getTaskResource(); - abstract String getQueriesResource(); - - abstract void postEvents() throws Exception; - - abstract int getNumExpectedRowsIngested(); -} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java index 8a382287d43..2bcf8ea5891 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java @@ -171,7 +171,7 @@ public abstract class AbstractIndexerTest public static InputStream getResourceAsStream(String resource) { - return ITRealtimeIndexTaskTest.class.getResourceAsStream(resource); + return ITCompactionTaskTest.class.getResourceAsStream(resource); } public static List listResources(String dir) throws IOException diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppenderatorDriverRealtimeIndexTaskTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppenderatorDriverRealtimeIndexTaskTest.java deleted file mode 100644 index b69ac027675..00000000000 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppenderatorDriverRealtimeIndexTaskTest.java +++ /dev/null @@ -1,153 +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.tests.indexer; - -import org.apache.druid.curator.discovery.ServerDiscoverySelector; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.jackson.JacksonUtils; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.testing.clients.EventReceiverFirehoseTestClient; -import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.testing.utils.ServerDiscoveryUtil; -import org.apache.druid.tests.TestNGGroup; -import org.joda.time.DateTime; -import org.testng.annotations.Guice; -import org.testng.annotations.Test; - -import javax.ws.rs.core.MediaType; -import java.io.BufferedReader; -import java.io.InputStreamReader; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Map; - -/** - * See {@link AbstractITRealtimeIndexTaskTest} for test details. - */ -@Test(groups = {TestNGGroup.REALTIME_INDEX, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_METADATA_QUERY_DISABLED}) -@Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITAppenderatorDriverRealtimeIndexTaskTest extends AbstractITRealtimeIndexTaskTest -{ - private static final Logger LOG = new Logger(ITAppenderatorDriverRealtimeIndexTaskTest.class); - private static final String REALTIME_TASK_RESOURCE = "/indexer/wikipedia_realtime_appenderator_index_task.json"; - private static final String REALTIME_QUERIES_RESOURCE = "/indexer/wikipedia_realtime_appenderator_index_queries.json"; - /** - * The expected number of rows ingested for this test. - * The total number of rows of raw data is 22 and there's no rollup. - */ - private static final int EXPECTED_NUM_ROWS = 22; - - @Test - public void testRealtimeIndexTask() - { - doTest(); - } - - @Override - void postEvents() throws Exception - { - final ServerDiscoverySelector eventReceiverSelector = factory.createSelector(EVENT_RECEIVER_SERVICE_NAME); - eventReceiverSelector.start(); - InputStreamReader isr; - try { - isr = new InputStreamReader( - ITRealtimeIndexTaskTest.class.getResourceAsStream(EVENT_DATA_FILE), - StandardCharsets.UTF_8 - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - try (BufferedReader reader = new BufferedReader(isr)) { - ServerDiscoveryUtil.waitUntilInstanceReady(eventReceiverSelector, "Event Receiver"); - // Use the host from the config file and the port announced in zookeeper - String host = config.getMiddleManagerHost() + ":" + eventReceiverSelector.pick().getPort(); - LOG.info("Event Receiver Found at host [%s]", host); - EventReceiverFirehoseTestClient client = new EventReceiverFirehoseTestClient( - host, - EVENT_RECEIVER_SERVICE_NAME, - jsonMapper, - httpClient, - smileMapper - ); - // there are 22 lines in the file - int i = 1; - DateTime dt = DateTimes.nowUtc(); // timestamp used for sending each event - dtFirst = dt; - dtLast = dt; - String line; - while ((line = reader.readLine()) != null) { - if (i == 15) { // for the 15th line, use a time before the window - dt = dt.minusMinutes(10); - dtFirst = dt; // oldest timestamp - } else if (i == 16) { // remember this time to use in the expected response from the groupBy query - dtGroupBy = dt; - } else if (i == 18) { // use a time 6 seconds ago so it will be out of order - dt = dt.minusSeconds(6); - } - String event = StringUtils.replace(line, TIME_PLACEHOLDER, EVENT_FMT.print(dt)); - LOG.info("sending event: [%s]\n", event); - Collection> events = new ArrayList<>(); - events.add(this.jsonMapper.readValue(event, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT)); - int eventsPosted = client.postEvents(events, this.jsonMapper, MediaType.APPLICATION_JSON); - if (eventsPosted != events.size()) { - throw new ISE("Event not posted"); - } - - try { - Thread.sleep(DELAY_BETWEEN_EVENTS_SECS * 1000); - } - catch (InterruptedException ignored) { - /* nothing */ - } - dtLast = dt; // latest timestamp - dt = DateTimes.nowUtc(); - i++; - } - } - catch (Exception e) { - throw new RuntimeException(e); - } - finally { - eventReceiverSelector.stop(); - } - } - - @Override - String getTaskResource() - { - return REALTIME_TASK_RESOURCE; - } - - @Override - String getQueriesResource() - { - return REALTIME_QUERIES_RESOURCE; - } - - @Override - int getNumExpectedRowsIngested() - { - return EXPECTED_NUM_ROWS; - } -} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITRealtimeIndexTaskTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITRealtimeIndexTaskTest.java deleted file mode 100644 index be3f518b098..00000000000 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITRealtimeIndexTaskTest.java +++ /dev/null @@ -1,152 +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.tests.indexer; - -import org.apache.druid.curator.discovery.ServerDiscoverySelector; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.jackson.JacksonUtils; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.testing.clients.EventReceiverFirehoseTestClient; -import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.testing.utils.ServerDiscoveryUtil; -import org.apache.druid.tests.TestNGGroup; -import org.joda.time.DateTime; -import org.testng.annotations.Guice; -import org.testng.annotations.Test; - -import javax.ws.rs.core.MediaType; -import java.io.BufferedReader; -import java.io.InputStreamReader; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Map; - -/** - * See {@link AbstractITRealtimeIndexTaskTest} for test details. - */ -@Test(groups = {TestNGGroup.REALTIME_INDEX, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_METADATA_QUERY_DISABLED}) -@Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITRealtimeIndexTaskTest extends AbstractITRealtimeIndexTaskTest -{ - private static final Logger LOG = new Logger(ITRealtimeIndexTaskTest.class); - private static final String REALTIME_TASK_RESOURCE = "/indexer/wikipedia_realtime_index_task.json"; - private static final String REALTIME_QUERIES_RESOURCE = "/indexer/wikipedia_realtime_index_queries.json"; - /** - * The expected number of rows ingested for this test. - * The total number of rows of raw data is 22, but two rows will be rolled up into one row. - */ - private static final int EXPECTED_NUM_ROWS = 21; - - @Test - public void testRealtimeIndexTask() - { - doTest(); - } - - @Override - String getTaskResource() - { - return REALTIME_TASK_RESOURCE; - } - - @Override - String getQueriesResource() - { - return REALTIME_QUERIES_RESOURCE; - } - - @Override - int getNumExpectedRowsIngested() - { - return EXPECTED_NUM_ROWS; - } - - @Override - void postEvents() throws Exception - { - final ServerDiscoverySelector eventReceiverSelector = factory.createSelector(EVENT_RECEIVER_SERVICE_NAME); - eventReceiverSelector.start(); - InputStreamReader isr; - try { - isr = new InputStreamReader( - ITRealtimeIndexTaskTest.class.getResourceAsStream(EVENT_DATA_FILE), - StandardCharsets.UTF_8 - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - try (BufferedReader reader = new BufferedReader(isr)) { - ServerDiscoveryUtil.waitUntilInstanceReady(eventReceiverSelector, "Event Receiver"); - // Use the host from the config file and the port announced in zookeeper - String host = config.getMiddleManagerHost() + ":" + eventReceiverSelector.pick().getPort(); - LOG.info("Event Receiver Found at host [%s]", host); - EventReceiverFirehoseTestClient client = new EventReceiverFirehoseTestClient( - host, - EVENT_RECEIVER_SERVICE_NAME, - jsonMapper, - httpClient, - smileMapper - ); - // there are 22 lines in the file - int i = 1; - DateTime dt = DateTimes.nowUtc(); // timestamp used for sending each event - dtFirst = dt; // timestamp of 1st event - dtLast = dt; // timestamp of last event - String line; - while ((line = reader.readLine()) != null) { - if (i == 15) { // for the 15th line, use a time before the window - dt = dt.minusMinutes(10); - } else if (i == 16) { // remember this time to use in the expected response from the groupBy query - dtGroupBy = dt; - } else if (i == 18) { // use a time 6 seconds ago so it will be out of order - dt = dt.minusSeconds(6); - } - String event = StringUtils.replace(line, TIME_PLACEHOLDER, EVENT_FMT.print(dt)); - LOG.info("sending event: [%s]\n", event); - Collection> events = new ArrayList<>(); - events.add(this.jsonMapper.readValue(event, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT)); - int eventsPosted = client.postEvents(events, this.jsonMapper, MediaType.APPLICATION_JSON); - if (eventsPosted != events.size()) { - throw new ISE("Event not posted"); - } - - try { - Thread.sleep(DELAY_BETWEEN_EVENTS_SECS * 1000); - } - catch (InterruptedException ignored) { - /* nothing */ - } - dtLast = dt; - dt = DateTimes.nowUtc(); - i++; - } - } - catch (Exception e) { - throw new RuntimeException(e); - } - finally { - eventReceiverSelector.stop(); - } - } -} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/query/ITUnionQueryTest.java b/integration-tests/src/test/java/org/apache/druid/tests/query/ITUnionQueryTest.java deleted file mode 100644 index 7eedcbb7317..00000000000 --- a/integration-tests/src/test/java/org/apache/druid/tests/query/ITUnionQueryTest.java +++ /dev/null @@ -1,248 +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.tests.query; - -import com.google.inject.Inject; -import org.apache.commons.io.IOUtils; -import org.apache.druid.curator.discovery.ServerDiscoveryFactory; -import org.apache.druid.curator.discovery.ServerDiscoverySelector; -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.io.Closer; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.java.util.http.client.Request; -import org.apache.druid.java.util.http.client.response.StatusResponseHandler; -import org.apache.druid.java.util.http.client.response.StatusResponseHolder; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.testing.IntegrationTestingConfig; -import org.apache.druid.testing.clients.EventReceiverFirehoseTestClient; -import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.testing.guice.TestClient; -import org.apache.druid.testing.utils.ITRetryUtil; -import org.apache.druid.testing.utils.ServerDiscoveryUtil; -import org.apache.druid.tests.TestNGGroup; -import org.apache.druid.tests.indexer.AbstractITBatchIndexTest; -import org.apache.druid.tests.indexer.AbstractIndexerTest; -import org.jboss.netty.handler.codec.http.HttpMethod; -import org.jboss.netty.handler.codec.http.HttpResponseStatus; -import org.joda.time.DateTime; -import org.testng.annotations.BeforeSuite; -import org.testng.annotations.Guice; -import org.testng.annotations.Test; - -import java.io.IOException; -import java.io.InputStream; -import java.net.URL; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeUnit; - -@Test(groups = {TestNGGroup.QUERY, TestNGGroup.CENTRALIZED_DATASOURCE_SCHEMA}) -@Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITUnionQueryTest extends AbstractIndexerTest -{ - private static final Logger LOG = new Logger(ITUnionQueryTest.class); - private static final String UNION_TASK_RESOURCE = "/indexer/wikipedia_union_index_task.json"; - private static final String EVENT_RECEIVER_SERVICE_PREFIX = "eventReceiverServiceName"; - private static final String UNION_DATA_FILE = "/data/union_query/wikipedia_index_data.json"; - private static final String UNION_QUERIES_RESOURCE = "/queries/union_queries.json"; - private static final String UNION_DATASOURCE = "wikipedia_index_test"; - - @Inject - ServerDiscoveryFactory factory; - - @Inject - @TestClient - HttpClient httpClient; - - @Inject - IntegrationTestingConfig config; - - private String fullDatasourceName; - - @BeforeSuite - public void setFullDatasourceName() - { - fullDatasourceName = UNION_DATASOURCE + config.getExtraDatasourceNameSuffix(); - } - - @Test - public void testUnionQuery() throws IOException - { - final int numTasks = 3; - final Closer closer = Closer.create(); - for (int i = 0; i < numTasks; i++) { - closer.register(unloader(fullDatasourceName + i)); - } - try { - // Load 3 datasources with same dimensions - String task = setShutOffTime( - getResourceAsString(UNION_TASK_RESOURCE), - DateTimes.utc(System.currentTimeMillis() + TimeUnit.MINUTES.toMillis(3)) - ); - List taskIDs = new ArrayList<>(); - for (int i = 0; i < numTasks; i++) { - taskIDs.add( - indexer.submitTask( - withServiceName( - withDataSource(task, fullDatasourceName + i), - EVENT_RECEIVER_SERVICE_PREFIX + i - ) - ) - ); - } - for (int i = 0; i < numTasks; i++) { - postEvents(i); - } - - // wait until all events are ingested - ITRetryUtil.retryUntil( - () -> { - for (int i = 0; i < numTasks; i++) { - final int countRows = queryHelper.countRows( - fullDatasourceName + i, - Intervals.of("2013-08-31/2013-09-01"), - name -> new LongSumAggregatorFactory(name, "count") - ); - - // there are 10 rows, but query only covers the first 5 - if (countRows < 5) { - LOG.warn("%d events have been ingested to %s so far", countRows, fullDatasourceName + i); - return false; - } - } - return true; - }, - true, - 1000, - 100, - "Waiting all events are ingested" - ); - - // should hit the queries on realtime task - LOG.info("Running Union Queries.."); - - String queryResponseTemplate; - try { - InputStream is = AbstractITBatchIndexTest.class.getResourceAsStream(UNION_QUERIES_RESOURCE); - queryResponseTemplate = IOUtils.toString(is, StandardCharsets.UTF_8); - } - catch (IOException e) { - throw new ISE(e, "could not read query file: %s", UNION_QUERIES_RESOURCE); - } - - queryResponseTemplate = StringUtils.replace( - queryResponseTemplate, - "%%DATASOURCE%%", - fullDatasourceName - ); - - this.queryHelper.testQueriesFromString(queryResponseTemplate); - - // wait for the task to complete - for (int i = 0; i < numTasks; i++) { - indexer.waitUntilTaskCompletes(taskIDs.get(i)); - } - // task should complete only after the segments are loaded by historical node - for (int i = 0; i < numTasks; i++) { - final int taskNum = i; - ITRetryUtil.retryUntil( - () -> coordinator.areSegmentsLoaded(fullDatasourceName + taskNum), - true, - 10000, - 10, - "Real-time generated segments loaded" - ); - } - // run queries on historical nodes - this.queryHelper.testQueriesFromString(queryResponseTemplate); - - } - catch (Throwable e) { - throw closer.rethrow(e); - } - finally { - closer.close(); - } - } - - private String setShutOffTime(String taskAsString, DateTime time) - { - return StringUtils.replace(taskAsString, "#SHUTOFFTIME", time.toString()); - } - - private String withDataSource(String taskAsString, String dataSource) - { - return StringUtils.replace(taskAsString, "%%DATASOURCE%%", dataSource); - } - - private String withServiceName(String taskAsString, String serviceName) - { - return StringUtils.replace(taskAsString, EVENT_RECEIVER_SERVICE_PREFIX, serviceName); - } - - private void postEvents(int id) throws Exception - { - final ServerDiscoverySelector eventReceiverSelector = factory.createSelector(EVENT_RECEIVER_SERVICE_PREFIX + id); - eventReceiverSelector.start(); - try { - ServerDiscoveryUtil.waitUntilInstanceReady(eventReceiverSelector, "Event Receiver"); - // Access the docker VM mapped host and port instead of service announced in zookeeper - String host = config.getMiddleManagerHost() + ":" + eventReceiverSelector.pick().getPort(); - - LOG.info("Event Receiver Found at host [%s]", host); - - LOG.info("Checking worker /status/health for [%s]", host); - ITRetryUtil.retryUntilTrue( - () -> { - try { - StatusResponseHolder response = httpClient.go( - new Request(HttpMethod.GET, new URL(StringUtils.format("https://%s/status/health", host))), - StatusResponseHandler.getInstance() - ).get(); - return response.getStatus().equals(HttpResponseStatus.OK); - } - catch (Throwable e) { - LOG.error(e, ""); - return false; - } - }, - StringUtils.format("Checking /status/health for worker [%s]", host) - ); - LOG.info("Finished checking worker /status/health for [%s], success", host); - - EventReceiverFirehoseTestClient client = new EventReceiverFirehoseTestClient( - host, - EVENT_RECEIVER_SERVICE_PREFIX + id, - jsonMapper, - httpClient, - smileMapper - ); - client.postEventsFromFile(UNION_DATA_FILE); - } - finally { - eventReceiverSelector.stop(); - } - } -} diff --git a/integration-tests/src/test/resources/data/broadcast/wikipedia_index_data.json b/integration-tests/src/test/resources/data/broadcast/wikipedia_index_data.json new file mode 100644 index 00000000000..d9a67cdfb70 --- /dev/null +++ b/integration-tests/src/test/resources/data/broadcast/wikipedia_index_data.json @@ -0,0 +1,10 @@ +{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} +{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} +{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} +{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} +{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} +{"timestamp": "2013-09-01T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} +{"timestamp": "2013-09-01T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} +{"timestamp": "2013-09-01T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} +{"timestamp": "2013-09-01T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} +{"timestamp": "2013-09-01T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} diff --git a/integration-tests/src/test/resources/indexer/broadcast_join_index_task.json b/integration-tests/src/test/resources/indexer/broadcast_join_index_task.json index 20c3b162ea6..699ca547ff4 100644 --- a/integration-tests/src/test/resources/indexer/broadcast_join_index_task.json +++ b/integration-tests/src/test/resources/indexer/broadcast_join_index_task.json @@ -61,7 +61,7 @@ "type": "index_parallel", "inputSource": { "type": "local", - "baseDir": "/resources/data/union_query/", + "baseDir": "/resources/data/broadcast/", "filter": "wikipedia_index_data*" }, "appendToExisting": false, diff --git a/integration-tests/src/test/resources/indexer/wikipedia_realtime_appenderator_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_realtime_appenderator_index_task.json deleted file mode 100644 index 9e773609cb7..00000000000 --- a/integration-tests/src/test/resources/indexer/wikipedia_realtime_appenderator_index_task.json +++ /dev/null @@ -1,94 +0,0 @@ -{ - "type": "index_realtime_appenderator", - "spec": { - "dataSchema": { - "dataSource": "%%DATASOURCE%%", - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - } - ], - "granularitySpec": { - "segmentGranularity": "minute", - "queryGranularity": "second" - }, - "parser": { - "type": "map", - "parseSpec": { - "format": "tsv", - "columns": [ - "timestamp", - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city", - "added", - "deleted", - "delta" - ], - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } - } - }, - "ioConfig": { - "type": "realtime", - "firehose": { - "type": "timed", - "shutoffTime": "#SHUTOFFTIME", - "delegate": { - "type": "receiver", - "serviceName": "eventReceiverServiceName", - "bufferSize": 100000 - } - } - }, - "tuningConfig": { - "type": "realtime_appenderator", - "maxRowsInMemory": 1, - "intermediatePersistPeriod": "PT1M" - } - } -} diff --git a/integration-tests/src/test/resources/indexer/wikipedia_realtime_index_data.json b/integration-tests/src/test/resources/indexer/wikipedia_realtime_index_data.json deleted file mode 100644 index 626bb2df861..00000000000 --- a/integration-tests/src/test/resources/indexer/wikipedia_realtime_index_data.json +++ /dev/null @@ -1,22 +0,0 @@ -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 1, "deleted": 1, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 3, "deleted": 3, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 5, "deleted": 5, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 7, "deleted": 3, "delta": 4} -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 11, "deleted": 11, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 13, "deleted": 13, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 17, "deleted": 17, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 19, "deleted": 19, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 23, "deleted": 23, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 29, "deleted": 31, "delta": -1} -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 31, "deleted": 31, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 37, "deleted": 37, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 41, "deleted": 41, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 43, "deleted": 43, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 47, "deleted": 47, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 59, "deleted": 59, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 61, "deleted": 61, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 67, "deleted": 67, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 71, "deleted": 71, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 73, "deleted": 73, "delta": 0}, -{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 79, "deleted": 79, "delta": 0} diff --git a/integration-tests/src/test/resources/indexer/wikipedia_realtime_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_realtime_index_task.json deleted file mode 100644 index 5f48162c488..00000000000 --- a/integration-tests/src/test/resources/indexer/wikipedia_realtime_index_task.json +++ /dev/null @@ -1,98 +0,0 @@ -{ - "type": "index_realtime", - "spec": { - "dataSchema": { - "dataSource": "%%DATASOURCE%%", - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - } - ], - "granularitySpec": { - "segmentGranularity": "minute", - "queryGranularity": "second" - }, - "parser": { - "type": "map", - "parseSpec": { - "format": "tsv", - "columns": [ - "timestamp", - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city", - "added", - "deleted", - "delta" - ], - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } - } - }, - "ioConfig": { - "type": "realtime", - "firehose": { - "type": "timed", - "shutoffTime": "#SHUTOFFTIME", - "delegate": { - "type": "receiver", - "serviceName": "eventReceiverServiceName", - "bufferSize": 100000 - } - } - }, - "tuningConfig": { - "type": "realtime", - "maxRowsInMemory": 1, - "intermediatePersistPeriod": "PT1M", - "windowPeriod": "PT1M", - "rejectionPolicy": { - "type": "serverTime" - } - } - } -} diff --git a/integration-tests/src/test/resources/indexer/wikipedia_union_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_union_index_task.json deleted file mode 100644 index 75c1281fcd2..00000000000 --- a/integration-tests/src/test/resources/indexer/wikipedia_union_index_task.json +++ /dev/null @@ -1,98 +0,0 @@ -{ - "type": "index_realtime", - "spec": { - "dataSchema": { - "dataSource": "%%DATASOURCE%%", - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - } - ], - "granularitySpec": { - "segmentGranularity": "DAY", - "queryGranularity": "second" - }, - "parser": { - "type": "map", - "parseSpec": { - "format": "tsv", - "columns": [ - "timestamp", - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city", - "added", - "deleted", - "delta" - ], - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } - } - }, - "ioConfig": { - "type": "realtime", - "firehose": { - "type": "timed", - "shutoffTime": "#SHUTOFFTIME", - "delegate": { - "type": "receiver", - "serviceName": "eventReceiverServiceName", - "bufferSize": 100000 - } - } - }, - "tuningConfig": { - "type": "realtime", - "maxRowsInMemory": 1, - "intermediatePersistPeriod": "PT1M", - "windowPeriod": "PT1M", - "rejectionPolicy": { - "type": "none" - } - } - } -} diff --git a/integration-tests/src/test/resources/queries/union_queries.json b/integration-tests/src/test/resources/queries/union_queries.json deleted file mode 100644 index 627af04edc9..00000000000 --- a/integration-tests/src/test/resources/queries/union_queries.json +++ /dev/null @@ -1,566 +0,0 @@ -[ - { - "description": "timeseries, filtered, all aggs, all", - "query": { - "queryType": "timeseries", - "dataSource": { - "type": "union", - "dataSources": [ - "%%DATASOURCE%%1", "%%DATASOURCE%%2", "%%DATASOURCE%%3", - "%%DATASOURCE%%0" - ] - }, - "intervals": ["2013-08-31/2013-09-01"], - "granularity": "all", - "filter": { - "type": "selector", - "dimension": "language", - "value": "en" - }, - "aggregations": [ - { - "type": "count", - "name": "rows" - }, - { - "type": "longSum", - "fieldName": "count", - "name": "count" - }, - { - "type": "doubleSum", - "fieldName": "added", - "name": "added" - }, - { - "type": "doubleSum", - "fieldName": "deleted", - "name": "deleted" - }, - { - "type": "doubleSum", - "fieldName": "delta", - "name": "delta" - } - ], - "context": { - "useCache": "true", - "populateCache": "true", - "timeout": 60000 - } - }, - "expectedResults": [ - { - "timestamp": "2013-08-31T01:02:33.000Z", - "result": { - "added": 1548.0, - "count": 6, - "delta": 561.0, - "deleted": 987.0, - "rows": 6 - } - } - ] - }, - { - "description": "topN, all aggs, page dim, uniques metric", - "query": { - "queryType": "topN", - "dataSource": { - "type": "union", - "dataSources": [ - "%%DATASOURCE%%1", "%%DATASOURCE%%2", "%%DATASOURCE%%3", - "%%DATASOURCE%%0" - ] - }, - "intervals": ["2013-08-31/2013-09-01"], - "granularity": "all", - "aggregations": [ - { - "type": "count", - "name": "rows" - }, - { - "type": "longSum", - "fieldName": "count", - "name": "count" - }, - { - "type": "doubleSum", - "fieldName": "added", - "name": "added" - }, - { - "type": "doubleSum", - "fieldName": "deleted", - "name": "deleted" - }, - { - "type": "doubleSum", - "fieldName": "delta", - "name": "delta" - } - ], - "dimension": "page", - "metric": "added", - "threshold": 3, - "context": { - "useCache": "true", - "populateCache": "true", - "timeout": 60000 - } - }, - "expectedResults": [ - { - "timestamp": "2013-08-31T01:02:33.000Z", - "result": [ - { - "added": 2715.0, - "count": 3, - "page": "Crimson Typhoon", - "delta": 2700.0, - "deleted": 15.0, - "rows": 3 - }, - { - "added": 1377.0, - "count": 3, - "page": "Striker Eureka", - "delta": 990.0, - "deleted": 387.0, - "rows": 3 - }, - { - "added": 369.0, - "count": 3, - "page": "Cherno Alpha", - "delta": 333.0, - "deleted": 36.0, - "rows": 3 - } - ] - } - ] - }, - { - "description": "topN, all aggs, page dim, count metric, postAggs", - "query": { - "queryType": "topN", - "dataSource": { - "type": "union", - "dataSources": [ - "%%DATASOURCE%%1", "%%DATASOURCE%%2", "%%DATASOURCE%%3", - "%%DATASOURCE%%0" - ] - }, - "intervals": ["2013-08-31/2013-09-01"], - "granularity": "all", - "aggregations": [ - { - "type": "count", - "name": "rows" - }, - { - "type": "longSum", - "fieldName": "count", - "name": "count" - }, - { - "type": "doubleSum", - "fieldName": "added", - "name": "added" - }, - { - "type": "doubleSum", - "fieldName": "deleted", - "name": "deleted" - }, - { - "type": "doubleSum", - "fieldName": "delta", - "name": "delta" - } - ], - "postAggregations": [ - { - "type": "arithmetic", - "name": "sumOfAddedDeletedConst", - "fn": "+", - "fields": [ - { - "type": "fieldAccess", - "name": "added", - "fieldName": "added" - }, - { - "type": "arithmetic", - "name": "", - "fn": "+", - "fields": [ - { - "type": "fieldAccess", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "constant", - "name": "constant", - "value": 1000 - } - ] - } - ] - } - ], - "dimension": "page", - "metric": "added", - "threshold": 3, - "context": { - "useCache": "true", - "populateCache": "true", - "timeout": 60000 - } - }, - "expectedResults": [ - { - "timestamp": "2013-08-31T01:02:33.000Z", - "result": [ - { - "added": 2715.0, - "count": 3, - "page": "Crimson Typhoon", - "delta": 2700.0, - "deleted": 15.0, - "sumOfAddedDeletedConst": 3730.0, - "rows": 3 - }, - { - "added": 1377.0, - "count": 3, - "page": "Striker Eureka", - "delta": 990.0, - "deleted": 387.0, - "sumOfAddedDeletedConst": 2764.0, - "rows": 3 - }, - { - "added": 369.0, - "count": 3, - "page": "Cherno Alpha", - "delta": 333.0, - "deleted": 36.0, - "sumOfAddedDeletedConst": 1405.0, - "rows": 3 - } - ] - } - ] - }, - { - "description": "topN, lexicographic, two aggs, language dim, postAggs", - "query": { - "queryType": "topN", - "dataSource": { - "type": "union", - "dataSources": [ - "%%DATASOURCE%%1", "%%DATASOURCE%%2", "%%DATASOURCE%%3", - "%%DATASOURCE%%0" - ] - }, - "intervals": ["2013-08-31/2013-09-01"], - "granularity": "all", - "aggregations": [ - { - "type": "count", - "name": "rows" - }, - { - "type": "longSum", - "fieldName": "count", - "name": "count" - } - ], - "postAggregations": [ - { - "type": "arithmetic", - "name": "sumOfRowsAndCount", - "fn": "+", - "fields": [ - { - "type": "fieldAccess", - "name": "rows", - "fieldName": "rows" - }, - { - "type": "fieldAccess", - "name": "count", - "fieldName": "count" - } - ] - } - ], - "dimension": "language", - "metric": { - "type": "lexicographic", - "previousStop": "a" - }, - "threshold": 3, - "context": { - "useCache": "true", - "populateCache": "true", - "timeout": 60000 - } - }, - "expectedResults": [ - { - "timestamp": "2013-08-31T01:02:33.000Z", - "result": [ - { - "sumOfRowsAndCount": 12.0, - "count": 6, - "language": "en", - "rows": 6 - }, - { - "sumOfRowsAndCount": 6.0, - "count": 3, - "language": "ja", - "rows": 3 - }, - { - "sumOfRowsAndCount": 6.0, - "count": 3, - "language": "ru", - "rows": 3 - } - ] - } - ] - }, - { - "description": "groupBy, two aggs, namespace dim, postAggs", - "query": { - "queryType": "groupBy", - "dataSource": { - "type": "union", - "dataSources": [ - "%%DATASOURCE%%1", "%%DATASOURCE%%2", "%%DATASOURCE%%3", - "%%DATASOURCE%%0" - ] - }, - "intervals": ["2013-08-31/2013-09-01"], - "granularity": "all", - "aggregations": [ - { - "type": "count", - "name": "rows" - }, - { - "type": "longSum", - "fieldName": "count", - "name": "count" - } - ], - "postAggregations": [ - { - "type": "arithmetic", - "name": "sumOfRowsAndCount", - "fn": "+", - "fields": [ - { - "type": "fieldAccess", - "name": "rows", - "fieldName": "rows" - }, - { - "type": "fieldAccess", - "name": "count", - "fieldName": "count" - } - ] - } - ], - "dimensions": ["namespace"], - "context": { - "useCache": "true", - "populateCache": "true", - "timeout": 60000 - } - }, - "expectedResults": [ - { - "version": "v1", - "timestamp": "2013-08-31T00:00:00.000Z", - "event": { - "sumOfRowsAndCount": 12.0, - "count": 6, - "rows": 6, - "namespace": "article" - } - }, - { - "version": "v1", - "timestamp": "2013-08-31T00:00:00.000Z", - "event": { - "sumOfRowsAndCount": 18.0, - "count": 9, - "rows": 9, - "namespace": "wikipedia" - } - } - ] - }, - { - "description": "groupBy, two aggs, namespace + robot dim, postAggs", - "query": { - "queryType": "groupBy", - "dataSource": { - "type": "union", - "dataSources": [ - "%%DATASOURCE%%1", "%%DATASOURCE%%2", "%%DATASOURCE%%3", - "%%DATASOURCE%%0" - ] - }, - "intervals": ["2013-08-31/2013-09-01"], - "granularity": "all", - "aggregations": [ - { - "type": "count", - "name": "rows" - }, - { - "type": "longSum", - "fieldName": "count", - "name": "count" - } - ], - "postAggregations": [ - { - "type": "arithmetic", - "name": "sumOfRowsAndCount", - "fn": "+", - "fields": [ - { - "type": "fieldAccess", - "name": "rows", - "fieldName": "rows" - }, - { - "type": "fieldAccess", - "name": "count", - "fieldName": "count" - } - ] - } - ], - "dimensions": ["namespace", "robot"], - "limitSpec": { - "type": "default", - "limit": 3, - "orderBy": ["robot", "namespace"] - }, - "context": { - "useCache": "true", - "populateCache": "true", - "timeout": 60000 - } - }, - "expectedResults": [ - { - "version": "v1", - "timestamp": "2013-08-31T00:00:00.000Z", - "event": { - "sumOfRowsAndCount": 6.0, - "count": 3, - "robot": "false", - "rows": 3, - "namespace": "article" - } - }, - { - "version": "v1", - "timestamp": "2013-08-31T00:00:00.000Z", - "event": { - "sumOfRowsAndCount": 6.0, - "count": 3, - "robot": "true", - "rows": 3, - "namespace": "article" - } - }, - { - "version": "v1", - "timestamp": "2013-08-31T00:00:00.000Z", - "event": { - "sumOfRowsAndCount": 18.0, - "count": 9, - "robot": "true", - "rows": 9, - "namespace": "wikipedia" - } - } - ] - }, - { - "query": { - "queryType": "search", - "intervals": ["2013-08-31/2013-09-01"], - "dataSource": { - "type": "union", - "dataSources": [ - "%%DATASOURCE%%1", "%%DATASOURCE%%2", "%%DATASOURCE%%3", - "%%DATASOURCE%%0" - ] - }, - "granularity": "all", - "query": { - "type": "insensitive_contains", - "value": "ip" - }, - "context": { - "useCache": "true", - "populateCache": "true", - "timeout": 60000 - } - }, - "expectedResults": [ - { - "timestamp": "2013-08-31T00:00:00.000Z", - "result": [ - { - "dimension": "user", - "value": "triplets", - "count":3 - }, - { - "dimension": "namespace", - "value": "wikipedia", - "count":9 - } - ] - } - ] - }, - { - "description": "timeboundary, 1 agg, union", - "query": { - "queryType": "timeBoundary", - "dataSource": { - "type": "union", - "dataSources": [ - "%%DATASOURCE%%1", "%%DATASOURCE%%2", "%%DATASOURCE%%3", - "%%DATASOURCE%%0" - ] - } - }, - "expectedResults": [ - { - "timestamp": "2013-08-31T01:02:33.000Z", - "result": { - "minTime": "2013-08-31T01:02:33.000Z", - "maxTime": "2013-09-01T12:41:27.000Z" - } - } - ] - } -] diff --git a/processing/src/main/java/org/apache/druid/common/guava/ThreadRenamingCallable.java b/processing/src/main/java/org/apache/druid/common/guava/ThreadRenamingCallable.java deleted file mode 100644 index 491d3890a57..00000000000 --- a/processing/src/main/java/org/apache/druid/common/guava/ThreadRenamingCallable.java +++ /dev/null @@ -1,50 +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.common.guava; - -import java.util.concurrent.Callable; - -public abstract class ThreadRenamingCallable implements Callable -{ - private final String name; - - public ThreadRenamingCallable( - String name - ) - { - this.name = name; - } - - @Override - public final T call() - { - final Thread currThread = Thread.currentThread(); - String currName = currThread.getName(); - try { - currThread.setName(name); - return doCall(); - } - finally { - currThread.setName(currName); - } - } - - public abstract T doCall(); -} diff --git a/processing/src/main/java/org/apache/druid/java/util/common/concurrent/ScheduledExecutors.java b/processing/src/main/java/org/apache/druid/java/util/common/concurrent/ScheduledExecutors.java index 97f43f6c9b9..72cdaf15edf 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/concurrent/ScheduledExecutors.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/concurrent/ScheduledExecutors.java @@ -64,16 +64,6 @@ public class ScheduledExecutors ); } - /** - * Run callable repeatedly with the given delay between calls, after the given - * initial delay, until it returns Signal.STOP. Exceptions are caught and - * logged as errors. - */ - public static void scheduleWithFixedDelay(ScheduledExecutorService exec, Duration delay, Callable callable) - { - scheduleWithFixedDelay(exec, delay, delay, callable); - } - /** * Run callable repeatedly with the given delay between calls, until it * returns Signal.STOP. Exceptions are caught and logged as errors. diff --git a/processing/src/main/java/org/apache/druid/query/filter/SelectorDimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/SelectorDimFilter.java index 05cd4730e4b..019862e5014 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/SelectorDimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/SelectorDimFilter.java @@ -34,7 +34,6 @@ import org.apache.druid.segment.filter.DimensionPredicateFilter; import org.apache.druid.segment.filter.SelectorFilter; import javax.annotation.Nullable; - import java.util.Objects; import java.util.Set; diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java index a2fc9cec8a6..7b28f782e31 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java @@ -55,7 +55,6 @@ import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; - import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collections; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java index 19bc6170c12..83952873050 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java @@ -39,7 +39,6 @@ import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import javax.annotation.Nonnull; import javax.annotation.Nullable; - import java.util.Arrays; import java.util.Iterator; import java.util.concurrent.atomic.AtomicInteger; diff --git a/processing/src/main/java/org/apache/druid/segment/Metadata.java b/processing/src/main/java/org/apache/druid/segment/Metadata.java index e8aa5e646de..e6b5b1f6588 100644 --- a/processing/src/main/java/org/apache/druid/segment/Metadata.java +++ b/processing/src/main/java/org/apache/druid/segment/Metadata.java @@ -108,11 +108,6 @@ public class Metadata return this; } - public Object get(String key) - { - return container.get(key); - } - // arbitrary key-value pairs from the metadata just follow the semantics of last one wins if same // key exists in multiple input Metadata containers // for others e.g. Aggregators, appropriate merging is done diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAddResult.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAddResult.java index 077f162f9da..5f84138495c 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAddResult.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAddResult.java @@ -97,10 +97,4 @@ public class IncrementalIndexAddResult { return reasonOfNotAdded == null && parseException == null; } - - @Nullable - public String getReasonOfNotAdded() - { - return reasonOfNotAdded; - } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java index 3085376b822..25fc3bae481 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java @@ -26,8 +26,8 @@ import java.util.Map; /** * A collection of meters for row ingestion stats, with support for moving average calculations. - * This can eventually replace FireDepartmentMetrics, but moving averages for other stats collected by - * FireDepartmentMetrics are not currently supported, so we continue to use FireDepartmentMetrics alongside + * This can eventually replace SegmentGenerationMetrics, but moving averages for other stats collected by + * SegmentGenerationMetrics are not currently supported, so we continue to use SegmentGenerationMetrics alongside * RowIngestionMeters to avoid unnecessary overhead from maintaining these moving averages. */ @ExtensionPoint diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java index 281527b3d1e..56be3d50f20 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java @@ -28,7 +28,6 @@ import org.junit.Assert; import org.junit.Test; import javax.annotation.Nonnull; - import java.util.ArrayList; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumnsTest.java index 3cb3c5772b3..cd1bb1b81ec 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumnsTest.java @@ -23,6 +23,7 @@ import org.apache.druid.query.operator.OffsetLimit; import org.apache.druid.query.rowsandcols.LazilyDecoratedRowsAndColumns; import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumnsTestBase; + import java.util.function.Function; public class ColumnBasedFrameRowsAndColumnsTest extends RowsAndColumnsTestBase diff --git a/server/src/main/java/org/apache/druid/guice/NoopSegmentPublisherProvider.java b/server/src/main/java/org/apache/druid/guice/NoopSegmentPublisherProvider.java deleted file mode 100644 index 254f2ca2823..00000000000 --- a/server/src/main/java/org/apache/druid/guice/NoopSegmentPublisherProvider.java +++ /dev/null @@ -1,35 +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.guice; - -import org.apache.druid.metadata.SegmentPublisherProvider; -import org.apache.druid.segment.realtime.NoopSegmentPublisher; -import org.apache.druid.segment.realtime.SegmentPublisher; - -/** - */ -public class NoopSegmentPublisherProvider implements SegmentPublisherProvider -{ - @Override - public SegmentPublisher get() - { - return new NoopSegmentPublisher(); - } -} diff --git a/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java b/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java index 7ece79ad40f..7894df83af4 100644 --- a/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java +++ b/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java @@ -29,8 +29,6 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.metadata.MetadataRuleManagerProvider; -import org.apache.druid.metadata.MetadataSegmentPublisher; -import org.apache.druid.metadata.MetadataSegmentPublisherProvider; import org.apache.druid.metadata.MetadataStorageActionHandlerFactory; import org.apache.druid.metadata.MetadataStorageConnector; import org.apache.druid.metadata.MetadataStorageProvider; @@ -38,8 +36,6 @@ import org.apache.druid.metadata.MetadataSupervisorManager; import org.apache.druid.metadata.SQLMetadataConnector; import org.apache.druid.metadata.SQLMetadataRuleManager; import org.apache.druid.metadata.SQLMetadataRuleManagerProvider; -import org.apache.druid.metadata.SQLMetadataSegmentPublisher; -import org.apache.druid.metadata.SQLMetadataSegmentPublisherProvider; import org.apache.druid.metadata.SQLMetadataSupervisorManager; import org.apache.druid.metadata.SegmentsMetadataManager; import org.apache.druid.metadata.SegmentsMetadataManagerProvider; @@ -76,8 +72,6 @@ public class SQLMetadataStorageDruidModule implements Module PolyBind.createChoiceWithDefault(binder, prop, Key.get(SegmentsMetadataManagerProvider.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataRuleManager.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataRuleManagerProvider.class), defaultValue); - PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataSegmentPublisher.class), defaultValue); - PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataSegmentPublisherProvider.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(IndexerMetadataStorageCoordinator.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataStorageActionHandlerFactory.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataStorageUpdaterJobHandler.class), defaultValue); @@ -109,16 +103,6 @@ public class SQLMetadataStorageDruidModule implements Module .to(SQLMetadataRuleManagerProvider.class) .in(LazySingleton.class); - PolyBind.optionBinder(binder, Key.get(MetadataSegmentPublisher.class)) - .addBinding(type) - .to(SQLMetadataSegmentPublisher.class) - .in(LazySingleton.class); - - PolyBind.optionBinder(binder, Key.get(MetadataSegmentPublisherProvider.class)) - .addBinding(type) - .to(SQLMetadataSegmentPublisherProvider.class) - .in(LazySingleton.class); - PolyBind.optionBinder(binder, Key.get(IndexerMetadataStorageCoordinator.class)) .addBinding(type) .to(IndexerSQLMetadataStorageCoordinator.class) 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 c02bba6082c..9452d19485f 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 @@ -204,7 +204,7 @@ public interface IndexerMetadataStorageCoordinator * * @return set of segments actually added */ - Set commitSegments(Set segments, @Nullable SegmentSchemaMapping segmentSchemaMapping) throws IOException; + Set commitSegments(Set segments, @Nullable SegmentSchemaMapping segmentSchemaMapping); /** * Allocates pending segments for the given requests in the pending segments table. @@ -316,7 +316,7 @@ public interface IndexerMetadataStorageCoordinator @Nullable DataSourceMetadata startMetadata, @Nullable DataSourceMetadata endMetadata, @Nullable SegmentSchemaMapping segmentSchemaMapping - ) throws IOException; + ); /** * Commits segments and corresponding schema created by an APPEND task. diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 72367330232..75d296923fb 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -450,7 +450,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor public Set commitSegments( final Set segments, @Nullable final SegmentSchemaMapping segmentSchemaMapping - ) throws IOException + ) { final SegmentPublishResult result = commitSegmentsAndMetadata( @@ -474,7 +474,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor @Nullable final DataSourceMetadata startMetadata, @Nullable final DataSourceMetadata endMetadata, @Nullable final SegmentSchemaMapping segmentSchemaMapping - ) throws IOException + ) { verifySegmentsToCommit(segments); diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentPublisherProvider.java b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentPublisherProvider.java deleted file mode 100644 index b493148ec90..00000000000 --- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentPublisherProvider.java +++ /dev/null @@ -1,26 +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.metadata; - -public interface MetadataSegmentPublisherProvider extends SegmentPublisherProvider -{ - @Override - MetadataSegmentPublisher get(); -} diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisher.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisher.java deleted file mode 100644 index 48a92ecba4e..00000000000 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisher.java +++ /dev/null @@ -1,126 +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.metadata; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.inject.Inject; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.NoneShardSpec; -import org.skife.jdbi.v2.DBI; - -import java.io.IOException; -import java.util.List; -import java.util.Map; - -public class SQLMetadataSegmentPublisher implements MetadataSegmentPublisher -{ - private static final Logger log = new Logger(SQLMetadataSegmentPublisher.class); - - private final ObjectMapper jsonMapper; - private final MetadataStorageTablesConfig config; - private final SQLMetadataConnector connector; - private final String statement; - - @Inject - public SQLMetadataSegmentPublisher( - ObjectMapper jsonMapper, - MetadataStorageTablesConfig config, - SQLMetadataConnector connector - ) - { - this.jsonMapper = jsonMapper; - this.config = config; - this.connector = connector; - this.statement = StringUtils.format( - "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload, used_status_last_updated) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload, :used_status_last_updated)", - config.getSegmentsTable(), connector.getQuoteString() - ); - } - - @Override - public void publishSegment(final DataSegment segment) throws IOException - { - String now = DateTimes.nowUtc().toString(); - publishSegment( - segment.getId().toString(), - segment.getDataSource(), - now, - segment.getInterval().getStart().toString(), - segment.getInterval().getEnd().toString(), - (segment.getShardSpec() instanceof NoneShardSpec) ? false : true, - segment.getVersion(), - true, - jsonMapper.writeValueAsBytes(segment), - now - ); - } - - private void publishSegment( - final String segmentId, - final String dataSource, - final String createdDate, - final String start, - final String end, - final boolean partitioned, - final String version, - final boolean used, - final byte[] payload, - final String usedFlagLastUpdated - ) - { - try { - final DBI dbi = connector.getDBI(); - List> exists = dbi.withHandle( - handle -> handle.createQuery( - StringUtils.format("SELECT id FROM %s WHERE id=:id", config.getSegmentsTable()) - ).bind("id", segmentId).list() - ); - - if (!exists.isEmpty()) { - log.info("Skipping publish of segment[%s] as it already exists in the metadata store.", segmentId); - return; - } - - dbi.withHandle( - handle -> - handle.createStatement(statement) - .bind("id", segmentId) - .bind("dataSource", dataSource) - .bind("created_date", createdDate) - .bind("start", start) - .bind("end", end) - .bind("partitioned", partitioned) - .bind("version", version) - .bind("used", used) - .bind("payload", payload) - .bind("used_status_last_updated", usedFlagLastUpdated) - .execute() - ); - } - catch (Exception e) { - log.error(e, "Exception inserting into DB"); - throw new RuntimeException(e); - } - } -} diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisherProvider.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisherProvider.java deleted file mode 100644 index 89ba4bf9bc7..00000000000 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisherProvider.java +++ /dev/null @@ -1,48 +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.metadata; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.databind.ObjectMapper; - -import javax.validation.constraints.NotNull; - -/** - */ -public class SQLMetadataSegmentPublisherProvider implements MetadataSegmentPublisherProvider -{ - @JacksonInject - @NotNull - private SQLMetadataConnector connector = null; - - @JacksonInject - @NotNull - private MetadataStorageTablesConfig config = null; - - @JacksonInject - @NotNull - private ObjectMapper jsonMapper = null; - - @Override - public MetadataSegmentPublisher get() - { - return new SQLMetadataSegmentPublisher(jsonMapper, config, connector); - } -} diff --git a/server/src/main/java/org/apache/druid/metadata/SegmentPublisherProvider.java b/server/src/main/java/org/apache/druid/metadata/SegmentPublisherProvider.java deleted file mode 100644 index c33c3a826f2..00000000000 --- a/server/src/main/java/org/apache/druid/metadata/SegmentPublisherProvider.java +++ /dev/null @@ -1,36 +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.metadata; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.google.inject.Provider; -import org.apache.druid.guice.NoopSegmentPublisherProvider; -import org.apache.druid.segment.realtime.SegmentPublisher; - -/** - */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = NoopSegmentPublisherProvider.class) -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "metadata", value = MetadataSegmentPublisherProvider.class) -}) -public interface SegmentPublisherProvider extends Provider -{ -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/NoopSegmentHandoffNotifierFactory.java b/server/src/main/java/org/apache/druid/segment/handoff/NoopSegmentHandoffNotifierFactory.java similarity index 90% rename from server/src/main/java/org/apache/druid/segment/realtime/plumber/NoopSegmentHandoffNotifierFactory.java rename to server/src/main/java/org/apache/druid/segment/handoff/NoopSegmentHandoffNotifierFactory.java index 9a51143d64b..bd04aa60839 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/NoopSegmentHandoffNotifierFactory.java +++ b/server/src/main/java/org/apache/druid/segment/handoff/NoopSegmentHandoffNotifierFactory.java @@ -17,12 +17,10 @@ * under the License. */ -package org.apache.druid.segment.realtime.plumber; +package org.apache.druid.segment.handoff; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.segment.handoff.SegmentHandoffNotifier; -import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import java.util.concurrent.Executor; diff --git a/server/src/main/java/org/apache/druid/segment/indexing/IOConfig.java b/server/src/main/java/org/apache/druid/segment/indexing/IOConfig.java index b1784806e7e..9a783980a42 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/IOConfig.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/IOConfig.java @@ -19,15 +19,11 @@ package org.apache.druid.segment.indexing; -import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; /** */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "realtime", value = RealtimeIOConfig.class) -}) public interface IOConfig { } diff --git a/server/src/main/java/org/apache/druid/segment/indexing/RealtimeIOConfig.java b/server/src/main/java/org/apache/druid/segment/indexing/RealtimeIOConfig.java deleted file mode 100644 index ba8eff09d1e..00000000000 --- a/server/src/main/java/org/apache/druid/segment/indexing/RealtimeIOConfig.java +++ /dev/null @@ -1,54 +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.segment.indexing; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.segment.realtime.plumber.PlumberSchool; - -/** - */ -public class RealtimeIOConfig implements IOConfig -{ - private final FirehoseFactory firehoseFactory; - private final PlumberSchool plumberSchool; - - @JsonCreator - public RealtimeIOConfig( - @JsonProperty("firehose") FirehoseFactory firehoseFactory, - @JsonProperty("plumber") PlumberSchool plumberSchool - ) - { - this.firehoseFactory = firehoseFactory; - this.plumberSchool = plumberSchool; - } - - @JsonProperty("firehose") - public FirehoseFactory getFirehoseFactory() - { - return firehoseFactory; - } - - public PlumberSchool getPlumberSchool() - { - return plumberSchool; - } -} diff --git a/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java deleted file mode 100644 index a75a79b4dd9..00000000000 --- a/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java +++ /dev/null @@ -1,420 +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.segment.indexing; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.incremental.AppendableIndexSpec; -import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; -import org.apache.druid.segment.realtime.plumber.IntervalStartVersioningPolicy; -import org.apache.druid.segment.realtime.plumber.RejectionPolicyFactory; -import org.apache.druid.segment.realtime.plumber.ServerTimeRejectionPolicyFactory; -import org.apache.druid.segment.realtime.plumber.VersioningPolicy; -import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; -import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.apache.druid.timeline.partition.ShardSpec; -import org.joda.time.Period; - -import javax.annotation.Nullable; -import java.io.File; -import java.time.Duration; - -/** - * - */ -public class RealtimeTuningConfig implements AppenderatorConfig -{ - private static final Period DEFAULT_INTERMEDIATE_PERSIST_PERIOD = new Period("PT10M"); - private static final Period DEFAULT_WINDOW_PERIOD = new Period("PT10M"); - private static final VersioningPolicy DEFAULT_VERSIONING_POLICY = new IntervalStartVersioningPolicy(); - private static final RejectionPolicyFactory DEFAULT_REJECTION_POLICY_FACTORY = new ServerTimeRejectionPolicyFactory(); - private static final int DEFAULT_MAX_PENDING_PERSISTS = 0; - private static final ShardSpec DEFAULT_SHARD_SPEC = new NumberedShardSpec(0, 1); - private static final IndexSpec DEFAULT_INDEX_SPEC = IndexSpec.DEFAULT; - private static final Boolean DEFAULT_REPORT_PARSE_EXCEPTIONS = Boolean.FALSE; - private static final long DEFAULT_HANDOFF_CONDITION_TIMEOUT = Duration.ofMinutes(15).toMillis(); - private static final long DEFAULT_ALERT_TIMEOUT = 0; - private static final String DEFAULT_DEDUP_COLUMN = null; - - // Might make sense for this to be a builder - public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File basePersistDirectory) - { - return new RealtimeTuningConfig( - DEFAULT_APPENDABLE_INDEX, - DEFAULT_MAX_ROWS_IN_MEMORY_REALTIME, - 0L, - DEFAULT_SKIP_BYTES_IN_MEMORY_OVERHEAD_CHECK, - DEFAULT_INTERMEDIATE_PERSIST_PERIOD, - DEFAULT_WINDOW_PERIOD, - basePersistDirectory, - DEFAULT_VERSIONING_POLICY, - DEFAULT_REJECTION_POLICY_FACTORY, - DEFAULT_MAX_PENDING_PERSISTS, - DEFAULT_SHARD_SPEC, - DEFAULT_INDEX_SPEC, - DEFAULT_INDEX_SPEC, - 0, - 0, - DEFAULT_REPORT_PARSE_EXCEPTIONS, - DEFAULT_HANDOFF_CONDITION_TIMEOUT, - DEFAULT_ALERT_TIMEOUT, - null, - DEFAULT_DEDUP_COLUMN, - DEFAULT_NUM_PERSIST_THREADS - ); - } - - private final AppendableIndexSpec appendableIndexSpec; - private final int maxRowsInMemory; - private final long maxBytesInMemory; - private final boolean skipBytesInMemoryOverheadCheck; - private final Period intermediatePersistPeriod; - private final Period windowPeriod; - private final File basePersistDirectory; - private final VersioningPolicy versioningPolicy; - private final RejectionPolicyFactory rejectionPolicyFactory; - private final int maxPendingPersists; - private final ShardSpec shardSpec; - private final IndexSpec indexSpec; - private final IndexSpec indexSpecForIntermediatePersists; - private final int persistThreadPriority; - private final int mergeThreadPriority; - private final boolean reportParseExceptions; - private final long handoffConditionTimeout; - private final long alertTimeout; - @Nullable - private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; - @Nullable - private final String dedupColumn; - private final int numPersistThreads; - - public RealtimeTuningConfig( - @Nullable AppendableIndexSpec appendableIndexSpec, - Integer maxRowsInMemory, - Long maxBytesInMemory, - @Nullable Boolean skipBytesInMemoryOverheadCheck, - Period intermediatePersistPeriod, - Period windowPeriod, - File basePersistDirectory, - VersioningPolicy versioningPolicy, - RejectionPolicyFactory rejectionPolicyFactory, - Integer maxPendingPersists, - ShardSpec shardSpec, - IndexSpec indexSpec, - @Nullable IndexSpec indexSpecForIntermediatePersists, - int persistThreadPriority, - int mergeThreadPriority, - Boolean reportParseExceptions, - Long handoffConditionTimeout, - Long alertTimeout, - @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, - @Nullable String dedupColumn, - @Nullable Integer numPersistThreads - ) - { - this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; - this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY_REALTIME : maxRowsInMemory; - // initializing this to 0, it will be lazily initialized to a value - // @see #getMaxBytesInMemoryOrDefault() - this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; - this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck == null ? - DEFAULT_SKIP_BYTES_IN_MEMORY_OVERHEAD_CHECK : skipBytesInMemoryOverheadCheck; - this.intermediatePersistPeriod = intermediatePersistPeriod == null - ? DEFAULT_INTERMEDIATE_PERSIST_PERIOD - : intermediatePersistPeriod; - this.windowPeriod = windowPeriod == null ? DEFAULT_WINDOW_PERIOD : windowPeriod; - this.basePersistDirectory = basePersistDirectory; - this.versioningPolicy = versioningPolicy; - this.rejectionPolicyFactory = rejectionPolicyFactory == null - ? DEFAULT_REJECTION_POLICY_FACTORY - : rejectionPolicyFactory; - this.maxPendingPersists = maxPendingPersists == null ? DEFAULT_MAX_PENDING_PERSISTS : maxPendingPersists; - this.shardSpec = shardSpec == null ? DEFAULT_SHARD_SPEC : shardSpec; - this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec; - this.indexSpecForIntermediatePersists = indexSpecForIntermediatePersists == null ? - this.indexSpec : indexSpecForIntermediatePersists; - this.mergeThreadPriority = mergeThreadPriority; - this.persistThreadPriority = persistThreadPriority; - this.reportParseExceptions = reportParseExceptions == null - ? DEFAULT_REPORT_PARSE_EXCEPTIONS - : reportParseExceptions; - this.handoffConditionTimeout = handoffConditionTimeout == null - ? DEFAULT_HANDOFF_CONDITION_TIMEOUT - : handoffConditionTimeout; - Preconditions.checkArgument(this.handoffConditionTimeout >= 0, "handoffConditionTimeout must be >= 0"); - - this.alertTimeout = alertTimeout == null ? DEFAULT_ALERT_TIMEOUT : alertTimeout; - Preconditions.checkArgument(this.alertTimeout >= 0, "alertTimeout must be >= 0"); - this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; - this.dedupColumn = dedupColumn == null ? DEFAULT_DEDUP_COLUMN : dedupColumn; - this.numPersistThreads = numPersistThreads == null ? - DEFAULT_NUM_PERSIST_THREADS : Math.max(numPersistThreads, DEFAULT_NUM_PERSIST_THREADS); - } - - @JsonCreator - private RealtimeTuningConfig( - @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, - @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, - @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, - @JsonProperty("skipBytesInMemoryOverheadCheck") @Nullable Boolean skipBytesInMemoryOverheadCheck, - @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, - @JsonProperty("windowPeriod") Period windowPeriod, - @JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory, - @JsonProperty("maxPendingPersists") Integer maxPendingPersists, - @JsonProperty("shardSpec") ShardSpec shardSpec, - @JsonProperty("indexSpec") IndexSpec indexSpec, - @JsonProperty("indexSpecForIntermediatePersists") @Nullable IndexSpec indexSpecForIntermediatePersists, - @JsonProperty("persistThreadPriority") int persistThreadPriority, - @JsonProperty("mergeThreadPriority") int mergeThreadPriority, - @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, - @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, - @JsonProperty("alertTimeout") Long alertTimeout, - @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, - @JsonProperty("dedupColumn") @Nullable String dedupColumn, - @JsonProperty("numPersistThreads") @Nullable Integer numPersistThreads - ) - { - this( - appendableIndexSpec, - maxRowsInMemory, - maxBytesInMemory, - skipBytesInMemoryOverheadCheck, - intermediatePersistPeriod, - windowPeriod, - null, - null, - rejectionPolicyFactory, - maxPendingPersists, - shardSpec, - indexSpec, - indexSpecForIntermediatePersists, - persistThreadPriority, - mergeThreadPriority, - reportParseExceptions, - handoffConditionTimeout, - alertTimeout, - segmentWriteOutMediumFactory, - dedupColumn, - numPersistThreads - ); - } - - @Override - @JsonProperty - public AppendableIndexSpec getAppendableIndexSpec() - { - return appendableIndexSpec; - } - - @Override - @JsonProperty - public int getMaxRowsInMemory() - { - return maxRowsInMemory; - } - - @Override - @JsonProperty - public long getMaxBytesInMemory() - { - return maxBytesInMemory; - } - - @JsonProperty - @Override - public boolean isSkipBytesInMemoryOverheadCheck() - { - return skipBytesInMemoryOverheadCheck; - } - - @Override - @JsonProperty - public Period getIntermediatePersistPeriod() - { - return intermediatePersistPeriod; - } - - @JsonProperty - public Period getWindowPeriod() - { - return windowPeriod; - } - - @Override - public File getBasePersistDirectory() - { - return Preconditions.checkNotNull(basePersistDirectory, "basePersistDirectory not set"); - } - - public VersioningPolicy getVersioningPolicy() - { - return Preconditions.checkNotNull(versioningPolicy, "versioningPolicy not set"); - } - - @JsonProperty("rejectionPolicy") - public RejectionPolicyFactory getRejectionPolicyFactory() - { - return rejectionPolicyFactory; - } - - @Override - @JsonProperty - public int getMaxPendingPersists() - { - return maxPendingPersists; - } - - @Override - public PartitionsSpec getPartitionsSpec() - { - throw new UnsupportedOperationException(); - } - - @JsonProperty - public ShardSpec getShardSpec() - { - return shardSpec; - } - - @Override - @JsonProperty - public IndexSpec getIndexSpec() - { - return indexSpec; - } - - @JsonProperty - @Override - public IndexSpec getIndexSpecForIntermediatePersists() - { - return indexSpecForIntermediatePersists; - } - - @JsonProperty - public int getPersistThreadPriority() - { - return this.persistThreadPriority; - } - - @JsonProperty - public int getMergeThreadPriority() - { - return this.mergeThreadPriority; - } - - @Override - @JsonProperty - public boolean isReportParseExceptions() - { - return reportParseExceptions; - } - - @JsonProperty - public long getHandoffConditionTimeout() - { - return handoffConditionTimeout; - } - - @JsonProperty - public long getAlertTimeout() - { - return alertTimeout; - } - - @Override - @JsonProperty - @Nullable - public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() - { - return segmentWriteOutMediumFactory; - } - - @JsonProperty - @Nullable - public String getDedupColumn() - { - return dedupColumn; - } - - @Override - @JsonProperty - public int getNumPersistThreads() - { - return numPersistThreads; - } - - public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy) - { - return new RealtimeTuningConfig( - appendableIndexSpec, - maxRowsInMemory, - maxBytesInMemory, - skipBytesInMemoryOverheadCheck, - intermediatePersistPeriod, - windowPeriod, - basePersistDirectory, - policy, - rejectionPolicyFactory, - maxPendingPersists, - shardSpec, - indexSpec, - indexSpecForIntermediatePersists, - persistThreadPriority, - mergeThreadPriority, - reportParseExceptions, - handoffConditionTimeout, - alertTimeout, - segmentWriteOutMediumFactory, - dedupColumn, - numPersistThreads - ); - } - - @Override - public RealtimeTuningConfig withBasePersistDirectory(File dir) - { - return new RealtimeTuningConfig( - appendableIndexSpec, - maxRowsInMemory, - maxBytesInMemory, - skipBytesInMemoryOverheadCheck, - intermediatePersistPeriod, - windowPeriod, - dir, - versioningPolicy, - rejectionPolicyFactory, - maxPendingPersists, - shardSpec, - indexSpec, - indexSpecForIntermediatePersists, - persistThreadPriority, - mergeThreadPriority, - reportParseExceptions, - handoffConditionTimeout, - alertTimeout, - segmentWriteOutMediumFactory, - dedupColumn, - numPersistThreads - ); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java b/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java index ba638c2c485..e190106281a 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java @@ -19,7 +19,6 @@ package org.apache.druid.segment.indexing; -import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.segment.IndexSpec; @@ -29,9 +28,6 @@ import org.apache.druid.segment.incremental.OnheapIncrementalIndex; /** */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "realtime", value = RealtimeTuningConfig.class) -}) public interface TuningConfig { boolean DEFAULT_LOG_PARSE_EXCEPTIONS = false; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/FireDepartment.java b/server/src/main/java/org/apache/druid/segment/realtime/FireDepartment.java deleted file mode 100644 index 2c25a1de174..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/FireDepartment.java +++ /dev/null @@ -1,100 +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.segment.realtime; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.IngestionSpec; -import org.apache.druid.segment.indexing.RealtimeIOConfig; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; - -import java.io.IOException; - -/** - * A Fire Department has a Firehose and a Plumber. - *

- * This is a metaphor for a realtime stream (Firehose) and a coordinator of sinks (Plumber). The Firehose provides the - * realtime stream of data. The Plumber directs each drop of water from the firehose into the correct sink and makes - * sure that the sinks don't overflow. - */ -public class FireDepartment extends IngestionSpec -{ - private final DataSchema dataSchema; - private final RealtimeIOConfig ioConfig; - private final RealtimeTuningConfig tuningConfig; - private final FireDepartmentMetrics metrics = new FireDepartmentMetrics(); - - @JsonCreator - public FireDepartment( - @JsonProperty("dataSchema") DataSchema dataSchema, - @JsonProperty("ioConfig") RealtimeIOConfig ioConfig, - @JsonProperty("tuningConfig") RealtimeTuningConfig tuningConfig - ) - { - super(dataSchema, ioConfig, tuningConfig); - Preconditions.checkNotNull(dataSchema, "dataSchema"); - Preconditions.checkNotNull(ioConfig, "ioConfig"); - - this.dataSchema = dataSchema; - this.ioConfig = ioConfig; - this.tuningConfig = tuningConfig == null ? RealtimeTuningConfig.makeDefaultTuningConfig(null) : tuningConfig; - - } - - /** - * Provides the data schema for the feed that this FireDepartment is in charge of. - * - * @return the Schema for this feed. - */ - @JsonProperty("dataSchema") - @Override - public DataSchema getDataSchema() - { - return dataSchema; - } - - @JsonProperty("ioConfig") - @Override - public RealtimeIOConfig getIOConfig() - { - return ioConfig; - } - - @JsonProperty("tuningConfig") - @Override - public RealtimeTuningConfig getTuningConfig() - { - return tuningConfig; - } - - public Firehose connect() throws IOException - { - return ioConfig.getFirehoseFactory() - .connect(Preconditions.checkNotNull(dataSchema.getParser(), "inputRowParser"), null); - } - - public FireDepartmentMetrics getMetrics() - { - return metrics; - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/FireDepartmentConfig.java b/server/src/main/java/org/apache/druid/segment/realtime/FireDepartmentConfig.java deleted file mode 100644 index 299bc6e5455..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/FireDepartmentConfig.java +++ /dev/null @@ -1,58 +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.segment.realtime; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import org.joda.time.Period; - -/** - */ -public class FireDepartmentConfig -{ - private final int maxRowsInMemory; - private final Period intermediatePersistPeriod; - - @JsonCreator - public FireDepartmentConfig( - @JsonProperty("maxRowsInMemory") int maxRowsInMemory, - @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod - ) - { - this.maxRowsInMemory = maxRowsInMemory; - this.intermediatePersistPeriod = intermediatePersistPeriod; - - Preconditions.checkArgument(maxRowsInMemory > 0, "maxRowsInMemory[%s] should be greater than 0", maxRowsInMemory); - Preconditions.checkNotNull(intermediatePersistPeriod, "intermediatePersistPeriod"); - } - - @JsonProperty - public int getMaxRowsInMemory() - { - return maxRowsInMemory; - } - - @JsonProperty - public Period getIntermediatePersistPeriod() - { - return intermediatePersistPeriod; - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/NoopSegmentPublisher.java b/server/src/main/java/org/apache/druid/segment/realtime/NoopSegmentPublisher.java deleted file mode 100644 index ef1283843a4..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/NoopSegmentPublisher.java +++ /dev/null @@ -1,33 +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.segment.realtime; - -import org.apache.druid.timeline.DataSegment; - -/** - */ -public class NoopSegmentPublisher implements SegmentPublisher -{ - @Override - public void publishSegment(DataSegment segment) - { - // do nothing - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/RealtimeMetricsMonitor.java b/server/src/main/java/org/apache/druid/segment/realtime/RealtimeMetricsMonitor.java deleted file mode 100644 index c923c9e7bbd..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/RealtimeMetricsMonitor.java +++ /dev/null @@ -1,132 +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.segment.realtime; - -import com.google.common.collect.ImmutableMap; -import com.google.inject.Inject; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; -import org.apache.druid.java.util.metrics.AbstractMonitor; -import org.apache.druid.java.util.metrics.MonitorUtils; -import org.apache.druid.query.DruidMetrics; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * RealtimeMetricsMonitor is only used by RealtimeIndexTask, this monitor only supports FireDepartmentMetrics. - * New ingestion task types should support RowIngestionMeters and use TaskRealtimeMetricsMonitor instead. - * Please see the comment on RowIngestionMeters for more information regarding the relationship between - * RowIngestionMeters and FireDepartmentMetrics. - */ -public class RealtimeMetricsMonitor extends AbstractMonitor -{ - private static final EmittingLogger log = new EmittingLogger(RealtimeMetricsMonitor.class); - - private final Map previousValues; - private final List fireDepartments; - private final Map dimensions; - - @Inject - public RealtimeMetricsMonitor(List fireDepartments) - { - this(fireDepartments, ImmutableMap.of()); - } - - public RealtimeMetricsMonitor(List fireDepartments, Map dimensions) - { - this.fireDepartments = fireDepartments; - this.previousValues = new HashMap<>(); - this.dimensions = ImmutableMap.copyOf(dimensions); - } - - @Override - public boolean doMonitor(ServiceEmitter emitter) - { - for (FireDepartment fireDepartment : fireDepartments) { - FireDepartmentMetrics metrics = fireDepartment.getMetrics().snapshot(); - FireDepartmentMetrics previous = previousValues.get(fireDepartment); - - if (previous == null) { - previous = new FireDepartmentMetrics(); - } - - final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DATASOURCE, fireDepartment.getDataSchema().getDataSource()); - MonitorUtils.addDimensionsToBuilder(builder, dimensions); - - final long thrownAway = metrics.thrownAway() - previous.thrownAway(); - if (thrownAway > 0) { - log.warn( - "[%,d] events thrown away. Possible causes: null events, events filtered out by transformSpec, or events outside windowPeriod.", - thrownAway - ); - } - emitter.emit(builder.setMetric("ingest/events/thrownAway", thrownAway)); - final long unparseable = metrics.unparseable() - previous.unparseable(); - if (unparseable > 0) { - log.error( - "[%,d] unparseable events discarded. Turn on debug logging to see exception stack trace.", - unparseable - ); - } - emitter.emit(builder.setMetric("ingest/events/unparseable", unparseable)); - final long dedup = metrics.dedup() - previous.dedup(); - if (dedup > 0) { - log.warn("[%,d] duplicate events!", dedup); - } - emitter.emit(builder.setMetric("ingest/events/duplicate", dedup)); - - emitter.emit(builder.setMetric("ingest/events/processed", metrics.processed() - previous.processed())); - emitter.emit(builder.setMetric("ingest/rows/output", metrics.rowOutput() - previous.rowOutput())); - emitter.emit(builder.setMetric("ingest/persists/count", metrics.numPersists() - previous.numPersists())); - emitter.emit(builder.setMetric("ingest/persists/time", metrics.persistTimeMillis() - previous.persistTimeMillis())); - emitter.emit(builder.setMetric("ingest/persists/cpu", metrics.persistCpuTime() - previous.persistCpuTime())); - emitter.emit( - builder.setMetric( - "ingest/persists/backPressure", - metrics.persistBackPressureMillis() - previous.persistBackPressureMillis() - ) - ); - emitter.emit(builder.setMetric("ingest/persists/failed", metrics.failedPersists() - previous.failedPersists())); - emitter.emit(builder.setMetric("ingest/handoff/failed", metrics.failedHandoffs() - previous.failedHandoffs())); - emitter.emit(builder.setMetric("ingest/merge/time", metrics.mergeTimeMillis() - previous.mergeTimeMillis())); - emitter.emit(builder.setMetric("ingest/merge/cpu", metrics.mergeCpuTime() - previous.mergeCpuTime())); - emitter.emit(builder.setMetric("ingest/handoff/count", metrics.handOffCount() - previous.handOffCount())); - emitter.emit(builder.setMetric("ingest/sink/count", metrics.sinkCount())); - - long messageGap = metrics.messageGap(); - if (messageGap >= 0) { - emitter.emit(builder.setMetric("ingest/events/messageGap", messageGap)); - } - - long maxSegmentHandoffTime = metrics.maxSegmentHandoffTime(); - if (maxSegmentHandoffTime >= 0) { - emitter.emit(builder.setMetric("ingest/handoff/time", maxSegmentHandoffTime)); - } - - previousValues.put(fireDepartment, metrics); - } - - return true; - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/FireDepartmentMetrics.java b/server/src/main/java/org/apache/druid/segment/realtime/SegmentGenerationMetrics.java similarity index 79% rename from server/src/main/java/org/apache/druid/segment/realtime/FireDepartmentMetrics.java rename to server/src/main/java/org/apache/druid/segment/realtime/SegmentGenerationMetrics.java index 8fb1f1309c1..564a3322bf9 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/FireDepartmentMetrics.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/SegmentGenerationMetrics.java @@ -25,17 +25,14 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; /** + * Metrics for segment generation. */ -public class FireDepartmentMetrics +public class SegmentGenerationMetrics { private static final long NO_EMIT_SEGMENT_HANDOFF_TIME = -1L; private static final long NO_EMIT_MESSAGE_GAP = -1L; - private final AtomicLong processedCount = new AtomicLong(0); - private final AtomicLong processedWithErrorsCount = new AtomicLong(0); - private final AtomicLong thrownAwayCount = new AtomicLong(0); - private final AtomicLong unparseableCount = new AtomicLong(0); private final AtomicLong dedupCount = new AtomicLong(0); private final AtomicLong rowOutputCount = new AtomicLong(0); private final AtomicLong numPersists = new AtomicLong(0); @@ -58,31 +55,6 @@ public class FireDepartmentMetrics private final AtomicLong maxSegmentHandoffTime = new AtomicLong(NO_EMIT_SEGMENT_HANDOFF_TIME); - public void incrementProcessed() - { - processedCount.incrementAndGet(); - } - - public void incrementProcessedWithErrors() - { - processedWithErrorsCount.incrementAndGet(); - } - - public void incrementThrownAway() - { - thrownAwayCount.incrementAndGet(); - } - - public void incrementDedup() - { - dedupCount.incrementAndGet(); - } - - public void incrementUnparseable() - { - unparseableCount.incrementAndGet(); - } - public void incrementRowOutputCount(long numRows) { rowOutputCount.addAndGet(numRows); @@ -113,11 +85,6 @@ public class FireDepartmentMetrics failedHandoffs.incrementAndGet(); } - public void incrementMergeTimeMillis(long millis) - { - mergeTimeMillis.addAndGet(millis); - } - public void incrementMergedRows(long rows) { mergedRows.addAndGet(rows); @@ -128,16 +95,6 @@ public class FireDepartmentMetrics pushedRows.addAndGet(rows); } - public void incrementMergeCpuTime(long mergeTime) - { - mergeCpuTime.addAndGet(mergeTime); - } - - public void incrementPersistCpuTime(long persistTime) - { - persistCpuTime.addAndGet(persistTime); - } - public void incrementHandOffCount() { handOffCount.incrementAndGet(); @@ -169,26 +126,6 @@ public class FireDepartmentMetrics return processingDone.get(); } - public long processed() - { - return processedCount.get(); - } - - public long processedWithErrors() - { - return processedWithErrorsCount.get(); - } - - public long thrownAway() - { - return thrownAwayCount.get(); - } - - public long unparseable() - { - return unparseableCount.get(); - } - public long dedup() { return dedupCount.get(); @@ -268,13 +205,9 @@ public class FireDepartmentMetrics return maxSegmentHandoffTime.get(); } - public FireDepartmentMetrics snapshot() + public SegmentGenerationMetrics snapshot() { - final FireDepartmentMetrics retVal = new FireDepartmentMetrics(); - retVal.processedCount.set(processedCount.get()); - retVal.processedWithErrorsCount.set(processedWithErrorsCount.get()); - retVal.thrownAwayCount.set(thrownAwayCount.get()); - retVal.unparseableCount.set(unparseableCount.get()); + final SegmentGenerationMetrics retVal = new SegmentGenerationMetrics(); retVal.dedupCount.set(dedupCount.get()); retVal.rowOutputCount.set(rowOutputCount.get()); retVal.numPersists.set(numPersists.get()); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/SegmentPublisher.java b/server/src/main/java/org/apache/druid/segment/realtime/SegmentPublisher.java deleted file mode 100644 index 7910e211ca5..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/SegmentPublisher.java +++ /dev/null @@ -1,29 +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.segment.realtime; - -import org.apache.druid.timeline.DataSegment; - -import java.io.IOException; - -public interface SegmentPublisher -{ - void publishSegment(DataSegment segment) throws IOException; -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorFactory.java deleted file mode 100644 index 4b8698490e9..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorFactory.java +++ /dev/null @@ -1,40 +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.segment.realtime.appenderator; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; - -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "default", value = DefaultRealtimeAppenderatorFactory.class), - @JsonSubTypes.Type(name = "offline", value = DefaultOfflineAppenderatorFactory.class) -}) -public interface AppenderatorFactory -{ - Appenderator build( - DataSchema schema, - RealtimeTuningConfig config, - FireDepartmentMetrics metrics - ); -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java index 764d7239736..734abc6ed54 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -73,9 +73,9 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.FingerprintGenerator; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.FireHydrant; -import org.apache.druid.segment.realtime.plumber.Sink; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; +import org.apache.druid.segment.realtime.sink.Sink; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -135,7 +135,7 @@ public class AppenderatorImpl implements Appenderator private final String myId; private final DataSchema schema; private final AppenderatorConfig tuningConfig; - private final FireDepartmentMetrics metrics; + private final SegmentGenerationMetrics metrics; private final DataSegmentPusher dataSegmentPusher; private final ObjectMapper objectMapper; private final DataSegmentAnnouncer segmentAnnouncer; @@ -211,7 +211,7 @@ public class AppenderatorImpl implements Appenderator String id, DataSchema schema, AppenderatorConfig tuningConfig, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, DataSegmentAnnouncer segmentAnnouncer, @@ -514,8 +514,7 @@ public class AppenderatorImpl implements Appenderator tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), maxBytesTuningConfig, - useMaxMemoryEstimates, - null + useMaxMemoryEstimates ); bytesCurrentlyInMemory.addAndGet(calculateSinkMemoryInUsed()); @@ -1340,7 +1339,6 @@ public class AppenderatorImpl implements Appenderator tuningConfig.getMaxRowsInMemory(), maxBytesTuningConfig, useMaxMemoryEstimates, - null, hydrants ); rowsSoFar += currSink.getNumRows(); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumber.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumber.java deleted file mode 100644 index 56af6382afa..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumber.java +++ /dev/null @@ -1,492 +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.segment.realtime.appenderator; - -import com.google.common.base.Function; -import com.google.common.base.Joiner; -import com.google.common.base.Stopwatch; -import com.google.common.base.Supplier; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.MoreExecutors; -import org.apache.druid.common.guava.ThreadRenamingCallable; -import org.apache.druid.data.input.Committer; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; -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.ScheduledExecutors; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.query.Query; -import org.apache.druid.query.QueryPlus; -import org.apache.druid.query.QueryRunner; -import org.apache.druid.query.context.ResponseContext; -import org.apache.druid.segment.handoff.SegmentHandoffNotifier; -import org.apache.druid.segment.incremental.IncrementalIndexAddResult; -import org.apache.druid.segment.incremental.IndexSizeExceededException; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.segment.realtime.SegmentPublisher; -import org.apache.druid.segment.realtime.plumber.Committers; -import org.apache.druid.segment.realtime.plumber.Plumber; -import org.apache.druid.segment.realtime.plumber.RejectionPolicy; -import org.apache.druid.segment.realtime.plumber.VersioningPolicy; -import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.apache.druid.timeline.DataSegment; -import org.joda.time.DateTime; -import org.joda.time.Duration; -import org.joda.time.Interval; -import org.joda.time.Period; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; - -public class AppenderatorPlumber implements Plumber -{ - private static final EmittingLogger log = new EmittingLogger(AppenderatorPlumber.class); - private static final int WARN_DELAY = 1000; - - private final DataSchema schema; - private final RealtimeTuningConfig config; - private final RejectionPolicy rejectionPolicy; - private final FireDepartmentMetrics metrics; - private final DataSegmentAnnouncer segmentAnnouncer; - private final SegmentPublisher segmentPublisher; - private final SegmentHandoffNotifier handoffNotifier; - private final Object handoffCondition = new Object(); - private final ConcurrentMap segments = new ConcurrentHashMap<>(); - private final Appenderator appenderator; - - private volatile boolean shuttingDown = false; - private volatile boolean stopped = false; - private volatile boolean cleanShutdown = true; - private volatile ScheduledExecutorService scheduledExecutor = null; - - private volatile Supplier lastCommitterSupplier = null; - - public AppenderatorPlumber( - DataSchema schema, - RealtimeTuningConfig config, - FireDepartmentMetrics metrics, - DataSegmentAnnouncer segmentAnnouncer, - SegmentPublisher segmentPublisher, - SegmentHandoffNotifier handoffNotifier, - Appenderator appenderator - ) - { - this.schema = schema; - this.config = config; - this.rejectionPolicy = config.getRejectionPolicyFactory().create(config.getWindowPeriod()); - this.metrics = metrics; - this.segmentAnnouncer = segmentAnnouncer; - this.segmentPublisher = segmentPublisher; - this.handoffNotifier = handoffNotifier; - this.appenderator = appenderator; - - log.info("Creating plumber using rejectionPolicy[%s]", getRejectionPolicy()); - } - - public Map getSegmentsView() - { - return ImmutableMap.copyOf(segments); - } - - public DataSchema getSchema() - { - return schema; - } - - public RealtimeTuningConfig getConfig() - { - return config; - } - - public RejectionPolicy getRejectionPolicy() - { - return rejectionPolicy; - } - - @Override - public Object startJob() - { - handoffNotifier.start(); - Object retVal = appenderator.startJob(); - initializeExecutors(); - startPersistThread(); - // Push pending sinks bootstrapped from previous run - mergeAndPush(); - return retVal; - } - - @Override - public IncrementalIndexAddResult add(InputRow row, Supplier committerSupplier) - throws IndexSizeExceededException - { - final SegmentIdWithShardSpec identifier = getSegmentIdentifier(row.getTimestampFromEpoch()); - if (identifier == null) { - return Plumber.THROWAWAY; - } - - try { - final Appenderator.AppenderatorAddResult addResult = appenderator.add(identifier, row, committerSupplier); - lastCommitterSupplier = committerSupplier; - return new IncrementalIndexAddResult(addResult.getNumRowsInSegment(), 0); - } - catch (SegmentNotWritableException e) { - // Segment already started handoff - return Plumber.NOT_WRITABLE; - } - } - - @Override - public QueryRunner getQueryRunner(final Query query) - { - return new QueryRunner() - { - @Override - public Sequence run(final QueryPlus queryPlus, final ResponseContext responseContext) - { - return queryPlus.run(appenderator, responseContext); - } - }; - } - - @Override - public void persist(final Committer committer) - { - final Stopwatch runExecStopwatch = Stopwatch.createStarted(); - appenderator.persistAll(committer); - - final long startDelay = runExecStopwatch.elapsed(TimeUnit.MILLISECONDS); - metrics.incrementPersistBackPressureMillis(startDelay); - if (startDelay > WARN_DELAY) { - log.warn("Ingestion was throttled for [%,d] millis because persists were pending.", startDelay); - } - runExecStopwatch.stop(); - } - - @Override - public void finishJob() - { - log.info("Shutting down..."); - - shuttingDown = true; - - List pending = appenderator.getSegments(); - if (pending.isEmpty()) { - log.info("No segments to hand off."); - } else { - log.info("Pushing segments: %s", Joiner.on(", ").join(pending)); - } - - try { - if (lastCommitterSupplier != null) { - // Push all remaining data - mergeAndPush(); - } - - synchronized (handoffCondition) { - while (!segments.isEmpty()) { - log.info("Waiting to hand off: %s", Joiner.on(", ").join(pending)); - handoffCondition.wait(); - pending = appenderator.getSegments(); - } - } - } - catch (Exception e) { - throw new RuntimeException(e); - } - finally { - stopped = true; - handoffNotifier.close(); - shutdownExecutors(); - appenderator.close(); - } - - if (!cleanShutdown) { - throw new ISE("Exception occurred during persist and merge."); - } - } - - private SegmentIdWithShardSpec getSegmentIdentifier(long timestamp) - { - if (!rejectionPolicy.accept(timestamp)) { - return null; - } - - final Granularity segmentGranularity = schema.getGranularitySpec().getSegmentGranularity(); - final VersioningPolicy versioningPolicy = config.getVersioningPolicy(); - - DateTime truncatedDateTime = segmentGranularity.bucketStart(DateTimes.utc(timestamp)); - final long truncatedTime = truncatedDateTime.getMillis(); - - SegmentIdWithShardSpec retVal = segments.get(truncatedTime); - - if (retVal == null) { - final Interval interval = new Interval( - truncatedDateTime, - segmentGranularity.increment(truncatedDateTime) - ); - - retVal = new SegmentIdWithShardSpec( - schema.getDataSource(), - interval, - versioningPolicy.getVersion(interval), - config.getShardSpec() - ); - addSegment(retVal); - - } - - return retVal; - } - - protected void initializeExecutors() - { - if (scheduledExecutor == null) { - scheduledExecutor = Execs.scheduledSingleThreaded("plumber_scheduled_%d"); - } - } - - protected void shutdownExecutors() - { - if (scheduledExecutor != null) { - scheduledExecutor.shutdown(); - } - } - - private void addSegment(final SegmentIdWithShardSpec identifier) - { - segments.put(identifier.getInterval().getStartMillis(), identifier); - try { - segmentAnnouncer.announceSegment( - new DataSegment( - identifier.getDataSource(), - identifier.getInterval(), - identifier.getVersion(), - ImmutableMap.of(), - ImmutableList.of(), - ImmutableList.of(), - identifier.getShardSpec(), - null, - 0 - ) - ); - } - catch (IOException e) { - log.makeAlert(e, "Failed to announce new segment[%s]", identifier.getDataSource()) - .addData("interval", identifier.getInterval()) - .emit(); - } - } - - public void dropSegment(final SegmentIdWithShardSpec identifier) - { - log.info("Dropping segment: %s", identifier); - segments.remove(identifier.getInterval().getStartMillis()); - - Futures.addCallback( - appenderator.drop(identifier), - new FutureCallback() - { - @Override - public void onSuccess(Object result) - { - log.info("Dropped segment: %s", identifier); - } - - @Override - public void onFailure(Throwable e) - { - // TODO: Retry? - log.warn(e, "Failed to drop segment: %s", identifier); - } - }, - MoreExecutors.directExecutor() - ); - } - - private void startPersistThread() - { - final Granularity segmentGranularity = schema.getGranularitySpec().getSegmentGranularity(); - final Period windowPeriod = config.getWindowPeriod(); - - final DateTime truncatedNow = segmentGranularity.bucketStart(DateTimes.nowUtc()); - final long windowMillis = windowPeriod.toStandardDuration().getMillis(); - - log.info( - "Expect to run at [%s]", - DateTimes.nowUtc().plus( - new Duration( - System.currentTimeMillis(), - segmentGranularity.increment(truncatedNow).getMillis() + windowMillis - ) - ) - ); - - String threadName = StringUtils.format( - "%s-overseer-%d", - schema.getDataSource(), - config.getShardSpec().getPartitionNum() - ); - ThreadRenamingCallable threadRenamingCallable = - new ThreadRenamingCallable(threadName) - { - @Override - public ScheduledExecutors.Signal doCall() - { - if (stopped) { - log.info("Stopping merge-n-push overseer thread"); - return ScheduledExecutors.Signal.STOP; - } - - mergeAndPush(); - - if (stopped) { - log.info("Stopping merge-n-push overseer thread"); - return ScheduledExecutors.Signal.STOP; - } else { - return ScheduledExecutors.Signal.REPEAT; - } - } - }; - Duration initialDelay = new Duration( - System.currentTimeMillis(), - segmentGranularity.increment(truncatedNow).getMillis() + windowMillis - ); - Duration rate = new Duration(truncatedNow, segmentGranularity.increment(truncatedNow)); - ScheduledExecutors.scheduleAtFixedRate(scheduledExecutor, initialDelay, rate, threadRenamingCallable); - } - - private void mergeAndPush() - { - final Granularity segmentGranularity = schema.getGranularitySpec().getSegmentGranularity(); - final Period windowPeriod = config.getWindowPeriod(); - - final long windowMillis = windowPeriod.toStandardDuration().getMillis(); - log.info("Starting merge and push."); - DateTime minTimestampAsDate = segmentGranularity.bucketStart( - DateTimes.utc(Math.max(windowMillis, rejectionPolicy.getCurrMaxTime().getMillis()) - windowMillis) - ); - long minTimestamp = minTimestampAsDate.getMillis(); - - final List appenderatorSegments = appenderator.getSegments(); - final List segmentsToPush = new ArrayList<>(); - - if (shuttingDown) { - log.info("Found [%,d] segments. Attempting to hand off all of them.", appenderatorSegments.size()); - segmentsToPush.addAll(appenderatorSegments); - } else { - log.info( - "Found [%,d] segments. Attempting to hand off segments that start before [%s].", - appenderatorSegments.size(), - minTimestampAsDate - ); - - for (SegmentIdWithShardSpec segment : appenderatorSegments) { - final Long intervalStart = segment.getInterval().getStartMillis(); - if (intervalStart < minTimestamp) { - log.info("Adding entry [%s] for merge and push.", segment); - segmentsToPush.add(segment); - } else { - log.info( - "Skipping persist and merge for entry [%s] : Start time [%s] >= [%s] min timestamp required in this run. Segment will be picked up in a future run.", - segment, - DateTimes.utc(intervalStart), - minTimestampAsDate - ); - } - } - } - - log.info("Found [%,d] segments to persist and merge", segmentsToPush.size()); - - final Function errorHandler = new Function() - { - @Override - public Void apply(Throwable throwable) - { - final List segmentIdentifierStrings = Lists.transform( - segmentsToPush, - SegmentIdWithShardSpec::toString - ); - - log.makeAlert(throwable, "Failed to publish merged indexes[%s]", schema.getDataSource()) - .addData("segments", segmentIdentifierStrings) - .emit(); - - if (shuttingDown) { - // We're trying to shut down, and these segments failed to push. Let's just get rid of them. - // This call will also delete possibly-partially-written files, so we don't need to do it explicitly. - cleanShutdown = false; - for (SegmentIdWithShardSpec identifier : segmentsToPush) { - dropSegment(identifier); - } - } - - return null; - } - }; - - // WARNING: Committers.nil() here means that on-disk data can get out of sync with committing. - Futures.addCallback( - appenderator.push(segmentsToPush, Committers.nil(), false), - new FutureCallback() - { - @Override - public void onSuccess(SegmentsAndCommitMetadata result) - { - // Immediately publish after pushing - for (DataSegment pushedSegment : result.getSegments()) { - try { - segmentPublisher.publishSegment(pushedSegment); - } - catch (Exception e) { - errorHandler.apply(e); - } - } - - log.info("Published [%,d] sinks.", segmentsToPush.size()); - } - - @Override - public void onFailure(Throwable e) - { - log.warn(e, "Failed to push [%,d] segments.", segmentsToPush.size()); - errorHandler.apply(e); - } - }, - MoreExecutors.directExecutor() - ); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberSchool.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberSchool.java deleted file mode 100644 index 8d188111f64..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberSchool.java +++ /dev/null @@ -1,84 +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.segment.realtime.appenderator; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.segment.realtime.SegmentPublisher; -import org.apache.druid.segment.realtime.plumber.Plumber; -import org.apache.druid.segment.realtime.plumber.PlumberSchool; -import org.apache.druid.server.coordination.DataSegmentAnnouncer; - -public class AppenderatorPlumberSchool implements PlumberSchool -{ - private final AppenderatorFactory appenderatorFactory; - private final DataSegmentAnnouncer segmentAnnouncer; - private final SegmentHandoffNotifierFactory handoffNotifierFactory; - private final SegmentPublisher segmentPublisher; - - @JsonCreator - public AppenderatorPlumberSchool( - @JsonProperty("appenderator") AppenderatorFactory appenderatorFactory, - @JacksonInject DataSegmentAnnouncer segmentAnnouncer, - @JacksonInject SegmentHandoffNotifierFactory handoffNotifierFactory, - @JacksonInject SegmentPublisher segmentPublisher - ) - { - this.appenderatorFactory = appenderatorFactory; - this.segmentAnnouncer = segmentAnnouncer; - this.handoffNotifierFactory = handoffNotifierFactory; - this.segmentPublisher = segmentPublisher; - } - - @Override - public Plumber findPlumber( - final DataSchema schema, - final RealtimeTuningConfig config, - final FireDepartmentMetrics metrics - ) - { - final Appenderator appenderator = appenderatorFactory.build( - schema, - config, - metrics - ); - - return new AppenderatorPlumber( - schema, - config, - metrics, - segmentAnnouncer, - segmentPublisher, - handoffNotifierFactory.createSegmentHandoffNotifier(schema.getDataSource()), - appenderator - ); - } - - @JsonProperty("appenderator") - public AppenderatorFactory getAppenderatorFactory() - { - return appenderatorFactory; - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java index 35cd526b1ea..56af5f3f62c 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java @@ -35,7 +35,7 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer; import org.apache.druid.timeline.VersionedIntervalTimeline; @@ -47,7 +47,7 @@ public class Appenderators String id, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -101,7 +101,7 @@ public class Appenderators String id, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -135,7 +135,7 @@ public class Appenderators String id, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -173,7 +173,7 @@ public class Appenderators String id, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java index aa11d85ef35..3bbf364656e 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java @@ -38,7 +38,7 @@ import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.joda.time.Interval; @@ -70,7 +70,7 @@ public interface AppenderatorsManager String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -96,7 +96,7 @@ public interface AppenderatorsManager String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -111,7 +111,7 @@ public interface AppenderatorsManager String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -126,7 +126,7 @@ public interface AppenderatorsManager String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 128de15196d..b544d33705c 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -67,9 +67,9 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.FingerprintGenerator; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.FireHydrant; -import org.apache.druid.segment.realtime.plumber.Sink; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; +import org.apache.druid.segment.realtime.sink.Sink; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -113,7 +113,7 @@ public class BatchAppenderator implements Appenderator private final String myId; private final DataSchema schema; private final AppenderatorConfig tuningConfig; - private final FireDepartmentMetrics metrics; + private final SegmentGenerationMetrics metrics; private final DataSegmentPusher dataSegmentPusher; private final ObjectMapper objectMapper; private final IndexIO indexIO; @@ -164,7 +164,7 @@ public class BatchAppenderator implements Appenderator String id, DataSchema schema, AppenderatorConfig tuningConfig, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -481,8 +481,7 @@ public class BatchAppenderator implements Appenderator tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), maxBytesTuningConfig, - useMaxMemoryEstimates, - null + useMaxMemoryEstimates ); bytesCurrentlyInMemory += calculateSinkMemoryInUsed(); sinks.put(identifier, retVal); @@ -1075,7 +1074,6 @@ public class BatchAppenderator implements Appenderator tuningConfig.getMaxRowsInMemory(), maxBytesTuningConfig, useMaxMemoryEstimates, - null, hydrants ); retVal.finishWriting(); // this sink is not writable diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java deleted file mode 100644 index b5574c686c4..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java +++ /dev/null @@ -1,92 +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.segment.realtime.appenderator; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.IndexMerger; -import org.apache.druid.segment.incremental.NoopRowIngestionMeters; -import org.apache.druid.segment.incremental.ParseExceptionHandler; -import org.apache.druid.segment.incremental.RowIngestionMeters; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; - -public class DefaultOfflineAppenderatorFactory implements AppenderatorFactory -{ - private final DataSegmentPusher dataSegmentPusher; - private final ObjectMapper objectMapper; - private final IndexIO indexIO; - private final IndexMerger indexMerger; - private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; - - @JsonCreator - public DefaultOfflineAppenderatorFactory( - @JacksonInject DataSegmentPusher dataSegmentPusher, - @JacksonInject ObjectMapper objectMapper, - @JacksonInject IndexIO indexIO, - @JacksonInject IndexMerger indexMerger, - @JsonProperty("centralizedDatasourceSchemaConfig") CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig - ) - { - this.dataSegmentPusher = dataSegmentPusher; - this.objectMapper = objectMapper; - this.indexIO = indexIO; - this.indexMerger = indexMerger; - this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; - - } - - @JsonProperty - public CentralizedDatasourceSchemaConfig getCentralizedDatasourceSchemaConfig() - { - return centralizedDatasourceSchemaConfig; - } - - @Override - public Appenderator build(DataSchema schema, RealtimeTuningConfig config, FireDepartmentMetrics metrics) - { - final RowIngestionMeters rowIngestionMeters = new NoopRowIngestionMeters(); - return Appenderators.createClosedSegmentsOffline( - schema.getDataSource(), - schema, - config, - metrics, - dataSegmentPusher, - objectMapper, - indexIO, - indexMerger, - rowIngestionMeters, - new ParseExceptionHandler( - rowIngestionMeters, - false, - config.isReportParseExceptions() ? 0 : Integer.MAX_VALUE, - 0 - ), - true, - centralizedDatasourceSchemaConfig - ); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java deleted file mode 100644 index e64c315484d..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java +++ /dev/null @@ -1,142 +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.segment.realtime.appenderator; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.client.cache.Cache; -import org.apache.druid.client.cache.CacheConfig; -import org.apache.druid.client.cache.CachePopulatorStats; -import org.apache.druid.guice.annotations.Json; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.query.QueryProcessingPool; -import org.apache.druid.query.QueryRunnerFactoryConglomerate; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.IndexMerger; -import org.apache.druid.segment.incremental.NoopRowIngestionMeters; -import org.apache.druid.segment.incremental.ParseExceptionHandler; -import org.apache.druid.segment.incremental.RowIngestionMeters; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.join.JoinableFactory; -import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.apache.druid.timeline.partition.ShardSpec; - -import java.io.File; - -public class DefaultRealtimeAppenderatorFactory implements AppenderatorFactory -{ - private final ServiceEmitter emitter; - private final QueryRunnerFactoryConglomerate conglomerate; - private final DataSegmentAnnouncer segmentAnnouncer; - private final QueryProcessingPool queryProcessingPool; - private final JoinableFactory joinableFactory; - private final DataSegmentPusher dataSegmentPusher; - private final ObjectMapper jsonMapper; - private final IndexIO indexIO; - private final IndexMerger indexMerger; - private final Cache cache; - private final CacheConfig cacheConfig; - private final CachePopulatorStats cachePopulatorStats; - - public DefaultRealtimeAppenderatorFactory( - @JacksonInject ServiceEmitter emitter, - @JacksonInject QueryRunnerFactoryConglomerate conglomerate, - @JacksonInject DataSegmentAnnouncer segmentAnnouncer, - @JacksonInject QueryProcessingPool queryProcessingPool, - @JacksonInject JoinableFactory joinableFactory, - @JacksonInject DataSegmentPusher dataSegmentPusher, - @JacksonInject @Json ObjectMapper jsonMapper, - @JacksonInject IndexIO indexIO, - @JacksonInject IndexMerger indexMerger, - @JacksonInject Cache cache, - @JacksonInject CacheConfig cacheConfig, - @JacksonInject CachePopulatorStats cachePopulatorStats - ) - { - this.emitter = emitter; - this.conglomerate = conglomerate; - this.segmentAnnouncer = segmentAnnouncer; - this.queryProcessingPool = queryProcessingPool; - this.joinableFactory = joinableFactory; - this.dataSegmentPusher = dataSegmentPusher; - this.jsonMapper = jsonMapper; - this.indexIO = indexIO; - this.indexMerger = indexMerger; - this.cache = cache; - this.cacheConfig = cacheConfig; - this.cachePopulatorStats = cachePopulatorStats; - } - - @Override - public Appenderator build( - final DataSchema schema, - final RealtimeTuningConfig config, - final FireDepartmentMetrics metrics - ) - { - final RowIngestionMeters rowIngestionMeters = new NoopRowIngestionMeters(); - return Appenderators.createRealtime( - null, - schema.getDataSource(), - schema, - config.withBasePersistDirectory( - makeBasePersistSubdirectory( - config.getBasePersistDirectory(), - schema.getDataSource(), - config.getShardSpec() - ) - ), - metrics, - dataSegmentPusher, - jsonMapper, - indexIO, - indexMerger, - conglomerate, - segmentAnnouncer, - emitter, - queryProcessingPool, - cache, - cacheConfig, - cachePopulatorStats, - rowIngestionMeters, - new ParseExceptionHandler( - rowIngestionMeters, - false, - config.isReportParseExceptions() ? 0 : Integer.MAX_VALUE, - 0 - ), - true, - null - ); - } - - private static File makeBasePersistSubdirectory( - final File basePersistDirectory, - final String dataSource, - final ShardSpec shardSpec - ) - { - final File dataSourceDirectory = new File(basePersistDirectory, dataSource); - return new File(dataSourceDirectory, String.valueOf(shardSpec.getPartitionNum())); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java index 988d77b6f70..f44fffe20e1 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java @@ -39,7 +39,7 @@ import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.joda.time.Interval; @@ -61,7 +61,7 @@ public class DummyForInjectionAppenderatorsManager implements AppenderatorsManag String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -88,7 +88,7 @@ public class DummyForInjectionAppenderatorsManager implements AppenderatorsManag String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -107,7 +107,7 @@ public class DummyForInjectionAppenderatorsManager implements AppenderatorsManag String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -126,7 +126,7 @@ public class DummyForInjectionAppenderatorsManager implements AppenderatorsManag String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java index c0f833ac594..52f75f72e47 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java @@ -39,7 +39,7 @@ import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.joda.time.Interval; @@ -67,7 +67,7 @@ public class PeonAppenderatorsManager implements AppenderatorsManager String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper jsonMapper, IndexIO indexIO, @@ -122,7 +122,7 @@ public class PeonAppenderatorsManager implements AppenderatorsManager String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -160,7 +160,7 @@ public class PeonAppenderatorsManager implements AppenderatorsManager String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -198,7 +198,7 @@ public class PeonAppenderatorsManager implements AppenderatorsManager String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentSchemas.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentSchemas.java index fbe2b6315ef..359da9bed76 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentSchemas.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentSchemas.java @@ -30,7 +30,7 @@ import java.util.Objects; /** * Encapsulates schema information for multiple segments. *

- * Primarily used to announce schema changes for all {@link org.apache.druid.segment.realtime.plumber.Sink} + * Primarily used to announce schema changes for all {@link org.apache.druid.segment.realtime.sink.Sink} * created by a task in {@link StreamAppenderator}. */ public class SegmentSchemas diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index aba071de1df..72cfe4de261 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -59,8 +59,8 @@ import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.realtime.FireHydrant; -import org.apache.druid.segment.realtime.plumber.Sink; -import org.apache.druid.segment.realtime.plumber.SinkSegmentReference; +import org.apache.druid.segment.realtime.sink.Sink; +import org.apache.druid.segment.realtime.sink.SinkSegmentReference; import org.apache.druid.server.ResourceIdPopulatingQueryRunner; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index 7a41ae3fb26..4d1253591e0 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -76,9 +76,9 @@ import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.FingerprintGenerator; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.FireHydrant; -import org.apache.druid.segment.realtime.plumber.Sink; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; +import org.apache.druid.segment.realtime.sink.Sink; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -134,7 +134,7 @@ public class StreamAppenderator implements Appenderator private final String myId; private final DataSchema schema; private final AppenderatorConfig tuningConfig; - private final FireDepartmentMetrics metrics; + private final SegmentGenerationMetrics metrics; private final DataSegmentPusher dataSegmentPusher; private final ObjectMapper objectMapper; private final DataSegmentAnnouncer segmentAnnouncer; @@ -221,7 +221,7 @@ public class StreamAppenderator implements Appenderator String id, DataSchema schema, AppenderatorConfig tuningConfig, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, DataSegmentAnnouncer segmentAnnouncer, @@ -533,8 +533,7 @@ public class StreamAppenderator implements Appenderator tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), maxBytesTuningConfig, - useMaxMemoryEstimates, - null + useMaxMemoryEstimates ); bytesCurrentlyInMemory.addAndGet(calculateSinkMemoryInUsed(retVal)); @@ -1391,7 +1390,6 @@ public class StreamAppenderator implements Appenderator tuningConfig.getMaxRowsInMemory(), maxBytesTuningConfig, useMaxMemoryEstimates, - null, hydrants ); rowsSoFar += currSink.getNumRows(); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java index 2b5c153d602..89115874916 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java @@ -40,7 +40,7 @@ import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.loading.DataSegmentKiller; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.SegmentWithState.SegmentState; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -82,7 +82,7 @@ public class StreamAppenderatorDriver extends BaseAppenderatorDriver private static final long HANDOFF_TIME_THRESHOLD = 600_000; private final SegmentHandoffNotifier handoffNotifier; - private final FireDepartmentMetrics metrics; + private final SegmentGenerationMetrics metrics; private final ObjectMapper objectMapper; /** @@ -102,7 +102,7 @@ public class StreamAppenderatorDriver extends BaseAppenderatorDriver UsedSegmentChecker usedSegmentChecker, DataSegmentKiller dataSegmentKiller, ObjectMapper objectMapper, - FireDepartmentMetrics metrics + SegmentGenerationMetrics metrics ) { super(appenderator, segmentAllocator, usedSegmentChecker, dataSegmentKiller); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java index 68ed2ae0d8f..ffdfb8d1eb0 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java @@ -64,8 +64,8 @@ import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.segment.realtime.plumber.Sink; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; +import org.apache.druid.segment.realtime.sink.Sink; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.timeline.VersionedIntervalTimeline; @@ -155,7 +155,7 @@ public class UnifiedIndexerAppenderatorsManager implements AppenderatorsManager String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -209,7 +209,7 @@ public class UnifiedIndexerAppenderatorsManager implements AppenderatorsManager String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -250,7 +250,7 @@ public class UnifiedIndexerAppenderatorsManager implements AppenderatorsManager String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, @@ -291,7 +291,7 @@ public class UnifiedIndexerAppenderatorsManager implements AppenderatorsManager String taskId, DataSchema schema, AppenderatorConfig config, - FireDepartmentMetrics metrics, + SegmentGenerationMetrics metrics, DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/CustomVersioningPolicy.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/CustomVersioningPolicy.java deleted file mode 100644 index 1974dcec473..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/CustomVersioningPolicy.java +++ /dev/null @@ -1,52 +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.segment.realtime.plumber; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.java.util.common.DateTimes; -import org.joda.time.Interval; - -/** - */ -public class CustomVersioningPolicy implements VersioningPolicy -{ - private final String version; - - @JsonCreator - public CustomVersioningPolicy( - @JsonProperty("version") String version - ) - { - this.version = version == null ? DateTimes.nowUtc().toString() : version; - } - - @Override - public String getVersion(Interval interval) - { - return version; - } - - @JsonProperty("version") - public String getVersion() - { - return version; - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumber.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumber.java deleted file mode 100644 index a271c4540c5..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumber.java +++ /dev/null @@ -1,238 +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.segment.realtime.plumber; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.client.cache.Cache; -import org.apache.druid.client.cache.CacheConfig; -import org.apache.druid.client.cache.CachePopulatorStats; -import org.apache.druid.common.guava.ThreadRenamingCallable; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.FileUtils; -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.ScheduledExecutors; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.query.QueryProcessingPool; -import org.apache.druid.query.QueryRunnerFactoryConglomerate; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.IndexMerger; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.join.JoinableFactory; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.joda.time.DateTime; -import org.joda.time.Duration; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.ScheduledExecutorService; - -/** - */ -public class FlushingPlumber extends RealtimePlumber -{ - private static final EmittingLogger log = new EmittingLogger(FlushingPlumber.class); - - private final DataSchema schema; - private final RealtimeTuningConfig config; - private final Duration flushDuration; - - private volatile ScheduledExecutorService flushScheduledExec = null; - private volatile boolean stopped = false; - - public FlushingPlumber( - Duration flushDuration, - DataSchema schema, - RealtimeTuningConfig config, - FireDepartmentMetrics metrics, - ServiceEmitter emitter, - QueryRunnerFactoryConglomerate conglomerate, - DataSegmentAnnouncer segmentAnnouncer, - QueryProcessingPool queryProcessingPool, - JoinableFactory joinableFactory, - IndexMerger indexMerger, - IndexIO indexIO, - Cache cache, - CacheConfig cacheConfig, - CachePopulatorStats cachePopulatorStats, - ObjectMapper objectMapper - - ) - { - super( - schema, - config, - metrics, - emitter, - conglomerate, - segmentAnnouncer, - queryProcessingPool, - null, - null, - null, - indexMerger, - indexIO, - cache, - cacheConfig, - cachePopulatorStats, - objectMapper - ); - - this.flushDuration = flushDuration; - this.schema = schema; - this.config = config; - } - - @Override - public Object startJob() - { - log.info("Starting job for %s", getSchema().getDataSource()); - - try { - FileUtils.mkdirp(computeBaseDir(getSchema())); - } - catch (IOException e) { - throw new RuntimeException(e); - } - - initializeExecutors(); - - if (flushScheduledExec == null) { - flushScheduledExec = Execs.scheduledSingleThreaded("flushing_scheduled_%d"); - } - - Object retVal = bootstrapSinksFromDisk(); - startFlushThread(); - return retVal; - } - - protected void flushAfterDuration(final long truncatedTime, final Sink sink) - { - log.info( - "Abandoning segment %s at %s", - sink.getSegment().getId(), - DateTimes.nowUtc().plusMillis((int) flushDuration.getMillis()) - ); - - ScheduledExecutors.scheduleWithFixedDelay( - flushScheduledExec, - flushDuration, - new Callable() - { - @Override - public ScheduledExecutors.Signal call() - { - log.info("Abandoning segment %s", sink.getSegment().getId()); - abandonSegment(truncatedTime, sink); - return ScheduledExecutors.Signal.STOP; - } - } - ); - } - - private void startFlushThread() - { - final Granularity segmentGranularity = schema.getGranularitySpec().getSegmentGranularity(); - final DateTime truncatedNow = segmentGranularity.bucketStart(DateTimes.nowUtc()); - final long windowMillis = config.getWindowPeriod().toStandardDuration().getMillis(); - - log.info( - "Expect to run at [%s]", - DateTimes.nowUtc().plus( - new Duration( - System.currentTimeMillis(), - schema.getGranularitySpec().getSegmentGranularity().increment(truncatedNow).getMillis() + windowMillis - ) - ) - ); - - String threadName = StringUtils.format( - "%s-flusher-%d", - getSchema().getDataSource(), - getConfig().getShardSpec().getPartitionNum() - ); - ThreadRenamingCallable threadRenamingCallable = - new ThreadRenamingCallable(threadName) - { - @Override - public ScheduledExecutors.Signal doCall() - { - if (stopped) { - log.info("Stopping flusher thread"); - return ScheduledExecutors.Signal.STOP; - } - - long minTimestamp = segmentGranularity.bucketStart( - getRejectionPolicy().getCurrMaxTime().minus(windowMillis) - ).getMillis(); - - List> sinksToPush = new ArrayList<>(); - for (Map.Entry entry : getSinks().entrySet()) { - final Long intervalStart = entry.getKey(); - if (intervalStart < minTimestamp) { - log.info("Adding entry[%s] to flush.", entry); - sinksToPush.add(entry); - } - } - - for (final Map.Entry entry : sinksToPush) { - flushAfterDuration(entry.getKey(), entry.getValue()); - } - - if (stopped) { - log.info("Stopping flusher thread"); - return ScheduledExecutors.Signal.STOP; - } else { - return ScheduledExecutors.Signal.REPEAT; - } - } - }; - Duration initialDelay = new Duration( - System.currentTimeMillis(), - schema.getGranularitySpec().getSegmentGranularity().increment(truncatedNow).getMillis() + windowMillis - ); - Duration rate = new Duration(truncatedNow, segmentGranularity.increment(truncatedNow)); - ScheduledExecutors.scheduleAtFixedRate(flushScheduledExec, initialDelay, rate, threadRenamingCallable); - } - - @Override - public void finishJob() - { - log.info("Stopping job"); - - for (final Map.Entry entry : getSinks().entrySet()) { - abandonSegment(entry.getKey(), entry.getValue()); - } - shutdownExecutors(); - - if (flushScheduledExec != null) { - flushScheduledExec.shutdown(); - } - - stopped = true; - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumberSchool.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumberSchool.java deleted file mode 100644 index 787ea26cbc6..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumberSchool.java +++ /dev/null @@ -1,145 +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.segment.realtime.plumber; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import org.apache.druid.client.cache.Cache; -import org.apache.druid.client.cache.CacheConfig; -import org.apache.druid.client.cache.CachePopulatorStats; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.query.QueryProcessingPool; -import org.apache.druid.query.QueryRunnerFactoryConglomerate; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.IndexMergerV9; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.join.JoinableFactory; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.joda.time.Duration; - -/** - * This plumber just drops segments at the end of a flush duration instead of handing them off. It is only useful if you want to run - * a real time node without the rest of the Druid cluster. - */ -public class FlushingPlumberSchool extends RealtimePlumberSchool -{ - private static final Duration DEFAULT_FLUSH_DURATION = new Duration("PT1H"); - - private final Duration flushDuration; - - private final ServiceEmitter emitter; - private final QueryRunnerFactoryConglomerate conglomerate; - private final DataSegmentAnnouncer segmentAnnouncer; - private final QueryProcessingPool queryProcessingPool; - private final JoinableFactory joinableFactory; - private final IndexMergerV9 indexMergerV9; - private final IndexIO indexIO; - private final Cache cache; - private final CacheConfig cacheConfig; - private final CachePopulatorStats cachePopulatorStats; - private final ObjectMapper objectMapper; - - @JsonCreator - public FlushingPlumberSchool( - @JsonProperty("flushDuration") Duration flushDuration, - @JacksonInject ServiceEmitter emitter, - @JacksonInject QueryRunnerFactoryConglomerate conglomerate, - @JacksonInject DataSegmentAnnouncer segmentAnnouncer, - @JacksonInject QueryProcessingPool queryProcessingPool, - @JacksonInject JoinableFactory joinableFactory, - @JacksonInject IndexMergerV9 indexMergerV9, - @JacksonInject IndexIO indexIO, - @JacksonInject Cache cache, - @JacksonInject CacheConfig cacheConfig, - @JacksonInject CachePopulatorStats cachePopulatorStats, - @JacksonInject ObjectMapper objectMapper - ) - { - super( - emitter, - conglomerate, - null, - segmentAnnouncer, - null, - null, - queryProcessingPool, - joinableFactory, - indexMergerV9, - indexIO, - cache, - cacheConfig, - cachePopulatorStats, - objectMapper - ); - - this.flushDuration = flushDuration == null ? DEFAULT_FLUSH_DURATION : flushDuration; - this.emitter = emitter; - this.conglomerate = conglomerate; - this.segmentAnnouncer = segmentAnnouncer; - this.queryProcessingPool = queryProcessingPool; - this.joinableFactory = joinableFactory; - this.indexMergerV9 = Preconditions.checkNotNull(indexMergerV9, "Null IndexMergerV9"); - this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO"); - this.cache = cache; - this.cacheConfig = cacheConfig; - this.cachePopulatorStats = cachePopulatorStats; - this.objectMapper = objectMapper; - } - - @Override - public Plumber findPlumber( - final DataSchema schema, - final RealtimeTuningConfig config, - final FireDepartmentMetrics metrics - ) - { - verifyState(); - - return new FlushingPlumber( - flushDuration, - schema, - config, - metrics, - emitter, - conglomerate, - segmentAnnouncer, - queryProcessingPool, - joinableFactory, - indexMergerV9, - indexIO, - cache, - cacheConfig, - cachePopulatorStats, - objectMapper - ); - } - - private void verifyState() - { - Preconditions.checkNotNull(conglomerate, "must specify a queryRunnerFactoryConglomerate to do this action."); - Preconditions.checkNotNull(segmentAnnouncer, "must specify a segmentAnnouncer to do this action."); - Preconditions.checkNotNull(emitter, "must specify a serviceEmitter to do this action."); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/IntervalStartVersioningPolicy.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/IntervalStartVersioningPolicy.java deleted file mode 100644 index e343d4fe0df..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/IntervalStartVersioningPolicy.java +++ /dev/null @@ -1,31 +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.segment.realtime.plumber; - -import org.joda.time.Interval; - -public class IntervalStartVersioningPolicy implements VersioningPolicy -{ - @Override - public String getVersion(Interval interval) - { - return interval.getStart().toString(); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/MessageTimeRejectionPolicyFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/MessageTimeRejectionPolicyFactory.java deleted file mode 100644 index 19fb37e3441..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/MessageTimeRejectionPolicyFactory.java +++ /dev/null @@ -1,90 +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.segment.realtime.plumber; - -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.JodaUtils; -import org.apache.druid.java.util.common.StringUtils; -import org.joda.time.DateTime; -import org.joda.time.Period; - -import java.util.concurrent.atomic.AtomicLongFieldUpdater; - -public class MessageTimeRejectionPolicyFactory implements RejectionPolicyFactory -{ - @Override - public RejectionPolicy create(final Period windowPeriod) - { - final long windowMillis = windowPeriod.toStandardDuration().getMillis(); - return new MessageTimeRejectionPolicy(windowMillis, windowPeriod); - } - - private static class MessageTimeRejectionPolicy implements RejectionPolicy - { - private static final AtomicLongFieldUpdater MAX_TIMESTAMP_UPDATER = - AtomicLongFieldUpdater.newUpdater(MessageTimeRejectionPolicy.class, "maxTimestamp"); - private final long windowMillis; - private final Period windowPeriod; - private volatile long maxTimestamp; - - public MessageTimeRejectionPolicy(long windowMillis, Period windowPeriod) - { - this.windowMillis = windowMillis; - this.windowPeriod = windowPeriod; - this.maxTimestamp = JodaUtils.MIN_INSTANT; - } - - @Override - public DateTime getCurrMaxTime() - { - return DateTimes.utc(maxTimestamp); - } - - @Override - public boolean accept(long timestamp) - { - long maxTimestamp = this.maxTimestamp; - if (timestamp > maxTimestamp) { - maxTimestamp = tryUpdateMaxTimestamp(timestamp); - } - - return timestamp >= (maxTimestamp - windowMillis); - } - - private long tryUpdateMaxTimestamp(long timestamp) - { - long currentMaxTimestamp; - do { - currentMaxTimestamp = maxTimestamp; - if (timestamp <= currentMaxTimestamp) { - return currentMaxTimestamp; - } - } while (!MAX_TIMESTAMP_UPDATER.compareAndSet(this, currentMaxTimestamp, timestamp)); - return timestamp; - } - - @Override - public String toString() - { - return StringUtils.format("messageTime-%s", windowPeriod); - } - } -} - diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/NoopRejectionPolicyFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/NoopRejectionPolicyFactory.java deleted file mode 100644 index 6acce9abaa3..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/NoopRejectionPolicyFactory.java +++ /dev/null @@ -1,46 +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.segment.realtime.plumber; - -import org.apache.druid.java.util.common.DateTimes; -import org.joda.time.DateTime; -import org.joda.time.Period; - -public class NoopRejectionPolicyFactory implements RejectionPolicyFactory -{ - @Override - public RejectionPolicy create(Period windowPeriod) - { - return new RejectionPolicy() - { - @Override - public DateTime getCurrMaxTime() - { - return DateTimes.EPOCH; - } - - @Override - public boolean accept(long timestamp) - { - return true; - } - }; - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Plumber.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Plumber.java deleted file mode 100644 index e7fab4ae8a0..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Plumber.java +++ /dev/null @@ -1,70 +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.segment.realtime.plumber; - -import com.google.common.base.Supplier; -import org.apache.druid.data.input.Committer; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.query.Query; -import org.apache.druid.query.QueryRunner; -import org.apache.druid.segment.incremental.IncrementalIndexAddResult; -import org.apache.druid.segment.incremental.IndexSizeExceededException; - -public interface Plumber -{ - IncrementalIndexAddResult THROWAWAY = new IncrementalIndexAddResult(-1, -1, "row too late"); - IncrementalIndexAddResult NOT_WRITABLE = new IncrementalIndexAddResult(-1, -1, "not writable"); - IncrementalIndexAddResult DUPLICATE = new IncrementalIndexAddResult(-2, -1, "duplicate row"); - - /** - * Perform any initial setup. Should be called before using any other methods, and should be paired - * with a corresponding call to {@link #finishJob}. - * - * @return the metadata of the "newest" segment that might have previously been persisted - */ - Object startJob(); - - /** - * @param row the row to insert - * @param committerSupplier supplier of a committer associated with all data that has been added, including this row - * - * @return IncrementalIndexAddResult whose rowCount - * - positive numbers indicate how many summarized rows exist in the index for that timestamp, - * -1 means a row was thrown away because it was too late - * -2 means a row was thrown away because it is duplicate - */ - IncrementalIndexAddResult add(InputRow row, Supplier committerSupplier) throws IndexSizeExceededException; - - QueryRunner getQueryRunner(Query query); - - /** - * Persist any in-memory indexed data to durable storage. This may be only somewhat durable, e.g. the - * machine's local disk. - * - * @param committer committer to use after persisting data - */ - void persist(Committer committer); - - /** - * Perform any final processing and clean up after ourselves. Should be called after all data has been - * fed into sinks and persisted. - */ - void finishJob(); -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/PlumberSchool.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/PlumberSchool.java deleted file mode 100644 index 849d5dbb184..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/PlumberSchool.java +++ /dev/null @@ -1,44 +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.segment.realtime.plumber; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; - -/** - */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = RealtimePlumberSchool.class) -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "realtime", value = RealtimePlumberSchool.class), - @JsonSubTypes.Type(name = "flushing", value = FlushingPlumberSchool.class) -}) -public interface PlumberSchool -{ - /** - * Creates a Plumber - * - * @return returns a plumber - */ - Plumber findPlumber(DataSchema schema, RealtimeTuningConfig config, FireDepartmentMetrics metrics); - -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Plumbers.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Plumbers.java deleted file mode 100644 index 44d611dfd0e..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Plumbers.java +++ /dev/null @@ -1,96 +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.segment.realtime.plumber; - -import com.google.common.base.Supplier; -import org.apache.druid.data.input.Committer; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.common.parsers.ParseException; -import org.apache.druid.segment.incremental.IncrementalIndexAddResult; -import org.apache.druid.segment.incremental.IndexSizeExceededException; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; - -import java.io.IOException; - -public class Plumbers -{ - private static final Logger log = new Logger(Plumbers.class); - - private Plumbers() - { - // No instantiation - } - - public static void addNextRow( - final Supplier committerSupplier, - final Firehose firehose, - final Plumber plumber, - final boolean reportParseExceptions, - final FireDepartmentMetrics metrics - ) throws IOException - { - final InputRow inputRow; - try { - inputRow = firehose.nextRow(); - } - catch (ParseException e) { - if (reportParseExceptions) { - throw e; - } else { - log.debug(e, "Discarded row due to exception, considering unparseable."); - metrics.incrementUnparseable(); - return; - } - } - - if (inputRow == null) { - log.debug("Discarded null row, considering thrownAway."); - metrics.incrementThrownAway(); - return; - } - - final IncrementalIndexAddResult addResult; - try { - addResult = plumber.add(inputRow, committerSupplier); - } - catch (IndexSizeExceededException e) { - // Shouldn't happen if this is only being called by a single thread. - // plumber.add should be swapping out indexes before they fill up. - throw new ISE(e, "Index size exceeded"); - } - - if (addResult.getRowCount() == -1) { - metrics.incrementThrownAway(); - log.debug("Discarded row[%s], considering thrownAway due to %s.", inputRow, addResult.getReasonOfNotAdded()); - return; - } - - if (addResult.getRowCount() == -2) { - metrics.incrementDedup(); - log.debug("Discarded row[%s], considering duplication.", inputRow); - return; - } - - metrics.incrementProcessed(); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java deleted file mode 100644 index 0380abf9f22..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java +++ /dev/null @@ -1,1014 +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.segment.realtime.plumber; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import com.google.common.base.Stopwatch; -import com.google.common.base.Supplier; -import com.google.common.collect.Collections2; -import com.google.common.collect.ImmutableMap; -import com.google.common.primitives.Ints; -import org.apache.druid.client.cache.Cache; -import org.apache.druid.client.cache.CacheConfig; -import org.apache.druid.client.cache.CachePopulatorStats; -import org.apache.druid.common.guava.ThreadRenamingCallable; -import org.apache.druid.common.guava.ThreadRenamingRunnable; -import org.apache.druid.concurrent.TaskThreadPriority; -import org.apache.druid.data.input.Committer; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.Pair; -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.ScheduledExecutors; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.query.Query; -import org.apache.druid.query.QueryProcessingPool; -import org.apache.druid.query.QueryRunner; -import org.apache.druid.query.QueryRunnerFactoryConglomerate; -import org.apache.druid.query.QuerySegmentWalker; -import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.segment.BaseProgressIndicator; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.IndexMerger; -import org.apache.druid.segment.Metadata; -import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.ReferenceCountingSegment; -import org.apache.druid.segment.handoff.SegmentHandoffNotifier; -import org.apache.druid.segment.incremental.IncrementalIndexAddResult; -import org.apache.druid.segment.incremental.IndexSizeExceededException; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.segment.realtime.FireHydrant; -import org.apache.druid.segment.realtime.SegmentPublisher; -import org.apache.druid.segment.realtime.appenderator.SinkQuerySegmentWalker; -import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; -import org.apache.druid.timeline.VersionedIntervalTimeline; -import org.apache.druid.timeline.partition.SingleElementPartitionChunk; -import org.apache.druid.utils.JvmUtils; -import org.joda.time.DateTime; -import org.joda.time.Duration; -import org.joda.time.Interval; -import org.joda.time.Period; - -import java.io.Closeable; -import java.io.File; -import java.io.FilenameFilter; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; - -/** - * - */ -public class RealtimePlumber implements Plumber -{ - private static final EmittingLogger log = new EmittingLogger(RealtimePlumber.class); - private static final int WARN_DELAY = 1000; - - private final DataSchema schema; - private final RealtimeTuningConfig config; - private final RejectionPolicy rejectionPolicy; - private final FireDepartmentMetrics metrics; - private final DataSegmentAnnouncer segmentAnnouncer; - private final DataSegmentPusher dataSegmentPusher; - private final SegmentPublisher segmentPublisher; - private final SegmentHandoffNotifier handoffNotifier; - private final Object handoffCondition = new Object(); - private final ConcurrentMap sinks = new ConcurrentHashMap<>(); - private final VersionedIntervalTimeline sinkTimeline = new VersionedIntervalTimeline( - String.CASE_INSENSITIVE_ORDER - ); - private final QuerySegmentWalker texasRanger; - private final Cache cache; - - private volatile long nextFlush = 0; - private volatile boolean shuttingDown = false; - private volatile boolean stopped = false; - private volatile boolean cleanShutdown = true; - private volatile ExecutorService persistExecutor = null; - private volatile ExecutorService mergeExecutor = null; - private volatile ScheduledExecutorService scheduledExecutor = null; - private volatile IndexMerger indexMerger; - private volatile IndexIO indexIO; - - private static final String COMMIT_METADATA_KEY = "%commitMetadata%"; - private static final String COMMIT_METADATA_TIMESTAMP_KEY = "%commitMetadataTimestamp%"; - - public RealtimePlumber( - DataSchema schema, - RealtimeTuningConfig config, - FireDepartmentMetrics metrics, - ServiceEmitter emitter, - QueryRunnerFactoryConglomerate conglomerate, - DataSegmentAnnouncer segmentAnnouncer, - QueryProcessingPool queryProcessingPool, - DataSegmentPusher dataSegmentPusher, - SegmentPublisher segmentPublisher, - SegmentHandoffNotifier handoffNotifier, - IndexMerger indexMerger, - IndexIO indexIO, - Cache cache, - CacheConfig cacheConfig, - CachePopulatorStats cachePopulatorStats, - ObjectMapper objectMapper - ) - { - this.schema = schema; - this.config = config; - this.rejectionPolicy = config.getRejectionPolicyFactory().create(config.getWindowPeriod()); - this.metrics = metrics; - this.segmentAnnouncer = segmentAnnouncer; - this.dataSegmentPusher = dataSegmentPusher; - this.segmentPublisher = segmentPublisher; - this.handoffNotifier = handoffNotifier; - this.indexMerger = Preconditions.checkNotNull(indexMerger, "Null IndexMerger"); - this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO"); - this.cache = cache; - this.texasRanger = new SinkQuerySegmentWalker( - schema.getDataSource(), - sinkTimeline, - objectMapper, - emitter, - conglomerate, - queryProcessingPool, - cache, - cacheConfig, - cachePopulatorStats - ); - - log.info("Creating plumber using rejectionPolicy[%s]", getRejectionPolicy()); - } - - public DataSchema getSchema() - { - return schema; - } - - public RealtimeTuningConfig getConfig() - { - return config; - } - - public RejectionPolicy getRejectionPolicy() - { - return rejectionPolicy; - } - - public Map getSinks() - { - return sinks; - } - - @Override - public Object startJob() - { - try { - FileUtils.mkdirp(computeBaseDir(schema)); - } - catch (IOException e) { - throw new RuntimeException(e); - } - - initializeExecutors(); - handoffNotifier.start(); - Object retVal = bootstrapSinksFromDisk(); - startPersistThread(); - // Push pending sinks bootstrapped from previous run - mergeAndPush(); - resetNextFlush(); - return retVal; - } - - @Override - public IncrementalIndexAddResult add(InputRow row, Supplier committerSupplier) - throws IndexSizeExceededException - { - long messageTimestamp = row.getTimestampFromEpoch(); - final Sink sink = getSink(messageTimestamp); - metrics.reportMessageMaxTimestamp(messageTimestamp); - if (sink == null) { - return Plumber.THROWAWAY; - } - - final IncrementalIndexAddResult addResult = sink.add(row, false); - if (config.isReportParseExceptions() && addResult.getParseException() != null) { - throw addResult.getParseException(); - } - - if (!sink.canAppendRow() || System.currentTimeMillis() > nextFlush) { - persist(committerSupplier.get()); - } - - return addResult; - } - - private Sink getSink(long timestamp) - { - if (!rejectionPolicy.accept(timestamp)) { - return null; - } - - final Granularity segmentGranularity = schema.getGranularitySpec().getSegmentGranularity(); - final VersioningPolicy versioningPolicy = config.getVersioningPolicy(); - - DateTime truncatedDateTime = segmentGranularity.bucketStart(DateTimes.utc(timestamp)); - final long truncatedTime = truncatedDateTime.getMillis(); - - Sink retVal = sinks.get(truncatedTime); - - if (retVal == null) { - final Interval sinkInterval = new Interval( - truncatedDateTime, - segmentGranularity.increment(truncatedDateTime) - ); - - retVal = new Sink( - sinkInterval, - schema, - config.getShardSpec(), - versioningPolicy.getVersion(sinkInterval), - config.getAppendableIndexSpec(), - config.getMaxRowsInMemory(), - config.getMaxBytesInMemoryOrDefault(), - true, - config.getDedupColumn() - ); - addSink(retVal); - - } - - return retVal; - } - - @Override - public QueryRunner getQueryRunner(final Query query) - { - // Calling getQueryRunnerForIntervals here works because there's only one segment per interval for RealtimePlumber. - return texasRanger.getQueryRunnerForIntervals(query, query.getIntervals()); - } - - @Override - public void persist(final Committer committer) - { - final List> indexesToPersist = new ArrayList<>(); - for (Sink sink : sinks.values()) { - if (sink.swappable()) { - indexesToPersist.add(Pair.of(sink.swap(), sink.getInterval())); - } - } - - log.info("Submitting persist runnable for dataSource[%s]", schema.getDataSource()); - - final Stopwatch runExecStopwatch = Stopwatch.createStarted(); - final Stopwatch persistStopwatch = Stopwatch.createStarted(); - - final Map metadataElems = committer.getMetadata() == null ? null : - ImmutableMap.of( - COMMIT_METADATA_KEY, - committer.getMetadata(), - COMMIT_METADATA_TIMESTAMP_KEY, - System.currentTimeMillis() - ); - - persistExecutor.execute( - new ThreadRenamingRunnable(StringUtils.format("%s-incremental-persist", schema.getDataSource())) - { - @Override - public void doRun() - { - /* Note: - If plumber crashes after storing a subset of all the hydrants then we will lose data and next - time we will start with the commitMetadata stored in those hydrants. - option#1: - maybe it makes sense to store the metadata outside the segments in a separate file. This is because the - commit metadata isn't really associated with an individual segment-- it's associated with a set of segments - that are persisted at the same time or maybe whole datasource. So storing it in the segments is asking for problems. - Sort of like this: - - { - "metadata" : {"foo": "bar"}, - "segments": [ - {"id": "datasource_2000_2001_2000_1", "hydrant": 10}, - {"id": "datasource_2001_2002_2001_1", "hydrant": 12}, - ] - } - When a realtime node crashes and starts back up, it would delete any hydrants numbered higher than the - ones in the commit file. - - option#2 - We could also just include the set of segments for the same chunk of metadata in more metadata on each - of the segments. we might also have to think about the hand-off in terms of the full set of segments being - handed off instead of individual segments being handed off (that is, if one of the set succeeds in handing - off and the others fail, the real-time would believe that it needs to re-ingest the data). - */ - long persistThreadCpuTime = JvmUtils.safeGetThreadCpuTime(); - try { - for (Pair pair : indexesToPersist) { - metrics.incrementRowOutputCount( - persistHydrant(pair.lhs, schema, pair.rhs, metadataElems) - ); - } - committer.run(); - } - catch (Exception e) { - metrics.incrementFailedPersists(); - throw e; - } - finally { - metrics.incrementPersistCpuTime(JvmUtils.safeGetThreadCpuTime() - persistThreadCpuTime); - metrics.incrementNumPersists(); - metrics.incrementPersistTimeMillis(persistStopwatch.elapsed(TimeUnit.MILLISECONDS)); - persistStopwatch.stop(); - } - } - } - ); - - final long startDelay = runExecStopwatch.elapsed(TimeUnit.MILLISECONDS); - metrics.incrementPersistBackPressureMillis(startDelay); - if (startDelay > WARN_DELAY) { - log.warn("Ingestion was throttled for [%,d] millis because persists were pending.", startDelay); - } - runExecStopwatch.stop(); - resetNextFlush(); - } - - // Submits persist-n-merge task for a Sink to the mergeExecutor - private void persistAndMerge(final long truncatedTime, final Sink sink) - { - final String threadName = StringUtils.format( - "%s-%s-persist-n-merge", schema.getDataSource(), DateTimes.utc(truncatedTime) - ); - mergeExecutor.execute( - new ThreadRenamingRunnable(threadName) - { - final Interval interval = sink.getInterval(); - Stopwatch mergeStopwatch = null; - - @Override - public void doRun() - { - try { - // Bail out if this sink has been abandoned by a previously-executed task. - if (sinks.get(truncatedTime) != sink) { - log.info("Sink[%s] was abandoned, bailing out of persist-n-merge.", sink); - return; - } - - // Use a file to indicate that pushing has completed. - final File persistDir = computePersistDir(schema, interval); - final File mergedTarget = new File(persistDir, "merged"); - final File isPushedMarker = new File(persistDir, "isPushedMarker"); - - if (!isPushedMarker.exists()) { - removeSegment(sink, mergedTarget); - if (mergedTarget.exists()) { - log.warn("Merged target[%s] still exists after attempt to delete it; skipping push.", mergedTarget); - return; - } - } else { - log.info("Already pushed sink[%s]", sink); - return; - } - - /* - Note: it the plumber crashes after persisting a subset of hydrants then might duplicate data as these - hydrants will be read but older commitMetadata will be used. fixing this possibly needs structural - changes to plumber. - */ - for (FireHydrant hydrant : sink) { - synchronized (hydrant) { - if (!hydrant.hasSwapped()) { - log.info("Hydrant[%s] hasn't swapped yet, swapping. Sink[%s]", hydrant, sink); - final int rowCount = persistHydrant(hydrant, schema, interval, null); - metrics.incrementRowOutputCount(rowCount); - } - } - } - final long mergeThreadCpuTime = JvmUtils.safeGetThreadCpuTime(); - mergeStopwatch = Stopwatch.createStarted(); - - final File mergedFile; - List indexes = new ArrayList<>(); - Closer closer = Closer.create(); - try { - for (FireHydrant fireHydrant : sink) { - Pair segmentAndCloseable = fireHydrant.getAndIncrementSegment(); - final QueryableIndex queryableIndex = segmentAndCloseable.lhs.asQueryableIndex(); - log.info("Adding hydrant[%s]", fireHydrant); - indexes.add(queryableIndex); - closer.register(segmentAndCloseable.rhs); - } - - mergedFile = indexMerger.mergeQueryableIndex( - indexes, - schema.getGranularitySpec().isRollup(), - schema.getAggregators(), - null, - mergedTarget, - config.getIndexSpec(), - config.getIndexSpecForIntermediatePersists(), - new BaseProgressIndicator(), - config.getSegmentWriteOutMediumFactory(), - -1 - ); - } - catch (Throwable t) { - throw closer.rethrow(t); - } - finally { - closer.close(); - } - - // emit merge metrics before publishing segment - metrics.incrementMergeCpuTime(JvmUtils.safeGetThreadCpuTime() - mergeThreadCpuTime); - metrics.incrementMergeTimeMillis(mergeStopwatch.elapsed(TimeUnit.MILLISECONDS)); - - log.info("Pushing [%s] to deep storage", sink.getSegment().getId()); - - DataSegment segment = dataSegmentPusher.push( - mergedFile, - sink.getSegment().withDimensions(IndexMerger.getMergedDimensionsFromQueryableIndexes(indexes, schema.getDimensionsSpec())), - false - ); - log.info("Inserting [%s] to the metadata store", sink.getSegment().getId()); - segmentPublisher.publishSegment(segment); - - if (!isPushedMarker.createNewFile()) { - log.makeAlert("Failed to create marker file for [%s]", schema.getDataSource()) - .addData("interval", sink.getInterval()) - .addData("partitionNum", segment.getShardSpec().getPartitionNum()) - .addData("marker", isPushedMarker) - .emit(); - } - } - catch (Exception e) { - metrics.incrementFailedHandoffs(); - log.makeAlert(e, "Failed to persist merged index[%s]", schema.getDataSource()) - .addData("interval", interval) - .emit(); - if (shuttingDown) { - // We're trying to shut down, and this segment failed to push. Let's just get rid of it. - // This call will also delete possibly-partially-written files, so we don't need to do it explicitly. - cleanShutdown = false; - abandonSegment(truncatedTime, sink); - } - } - finally { - if (mergeStopwatch != null) { - mergeStopwatch.stop(); - } - } - } - } - ); - handoffNotifier.registerSegmentHandoffCallback( - new SegmentDescriptor(sink.getInterval(), sink.getVersion(), config.getShardSpec().getPartitionNum()), - mergeExecutor, new Runnable() - { - @Override - public void run() - { - abandonSegment(sink.getInterval().getStartMillis(), sink); - metrics.incrementHandOffCount(); - } - } - ); - } - - @Override - public void finishJob() - { - log.info("Shutting down..."); - - shuttingDown = true; - - for (final Map.Entry entry : sinks.entrySet()) { - entry.getValue().clearDedupCache(); - persistAndMerge(entry.getKey(), entry.getValue()); - } - - final long forceEndWaitTime = System.currentTimeMillis() + config.getHandoffConditionTimeout(); - while (!sinks.isEmpty()) { - try { - log.info( - "Cannot shut down yet! Sinks remaining: %s", - Collections2.transform(sinks.values(), sink -> sink.getSegment().getId()) - ); - - synchronized (handoffCondition) { - while (!sinks.isEmpty()) { - if (config.getHandoffConditionTimeout() == 0) { - handoffCondition.wait(); - } else { - long curr = System.currentTimeMillis(); - if (forceEndWaitTime - curr > 0) { - handoffCondition.wait(forceEndWaitTime - curr); - } else { - throw new ISE( - "Segment handoff wait timeout. [%s] segments might not have completed handoff.", - sinks.size() - ); - } - } - } - } - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - - handoffNotifier.close(); - shutdownExecutors(); - - stopped = true; - - if (!cleanShutdown) { - throw new ISE("Exception occurred during persist and merge."); - } - } - - private void resetNextFlush() - { - nextFlush = DateTimes.nowUtc().plus(config.getIntermediatePersistPeriod()).getMillis(); - } - - protected void initializeExecutors() - { - final int maxPendingPersists = config.getMaxPendingPersists(); - - if (persistExecutor == null) { - // use a blocking single threaded executor to throttle the firehose when write to disk is slow - persistExecutor = Execs.newBlockingSingleThreaded( - "plumber_persist_%d", - maxPendingPersists, - TaskThreadPriority.getThreadPriorityFromTaskPriority(config.getPersistThreadPriority()) - ); - } - if (mergeExecutor == null) { - // use a blocking single threaded executor to throttle the firehose when write to disk is slow - mergeExecutor = Execs.newBlockingSingleThreaded( - "plumber_merge_%d", - 1, - TaskThreadPriority.getThreadPriorityFromTaskPriority(config.getMergeThreadPriority()) - ); - } - - if (scheduledExecutor == null) { - scheduledExecutor = Execs.scheduledSingleThreaded("plumber_scheduled_%d"); - } - } - - protected void shutdownExecutors() - { - // scheduledExecutor is shutdown here - if (scheduledExecutor != null) { - scheduledExecutor.shutdown(); - persistExecutor.shutdown(); - mergeExecutor.shutdown(); - } - } - - protected Object bootstrapSinksFromDisk() - { - final VersioningPolicy versioningPolicy = config.getVersioningPolicy(); - - File baseDir = computeBaseDir(schema); - if (baseDir == null || !baseDir.exists()) { - return null; - } - - File[] files = baseDir.listFiles(); - if (files == null) { - return null; - } - - Object metadata = null; - long latestCommitTime = 0; - for (File sinkDir : files) { - final Interval sinkInterval = Intervals.of(sinkDir.getName().replace('_', '/')); - - //final File[] sinkFiles = sinkDir.listFiles(); - // To avoid reading and listing of "merged" dir - final File[] sinkFiles = sinkDir.listFiles( - new FilenameFilter() - { - @Override - public boolean accept(File dir, String fileName) - { - return !(Ints.tryParse(fileName) == null); - } - } - ); - Arrays.sort( - sinkFiles, - new Comparator() - { - @Override - public int compare(File o1, File o2) - { - try { - return Ints.compare(Integer.parseInt(o1.getName()), Integer.parseInt(o2.getName())); - } - catch (NumberFormatException e) { - log.error(e, "Couldn't compare as numbers? [%s][%s]", o1, o2); - return o1.compareTo(o2); - } - } - } - ); - boolean isCorrupted = false; - List hydrants = new ArrayList<>(); - for (File segmentDir : sinkFiles) { - log.info("Loading previously persisted segment at [%s]", segmentDir); - - // Although this has been tackled at start of this method. - // Just a doubly-check added to skip "merged" dir. from being added to hydrants - // If 100% sure that this is not needed, this check can be removed. - if (Ints.tryParse(segmentDir.getName()) == null) { - continue; - } - QueryableIndex queryableIndex = null; - try { - queryableIndex = indexIO.loadIndex(segmentDir); - } - catch (IOException e) { - log.error(e, "Problem loading segmentDir from disk."); - isCorrupted = true; - } - if (isCorrupted) { - try { - File corruptSegmentDir = computeCorruptedFileDumpDir(segmentDir, schema); - log.info("Renaming %s to %s", segmentDir.getAbsolutePath(), corruptSegmentDir.getAbsolutePath()); - org.apache.commons.io.FileUtils.copyDirectory(segmentDir, corruptSegmentDir); - FileUtils.deleteDirectory(segmentDir); - } - catch (Exception e1) { - log.error(e1, "Failed to rename %s", segmentDir.getAbsolutePath()); - } - //Note: skipping corrupted segment might lead to dropping some data. This strategy should be changed - //at some point. - continue; - } - Metadata segmentMetadata = queryableIndex.getMetadata(); - if (segmentMetadata != null) { - Object timestampObj = segmentMetadata.get(COMMIT_METADATA_TIMESTAMP_KEY); - if (timestampObj != null) { - long timestamp = ((Long) timestampObj).longValue(); - if (timestamp > latestCommitTime) { - log.info( - "Found metaData [%s] with latestCommitTime [%s] greater than previous recorded [%s]", - queryableIndex.getMetadata(), - timestamp, - latestCommitTime - ); - latestCommitTime = timestamp; - metadata = queryableIndex.getMetadata().get(COMMIT_METADATA_KEY); - } - } - } - hydrants.add( - new FireHydrant( - new QueryableIndexSegment( - queryableIndex, - SegmentId.of( - schema.getDataSource(), - sinkInterval, - versioningPolicy.getVersion(sinkInterval), - config.getShardSpec() - ) - ), - Integer.parseInt(segmentDir.getName()) - ) - ); - } - if (hydrants.isEmpty()) { - // Probably encountered a corrupt sink directory - log.warn( - "Found persisted segment directory with no intermediate segments present at %s, skipping sink creation.", - sinkDir.getAbsolutePath() - ); - continue; - } - final Sink currSink = new Sink( - sinkInterval, - schema, - config.getShardSpec(), - versioningPolicy.getVersion(sinkInterval), - config.getAppendableIndexSpec(), - config.getMaxRowsInMemory(), - config.getMaxBytesInMemoryOrDefault(), - true, - config.getDedupColumn(), - hydrants - ); - addSink(currSink); - } - return metadata; - } - - private void addSink(final Sink sink) - { - sinks.put(sink.getInterval().getStartMillis(), sink); - metrics.setSinkCount(sinks.size()); - sinkTimeline.add( - sink.getInterval(), - sink.getVersion(), - new SingleElementPartitionChunk<>(sink) - ); - try { - segmentAnnouncer.announceSegment(sink.getSegment()); - } - catch (IOException e) { - log.makeAlert(e, "Failed to announce new segment[%s]", schema.getDataSource()) - .addData("interval", sink.getInterval()) - .emit(); - } - clearDedupCache(); - } - - protected void startPersistThread() - { - final Granularity segmentGranularity = schema.getGranularitySpec().getSegmentGranularity(); - final Period windowPeriod = config.getWindowPeriod(); - - final DateTime truncatedNow = segmentGranularity.bucketStart(DateTimes.nowUtc()); - final long windowMillis = windowPeriod.toStandardDuration().getMillis(); - - log.info( - "Expect to run at [%s]", - DateTimes.nowUtc().plus( - new Duration( - System.currentTimeMillis(), - segmentGranularity.increment(truncatedNow).getMillis() + windowMillis - ) - ) - ); - - String threadName = StringUtils.format( - "%s-overseer-%d", - schema.getDataSource(), - config.getShardSpec().getPartitionNum() - ); - ThreadRenamingCallable threadRenamingCallable = - new ThreadRenamingCallable(threadName) - { - @Override - public ScheduledExecutors.Signal doCall() - { - if (stopped) { - log.info("Stopping merge-n-push overseer thread"); - return ScheduledExecutors.Signal.STOP; - } - - mergeAndPush(); - - if (stopped) { - log.info("Stopping merge-n-push overseer thread"); - return ScheduledExecutors.Signal.STOP; - } else { - return ScheduledExecutors.Signal.REPEAT; - } - } - }; - Duration initialDelay = new Duration( - System.currentTimeMillis(), - segmentGranularity.increment(truncatedNow).getMillis() + windowMillis - ); - Duration rate = new Duration(truncatedNow, segmentGranularity.increment(truncatedNow)); - ScheduledExecutors.scheduleAtFixedRate(scheduledExecutor, initialDelay, rate, threadRenamingCallable); - } - - private void clearDedupCache() - { - long minTimestamp = getAllowedMinTime().getMillis(); - - for (Map.Entry entry : sinks.entrySet()) { - final Long intervalStart = entry.getKey(); - if (intervalStart < minTimestamp) { - entry.getValue().clearDedupCache(); - } - } - } - - private DateTime getAllowedMinTime() - { - final Granularity segmentGranularity = schema.getGranularitySpec().getSegmentGranularity(); - final Period windowPeriod = config.getWindowPeriod(); - - final long windowMillis = windowPeriod.toStandardDuration().getMillis(); - return segmentGranularity.bucketStart( - DateTimes.utc(Math.max(windowMillis, rejectionPolicy.getCurrMaxTime().getMillis()) - windowMillis) - ); - } - - private void mergeAndPush() - { - log.info("Starting merge and push."); - DateTime minTimestampAsDate = getAllowedMinTime(); - long minTimestamp = minTimestampAsDate.getMillis(); - - log.info( - "Found [%,d] segments. Attempting to hand off segments that start before [%s].", - sinks.size(), - minTimestampAsDate - ); - - List> sinksToPush = new ArrayList<>(); - for (Map.Entry entry : sinks.entrySet()) { - final Long intervalStart = entry.getKey(); - if (intervalStart < minTimestamp) { - log.info("Adding entry [%s] for merge and push.", entry); - sinksToPush.add(entry); - entry.getValue().clearDedupCache(); - } else { - log.info( - "Skipping persist and merge for entry [%s] : Start time [%s] >= [%s] min timestamp required in this run. Segment will be picked up in a future run.", - entry, - DateTimes.utc(intervalStart), - minTimestampAsDate - ); - } - } - - log.info("Found [%,d] sinks to persist and merge", sinksToPush.size()); - - for (final Map.Entry entry : sinksToPush) { - persistAndMerge(entry.getKey(), entry.getValue()); - } - } - - /** - * Unannounces a given sink and removes all local references to it. It is important that this is only called - * from the single-threaded mergeExecutor, since otherwise chaos may ensue if merged segments are deleted while - * being created. - * - * @param truncatedTime sink key - * @param sink sink to unannounce - */ - protected void abandonSegment(final long truncatedTime, final Sink sink) - { - if (sinks.containsKey(truncatedTime)) { - try { - segmentAnnouncer.unannounceSegment(sink.getSegment()); - removeSegment(sink, computePersistDir(schema, sink.getInterval())); - log.info("Removing sinkKey %d for segment %s", truncatedTime, sink.getSegment().getId()); - sinks.remove(truncatedTime); - metrics.setSinkCount(sinks.size()); - sinkTimeline.remove( - sink.getInterval(), - sink.getVersion(), - new SingleElementPartitionChunk<>(sink) - ); - for (FireHydrant hydrant : sink) { - cache.close(SinkQuerySegmentWalker.makeHydrantCacheIdentifier(hydrant)); - hydrant.swapSegment(null); - } - synchronized (handoffCondition) { - handoffCondition.notifyAll(); - } - } - catch (Exception e) { - log.makeAlert(e, "Unable to abandon old segment for dataSource[%s]", schema.getDataSource()) - .addData("interval", sink.getInterval()) - .emit(); - } - } - } - - protected File computeBaseDir(DataSchema schema) - { - return new File(config.getBasePersistDirectory(), schema.getDataSource()); - } - - protected File computeCorruptedFileDumpDir(File persistDir, DataSchema schema) - { - return new File( - StringUtils.replace( - persistDir.getAbsolutePath(), - schema.getDataSource(), - "corrupted" + File.pathSeparator + schema.getDataSource() - ) - ); - } - - protected File computePersistDir(DataSchema schema, Interval interval) - { - return new File(computeBaseDir(schema), interval.toString().replace('/', '_')); - } - - /** - * Persists the given hydrant and returns the number of rows persisted - * - * @param indexToPersist hydrant to persist - * @param schema datasource schema - * @param interval interval to persist - * - * @return the number of rows persisted - */ - protected int persistHydrant( - FireHydrant indexToPersist, - DataSchema schema, - Interval interval, - Map metadataElems - ) - { - synchronized (indexToPersist) { - if (indexToPersist.hasSwapped()) { - log.info( - "DataSource[%s], Interval[%s], Hydrant[%s] already swapped. Ignoring request to persist.", - schema.getDataSource(), interval, indexToPersist - ); - return 0; - } - - log.info( - "DataSource[%s], Interval[%s], Metadata [%s] persisting Hydrant[%s]", - schema.getDataSource(), - interval, - metadataElems, - indexToPersist - ); - try { - int numRows = indexToPersist.getIndex().size(); - - indexToPersist.getIndex().getMetadata().putAll(metadataElems); - final File persistedFile = indexMerger.persist( - indexToPersist.getIndex(), - interval, - new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())), - config.getIndexSpecForIntermediatePersists(), - config.getSegmentWriteOutMediumFactory() - ); - - indexToPersist.swapSegment( - new QueryableIndexSegment(indexIO.loadIndex(persistedFile), indexToPersist.getSegmentId()) - ); - return numRows; - } - catch (IOException e) { - log.makeAlert("dataSource[%s] -- incremental persist failed", schema.getDataSource()) - .addData("interval", interval) - .addData("count", indexToPersist.getCount()) - .emit(); - - throw new RuntimeException(e); - } - } - } - - private void removeSegment(final Sink sink, final File target) - { - if (target.exists()) { - try { - log.info("Deleting Index File[%s]", target); - FileUtils.deleteDirectory(target); - } - catch (Exception e) { - log.makeAlert(e, "Unable to remove file for dataSource[%s]", schema.getDataSource()) - .addData("file", target) - .addData("interval", sink.getInterval()) - .emit(); - } - } - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchool.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchool.java deleted file mode 100644 index 8b19153a9de..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchool.java +++ /dev/null @@ -1,136 +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.segment.realtime.plumber; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import org.apache.druid.client.cache.Cache; -import org.apache.druid.client.cache.CacheConfig; -import org.apache.druid.client.cache.CachePopulatorStats; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.query.QueryProcessingPool; -import org.apache.druid.query.QueryRunnerFactoryConglomerate; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.IndexMergerV9; -import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.join.JoinableFactory; -import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.segment.realtime.SegmentPublisher; -import org.apache.druid.server.coordination.DataSegmentAnnouncer; - -/** - * - */ -public class RealtimePlumberSchool implements PlumberSchool -{ - private final ServiceEmitter emitter; - private final QueryRunnerFactoryConglomerate conglomerate; - private final DataSegmentPusher dataSegmentPusher; - private final DataSegmentAnnouncer segmentAnnouncer; - private final SegmentPublisher segmentPublisher; - private final SegmentHandoffNotifierFactory handoffNotifierFactory; - private final QueryProcessingPool queryProcessingPool; - private final JoinableFactory joinableFactory; - private final IndexMergerV9 indexMergerV9; - private final IndexIO indexIO; - private final Cache cache; - private final CacheConfig cacheConfig; - private final CachePopulatorStats cachePopulatorStats; - private final ObjectMapper objectMapper; - - @JsonCreator - public RealtimePlumberSchool( - @JacksonInject ServiceEmitter emitter, - @JacksonInject QueryRunnerFactoryConglomerate conglomerate, - @JacksonInject DataSegmentPusher dataSegmentPusher, - @JacksonInject DataSegmentAnnouncer segmentAnnouncer, - @JacksonInject SegmentPublisher segmentPublisher, - @JacksonInject SegmentHandoffNotifierFactory handoffNotifierFactory, - @JacksonInject QueryProcessingPool queryProcessingPool, - @JacksonInject JoinableFactory joinableFactory, - @JacksonInject IndexMergerV9 indexMergerV9, - @JacksonInject IndexIO indexIO, - @JacksonInject Cache cache, - @JacksonInject CacheConfig cacheConfig, - @JacksonInject CachePopulatorStats cachePopulatorStats, - @JacksonInject ObjectMapper objectMapper - ) - { - this.emitter = emitter; - this.conglomerate = conglomerate; - this.dataSegmentPusher = dataSegmentPusher; - this.segmentAnnouncer = segmentAnnouncer; - this.segmentPublisher = segmentPublisher; - this.handoffNotifierFactory = handoffNotifierFactory; - this.queryProcessingPool = queryProcessingPool; - this.joinableFactory = joinableFactory; - this.indexMergerV9 = Preconditions.checkNotNull(indexMergerV9, "Null IndexMergerV9"); - this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO"); - - this.cache = cache; - this.cacheConfig = cacheConfig; - this.cachePopulatorStats = cachePopulatorStats; - this.objectMapper = objectMapper; - } - - @Override - public Plumber findPlumber( - final DataSchema schema, - final RealtimeTuningConfig config, - final FireDepartmentMetrics metrics - ) - { - verifyState(); - - return new RealtimePlumber( - schema, - config, - metrics, - emitter, - conglomerate, - segmentAnnouncer, - queryProcessingPool, - dataSegmentPusher, - segmentPublisher, - handoffNotifierFactory.createSegmentHandoffNotifier(schema.getDataSource()), - indexMergerV9, - indexIO, - cache, - cacheConfig, - cachePopulatorStats, - objectMapper - ); - } - - private void verifyState() - { - Preconditions.checkNotNull(conglomerate, "must specify a queryRunnerFactoryConglomerate to do this action."); - Preconditions.checkNotNull(dataSegmentPusher, "must specify a segmentPusher to do this action."); - Preconditions.checkNotNull(segmentAnnouncer, "must specify a segmentAnnouncer to do this action."); - Preconditions.checkNotNull(segmentPublisher, "must specify a segmentPublisher to do this action."); - Preconditions.checkNotNull(handoffNotifierFactory, "must specify a handoffNotifierFactory to do this action."); - Preconditions.checkNotNull(emitter, "must specify a serviceEmitter to do this action."); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RejectionPolicy.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RejectionPolicy.java deleted file mode 100644 index e1dd6185bef..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RejectionPolicy.java +++ /dev/null @@ -1,28 +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.segment.realtime.plumber; - -import org.joda.time.DateTime; - -public interface RejectionPolicy -{ - DateTime getCurrMaxTime(); - boolean accept(long timestamp); -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RejectionPolicyFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RejectionPolicyFactory.java deleted file mode 100644 index f9b559f179d..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RejectionPolicyFactory.java +++ /dev/null @@ -1,35 +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.segment.realtime.plumber; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.joda.time.Period; - -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "serverTime", value = ServerTimeRejectionPolicyFactory.class), - @JsonSubTypes.Type(name = "messageTime", value = MessageTimeRejectionPolicyFactory.class), - @JsonSubTypes.Type(name = "none", value = NoopRejectionPolicyFactory.class) -}) -public interface RejectionPolicyFactory -{ - RejectionPolicy create(Period windowPeriod); -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/ServerTimeRejectionPolicyFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/ServerTimeRejectionPolicyFactory.java deleted file mode 100644 index 9cbd2cca23f..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/ServerTimeRejectionPolicyFactory.java +++ /dev/null @@ -1,60 +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.segment.realtime.plumber; - -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.StringUtils; -import org.joda.time.DateTime; -import org.joda.time.Period; - -public class ServerTimeRejectionPolicyFactory implements RejectionPolicyFactory -{ - @Override - public RejectionPolicy create(final Period windowPeriod) - { - final long windowMillis = windowPeriod.toStandardDuration().getMillis(); - - return new RejectionPolicy() - { - @Override - public DateTime getCurrMaxTime() - { - return DateTimes.nowUtc(); - } - - @Override - public boolean accept(long timestamp) - { - long now = System.currentTimeMillis(); - - boolean notTooOld = timestamp >= (now - windowMillis); - boolean notTooYoung = timestamp <= (now + windowMillis); - - return notTooOld && notTooYoung; - } - - @Override - public String toString() - { - return StringUtils.format("serverTime-%s", windowPeriod); - } - }; - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/VersioningPolicy.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/VersioningPolicy.java deleted file mode 100644 index 7d6e539f33a..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/VersioningPolicy.java +++ /dev/null @@ -1,35 +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.segment.realtime.plumber; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.joda.time.Interval; - -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "intervalStart", value = IntervalStartVersioningPolicy.class), - @JsonSubTypes.Type(name = "custom", value = CustomVersioningPolicy.class) - -}) -public interface VersioningPolicy -{ - String getVersion(Interval interval); -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Committers.java b/server/src/main/java/org/apache/druid/segment/realtime/sink/Committers.java similarity index 96% rename from server/src/main/java/org/apache/druid/segment/realtime/plumber/Committers.java rename to server/src/main/java/org/apache/druid/segment/realtime/sink/Committers.java index aa0bcbe50fb..8efd117b06c 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Committers.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/sink/Committers.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.plumber; +package org.apache.druid.segment.realtime.sink; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java similarity index 92% rename from server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java rename to server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java index 4035f606434..44d0db3cb7f 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.plumber; +package org.apache.druid.segment.realtime.sink; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Predicate; @@ -60,13 +60,11 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -74,6 +72,8 @@ import java.util.function.Function; public class Sink implements Iterable, Overshadowable { + private static final IncrementalIndexAddResult NOT_WRITABLE = new IncrementalIndexAddResult(-1, -1, "not writable"); + private static final IncrementalIndexAddResult ALREADY_SWAPPED = new IncrementalIndexAddResult(-1, -1, "write after index swapped"); private static final Logger log = new Logger(Sink.class); @@ -98,8 +98,6 @@ public class Sink implements Iterable, Overshadowable private final Map columnTypeExcludingCurrIndex = new HashMap<>(); private final AtomicInteger numRowsExcludingCurrIndex = new AtomicInteger(); - private final String dedupColumn; - private final Set dedupSet = new HashSet<>(); private volatile FireHydrant currHydrant; private volatile boolean writable = true; @@ -112,8 +110,7 @@ public class Sink implements Iterable, Overshadowable AppendableIndexSpec appendableIndexSpec, int maxRowsInMemory, long maxBytesInMemory, - boolean useMaxMemoryEstimates, - String dedupColumn + boolean useMaxMemoryEstimates ) { this( @@ -125,7 +122,6 @@ public class Sink implements Iterable, Overshadowable maxRowsInMemory, maxBytesInMemory, useMaxMemoryEstimates, - dedupColumn, Collections.emptyList() ); } @@ -139,7 +135,6 @@ public class Sink implements Iterable, Overshadowable int maxRowsInMemory, long maxBytesInMemory, boolean useMaxMemoryEstimates, - String dedupColumn, List hydrants ) { @@ -151,7 +146,6 @@ public class Sink implements Iterable, Overshadowable this.maxRowsInMemory = maxRowsInMemory; this.maxBytesInMemory = maxBytesInMemory; this.useMaxMemoryEstimates = useMaxMemoryEstimates; - this.dedupColumn = dedupColumn; int maxCount = -1; for (int i = 0; i < hydrants.size(); ++i) { @@ -175,11 +169,6 @@ public class Sink implements Iterable, Overshadowable makeNewCurrIndex(interval.getStartMillis(), schema); } - public void clearDedupCache() - { - dedupSet.clear(); - } - public Interval getInterval() { return interval; @@ -198,7 +187,7 @@ public class Sink implements Iterable, Overshadowable synchronized (hydrantLock) { if (!writable) { - return Plumber.NOT_WRITABLE; + return NOT_WRITABLE; } IncrementalIndex index = currHydrant.getIndex(); @@ -206,10 +195,6 @@ public class Sink implements Iterable, Overshadowable return ALREADY_SWAPPED; // the hydrant was swapped without being replaced } - if (checkInDedupSet(row)) { - return Plumber.DUPLICATE; - } - return index.add(row, skipMaxRowsInMemoryCheck); } } @@ -267,7 +252,6 @@ public class Sink implements Iterable, Overshadowable return false; } writable = false; - clearDedupCache(); } return true; } @@ -334,41 +318,6 @@ public class Sink implements Iterable, Overshadowable return acquireSegmentReferences(hydrants, segmentMapFn, skipIncrementalSegment); } - private boolean checkInDedupSet(InputRow row) - { - if (dedupColumn != null) { - Object value = row.getRaw(dedupColumn); - if (value != null) { - if (value instanceof List) { - throw new IAE("Dedup on multi-value field not support"); - } - Long pk; - if (value instanceof Long || value instanceof Integer) { - pk = ((Number) value).longValue(); - } else { - // use long type hashcode to reduce heap cost. - // maybe hash collision, but it's more important to avoid OOM - pk = pkHash(String.valueOf(value)); - } - if (dedupSet.contains(pk)) { - return true; - } - dedupSet.add(pk); - } - } - return false; - } - - private long pkHash(String s) - { - long seed = 131; // 31 131 1313 13131 131313 etc.. BKDRHash - long hash = 0; - for (int i = 0; i < s.length(); i++) { - hash = (hash * seed) + s.charAt(i); - } - return hash; - } - private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) { final IncrementalIndexSchema indexSchema = new IncrementalIndexSchema.Builder() diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/SinkSegmentReference.java b/server/src/main/java/org/apache/druid/segment/realtime/sink/SinkSegmentReference.java similarity index 97% rename from server/src/main/java/org/apache/druid/segment/realtime/plumber/SinkSegmentReference.java rename to server/src/main/java/org/apache/druid/segment/realtime/sink/SinkSegmentReference.java index 10dfc2b275e..c24cf118661 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/SinkSegmentReference.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/sink/SinkSegmentReference.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.plumber; +package org.apache.druid.segment.realtime.sink; import org.apache.druid.segment.SegmentReference; diff --git a/server/src/main/java/org/apache/druid/server/coordination/DataSegmentAnnouncer.java b/server/src/main/java/org/apache/druid/server/coordination/DataSegmentAnnouncer.java index bf708206651..08368cbcabe 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/DataSegmentAnnouncer.java +++ b/server/src/main/java/org/apache/druid/server/coordination/DataSegmentAnnouncer.java @@ -29,11 +29,11 @@ public interface DataSegmentAnnouncer { void announceSegment(DataSegment segment) throws IOException; - void unannounceSegment(DataSegment segment) throws IOException; + void unannounceSegment(DataSegment segment); void announceSegments(Iterable segments) throws IOException; - void unannounceSegments(Iterable segments) throws IOException; + void unannounceSegments(Iterable segments); /** * Announces schema associated with all segments for the specified realtime task. diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 20a74e6c026..4c2fc76533e 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -582,7 +582,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testTransactionalAnnounceFailDbNullWantNotNull() throws IOException + public void testTransactionalAnnounceFailDbNullWantNotNull() { final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), @@ -604,7 +604,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testTransactionalAnnounceFailDbNotNullWantNull() throws IOException + public void testTransactionalAnnounceFailDbNotNullWantNull() { final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), @@ -682,7 +682,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testTransactionalAnnounceFailDbNotNullWantDifferent() throws IOException + public void testTransactionalAnnounceFailDbNotNullWantDifferent() { final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), @@ -712,7 +712,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testSimpleUsedList() throws IOException + public void testSimpleUsedList() { coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); Assert.assertEquals( @@ -728,7 +728,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testMultiIntervalUsedList() throws IOException + public void testMultiIntervalUsedList() { coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); coordinator.commitSegments(ImmutableSet.of(defaultSegment3), new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); @@ -771,7 +771,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUsedSegmentsUsingMultipleIntervals() throws IOException + public void testRetrieveUsedSegmentsUsingMultipleIntervals() { final List segments = createAndGetUsedYearSegments(1900, 2133); final List intervals = segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()); @@ -787,7 +787,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveAllUsedSegmentsUsingIntervalsOutOfRange() throws IOException + public void testRetrieveAllUsedSegmentsUsingIntervalsOutOfRange() { final List segments = createAndGetUsedYearSegments(1905, 1910); @@ -805,7 +805,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveAllUsedSegmentsUsingNoIntervals() throws IOException + public void testRetrieveAllUsedSegmentsUsingNoIntervals() { final List segments = createAndGetUsedYearSegments(1900, 2133); @@ -819,7 +819,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUnusedSegmentsUsingSingleIntervalAndNoLimit() throws IOException + public void testRetrieveUnusedSegmentsUsingSingleIntervalAndNoLimit() { final List segments = createAndGetUsedYearSegments(1900, 2133); markAllSegmentsUnused(new HashSet<>(segments), DateTimes.nowUtc()); @@ -836,7 +836,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUnusedSegmentsUsingSingleIntervalAndLimitAtRange() throws IOException + public void testRetrieveUnusedSegmentsUsingSingleIntervalAndLimitAtRange() { final List segments = createAndGetUsedYearSegments(1900, 2133); markAllSegmentsUnused(new HashSet<>(segments), DateTimes.nowUtc()); @@ -854,7 +854,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUnusedSegmentsUsingSingleIntervalAndLimitInRange() throws IOException + public void testRetrieveUnusedSegmentsUsingSingleIntervalAndLimitInRange() { final List segments = createAndGetUsedYearSegments(1900, 2133); markAllSegmentsUnused(new HashSet<>(segments), DateTimes.nowUtc()); @@ -872,26 +872,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUnusedSegmentsUsingSingleIntervalVersionAndLimitInRange() throws IOException - { - final List segments = createAndGetUsedYearSegments(1900, 2133); - markAllSegmentsUnused(new HashSet<>(segments), DateTimes.nowUtc()); - - final int requestedLimit = 10; - final List actualUnusedSegments = coordinator.retrieveUnusedSegmentsForInterval( - DS.WIKI, - Intervals.of("1900/3000"), - ImmutableList.of("version"), - requestedLimit, - null - ); - - Assert.assertEquals(requestedLimit, actualUnusedSegments.size()); - Assert.assertTrue(actualUnusedSegments.containsAll(segments.stream().limit(requestedLimit).collect(Collectors.toList()))); - } - - @Test - public void testRetrieveUnusedSegmentsUsingSingleIntervalAndLimitOutOfRange() throws IOException + public void testRetrieveUnusedSegmentsUsingSingleIntervalAndLimitOutOfRange() { final List segments = createAndGetUsedYearSegments(1900, 2133); markAllSegmentsUnused(new HashSet<>(segments), DateTimes.nowUtc()); @@ -908,7 +889,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUnusedSegmentsUsingSingleIntervalOutOfRange() throws IOException + public void testRetrieveUnusedSegmentsUsingSingleIntervalOutOfRange() { final List segments = createAndGetUsedYearSegments(1905, 1910); markAllSegmentsUnused(new HashSet<>(segments), DateTimes.nowUtc()); @@ -928,7 +909,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUnusedSegmentsUsingMultipleIntervalsAndNoLimit() throws IOException + public void testRetrieveUnusedSegmentsUsingMultipleIntervalsAndNoLimit() { final List segments = createAndGetUsedYearSegments(1900, 2133); DateTime usedStatusLastUpdatedTime = DateTimes.nowUtc(); @@ -958,7 +939,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUnusedSegmentsUsingNoIntervalsNoLimitAndNoLastSegmentId() throws IOException + public void testRetrieveUnusedSegmentsUsingNoIntervalsNoLimitAndNoLastSegmentId() { final List segments = createAndGetUsedYearSegments(1900, 2133); DateTime usedStatusLastUpdatedTime = DateTimes.nowUtc(); @@ -988,7 +969,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUnusedSegmentsUsingNoIntervalsAndNoLimitAndNoLastSegmentId() throws IOException + public void testRetrieveUnusedSegmentsUsingNoIntervalsAndNoLimitAndNoLastSegmentId() { final List segments = createAndGetUsedYearSegments(2033, 2133); DateTime usedStatusLastUpdatedTime = DateTimes.nowUtc(); @@ -1071,7 +1052,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUnusedSegmentsUsingMultipleIntervalsAndLimitAtRange() throws IOException + public void testRetrieveUnusedSegmentsUsingMultipleIntervalsAndLimitAtRange() { final List segments = createAndGetUsedYearSegments(1900, 2133); DateTime usedStatusLastUpdatedTime = DateTimes.nowUtc(); @@ -1101,7 +1082,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUnusedSegmentsUsingMultipleIntervalsAndLimitInRange() throws IOException + public void testRetrieveUnusedSegmentsUsingMultipleIntervalsAndLimitInRange() { final List segments = createAndGetUsedYearSegments(1900, 2133); DateTime usedStatusLastUpdatedTime = DateTimes.nowUtc(); @@ -1133,7 +1114,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUnusedSegmentsUsingMultipleIntervalsInSingleBatchLimitAndLastSegmentId() throws IOException + public void testRetrieveUnusedSegmentsUsingMultipleIntervalsInSingleBatchLimitAndLastSegmentId() { final List segments = createAndGetUsedYearSegments(2034, 2133); DateTime usedStatusLastUpdatedTime = DateTimes.nowUtc(); @@ -1169,7 +1150,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUnusedSegmentsUsingMultipleIntervalsLimitAndLastSegmentId() throws IOException + public void testRetrieveUnusedSegmentsUsingMultipleIntervalsLimitAndLastSegmentId() { final List segments = createAndGetUsedYearSegments(1900, 2133); DateTime usedStatusLastUpdatedTime = DateTimes.nowUtc(); @@ -1205,7 +1186,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUnusedSegmentsUsingMultipleIntervals() throws IOException + public void testRetrieveUnusedSegmentsUsingMultipleIntervals() { final List segments = createAndGetUsedYearSegments(1900, 2133); DateTime usedStatusLastUpdatedTime = DateTimes.nowUtc(); @@ -1235,7 +1216,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUnusedSegmentsUsingIntervalOutOfRange() throws IOException + public void testRetrieveUnusedSegmentsUsingIntervalOutOfRange() { final List segments = createAndGetUsedYearSegments(1905, 1910); markAllSegmentsUnused(new HashSet<>(segments), DateTimes.nowUtc()); @@ -1266,7 +1247,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUnusedSegmentsWithMaxUsedStatusLastUpdatedTime() throws IOException + public void testRetrieveUnusedSegmentsWithMaxUsedStatusLastUpdatedTime() { final List segments = createAndGetUsedYearSegments(1905, 1910); DateTime usedStatusLastUpdatedTime = DateTimes.nowUtc(); @@ -1316,7 +1297,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUnusedSegmentsWithMaxUsedStatusLastUpdatedTime2() throws IOException + public void testRetrieveUnusedSegmentsWithMaxUsedStatusLastUpdatedTime2() { final List segments = createAndGetUsedYearSegments(1900, 1950); final List evenYearSegments = new ArrayList<>(); @@ -1381,7 +1362,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testSimpleUnusedList() throws IOException + public void testSimpleUnusedList() { coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); @@ -1399,7 +1380,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRetrieveUnusedSegmentsWithVersions() throws IOException + public void testRetrieveUnusedSegmentsWithVersions() { final DateTime now = DateTimes.nowUtc(); final String v1 = now.toString(); @@ -1476,7 +1457,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testSimpleUnusedListWithLimit() throws IOException + public void testSimpleUnusedListWithLimit() { coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); @@ -1495,7 +1476,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testUsedOverlapLow() throws IOException + public void testUsedOverlapLow() { coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); Set actualSegments = ImmutableSet.copyOf( @@ -1513,7 +1494,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata @Test - public void testUsedOverlapHigh() throws IOException + public void testUsedOverlapHigh() { coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); Assert.assertEquals( @@ -1529,7 +1510,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testUsedOutOfBoundsLow() throws IOException + public void testUsedOutOfBoundsLow() { coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); Assert.assertTrue( @@ -1543,7 +1524,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata @Test - public void testUsedOutOfBoundsHigh() throws IOException + public void testUsedOutOfBoundsHigh() { coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); Assert.assertTrue( @@ -1556,7 +1537,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testUsedWithinBoundsEnd() throws IOException + public void testUsedWithinBoundsEnd() { coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); Assert.assertEquals( @@ -1572,7 +1553,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testUsedOverlapEnd() throws IOException + public void testUsedOverlapEnd() { coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); Assert.assertEquals( @@ -1588,7 +1569,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testUnusedOverlapLow() throws IOException + public void testUnusedOverlapLow() { coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); @@ -1606,7 +1587,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testUnusedUnderlapLow() throws IOException + public void testUnusedUnderlapLow() { coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); @@ -1622,7 +1603,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata @Test - public void testUnusedUnderlapHigh() throws IOException + public void testUnusedUnderlapHigh() { coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); @@ -1637,7 +1618,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testUnusedOverlapHigh() throws IOException + public void testUnusedOverlapHigh() { coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); @@ -1652,7 +1633,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testUnusedBigOverlap() throws IOException + public void testUnusedBigOverlap() { coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); @@ -1670,7 +1651,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testUnusedLowRange() throws IOException + public void testUnusedLowRange() { coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); @@ -1699,7 +1680,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testUnusedHighRange() throws IOException + public void testUnusedHighRange() { coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); markAllSegmentsUnused(); @@ -1728,7 +1709,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testUsedHugeTimeRangeEternityFilter() throws IOException + public void testUsedHugeTimeRangeEternityFilter() { coordinator.commitSegments( ImmutableSet.of( @@ -1752,7 +1733,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testUsedHugeTimeRangeTrickyFilter1() throws IOException + public void testUsedHugeTimeRangeTrickyFilter1() { coordinator.commitSegments( ImmutableSet.of( @@ -1776,7 +1757,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testUsedHugeTimeRangeTrickyFilter2() throws IOException + public void testUsedHugeTimeRangeTrickyFilter2() { coordinator.commitSegments( ImmutableSet.of( @@ -1800,7 +1781,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testEternitySegmentWithStringComparison() throws IOException + public void testEternitySegmentWithStringComparison() { coordinator.commitSegments( ImmutableSet.of( @@ -1822,7 +1803,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testEternityMultipleSegmentWithStringComparison() throws IOException + public void testEternityMultipleSegmentWithStringComparison() { coordinator.commitSegments( ImmutableSet.of( @@ -1845,7 +1826,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testFirstHalfEternitySegmentWithStringComparison() throws IOException + public void testFirstHalfEternitySegmentWithStringComparison() { coordinator.commitSegments( ImmutableSet.of( @@ -1867,7 +1848,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testFirstHalfEternityMultipleSegmentWithStringComparison() throws IOException + public void testFirstHalfEternityMultipleSegmentWithStringComparison() { coordinator.commitSegments( ImmutableSet.of( @@ -1890,7 +1871,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testSecondHalfEternitySegmentWithStringComparison() throws IOException + public void testSecondHalfEternitySegmentWithStringComparison() { coordinator.commitSegments( ImmutableSet.of( @@ -1914,7 +1895,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata // Known Issue: https://github.com/apache/druid/issues/12860 @Ignore @Test - public void testLargeIntervalWithStringComparison() throws IOException + public void testLargeIntervalWithStringComparison() { coordinator.commitSegments( ImmutableSet.of( @@ -1936,7 +1917,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testSecondHalfEternityMultipleSegmentWithStringComparison() throws IOException + public void testSecondHalfEternityMultipleSegmentWithStringComparison() { coordinator.commitSegments( ImmutableSet.of( @@ -1959,7 +1940,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testDeleteDataSourceMetadata() throws IOException + public void testDeleteDataSourceMetadata() { coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), @@ -1980,7 +1961,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testDeleteSegmentsInMetaDataStorage() throws IOException + public void testDeleteSegmentsInMetaDataStorage() { // Published segments to MetaDataStorage coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); @@ -2013,7 +1994,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testUpdateSegmentsInMetaDataStorage() throws IOException + public void testUpdateSegmentsInMetaDataStorage() { // Published segments to MetaDataStorage coordinator.commitSegments(SEGMENTS, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); @@ -2646,7 +2627,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testAllocatePendingSegmentsWithOvershadowingSegments() throws IOException + public void testAllocatePendingSegmentsWithOvershadowingSegments() { final String dataSource = "ds"; final Interval interval = Intervals.of("2017-01-01/2017-02-01"); @@ -2716,7 +2697,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IOException + public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() { final PartialShardSpec partialShardSpec = new HashBasedNumberedPartialShardSpec(null, 2, 5, null); final String dataSource = "ds"; @@ -2806,7 +2787,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testAddNumberedShardSpecAfterMultiDimensionsShardSpecWithUnknownCorePartitionSize() throws IOException + public void testAddNumberedShardSpecAfterMultiDimensionsShardSpecWithUnknownCorePartitionSize() { final String datasource = "datasource"; final Interval interval = Intervals.of("2020-01-01/P1D"); @@ -2850,7 +2831,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testAddNumberedShardSpecAfterSingleDimensionsShardSpecWithUnknownCorePartitionSize() throws IOException + public void testAddNumberedShardSpecAfterSingleDimensionsShardSpecWithUnknownCorePartitionSize() { final String datasource = "datasource"; final Interval interval = Intervals.of("2020-01-01/P1D"); @@ -2896,7 +2877,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRemoveDataSourceMetadataOlderThanDatasourceActiveShouldNotBeDeleted() throws Exception + public void testRemoveDataSourceMetadataOlderThanDatasourceActiveShouldNotBeDeleted() { coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), @@ -2925,7 +2906,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testRemoveDataSourceMetadataOlderThanDatasourceNotActiveAndOlderThanTimeShouldBeDeleted() throws Exception + public void testRemoveDataSourceMetadataOlderThanDatasourceNotActiveAndOlderThanTimeShouldBeDeleted() { coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), @@ -2951,7 +2932,6 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata @Test public void testRemoveDataSourceMetadataOlderThanDatasourceNotActiveButNotOlderThanTimeShouldNotBeDeleted() - throws Exception { coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), @@ -2981,7 +2961,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testMarkSegmentsAsUnusedWithinIntervalOneYear() throws IOException + public void testMarkSegmentsAsUnusedWithinIntervalOneYear() { coordinator.commitSegments(ImmutableSet.of(existingSegment1, existingSegment2), new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); @@ -3018,7 +2998,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testMarkSegmentsAsUnusedWithinIntervalTwoYears() throws IOException + public void testMarkSegmentsAsUnusedWithinIntervalTwoYears() { coordinator.commitSegments(ImmutableSet.of(existingSegment1, existingSegment2), new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); @@ -3116,7 +3096,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testTimelineVisibilityWith0CorePartitionTombstone() throws IOException + public void testTimelineVisibilityWith0CorePartitionTombstone() { final Interval interval = Intervals.of("2020/2021"); // Create and commit a tombstone segment @@ -3168,7 +3148,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testTimelineWith1CorePartitionTombstone() throws IOException + public void testTimelineWith1CorePartitionTombstone() { // Register the old generation tombstone spec for this test. mapper.registerSubtypes(TombstoneShardSpecWith1CorePartition.class); @@ -3222,7 +3202,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata } @Test - public void testSegmentIdShouldNotBeReallocated() throws IOException + public void testSegmentIdShouldNotBeReallocated() { final SegmentIdWithShardSpec idWithNullTaskAllocator = coordinator.allocatePendingSegment( DS.WIKI, diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java index a2f91378bb7..ce0e0686058 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java @@ -52,7 +52,6 @@ import org.skife.jdbi.v2.PreparedBatch; import org.skife.jdbi.v2.ResultIterator; import org.skife.jdbi.v2.util.StringMapper; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -326,7 +325,7 @@ public class IndexerSqlMetadataStorageCoordinatorTestBase .build(); } - protected List createAndGetUsedYearSegments(final int startYear, final int endYear) throws IOException + protected List createAndGetUsedYearSegments(final int startYear, final int endYear) { final List segments = new ArrayList<>(); diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataRuleManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataRuleManagerTest.java index 8cdf32bddf9..efeadbdc04e 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataRuleManagerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataRuleManagerTest.java @@ -62,7 +62,6 @@ public class SQLMetadataRuleManagerTest private MetadataRuleManagerConfig managerConfig; private SQLMetadataRuleManager ruleManager; private AuditManager auditManager; - private SQLMetadataSegmentPublisher publisher; private final ObjectMapper mapper = new DefaultObjectMapper(); @Before @@ -87,11 +86,6 @@ public class SQLMetadataRuleManagerTest ruleManager = new SQLMetadataRuleManager(mapper, managerConfig, tablesConfig, connector, auditManager); connector.createSegmentSchemasTable(); connector.createSegmentTable(); - publisher = new SQLMetadataSegmentPublisher( - mapper, - derbyConnectorRule.metadataTablesConfigSupplier().get(), - connector - ); } @Test @@ -321,7 +315,7 @@ public class SQLMetadataRuleManagerTest 1, 1234L ); - publisher.publishSegment(dataSegment); + SqlSegmentsMetadataManagerTestBase.publishSegment(connector, tablesConfig, mapper, dataSegment); // This will not delete the rule as the datasource has segment in the segment metadata table ruleManager.removeRulesForEmptyDatasourcesOlderThan(System.currentTimeMillis()); diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerSchemaPollTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerSchemaPollTest.java index 8595f3e7194..18095305ad9 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerSchemaPollTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerSchemaPollTest.java @@ -73,18 +73,13 @@ public class SqlSegmentsMetadataManagerSchemaPollTest extends SqlSegmentsMetadat CentralizedDatasourceSchemaConfig.create() ); sqlSegmentsMetadataManager.start(); - - publisher = new SQLMetadataSegmentPublisher( - jsonMapper, - derbyConnectorRule.metadataTablesConfigSupplier().get(), - connector - ); + storageConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); connector.createSegmentSchemasTable(); connector.createSegmentTable(); - publisher.publishSegment(segment1); - publisher.publishSegment(segment2); + publishSegment(segment1); + publishSegment(segment2); } @After diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java index 486e58e2a2a..d101a7a74b4 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java @@ -19,7 +19,6 @@ package org.apache.druid.metadata; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; @@ -34,7 +33,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; @@ -89,10 +87,6 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - private SqlSegmentsMetadataManager sqlSegmentsMetadataManager; - private SQLMetadataSegmentPublisher publisher; - private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); - private final DataSegment wikiSegment1 = CreateDataSegments.ofDatasource(DS.WIKI).startingAt("2012-03-15").eachOfSizeInMb(500).get(0); private final DataSegment wikiSegment2 = @@ -101,7 +95,7 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe private void publishUnusedSegments(DataSegment... segments) throws IOException { for (DataSegment segment : segments) { - publisher.publishSegment(segment); + publishSegment(segment); sqlSegmentsMetadataManager.markSegmentAsUnused(segment.getId()); } } @@ -109,8 +103,8 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe private void publishWikiSegments() { try { - publisher.publishSegment(wikiSegment1); - publisher.publishSegment(wikiSegment2); + publishSegment(wikiSegment1); + publishSegment(wikiSegment2); } catch (Exception e) { throw new RuntimeException(e); @@ -123,6 +117,7 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe connector = derbyConnectorRule.getConnector(); SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(); config.setPollDuration(Period.seconds(3)); + storageConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); segmentSchemaCache = new SegmentSchemaCache(new NoopServiceEmitter()); segmentSchemaManager = new SegmentSchemaManager( @@ -131,10 +126,8 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe connector ); - final TestDerbyConnector connector = derbyConnectorRule.getConnector(); - sqlSegmentsMetadataManager = new SqlSegmentsMetadataManager( - JSON_MAPPER, + jsonMapper, Suppliers.ofInstance(config), derbyConnectorRule.metadataTablesConfigSupplier(), connector, @@ -143,12 +136,6 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe ); sqlSegmentsMetadataManager.start(); - publisher = new SQLMetadataSegmentPublisher( - JSON_MAPPER, - derbyConnectorRule.metadataTablesConfigSupplier().get(), - connector - ); - connector.createSegmentSchemasTable(); connector.createSegmentTable(); } @@ -271,7 +258,7 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe .map(ImmutableDruidDataSource::getName) .collect(Collectors.toList()) ); - publisher.publishSegment(createNewSegment1(DS.KOALA)); + publishSegment(createNewSegment1(DS.KOALA)); // This call will force on demand poll sqlSegmentsMetadataManager.forceOrWaitOngoingDatabasePoll(); @@ -288,7 +275,7 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe ); final String newDataSource3 = "wikipedia3"; - publisher.publishSegment(createNewSegment1(newDataSource3)); + publishSegment(createNewSegment1(newDataSource3)); // This time wait for periodic poll (not doing on demand poll so we have to wait a bit...) while (sqlSegmentsMetadataManager.getDataSourcesSnapshot().getDataSource(newDataSource3) == null) { @@ -367,10 +354,11 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe sqlSegmentsMetadataManager.retrieveAllDataSourceNames() ); final DataSegment koalaSegment = createNewSegment1(DS.KOALA); - publisher.publishSegment(koalaSegment); + publishSegment(koalaSegment); sqlSegmentsMetadataManager.startPollingDatabasePeriodically(); return koalaSegment; } + /** * Create a corrupted segment entry in the segments table to test * whether the overall loading of segments from the database continues to work @@ -382,7 +370,7 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe publishWikiSegments(); final DataSegment corruptSegment = DataSegment.builder(wikiSegment1).dataSource("corrupt-datasource").build(); - publisher.publishSegment(corruptSegment); + publishSegment(corruptSegment); updateSegmentPayload(corruptSegment, StringUtils.toUtf8("corrupt-payload")); EmittingLogger.registerEmitter(new NoopServiceEmitter()); @@ -444,26 +432,56 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe Assert.assertEquals( ImmutableList.of(wikiSegment2.getInterval()), - sqlSegmentsMetadataManager.getUnusedSegmentIntervals(DS.WIKI, null, DateTimes.of("3000"), 1, DateTimes.COMPARE_DATE_AS_STRING_MAX) + sqlSegmentsMetadataManager.getUnusedSegmentIntervals( + DS.WIKI, + null, + DateTimes.of("3000"), + 1, + DateTimes.COMPARE_DATE_AS_STRING_MAX + ) ); // Test the DateTime maxEndTime argument of getUnusedSegmentIntervals Assert.assertEquals( ImmutableList.of(wikiSegment2.getInterval()), - sqlSegmentsMetadataManager.getUnusedSegmentIntervals(DS.WIKI, null, DateTimes.of(2012, 1, 7, 0, 0), 1, DateTimes.COMPARE_DATE_AS_STRING_MAX) + sqlSegmentsMetadataManager.getUnusedSegmentIntervals( + DS.WIKI, + null, + DateTimes.of(2012, 1, 7, 0, 0), + 1, + DateTimes.COMPARE_DATE_AS_STRING_MAX + ) ); Assert.assertEquals( ImmutableList.of(wikiSegment1.getInterval()), - sqlSegmentsMetadataManager.getUnusedSegmentIntervals(DS.WIKI, DateTimes.of(2012, 1, 7, 0, 0), DateTimes.of(2012, 4, 7, 0, 0), 1, DateTimes.COMPARE_DATE_AS_STRING_MAX) + sqlSegmentsMetadataManager.getUnusedSegmentIntervals( + DS.WIKI, + DateTimes.of(2012, 1, 7, 0, 0), + DateTimes.of(2012, 4, 7, 0, 0), + 1, + DateTimes.COMPARE_DATE_AS_STRING_MAX + ) ); Assert.assertEquals( ImmutableList.of(), - sqlSegmentsMetadataManager.getUnusedSegmentIntervals(DS.WIKI, DateTimes.of(2012, 1, 7, 0, 0), DateTimes.of(2012, 1, 7, 0, 0), 1, DateTimes.COMPARE_DATE_AS_STRING_MAX) + sqlSegmentsMetadataManager.getUnusedSegmentIntervals( + DS.WIKI, + DateTimes.of(2012, 1, 7, 0, 0), + DateTimes.of(2012, 1, 7, 0, 0), + 1, + DateTimes.COMPARE_DATE_AS_STRING_MAX + ) ); Assert.assertEquals( ImmutableList.of(wikiSegment2.getInterval(), wikiSegment1.getInterval()), - sqlSegmentsMetadataManager.getUnusedSegmentIntervals(DS.WIKI, null, DateTimes.of("3000"), 5, DateTimes.COMPARE_DATE_AS_STRING_MAX) + sqlSegmentsMetadataManager.getUnusedSegmentIntervals( + DS.WIKI, + null, + DateTimes.of("3000"), + 5, + DateTimes.COMPARE_DATE_AS_STRING_MAX + ) ); // Test a buffer period that should exclude some segments @@ -471,7 +489,13 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe // The wikipedia datasource has segments generated with last used time equal to roughly the time of test run. None of these segments should be selected with a bufer period of 1 day Assert.assertEquals( ImmutableList.of(), - sqlSegmentsMetadataManager.getUnusedSegmentIntervals(DS.WIKI, DateTimes.COMPARE_DATE_AS_STRING_MIN, DateTimes.of("3000"), 5, DateTimes.nowUtc().minus(Duration.parse("PT86400S"))) + sqlSegmentsMetadataManager.getUnusedSegmentIntervals( + DS.WIKI, + DateTimes.COMPARE_DATE_AS_STRING_MIN, + DateTimes.of("3000"), + 5, + DateTimes.nowUtc().minus(Duration.parse("PT86400S")) + ) ); // koalaSegment3 has a null used_status_last_updated which should mean getUnusedSegmentIntervals never returns it @@ -496,7 +520,7 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe sqlSegmentsMetadataManager.poll(); Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); - publisher.publishSegment(createNewSegment1(DS.KOALA)); + publishSegment(createNewSegment1(DS.KOALA)); awaitDataSourceAppeared(DS.KOALA); int numChangedSegments = sqlSegmentsMetadataManager.markAsUnusedAllSegmentsInDataSource(DS.KOALA); @@ -536,7 +560,7 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe "2017-10-15T20:19:12.565Z" ); - publisher.publishSegment(koalaSegment); + publishSegment(koalaSegment); awaitDataSourceAppeared(DS.KOALA); Assert.assertNotNull(sqlSegmentsMetadataManager.getImmutableDataSourceWithUsedSegments(DS.KOALA)); @@ -1018,8 +1042,8 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe final DataSegment koalaSegment1 = createNewSegment1(DS.KOALA); final DataSegment koalaSegment2 = createNewSegment1(DS.KOALA); - publisher.publishSegment(koalaSegment1); - publisher.publishSegment(koalaSegment2); + publishSegment(koalaSegment1); + publishSegment(koalaSegment2); final ImmutableSet segmentIds = ImmutableSet.of(koalaSegment1.getId(), koalaSegment1.getId()); @@ -1048,9 +1072,9 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe "2017-10-15T20:19:12.565Z" ); - publisher.publishSegment(koalaSegment1); - publisher.publishSegment(koalaSegment2); - publisher.publishSegment(koalaSegment3); + publishSegment(koalaSegment1); + publishSegment(koalaSegment2); + publishSegment(koalaSegment3); final Interval theInterval = Intervals.of("2017-10-15T00:00:00.000/2017-10-18T00:00:00.000"); // 2 out of 3 segments match the interval @@ -1091,9 +1115,9 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe v2 ); - publisher.publishSegment(koalaSegment1); - publisher.publishSegment(koalaSegment2); - publisher.publishSegment(koalaSegment3); + publishSegment(koalaSegment1); + publishSegment(koalaSegment2); + publishSegment(koalaSegment3); final Interval theInterval = Intervals.of("2017-10-15/2017-10-18"); Assert.assertEquals( @@ -1140,9 +1164,9 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe v2 ); - publisher.publishSegment(koalaSegment1); - publisher.publishSegment(koalaSegment2); - publisher.publishSegment(koalaSegment3); + publishSegment(koalaSegment1); + publishSegment(koalaSegment2); + publishSegment(koalaSegment3); final Interval theInterval = Intervals.of("2017-10-15/2017-10-18"); Assert.assertEquals( @@ -1189,9 +1213,9 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe v2 ); - publisher.publishSegment(koalaSegment1); - publisher.publishSegment(koalaSegment2); - publisher.publishSegment(koalaSegment3); + publishSegment(koalaSegment1); + publishSegment(koalaSegment2); + publishSegment(koalaSegment3); final Interval theInterval = Intervals.of("2017-10-15/2017-10-18"); Assert.assertEquals( @@ -1238,9 +1262,9 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe v2 ); - publisher.publishSegment(koalaSegment1); - publisher.publishSegment(koalaSegment2); - publisher.publishSegment(koalaSegment3); + publishSegment(koalaSegment1); + publishSegment(koalaSegment2); + publishSegment(koalaSegment3); final Interval theInterval = Intervals.of("2017-10-15/2017-10-18"); Assert.assertEquals( @@ -1279,9 +1303,9 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe "2017-10-15T20:19:12.565Z" ); - publisher.publishSegment(koalaSegment1); - publisher.publishSegment(koalaSegment2); - publisher.publishSegment(koalaSegment3); + publishSegment(koalaSegment1); + publishSegment(koalaSegment2); + publishSegment(koalaSegment3); final Interval theInterval = Intervals.of("2017-10-16T00:00:00.000/2017-10-20T00:00:00.000"); // 1 out of 3 segments match the interval, other 2 overlap, only the segment fully contained will be marked unused @@ -1314,7 +1338,7 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe final SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(); config.setPollDuration(Period.seconds(1)); sqlSegmentsMetadataManager = new SqlSegmentsMetadataManager( - JSON_MAPPER, + jsonMapper, Suppliers.ofInstance(config), derbyConnectorRule.metadataTablesConfigSupplier(), derbyConnectorRule.getConnector(), @@ -1335,7 +1359,7 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTe "2012-03-16T00:00:00.000/2012-03-17T00:00:00.000", "2017-10-15T20:19:12.565Z" ); - publisher.publishSegment(wikiSegment3); + publishSegment(wikiSegment3); // New segment is not returned since we call without force poll segments = sqlSegmentsMetadataManager diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTestBase.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTestBase.java index f166befde73..18112c29b4c 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTestBase.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTestBase.java @@ -22,23 +22,53 @@ package org.apache.druid.metadata; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; +import org.skife.jdbi.v2.DBI; +import org.skife.jdbi.v2.tweak.HandleCallback; + +import java.io.IOException; +import java.util.List; +import java.util.Map; public class SqlSegmentsMetadataManagerTestBase { protected SqlSegmentsMetadataManager sqlSegmentsMetadataManager; - protected SQLMetadataSegmentPublisher publisher; + protected SegmentSchemaCache segmentSchemaCache; protected SegmentSchemaManager segmentSchemaManager; protected TestDerbyConnector connector; protected SegmentsMetadataManagerConfig config; + protected MetadataStorageTablesConfig storageConfig; protected final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + protected final DataSegment segment1 = createSegment( + "wikipedia", + "2012-03-15T00:00:00.000/2012-03-16T00:00:00.000", + "2012-03-16T00:36:30.848Z", + "index/y=2012/m=03/d=15/2012-03-16T00:36:30.848Z/0/index.zip", + 0 + ); + + protected final DataSegment segment2 = createSegment( + "wikipedia", + "2012-01-05T00:00:00.000/2012-01-06T00:00:00.000", + "2012-01-06T22:19:12.565Z", + "wikipedia/index/y=2012/m=01/d=05/2012-01-06T22:19:12.565Z/0/index.zip", + 0 + ); + + protected void publishSegment(final DataSegment segment) throws IOException + { + publishSegment(connector, storageConfig, jsonMapper, segment); + } + protected static DataSegment createSegment( String dataSource, String interval, @@ -64,19 +94,86 @@ public class SqlSegmentsMetadataManagerTestBase ); } - protected final DataSegment segment1 = createSegment( - "wikipedia", - "2012-03-15T00:00:00.000/2012-03-16T00:00:00.000", - "2012-03-16T00:36:30.848Z", - "index/y=2012/m=03/d=15/2012-03-16T00:36:30.848Z/0/index.zip", - 0 - ); + public static void publishSegment( + final SQLMetadataConnector connector, + final MetadataStorageTablesConfig config, + final ObjectMapper jsonMapper, + final DataSegment segment + ) throws IOException + { + String now = DateTimes.nowUtc().toString(); + publishSegment( + connector, + config, + segment.getId().toString(), + segment.getDataSource(), + now, + segment.getInterval().getStart().toString(), + segment.getInterval().getEnd().toString(), + (segment.getShardSpec() instanceof NoneShardSpec) ? false : true, + segment.getVersion(), + true, + jsonMapper.writeValueAsBytes(segment), + now + ); + } - protected final DataSegment segment2 = createSegment( - "wikipedia", - "2012-01-05T00:00:00.000/2012-01-06T00:00:00.000", - "2012-01-06T22:19:12.565Z", - "wikipedia/index/y=2012/m=01/d=05/2012-01-06T22:19:12.565Z/0/index.zip", - 0 - ); + private static void publishSegment( + final SQLMetadataConnector connector, + final MetadataStorageTablesConfig config, + final String segmentId, + final String dataSource, + final String createdDate, + final String start, + final String end, + final boolean partitioned, + final String version, + final boolean used, + final byte[] payload, + final String usedFlagLastUpdated + ) + { + try { + final DBI dbi = connector.getDBI(); + List> exists = dbi.withHandle( + handle -> + handle.createQuery(StringUtils.format("SELECT id FROM %s WHERE id=:id", config.getSegmentsTable())) + .bind("id", segmentId) + .list() + ); + + if (!exists.isEmpty()) { + return; + } + + final String publishStatement = StringUtils.format( + "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload, used_status_last_updated) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload, :used_status_last_updated)", + config.getSegmentsTable(), + connector.getQuoteString() + ); + + dbi.withHandle( + (HandleCallback) handle -> { + handle.createStatement(publishStatement) + .bind("id", segmentId) + .bind("dataSource", dataSource) + .bind("created_date", createdDate) + .bind("start", start) + .bind("end", end) + .bind("partitioned", partitioned) + .bind("version", version) + .bind("used", used) + .bind("payload", payload) + .bind("used_status_last_updated", usedFlagLastUpdated) + .execute(); + + return null; + } + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } } diff --git a/server/src/test/java/org/apache/druid/segment/indexing/RealtimeTuningConfigTest.java b/server/src/test/java/org/apache/druid/segment/indexing/RealtimeTuningConfigTest.java deleted file mode 100644 index abc95719707..00000000000 --- a/server/src/test/java/org/apache/druid/segment/indexing/RealtimeTuningConfigTest.java +++ /dev/null @@ -1,151 +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.segment.indexing; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.data.CompressionStrategy; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; -import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.hamcrest.CoreMatchers; -import org.joda.time.Duration; -import org.joda.time.Period; -import org.junit.Assert; -import org.junit.Test; - -import java.io.File; -import java.util.UUID; - -public class RealtimeTuningConfigTest -{ - @Test - public void testErrorMessageIsMeaningfulWhenUnableToCreateTemporaryDirectory() - { - String propertyName = "java.io.tmpdir"; - String originalValue = System.getProperty(propertyName); - String nonExistedDirectory = "/tmp/" + UUID.randomUUID(); - try { - System.setProperty(propertyName, nonExistedDirectory); - RealtimeTuningConfig.makeDefaultTuningConfig(null); - } - catch (IllegalStateException e) { - Assert.assertThat( - e.getMessage(), - CoreMatchers.startsWith("java.io.tmpdir (" + nonExistedDirectory + ") does not exist") - ); - } - finally { - System.setProperty(propertyName, originalValue); - } - } - - @Test - public void testSpecificBasePersistDirectory() - { - final RealtimeTuningConfig tuningConfig = RealtimeTuningConfig.makeDefaultTuningConfig( - new File("/tmp/nonexistent") - ); - Assert.assertEquals(new File("/tmp/nonexistent"), tuningConfig.getBasePersistDirectory()); - } - - @Test - public void testSerdeWithDefaults() throws Exception - { - String jsonStr = "{\"type\":\"realtime\"}"; - - ObjectMapper mapper = TestHelper.makeJsonMapper(); - RealtimeTuningConfig config = (RealtimeTuningConfig) mapper.readValue( - mapper.writeValueAsString( - mapper.readValue( - jsonStr, - TuningConfig.class - ) - ), - TuningConfig.class - ); - - Assert.assertEquals(new OnheapIncrementalIndex.Spec(), config.getAppendableIndexSpec()); - Assert.assertEquals(Duration.standardMinutes(15).getMillis(), config.getHandoffConditionTimeout()); - Assert.assertEquals(0, config.getAlertTimeout()); - Assert.assertEquals(IndexSpec.DEFAULT, config.getIndexSpec()); - Assert.assertEquals(IndexSpec.DEFAULT, config.getIndexSpecForIntermediatePersists()); - Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); - Assert.assertEquals(new NumberedShardSpec(0, 1), config.getShardSpec()); - Assert.assertEquals(0, config.getMaxPendingPersists()); - Assert.assertEquals(150000, config.getMaxRowsInMemory()); - Assert.assertEquals(0, config.getMergeThreadPriority()); - Assert.assertEquals(0, config.getPersistThreadPriority()); - Assert.assertEquals(new Period("PT10M"), config.getWindowPeriod()); - Assert.assertFalse(config.isReportParseExceptions()); - } - - @Test - public void testSerdeWithNonDefaults() throws Exception - { - String jsonStr = "{\n" - + " \"type\": \"realtime\",\n" - + " \"maxRowsInMemory\": 100,\n" - + " \"intermediatePersistPeriod\": \"PT1H\",\n" - + " \"windowPeriod\": \"PT1H\",\n" - + " \"maxPendingPersists\": 100,\n" - + " \"persistThreadPriority\": 100,\n" - + " \"mergeThreadPriority\": 100,\n" - + " \"reportParseExceptions\": true,\n" - + " \"handoffConditionTimeout\": 100,\n" - + " \"alertTimeout\": 70,\n" - + " \"indexSpec\": { \"metricCompression\" : \"NONE\" },\n" - + " \"indexSpecForIntermediatePersists\": { \"dimensionCompression\" : \"uncompressed\" },\n" - + " \"appendableIndexSpec\": { \"type\" : \"onheap\" }\n" - + "}"; - - ObjectMapper mapper = TestHelper.makeJsonMapper(); - RealtimeTuningConfig config = (RealtimeTuningConfig) mapper.readValue( - mapper.writeValueAsString( - mapper.readValue( - jsonStr, - TuningConfig.class - ) - ), - TuningConfig.class - ); - - Assert.assertEquals(new OnheapIncrementalIndex.Spec(), config.getAppendableIndexSpec()); - Assert.assertEquals(100, config.getHandoffConditionTimeout()); - Assert.assertEquals(70, config.getAlertTimeout()); - Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); - Assert.assertEquals(new NumberedShardSpec(0, 1), config.getShardSpec()); - Assert.assertEquals(100, config.getMaxPendingPersists()); - Assert.assertEquals(100, config.getMaxRowsInMemory()); - Assert.assertEquals(100, config.getMergeThreadPriority()); - Assert.assertEquals(100, config.getPersistThreadPriority()); - Assert.assertEquals(new Period("PT1H"), config.getWindowPeriod()); - Assert.assertEquals(true, config.isReportParseExceptions()); - Assert.assertEquals( - IndexSpec.builder().withMetricCompression(CompressionStrategy.NONE).build(), - config.getIndexSpec() - ); - Assert.assertEquals( - IndexSpec.builder().withDimensionCompression(CompressionStrategy.UNCOMPRESSED).build(), - config.getIndexSpecForIntermediatePersists() - ); - - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentTest.java deleted file mode 100644 index 9c37fe7a37c..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentTest.java +++ /dev/null @@ -1,135 +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.segment.realtime; - -import com.fasterxml.jackson.databind.InjectableValues; -import com.fasterxml.jackson.databind.ObjectMapper; -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.JSONParseSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.CountAggregatorFactory; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.join.NoopJoinableFactory; -import org.apache.druid.segment.realtime.plumber.RealtimePlumberSchool; -import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; -import org.junit.Assert; -import org.junit.Test; - -import java.io.File; -import java.util.Arrays; -import java.util.Map; - -/** - */ -public class FireDepartmentTest -{ - - public static final CacheConfig NO_CACHE_CONFIG = new CacheConfig() - { - @Override - public boolean isPopulateCache() - { - return false; - } - - @Override - public boolean isUseCache() - { - return false; - } - }; - - @Test - public void testSerde() throws Exception - { - ObjectMapper jsonMapper = new DefaultObjectMapper(); - jsonMapper.setInjectableValues(new InjectableValues.Std().addValue(ObjectMapper.class, jsonMapper)); - - FireDepartment schema = new FireDepartment( - new DataSchema( - "foo", - jsonMapper.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec( - "timestamp", - "auto", - null - ), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(Arrays.asList("dim1", "dim2")) - ), - null, - null, - null - ), - null - ), - Map.class - ), - new AggregatorFactory[]{ - new CountAggregatorFactory("count") - }, - new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null), - null, - jsonMapper - ), - new RealtimeIOConfig( - null, - new RealtimePlumberSchool( - null, - null, - null, - null, - null, - null, - null, - NoopJoinableFactory.INSTANCE, - TestHelper.getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance()), - TestHelper.getTestIndexIO(), - MapCache.create(0), - NO_CACHE_CONFIG, - new CachePopulatorStats(), - TestHelper.makeJsonMapper() - - ) - ), - RealtimeTuningConfig.makeDefaultTuningConfig(new File("/tmp/nonexistent")) - ); - - String json = jsonMapper.writeValueAsString(schema); - - FireDepartment newSchema = jsonMapper.readValue(json, FireDepartment.class); - - Assert.assertEquals(schema.getDataSchema().getDataSource(), newSchema.getDataSchema().getDataSource()); - Assert.assertEquals("/tmp/nonexistent", schema.getTuningConfig().getBasePersistDirectory().toString()); - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/RealtimeMetricsMonitorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/RealtimeMetricsMonitorTest.java deleted file mode 100644 index cca07bc9708..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/RealtimeMetricsMonitorTest.java +++ /dev/null @@ -1,107 +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.segment.realtime; - -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.metrics.StubServiceEmitter; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; -import org.junit.Before; -import org.junit.Test; - -import java.util.Collections; -import java.util.Random; - -public class RealtimeMetricsMonitorTest -{ - - private StubServiceEmitter emitter; - private Random random; - - @Before - public void setup() - { - random = new Random(100); - emitter = new StubServiceEmitter("test", "localhost"); - } - - @Test - public void testDoMonitor() - { - FireDepartment fireDepartment = new FireDepartment( - new DataSchema("wiki", null, null, null, null, null, null, new DefaultObjectMapper()), - new RealtimeIOConfig(null, null), - null - ); - - // Add some metrics and invoke monitoring - final FireDepartmentMetrics metrics = fireDepartment.getMetrics(); - invokeRandomTimes(metrics::incrementThrownAway); - invokeRandomTimes(metrics::incrementUnparseable); - invokeRandomTimes(metrics::incrementProcessed); - invokeRandomTimes(metrics::incrementDedup); - invokeRandomTimes(metrics::incrementFailedHandoffs); - invokeRandomTimes(metrics::incrementFailedPersists); - invokeRandomTimes(metrics::incrementHandOffCount); - invokeRandomTimes(metrics::incrementNumPersists); - - metrics.incrementPushedRows(random.nextInt()); - metrics.incrementRowOutputCount(random.nextInt()); - metrics.incrementMergedRows(random.nextInt()); - metrics.incrementMergeCpuTime(random.nextInt()); - metrics.setSinkCount(random.nextInt()); - - RealtimeMetricsMonitor monitor = new RealtimeMetricsMonitor(Collections.singletonList(fireDepartment)); - monitor.doMonitor(emitter); - - // Verify the metrics - emitter.verifyValue("ingest/events/thrownAway", metrics.thrownAway()); - emitter.verifyValue("ingest/events/unparseable", metrics.unparseable()); - - emitter.verifyValue("ingest/events/duplicate", metrics.dedup()); - emitter.verifyValue("ingest/events/processed", metrics.processed()); - emitter.verifyValue("ingest/rows/output", metrics.rowOutput()); - emitter.verifyValue("ingest/persists/count", metrics.numPersists()); - emitter.verifyValue("ingest/persists/time", metrics.persistTimeMillis()); - emitter.verifyValue("ingest/persists/cpu", metrics.persistCpuTime()); - emitter.verifyValue("ingest/persists/backPressure", metrics.persistBackPressureMillis()); - emitter.verifyValue("ingest/persists/failed", metrics.failedPersists()); - emitter.verifyValue("ingest/handoff/failed", metrics.failedHandoffs()); - emitter.verifyValue("ingest/merge/time", metrics.mergeTimeMillis()); - emitter.verifyValue("ingest/merge/cpu", metrics.mergeCpuTime()); - emitter.verifyValue("ingest/handoff/count", metrics.handOffCount()); - emitter.verifyValue("ingest/sink/count", metrics.sinkCount()); - } - - private void invokeRandomTimes(Action action) - { - int limit = random.nextInt(20); - for (int i = 0; i < limit; ++i) { - action.perform(); - } - } - - @FunctionalInterface - private interface Action - { - void perform(); - } - -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentMetricsTest.java b/server/src/test/java/org/apache/druid/segment/realtime/SegmentGenerationMetricsTest.java similarity index 85% rename from server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentMetricsTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/SegmentGenerationMetricsTest.java index d9c1c461abe..1928c08b2ac 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentMetricsTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/SegmentGenerationMetricsTest.java @@ -23,20 +23,20 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -public class FireDepartmentMetricsTest +public class SegmentGenerationMetricsTest { - private FireDepartmentMetrics metrics; + private SegmentGenerationMetrics metrics; @Before public void setup() { - metrics = new FireDepartmentMetrics(); + metrics = new SegmentGenerationMetrics(); } @Test public void testSnapshotBeforeProcessing() { - FireDepartmentMetrics snapshot = metrics.snapshot(); + SegmentGenerationMetrics snapshot = metrics.snapshot(); Assert.assertEquals(0L, snapshot.messageGap()); // invalid value Assert.assertTrue(0 > snapshot.maxSegmentHandoffTime()); @@ -47,7 +47,7 @@ public class FireDepartmentMetricsTest { metrics.reportMessageMaxTimestamp(System.currentTimeMillis() - 20L); metrics.reportMaxSegmentHandoffTime(7L); - FireDepartmentMetrics snapshot = metrics.snapshot(); + SegmentGenerationMetrics snapshot = metrics.snapshot(); Assert.assertTrue(snapshot.messageGap() >= 20L); Assert.assertEquals(7, snapshot.maxSegmentHandoffTime()); } @@ -60,7 +60,7 @@ public class FireDepartmentMetricsTest // Should reset to invalid value metrics.snapshot(); metrics.markProcessingDone(); - FireDepartmentMetrics snapshot = metrics.snapshot(); + SegmentGenerationMetrics snapshot = metrics.snapshot(); // Message gap must be invalid after processing is done Assert.assertTrue(0 > snapshot.messageGap()); // value must be invalid diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java deleted file mode 100644 index f795492819a..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java +++ /dev/null @@ -1,139 +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.segment.realtime.appenderator; - -import org.apache.druid.data.input.InputRow; -import org.apache.druid.segment.handoff.SegmentHandoffNotifier; -import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.realtime.SegmentPublisher; -import org.apache.druid.segment.realtime.plumber.IntervalStartVersioningPolicy; -import org.apache.druid.segment.realtime.plumber.NoopRejectionPolicyFactory; -import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.easymock.EasyMock; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class AppenderatorPlumberTest -{ - private AppenderatorPlumber plumber; - private StreamAppenderatorTester streamAppenderatorTester; - - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); - - @Before - public void setUp() throws Exception - { - this.streamAppenderatorTester = - new StreamAppenderatorTester.Builder() - .maxRowsInMemory(10) - .basePersistDirectory(temporaryFolder.newFolder()) - .build(); - DataSegmentAnnouncer segmentAnnouncer = EasyMock - .createMock(DataSegmentAnnouncer.class); - segmentAnnouncer.announceSegment(EasyMock.anyObject()); - EasyMock.expectLastCall().anyTimes(); - - SegmentPublisher segmentPublisher = EasyMock - .createNiceMock(SegmentPublisher.class); - SegmentHandoffNotifierFactory handoffNotifierFactory = EasyMock - .createNiceMock(SegmentHandoffNotifierFactory.class); - SegmentHandoffNotifier handoffNotifier = EasyMock - .createNiceMock(SegmentHandoffNotifier.class); - EasyMock - .expect( - handoffNotifierFactory.createSegmentHandoffNotifier(EasyMock - .anyString())).andReturn(handoffNotifier).anyTimes(); - EasyMock - .expect( - handoffNotifier.registerSegmentHandoffCallback( - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject())).andReturn(true).anyTimes(); - - RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( - null, - 1, - null, - null, - null, - null, - temporaryFolder.newFolder(), - new IntervalStartVersioningPolicy(), - new NoopRejectionPolicyFactory(), - null, - null, - null, - null, - 0, - 0, - false, - null, - null, - null, - null, - null - ); - - this.plumber = new AppenderatorPlumber(streamAppenderatorTester.getSchema(), - tuningConfig, streamAppenderatorTester.getMetrics(), - segmentAnnouncer, segmentPublisher, handoffNotifier, - streamAppenderatorTester.getAppenderator()); - } - - @Test - public void testSimpleIngestion() throws Exception - { - Appenderator appenderator = streamAppenderatorTester.getAppenderator(); - - // startJob - Assert.assertEquals(null, plumber.startJob()); - - // getDataSource - Assert.assertEquals(StreamAppenderatorTester.DATASOURCE, appenderator.getDataSource()); - - InputRow[] rows = new InputRow[] { - StreamAppenderatorTest.ir("2000", "foo", 1), - StreamAppenderatorTest.ir("2000", "bar", 2), StreamAppenderatorTest.ir("2000", "qux", 4)}; - // add - Assert.assertEquals(1, plumber.add(rows[0], null).getRowCount()); - - Assert.assertEquals(2, plumber.add(rows[1], null).getRowCount()); - - Assert.assertEquals(3, plumber.add(rows[2], null).getRowCount()); - - - Assert.assertEquals(1, plumber.getSegmentsView().size()); - - SegmentIdWithShardSpec si = plumber.getSegmentsView().values().toArray(new SegmentIdWithShardSpec[0])[0]; - - Assert.assertEquals(3, appenderator.getRowCount(si)); - - appenderator.clear(); - Assert.assertTrue(appenderator.getSegments().isEmpty()); - - plumber.dropSegment(si); - plumber.finishJob(); - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsTest.java new file mode 100644 index 00000000000..66b2281d33c --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsTest.java @@ -0,0 +1,245 @@ +/* + * 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.realtime.appenderator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.MapInputRowParser; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +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.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMerger; +import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.incremental.ParseExceptionHandler; +import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.TuningConfig; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.loading.NoopDataSegmentPusher; +import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; +import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.Map; + + +public class AppenderatorsTest +{ + @Test + public void testOpenSegmentsOfflineAppenderator() throws Exception + { + try (final AppenderatorTester tester = new AppenderatorTester("OPEN_SEGMENTS")) { + Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl); + AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator; + Assert.assertTrue(appenderator.isOpenSegments()); + } + } + + @Test + public void testClosedSegmentsOfflineAppenderator() throws Exception + { + try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS")) { + Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl); + AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator; + Assert.assertFalse(appenderator.isOpenSegments()); + } + } + + @Test + public void testClosedSegmentsSinksOfflineAppenderator() throws Exception + { + try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS_SINKS")) { + Assert.assertTrue(tester.appenderator instanceof BatchAppenderator); + } + } + + private static class AppenderatorTester implements AutoCloseable + { + public static final String DATASOURCE = "foo"; + + private final AppenderatorConfig tuningConfig; + private final Appenderator appenderator; + private final ServiceEmitter emitter; + + public AppenderatorTester(final String batchMode) + { + this(100, 100, null, new SimpleRowIngestionMeters(), false, batchMode); + } + + public AppenderatorTester( + final int maxRowsInMemory, + final long maxSizeInBytes, + @Nullable final File basePersistDirectory, + final RowIngestionMeters rowIngestionMeters, + final boolean skipBytesInMemoryOverheadCheck, + String batchMode + ) + { + ObjectMapper objectMapper = new DefaultObjectMapper(); + objectMapper.registerSubtypes(LinearShardSpec.class); + + final Map parserMap = objectMapper.convertValue( + new MapInputRowParser( + new JSONParseSpec( + new TimestampSpec("ts", "auto", null), + DimensionsSpec.EMPTY, + null, + null, + null + ) + ), + Map.class + ); + + DataSchema schema = new DataSchema( + DATASOURCE, + null, + null, + new AggregatorFactory[]{ + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("met", "met") + }, + new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), + null, + parserMap, + objectMapper + ); + + tuningConfig = new TestAppenderatorConfig( + TuningConfig.DEFAULT_APPENDABLE_INDEX, + maxRowsInMemory, + maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, + skipBytesInMemoryOverheadCheck, + IndexSpec.DEFAULT, + 0, + false, + 0L, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), + IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE, + basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory + ); + SegmentGenerationMetrics metrics = new SegmentGenerationMetrics(); + + IndexIO indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT); + IndexMergerV9 indexMerger = new IndexMergerV9( + objectMapper, + indexIO, + OffHeapMemorySegmentWriteOutMediumFactory.instance() + ); + + emitter = new ServiceEmitter( + "test", + "test", + new NoopEmitter() + ); + emitter.start(); + EmittingLogger.registerEmitter(emitter); + + switch (batchMode) { + case "OPEN_SEGMENTS": + appenderator = Appenderators.createOpenSegmentsOffline( + schema.getDataSource(), + schema, + tuningConfig, + metrics, + new NoopDataSegmentPusher(), + objectMapper, + indexIO, + indexMerger, + rowIngestionMeters, + new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), + false, + CentralizedDatasourceSchemaConfig.create() + ); + break; + case "CLOSED_SEGMENTS": + appenderator = Appenderators.createClosedSegmentsOffline( + schema.getDataSource(), + schema, + tuningConfig, + metrics, + new NoopDataSegmentPusher(), + objectMapper, + indexIO, + indexMerger, + rowIngestionMeters, + new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), + false, + CentralizedDatasourceSchemaConfig.create() + ); + + break; + case "CLOSED_SEGMENTS_SINKS": + appenderator = Appenderators.createOffline( + schema.getDataSource(), + schema, + tuningConfig, + metrics, + new NoopDataSegmentPusher(), + objectMapper, + indexIO, + indexMerger, + rowIngestionMeters, + new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), + false, + CentralizedDatasourceSchemaConfig.create() + ); + break; + default: + throw new IllegalArgumentException("Unrecognized batchMode: " + batchMode); + } + } + + private long getDefaultMaxBytesInMemory() + { + return (Runtime.getRuntime().totalMemory()) / 3; + } + + @Override + public void close() throws Exception + { + appenderator.close(); + emitter.close(); + FileUtils.deleteDirectory(tuningConfig.getBasePersistDirectory()); + } + + private static File createNewBasePersistDirectory() + { + return FileUtils.createTempDir("druid-batch-persist"); + } + } +} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java index f5ae98dd6a0..cf2d7f79898 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java @@ -24,7 +24,6 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.MapInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.granularity.Granularities; @@ -39,7 +38,6 @@ import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.column.ColumnConfig; -import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; @@ -48,12 +46,10 @@ import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; -import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; -import org.joda.time.Period; import javax.annotation.Nullable; import java.io.File; @@ -61,7 +57,6 @@ import java.io.IOException; import java.net.URI; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.concurrent.CopyOnWriteArrayList; public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable @@ -70,7 +65,7 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable private final DataSchema schema; private final AppenderatorConfig tuningConfig; - private final FireDepartmentMetrics metrics; + private final SegmentGenerationMetrics metrics; private final ObjectMapper objectMapper; private final Appenderator appenderator; private final ServiceEmitter emitter; @@ -170,7 +165,7 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable objectMapper ); - tuningConfig = new TestIndexTuningConfig( + tuningConfig = new TestAppenderatorConfig( TuningConfig.DEFAULT_APPENDABLE_INDEX, maxRowsInMemory, maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, @@ -181,10 +176,9 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable 0L, OffHeapMemorySegmentWriteOutMediumFactory.instance(), IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE, - basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory, - null + basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory ); - metrics = new FireDepartmentMetrics(); + metrics = new SegmentGenerationMetrics(); IndexIO indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT); IndexMergerV9 indexMerger = new IndexMergerV9( @@ -267,7 +261,7 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable return tuningConfig; } - public FireDepartmentMetrics getMetrics() + public SegmentGenerationMetrics getMetrics() { return metrics; } @@ -299,216 +293,4 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable { return FileUtils.createTempDir("druid-batch-persist"); } - - - static class TestIndexTuningConfig implements AppenderatorConfig - { - private final AppendableIndexSpec appendableIndexSpec; - private final int maxRowsInMemory; - private final long maxBytesInMemory; - private final boolean skipBytesInMemoryOverheadCheck; - private final int maxColumnsToMerge; - private final PartitionsSpec partitionsSpec; - private final IndexSpec indexSpec; - private final File basePersistDirectory; - private final int maxPendingPersists; - private final boolean reportParseExceptions; - private final long pushTimeout; - private final IndexSpec indexSpecForIntermediatePersists; - @Nullable - private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; - private final int numPersistThreads; - - public TestIndexTuningConfig( - AppendableIndexSpec appendableIndexSpec, - Integer maxRowsInMemory, - Long maxBytesInMemory, - Boolean skipBytesInMemoryOverheadCheck, - IndexSpec indexSpec, - Integer maxPendingPersists, - Boolean reportParseExceptions, - Long pushTimeout, - @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, - Integer maxColumnsToMerge, - File basePersistDirectory, - @Nullable Integer numPersistThreads - ) - { - this.appendableIndexSpec = appendableIndexSpec; - this.maxRowsInMemory = maxRowsInMemory; - this.maxBytesInMemory = maxBytesInMemory; - this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck; - this.indexSpec = indexSpec; - this.maxPendingPersists = maxPendingPersists; - this.reportParseExceptions = reportParseExceptions; - this.pushTimeout = pushTimeout; - this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; - this.maxColumnsToMerge = maxColumnsToMerge; - this.basePersistDirectory = basePersistDirectory; - - this.partitionsSpec = null; - this.indexSpecForIntermediatePersists = this.indexSpec; - this.numPersistThreads = numPersistThreads == null ? DEFAULT_NUM_PERSIST_THREADS : numPersistThreads; - } - - @Override - public TestIndexTuningConfig withBasePersistDirectory(File dir) - { - throw new UnsupportedOperationException(); - } - - @Override - public AppendableIndexSpec getAppendableIndexSpec() - { - return appendableIndexSpec; - } - - @Override - public int getMaxRowsInMemory() - { - return maxRowsInMemory; - } - - @Override - public long getMaxBytesInMemory() - { - return maxBytesInMemory; - } - - @Override - public boolean isSkipBytesInMemoryOverheadCheck() - { - return skipBytesInMemoryOverheadCheck; - } - - @Nullable - @Override - public PartitionsSpec getPartitionsSpec() - { - return partitionsSpec; - } - - @Override - public IndexSpec getIndexSpec() - { - return indexSpec; - } - - @Override - public IndexSpec getIndexSpecForIntermediatePersists() - { - return indexSpecForIntermediatePersists; - } - - @Override - public int getMaxPendingPersists() - { - return maxPendingPersists; - } - - @Override - public boolean isReportParseExceptions() - { - return reportParseExceptions; - } - - @Nullable - @Override - public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() - { - return segmentWriteOutMediumFactory; - } - - @Override - public int getMaxColumnsToMerge() - { - return maxColumnsToMerge; - } - - @Override - public File getBasePersistDirectory() - { - return basePersistDirectory; - } - - @Override - public Period getIntermediatePersistPeriod() - { - return new Period(Integer.MAX_VALUE); // intermediate persist doesn't make much sense for batch jobs - } - - @Override - public int getNumPersistThreads() - { - return numPersistThreads; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - TestIndexTuningConfig that = (TestIndexTuningConfig) o; - return Objects.equals(appendableIndexSpec, that.appendableIndexSpec) && - maxRowsInMemory == that.maxRowsInMemory && - maxBytesInMemory == that.maxBytesInMemory && - skipBytesInMemoryOverheadCheck == that.skipBytesInMemoryOverheadCheck && - maxColumnsToMerge == that.maxColumnsToMerge && - maxPendingPersists == that.maxPendingPersists && - reportParseExceptions == that.reportParseExceptions && - pushTimeout == that.pushTimeout && - numPersistThreads == that.numPersistThreads && - Objects.equals(partitionsSpec, that.partitionsSpec) && - Objects.equals(indexSpec, that.indexSpec) && - Objects.equals(indexSpecForIntermediatePersists, that.indexSpecForIntermediatePersists) && - Objects.equals(basePersistDirectory, that.basePersistDirectory) && - Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory); - } - - @Override - public int hashCode() - { - return Objects.hash( - appendableIndexSpec, - maxRowsInMemory, - maxBytesInMemory, - skipBytesInMemoryOverheadCheck, - maxColumnsToMerge, - partitionsSpec, - indexSpec, - indexSpecForIntermediatePersists, - basePersistDirectory, - maxPendingPersists, - reportParseExceptions, - pushTimeout, - segmentWriteOutMediumFactory, - numPersistThreads - ); - } - - @Override - public String toString() - { - return "IndexTuningConfig{" + - "maxRowsInMemory=" + maxRowsInMemory + - ", maxBytesInMemory=" + maxBytesInMemory + - ", skipBytesInMemoryOverheadCheck=" + skipBytesInMemoryOverheadCheck + - ", maxColumnsToMerge=" + maxColumnsToMerge + - ", partitionsSpec=" + partitionsSpec + - ", indexSpec=" + indexSpec + - ", indexSpecForIntermediatePersists=" + indexSpecForIntermediatePersists + - ", basePersistDirectory=" + basePersistDirectory + - ", maxPendingPersists=" + maxPendingPersists + - ", reportParseExceptions=" + reportParseExceptions + - ", pushTimeout=" + pushTimeout + - ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + - ", numPersistThreads=" + numPersistThreads + - '}'; - } - } - } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java deleted file mode 100644 index 97578426812..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java +++ /dev/null @@ -1,179 +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.segment.realtime.appenderator; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; -import com.google.inject.Binder; -import com.google.inject.Injector; -import com.google.inject.Module; -import com.google.inject.name.Names; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.GuiceInjectors; -import org.apache.druid.initialization.Initialization; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.DruidProcessingConfig; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.CountAggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.segment.column.ColumnConfig; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.timeline.partition.LinearShardSpec; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.IOException; -import java.util.Map; - -public class DefaultOfflineAppenderatorFactoryTest -{ - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); - - @Test - public void testBuild() throws IOException, SegmentNotWritableException - { - Injector injector = Initialization.makeInjectorWithModules( - GuiceInjectors.makeStartupInjector(), - ImmutableList.of( - new Module() - { - @Override - public void configure(Binder binder) - { - binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/tool"); - binder.bindConstant().annotatedWith(Names.named("servicePort")).to(9999); - binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); - binder.bind(DruidProcessingConfig.class).toInstance( - new DruidProcessingConfig() - { - @Override - public String getFormatString() - { - return "processing-%s"; - } - - @Override - public int intermediateComputeSizeBytes() - { - return 100 * 1024 * 1024; - } - - @Override - public int getNumThreads() - { - return 1; - } - - } - ); - binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class); - } - } - ) - ); - ObjectMapper objectMapper = injector.getInstance(ObjectMapper.class); - AppenderatorFactory defaultOfflineAppenderatorFactory = objectMapper.readerFor(AppenderatorFactory.class) - .readValue("{\"type\":\"offline\"}"); - - final Map parserMap = objectMapper.convertValue( - new MapInputRowParser( - new JSONParseSpec( - new TimestampSpec("ts", "auto", null), - DimensionsSpec.EMPTY, - null, - null, - null - ) - ), - Map.class - ); - DataSchema schema = new DataSchema( - "dataSourceName", - parserMap, - new AggregatorFactory[]{ - new CountAggregatorFactory("count"), - new LongSumAggregatorFactory("met", "met") - }, - new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), - null, - objectMapper - ); - - RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( - null, - 75000, - null, - null, - null, - null, - temporaryFolder.newFolder(), - null, - null, - null, - null, - null, - null, - 0, - 0, - null, - null, - null, - null, - null, - null - ); - - Appenderator appenderator = defaultOfflineAppenderatorFactory.build( - schema, - tuningConfig, - new FireDepartmentMetrics() - ); - try { - Assert.assertEquals("dataSourceName", appenderator.getDataSource()); - Assert.assertEquals(null, appenderator.startJob()); - SegmentIdWithShardSpec identifier = new SegmentIdWithShardSpec( - "dataSourceName", - Intervals.of("2000/2001"), - "A", - new LinearShardSpec(0) - ); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); - appenderator.add(identifier, StreamAppenderatorTest.ir("2000", "bar", 1), null); - Assert.assertEquals(1, ((AppenderatorImpl) appenderator).getRowsInMemory()); - appenderator.add(identifier, StreamAppenderatorTest.ir("2000", "baz", 1), null); - Assert.assertEquals(2, ((AppenderatorImpl) appenderator).getRowsInMemory()); - appenderator.close(); - Assert.assertEquals(0, ((AppenderatorImpl) appenderator).getRowsInMemory()); - } - finally { - appenderator.close(); - } - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java index 0d210f21a2c..33a0ed2f8a4 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java @@ -46,7 +46,7 @@ import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; @@ -63,7 +63,7 @@ public class OpenAndClosedSegmentsAppenderatorTester implements AutoCloseable public static final String DATASOURCE = "foo"; private final DataSchema schema; - private final FireDepartmentMetrics metrics; + private final SegmentGenerationMetrics metrics; private final DataSegmentPusher dataSegmentPusher; private final ObjectMapper objectMapper; private final Appenderator appenderator; @@ -139,23 +139,21 @@ public class OpenAndClosedSegmentsAppenderatorTester implements AutoCloseable null, objectMapper ); - tuningConfig = - new ClosedSegmensSinksBatchAppenderatorTester.TestIndexTuningConfig( - TuningConfig.DEFAULT_APPENDABLE_INDEX, - maxRowsInMemory, - maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, - skipBytesInMemoryOverheadCheck, - IndexSpec.DEFAULT, - 0, - false, - 0L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE, - basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory, - null - ); + tuningConfig = new TestAppenderatorConfig( + TuningConfig.DEFAULT_APPENDABLE_INDEX, + maxRowsInMemory, + maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, + skipBytesInMemoryOverheadCheck, + IndexSpec.DEFAULT, + 0, + false, + 0L, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), + IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE, + basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory + ); - metrics = new FireDepartmentMetrics(); + metrics = new SegmentGenerationMetrics(); indexIO = new IndexIO( objectMapper, @@ -251,7 +249,7 @@ public class OpenAndClosedSegmentsAppenderatorTester implements AutoCloseable return tuningConfig; } - public FireDepartmentMetrics getMetrics() + public SegmentGenerationMetrics getMetrics() { return metrics; } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java index e3ed8fd49d7..bdc6f64c90a 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java @@ -41,7 +41,7 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.loading.DataSegmentKiller; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriverTest.TestCommitterSupplier; import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriverTest.TestSegmentAllocator; import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriverTest.TestSegmentHandoffNotifierFactory; @@ -136,7 +136,7 @@ public class StreamAppenderatorDriverFailTest extends EasyMockSupport new NoopUsedSegmentChecker(), dataSegmentKiller, OBJECT_MAPPER, - new FireDepartmentMetrics() + new SegmentGenerationMetrics() ); driver.startJob(null); @@ -174,7 +174,7 @@ public class StreamAppenderatorDriverFailTest extends EasyMockSupport new NoopUsedSegmentChecker(), dataSegmentKiller, OBJECT_MAPPER, - new FireDepartmentMetrics() + new SegmentGenerationMetrics() ); driver.startJob(null); @@ -212,7 +212,7 @@ public class StreamAppenderatorDriverFailTest extends EasyMockSupport new NoopUsedSegmentChecker(), dataSegmentKiller, OBJECT_MAPPER, - new FireDepartmentMetrics() + new SegmentGenerationMetrics() ); driver.startJob(null); @@ -263,7 +263,7 @@ public class StreamAppenderatorDriverFailTest extends EasyMockSupport new NoopUsedSegmentChecker(), dataSegmentKiller, OBJECT_MAPPER, - new FireDepartmentMetrics() + new SegmentGenerationMetrics() ); driver.startJob(null); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java index 63775e2dc3b..eae1148027d 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java @@ -41,7 +41,7 @@ import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.loading.DataSegmentKiller; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.NumberedShardSpec; @@ -130,7 +130,7 @@ public class StreamAppenderatorDriverTest extends EasyMockSupport new TestUsedSegmentChecker(streamAppenderatorTester.getPushedSegments()), dataSegmentKiller, OBJECT_MAPPER, - new FireDepartmentMetrics() + new SegmentGenerationMetrics() ); EasyMock.replay(dataSegmentKiller); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java index 4e057c5c897..538784a8853 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java @@ -48,9 +48,8 @@ import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.plumber.Committers; +import org.apache.druid.segment.realtime.sink.Committers; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.DataSegment; @@ -909,7 +908,7 @@ public class StreamAppenderatorTest extends InitializedNullHandlingTest @Test public void testRestoreFromDisk() throws Exception { - final RealtimeTuningConfig tuningConfig; + final AppenderatorConfig tuningConfig; try ( final StreamAppenderatorTester tester = new StreamAppenderatorTester.Builder().maxRowsInMemory(2) diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java index da4a76f1a10..2a39718667c 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java @@ -53,18 +53,20 @@ 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.segment.IndexIO; +import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; +import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer; @@ -84,8 +86,8 @@ public class StreamAppenderatorTester implements AutoCloseable public static final String DATASOURCE = "foo"; private final DataSchema schema; - private final RealtimeTuningConfig tuningConfig; - private final FireDepartmentMetrics metrics; + private final AppenderatorConfig tuningConfig; + private final SegmentGenerationMetrics metrics; private final DataSegmentPusher dataSegmentPusher; private final ObjectMapper objectMapper; private final Appenderator appenderator; @@ -132,31 +134,21 @@ public class StreamAppenderatorTester implements AutoCloseable null, objectMapper ); - tuningConfig = new RealtimeTuningConfig( - null, - maxRowsInMemory, - maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, - skipBytesInMemoryOverheadCheck, - null, - null, - basePersistDirectory, - null, - null, - null, - null, - null, - null, - 0, - 0, - null, - null, - null, - null, - null, - null - ); + tuningConfig = new TestAppenderatorConfig( + TuningConfig.DEFAULT_APPENDABLE_INDEX, + maxRowsInMemory, + maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, + skipBytesInMemoryOverheadCheck, + IndexSpec.DEFAULT, + 0, + false, + 0L, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), + IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE, + basePersistDirectory + ); - metrics = new FireDepartmentMetrics(); + metrics = new SegmentGenerationMetrics(); queryExecutor = Execs.singleThreaded("queryExecutor(%d)"); IndexIO indexIO = new IndexIO( @@ -310,12 +302,12 @@ public class StreamAppenderatorTester implements AutoCloseable return schema; } - public RealtimeTuningConfig getTuningConfig() + public AppenderatorConfig getTuningConfig() { return tuningConfig; } - public FireDepartmentMetrics getMetrics() + public SegmentGenerationMetrics getMetrics() { return metrics; } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestAppenderatorConfig.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestAppenderatorConfig.java new file mode 100644 index 00000000000..5dba99de500 --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestAppenderatorConfig.java @@ -0,0 +1,228 @@ +/* + * 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.realtime.appenderator; + +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.joda.time.Period; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.Objects; + +public class TestAppenderatorConfig implements AppenderatorConfig +{ + private final AppendableIndexSpec appendableIndexSpec; + private final int maxRowsInMemory; + private final long maxBytesInMemory; + private final boolean skipBytesInMemoryOverheadCheck; + private final int maxColumnsToMerge; + private final PartitionsSpec partitionsSpec; + private final IndexSpec indexSpec; + private final File basePersistDirectory; + private final int maxPendingPersists; + private final boolean reportParseExceptions; + private final long pushTimeout; + private final IndexSpec indexSpecForIntermediatePersists; + @Nullable + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; + + public TestAppenderatorConfig( + AppendableIndexSpec appendableIndexSpec, + Integer maxRowsInMemory, + Long maxBytesInMemory, + Boolean skipBytesInMemoryOverheadCheck, + IndexSpec indexSpec, + Integer maxPendingPersists, + Boolean reportParseExceptions, + Long pushTimeout, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + Integer maxColumnsToMerge, + File basePersistDirectory + ) + { + this.appendableIndexSpec = appendableIndexSpec; + this.maxRowsInMemory = maxRowsInMemory; + this.maxBytesInMemory = maxBytesInMemory; + this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck; + this.indexSpec = indexSpec; + this.maxPendingPersists = maxPendingPersists; + this.reportParseExceptions = reportParseExceptions; + this.pushTimeout = pushTimeout; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + this.maxColumnsToMerge = maxColumnsToMerge; + this.basePersistDirectory = basePersistDirectory; + + this.partitionsSpec = null; + this.indexSpecForIntermediatePersists = this.indexSpec; + } + + @Override + public TestAppenderatorConfig withBasePersistDirectory(File dir) + { + throw new UnsupportedOperationException(); + } + + @Override + public AppendableIndexSpec getAppendableIndexSpec() + { + return appendableIndexSpec; + } + + @Override + public int getMaxRowsInMemory() + { + return maxRowsInMemory; + } + + @Override + public long getMaxBytesInMemory() + { + return maxBytesInMemory; + } + + @Override + public boolean isSkipBytesInMemoryOverheadCheck() + { + return skipBytesInMemoryOverheadCheck; + } + + @Nullable + @Override + public PartitionsSpec getPartitionsSpec() + { + return partitionsSpec; + } + + @Override + public IndexSpec getIndexSpec() + { + return indexSpec; + } + + @Override + public IndexSpec getIndexSpecForIntermediatePersists() + { + return indexSpecForIntermediatePersists; + } + + @Override + public int getMaxPendingPersists() + { + return maxPendingPersists; + } + + @Override + public boolean isReportParseExceptions() + { + return reportParseExceptions; + } + + @Nullable + @Override + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() + { + return segmentWriteOutMediumFactory; + } + + @Override + public int getMaxColumnsToMerge() + { + return maxColumnsToMerge; + } + + @Override + public File getBasePersistDirectory() + { + return basePersistDirectory; + } + + @Override + public Period getIntermediatePersistPeriod() + { + return new Period(Integer.MAX_VALUE); // intermediate persist doesn't make much sense for batch jobs + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TestAppenderatorConfig that = (TestAppenderatorConfig) o; + return Objects.equals(appendableIndexSpec, that.appendableIndexSpec) && + maxRowsInMemory == that.maxRowsInMemory && + maxBytesInMemory == that.maxBytesInMemory && + skipBytesInMemoryOverheadCheck == that.skipBytesInMemoryOverheadCheck && + maxColumnsToMerge == that.maxColumnsToMerge && + maxPendingPersists == that.maxPendingPersists && + reportParseExceptions == that.reportParseExceptions && + pushTimeout == that.pushTimeout && + Objects.equals(partitionsSpec, that.partitionsSpec) && + Objects.equals(indexSpec, that.indexSpec) && + Objects.equals(indexSpecForIntermediatePersists, that.indexSpecForIntermediatePersists) && + Objects.equals(basePersistDirectory, that.basePersistDirectory) && + Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory); + } + + @Override + public int hashCode() + { + return Objects.hash( + appendableIndexSpec, + maxRowsInMemory, + maxBytesInMemory, + skipBytesInMemoryOverheadCheck, + maxColumnsToMerge, + partitionsSpec, + indexSpec, + indexSpecForIntermediatePersists, + basePersistDirectory, + maxPendingPersists, + reportParseExceptions, + pushTimeout, + segmentWriteOutMediumFactory + ); + } + + @Override + public String toString() + { + return "TestAppenderatorConfig{" + + "maxRowsInMemory=" + maxRowsInMemory + + ", maxBytesInMemory=" + maxBytesInMemory + + ", skipBytesInMemoryOverheadCheck=" + skipBytesInMemoryOverheadCheck + + ", maxColumnsToMerge=" + maxColumnsToMerge + + ", partitionsSpec=" + partitionsSpec + + ", indexSpec=" + indexSpec + + ", indexSpecForIntermediatePersists=" + indexSpecForIntermediatePersists + + ", basePersistDirectory=" + basePersistDirectory + + ", maxPendingPersists=" + maxPendingPersists + + ", reportParseExceptions=" + reportParseExceptions + + ", pushTimeout=" + pushTimeout + + ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + + '}'; + } +} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java index ce77db1d7cf..9bf629931b5 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java @@ -49,7 +49,7 @@ import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.JoinableFactoryWrapperTest; import org.apache.druid.segment.loading.NoopDataSegmentPusher; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -107,7 +107,7 @@ public class UnifiedIndexerAppenderatorsManagerTest extends InitializedNullHandl null ), appenderatorConfig, - new FireDepartmentMetrics(), + new SegmentGenerationMetrics(), new NoopDataSegmentPusher(), TestHelper.makeJsonMapper(), TestHelper.getTestIndexIO(), diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/CustomVersioningPolicyTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/CustomVersioningPolicyTest.java deleted file mode 100644 index f145b0e1d79..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/CustomVersioningPolicyTest.java +++ /dev/null @@ -1,50 +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.segment.realtime.plumber; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.segment.TestHelper; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.joda.time.Interval; -import org.junit.Assert; -import org.junit.Test; - -public class CustomVersioningPolicyTest -{ - - @Test - public void testSerialization() throws Exception - { - Interval interval = new Interval(DateTime.now(DateTimeZone.UTC), DateTime.now(DateTimeZone.UTC)); - String version = "someversion"; - - CustomVersioningPolicy policy = new CustomVersioningPolicy(version); - - final ObjectMapper mapper = TestHelper.makeJsonMapper(); - CustomVersioningPolicy serialized = mapper.readValue( - mapper.writeValueAsBytes(policy), - CustomVersioningPolicy.class - ); - - Assert.assertEquals(version, policy.getVersion(interval)); - Assert.assertEquals(version, serialized.getVersion(interval)); - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/IntervalStartVersioningPolicyTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/IntervalStartVersioningPolicyTest.java deleted file mode 100644 index 15c05fd9c56..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/IntervalStartVersioningPolicyTest.java +++ /dev/null @@ -1,37 +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.segment.realtime.plumber; - -import org.apache.druid.java.util.common.Intervals; -import org.junit.Assert; -import org.junit.Test; - -/** - */ -public class IntervalStartVersioningPolicyTest -{ - @Test - public void testGetVersion() - { - IntervalStartVersioningPolicy policy = new IntervalStartVersioningPolicy(); - String version = policy.getVersion(Intervals.of("2013-01-01/2013-01-02")); - Assert.assertEquals("2013-01-01T00:00:00.000Z", version); - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/MessageTimeRejectionPolicyFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/MessageTimeRejectionPolicyFactoryTest.java deleted file mode 100644 index 36e73dc60bc..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/MessageTimeRejectionPolicyFactoryTest.java +++ /dev/null @@ -1,47 +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.segment.realtime.plumber; - -import org.apache.druid.java.util.common.DateTimes; -import org.joda.time.DateTime; -import org.joda.time.Period; -import org.junit.Assert; -import org.junit.Test; - -/** - */ -public class MessageTimeRejectionPolicyFactoryTest -{ - @Test - public void testAccept() - { - Period period = new Period("PT10M"); - RejectionPolicy rejectionPolicy = new MessageTimeRejectionPolicyFactory().create(period); - - DateTime now = DateTimes.nowUtc(); - DateTime past = now.minus(period).minus(1); - DateTime future = now.plus(period).plus(1); - - Assert.assertTrue(rejectionPolicy.accept(now.getMillis())); - Assert.assertFalse(rejectionPolicy.accept(past.getMillis())); - Assert.assertTrue(rejectionPolicy.accept(future.getMillis())); - Assert.assertFalse(rejectionPolicy.accept(now.getMillis())); - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java deleted file mode 100644 index 74a1217ed27..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ /dev/null @@ -1,714 +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.segment.realtime.plumber; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Supplier; -import com.google.common.base.Suppliers; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import org.apache.druid.client.cache.CachePopulatorStats; -import org.apache.druid.client.cache.MapCache; -import org.apache.druid.data.input.Committer; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.Row; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; -import org.apache.druid.query.DirectQueryProcessingPool; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.CountAggregatorFactory; -import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.ReferenceCountingSegment; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.handoff.SegmentHandoffNotifier; -import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.join.NoopJoinableFactory; -import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.segment.realtime.FireDepartmentTest; -import org.apache.druid.segment.realtime.FireHydrant; -import org.apache.druid.segment.realtime.SegmentPublisher; -import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; -import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; -import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; -import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.apache.druid.testing.InitializedNullHandlingTest; -import org.easymock.EasyMock; -import org.joda.time.DateTime; -import org.joda.time.Interval; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.io.File; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; - -/** - * - */ -@RunWith(Parameterized.class) -public class RealtimePlumberSchoolTest extends InitializedNullHandlingTest -{ - @Parameterized.Parameters(name = "rejectionPolicy = {0}, segmentWriteOutMediumFactory = {1}") - public static Collection constructorFeeder() - { - final RejectionPolicyFactory[] rejectionPolicies = new RejectionPolicyFactory[]{ - new NoopRejectionPolicyFactory(), - new MessageTimeRejectionPolicyFactory() - }; - - final List constructors = new ArrayList<>(); - for (RejectionPolicyFactory rejectionPolicy : rejectionPolicies) { - constructors.add(new Object[]{rejectionPolicy, OffHeapMemorySegmentWriteOutMediumFactory.instance()}); - constructors.add(new Object[]{rejectionPolicy, TmpFileSegmentWriteOutMediumFactory.instance()}); - } - return constructors; - } - - private final RejectionPolicyFactory rejectionPolicy; - private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; - private RealtimePlumber plumber; - private RealtimePlumberSchool realtimePlumberSchool; - private DataSegmentAnnouncer announcer; - private SegmentPublisher segmentPublisher; - private DataSegmentPusher dataSegmentPusher; - private SegmentHandoffNotifier handoffNotifier; - private SegmentHandoffNotifierFactory handoffNotifierFactory; - private ServiceEmitter emitter; - private RealtimeTuningConfig tuningConfig; - private DataSchema schema; - private DataSchema schema2; - private FireDepartmentMetrics metrics; - private File tmpDir; - - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); - - public RealtimePlumberSchoolTest( - RejectionPolicyFactory rejectionPolicy, - SegmentWriteOutMediumFactory segmentWriteOutMediumFactory - ) - { - this.rejectionPolicy = rejectionPolicy; - this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; - } - - @Before - public void setUp() throws Exception - { - tmpDir = FileUtils.createTempDir(); - - ObjectMapper jsonMapper = new DefaultObjectMapper(); - - schema = new DataSchema( - "test", - jsonMapper.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec("timestamp", "auto", null), - DimensionsSpec.EMPTY, - null, - null, - null - ), - null - ), - Map.class - ), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null), - null, - jsonMapper - ); - - schema2 = new DataSchema( - "test", - jsonMapper.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec("timestamp", "auto", null), - DimensionsSpec.EMPTY, - null, - null, - null - ), - null - ), - Map.class - ), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularities.YEAR, Granularities.NONE, null), - null, - jsonMapper - ); - - announcer = EasyMock.createMock(DataSegmentAnnouncer.class); - announcer.announceSegment(EasyMock.anyObject()); - EasyMock.expectLastCall().anyTimes(); - - segmentPublisher = EasyMock.createNiceMock(SegmentPublisher.class); - dataSegmentPusher = EasyMock.createNiceMock(DataSegmentPusher.class); - handoffNotifierFactory = EasyMock.createNiceMock(SegmentHandoffNotifierFactory.class); - handoffNotifier = EasyMock.createNiceMock(SegmentHandoffNotifier.class); - EasyMock.expect(handoffNotifierFactory.createSegmentHandoffNotifier(EasyMock.anyString())) - .andReturn(handoffNotifier) - .anyTimes(); - EasyMock.expect( - handoffNotifier.registerSegmentHandoffCallback( - EasyMock.anyObject(), - EasyMock.anyObject(), - EasyMock.anyObject() - ) - ).andReturn(true).anyTimes(); - - emitter = EasyMock.createMock(ServiceEmitter.class); - - EasyMock.replay(announcer, segmentPublisher, dataSegmentPusher, handoffNotifierFactory, handoffNotifier, emitter); - - tuningConfig = new RealtimeTuningConfig( - null, - 1, - null, - null, - null, - null, - temporaryFolder.newFolder(), - new IntervalStartVersioningPolicy(), - rejectionPolicy, - null, - null, - null, - null, - 0, - 0, - false, - null, - null, - null, - null, - null - ); - - realtimePlumberSchool = new RealtimePlumberSchool( - emitter, - new DefaultQueryRunnerFactoryConglomerate(new HashMap<>()), - dataSegmentPusher, - announcer, - segmentPublisher, - handoffNotifierFactory, - DirectQueryProcessingPool.INSTANCE, - NoopJoinableFactory.INSTANCE, - TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory), - TestHelper.getTestIndexIO(), - MapCache.create(0), - FireDepartmentTest.NO_CACHE_CONFIG, - new CachePopulatorStats(), - TestHelper.makeJsonMapper() - ); - - metrics = new FireDepartmentMetrics(); - plumber = (RealtimePlumber) realtimePlumberSchool.findPlumber(schema, tuningConfig, metrics); - } - - @After - public void tearDown() throws Exception - { - EasyMock.verify(announcer, segmentPublisher, dataSegmentPusher, handoffNotifierFactory, handoffNotifier, emitter); - FileUtils.deleteDirectory( - new File( - tuningConfig.getBasePersistDirectory(), - schema.getDataSource() - ) - ); - FileUtils.deleteDirectory(tmpDir); - } - - @Test(timeout = 60_000L) - public void testPersist() throws Exception - { - testPersist(null); - } - - @Test(timeout = 60_000L) - public void testPersistWithCommitMetadata() throws Exception - { - final Object commitMetadata = "dummyCommitMetadata"; - testPersist(commitMetadata); - - plumber = (RealtimePlumber) realtimePlumberSchool.findPlumber(schema, tuningConfig, metrics); - Assert.assertEquals(commitMetadata, plumber.startJob()); - } - - private void testPersist(final Object commitMetadata) throws Exception - { - Sink sink = new Sink( - Intervals.utc(0, TimeUnit.HOURS.toMillis(1)), - schema, - tuningConfig.getShardSpec(), - DateTimes.of("2014-12-01T12:34:56.789").toString(), - tuningConfig.getAppendableIndexSpec(), - tuningConfig.getMaxRowsInMemory(), - tuningConfig.getMaxBytesInMemoryOrDefault(), - true, - tuningConfig.getDedupColumn() - ); - plumber.getSinks().put(0L, sink); - Assert.assertNull(plumber.startJob()); - - final InputRow row = EasyMock.createNiceMock(InputRow.class); - EasyMock.expect(row.getTimestampFromEpoch()).andReturn(0L); - EasyMock.expect(row.getDimensions()).andReturn(new ArrayList()); - EasyMock.replay(row); - - final CountDownLatch doneSignal = new CountDownLatch(1); - - final Committer committer = new Committer() - { - @Override - public Object getMetadata() - { - return commitMetadata; - } - - @Override - public void run() - { - doneSignal.countDown(); - } - }; - plumber.add(row, Suppliers.ofInstance(committer)); - plumber.persist(committer); - - doneSignal.await(); - - plumber.getSinks().clear(); - plumber.finishJob(); - } - - @Test(timeout = 60_000L) - public void testPersistFails() throws Exception - { - Sink sink = new Sink( - Intervals.utc(0, TimeUnit.HOURS.toMillis(1)), - schema, - tuningConfig.getShardSpec(), - DateTimes.of("2014-12-01T12:34:56.789").toString(), - tuningConfig.getAppendableIndexSpec(), - tuningConfig.getMaxRowsInMemory(), - tuningConfig.getMaxBytesInMemoryOrDefault(), - true, - tuningConfig.getDedupColumn() - ); - plumber.getSinks().put(0L, sink); - plumber.startJob(); - final InputRow row = EasyMock.createNiceMock(InputRow.class); - EasyMock.expect(row.getTimestampFromEpoch()).andReturn(0L); - EasyMock.expect(row.getDimensions()).andReturn(new ArrayList()); - EasyMock.replay(row); - plumber.add(row, Suppliers.ofInstance(Committers.nil())); - - final CountDownLatch doneSignal = new CountDownLatch(1); - - plumber.persist( - supplierFromRunnable( - () -> { - doneSignal.countDown(); - throw new RuntimeException(); - } - ).get() - ); - - doneSignal.await(); - - // Exception may need time to propagate - while (metrics.failedPersists() < 1) { - Thread.sleep(100); - } - - Assert.assertEquals(1, metrics.failedPersists()); - } - - @Test(timeout = 60_000L) - public void testPersistHydrantGaps() throws Exception - { - final Object commitMetadata = "dummyCommitMetadata"; - testPersistHydrantGapsHelper(commitMetadata); - } - - private void testPersistHydrantGapsHelper(final Object commitMetadata) throws Exception - { - Interval testInterval = new Interval(DateTimes.of("1970-01-01"), DateTimes.of("1971-01-01")); - - RealtimePlumber plumber2 = (RealtimePlumber) realtimePlumberSchool.findPlumber(schema2, tuningConfig, metrics); - Sink sink = new Sink( - testInterval, - schema2, - tuningConfig.getShardSpec(), - DateTimes.of("2014-12-01T12:34:56.789").toString(), - tuningConfig.getAppendableIndexSpec(), - tuningConfig.getMaxRowsInMemory(), - tuningConfig.getMaxBytesInMemoryOrDefault(), - true, - tuningConfig.getDedupColumn() - ); - plumber2.getSinks().put(0L, sink); - Assert.assertNull(plumber2.startJob()); - final CountDownLatch doneSignal = new CountDownLatch(1); - final Committer committer = new Committer() - { - @Override - public Object getMetadata() - { - return commitMetadata; - } - - @Override - public void run() - { - doneSignal.countDown(); - } - }; - plumber2.add(getTestInputRow("1970-01-01"), Suppliers.ofInstance(committer)); - plumber2.add(getTestInputRow("1970-02-01"), Suppliers.ofInstance(committer)); - plumber2.add(getTestInputRow("1970-03-01"), Suppliers.ofInstance(committer)); - plumber2.add(getTestInputRow("1970-04-01"), Suppliers.ofInstance(committer)); - plumber2.add(getTestInputRow("1970-05-01"), Suppliers.ofInstance(committer)); - - plumber2.persist(committer); - - doneSignal.await(); - plumber2.getSinks().clear(); - plumber2.finishJob(); - - File persistDir = plumber2.computePersistDir(schema2, testInterval); - - /* Check that all hydrants were persisted */ - for (int i = 0; i < 5; i++) { - Assert.assertTrue(new File(persistDir, String.valueOf(i)).exists()); - } - - /* Create some gaps in the persisted hydrants and reload */ - FileUtils.deleteDirectory(new File(persistDir, "1")); - FileUtils.deleteDirectory(new File(persistDir, "3")); - RealtimePlumber restoredPlumber = (RealtimePlumber) realtimePlumberSchool.findPlumber( - schema2, - tuningConfig, - metrics - ); - restoredPlumber.bootstrapSinksFromDisk(); - - Map sinks = restoredPlumber.getSinks(); - Assert.assertEquals(1, sinks.size()); - - - List hydrants = Lists.newArrayList(sinks.get(new Long(0))); - DateTime startTime = DateTimes.of("1970-01-01T00:00:00.000Z"); - Interval expectedInterval = new Interval(startTime, DateTimes.of("1971-01-01T00:00:00.000Z")); - Assert.assertEquals(0, hydrants.get(0).getCount()); - Assert.assertEquals( - expectedInterval, - hydrants.get(0).getSegmentDataInterval() - ); - Assert.assertEquals(2, hydrants.get(1).getCount()); - Assert.assertEquals( - expectedInterval, - hydrants.get(1).getSegmentDataInterval() - ); - Assert.assertEquals(4, hydrants.get(2).getCount()); - Assert.assertEquals( - expectedInterval, - hydrants.get(2).getSegmentDataInterval() - ); - - /* Delete all the hydrants and reload, no sink should be created */ - FileUtils.deleteDirectory(new File(persistDir, "0")); - FileUtils.deleteDirectory(new File(persistDir, "2")); - FileUtils.deleteDirectory(new File(persistDir, "4")); - RealtimePlumber restoredPlumber2 = (RealtimePlumber) realtimePlumberSchool.findPlumber( - schema2, - tuningConfig, - metrics - ); - restoredPlumber2.bootstrapSinksFromDisk(); - - Assert.assertEquals(0, restoredPlumber2.getSinks().size()); - } - - @Test(timeout = 60_000L) - public void testDimOrderInheritance() throws Exception - { - final Object commitMetadata = "dummyCommitMetadata"; - testDimOrderInheritanceHelper(commitMetadata); - } - - private void testDimOrderInheritanceHelper(final Object commitMetadata) throws Exception - { - List> expectedDims = ImmutableList.of( - ImmutableList.of("dimD"), - ImmutableList.of("dimC"), - ImmutableList.of("dimA"), - ImmutableList.of("dimB"), - ImmutableList.of("dimE"), - ImmutableList.of("dimD", "dimC", "dimA", "dimB", "dimE") - ); - - QueryableIndex qindex; - FireHydrant hydrant; - Map sinks; - - RealtimePlumber plumber = (RealtimePlumber) realtimePlumberSchool.findPlumber(schema2, tuningConfig, metrics); - Assert.assertNull(plumber.startJob()); - - final CountDownLatch doneSignal = new CountDownLatch(1); - - final Committer committer = new Committer() - { - @Override - public Object getMetadata() - { - return commitMetadata; - } - - @Override - public void run() - { - doneSignal.countDown(); - } - }; - - plumber.add( - getTestInputRowFull( - "1970-01-01", - ImmutableList.of("dimD"), - ImmutableList.of("1") - ), - Suppliers.ofInstance(committer) - ); - plumber.add( - getTestInputRowFull( - "1970-01-01", - ImmutableList.of("dimC"), - ImmutableList.of("1") - ), - Suppliers.ofInstance(committer) - ); - plumber.add( - getTestInputRowFull( - "1970-01-01", - ImmutableList.of("dimA"), - ImmutableList.of("1") - ), - Suppliers.ofInstance(committer) - ); - plumber.add( - getTestInputRowFull( - "1970-01-01", - ImmutableList.of("dimB"), - ImmutableList.of("1") - ), - Suppliers.ofInstance(committer) - ); - plumber.add( - getTestInputRowFull( - "1970-01-01", - ImmutableList.of("dimE"), - ImmutableList.of("1") - ), - Suppliers.ofInstance(committer) - ); - plumber.add( - getTestInputRowFull( - "1970-01-01", - ImmutableList.of("dimA", "dimB", "dimC", "dimD", "dimE"), - ImmutableList.of("1") - ), - Suppliers.ofInstance(committer) - ); - - plumber.persist(committer); - - doneSignal.await(); - - plumber.getSinks().clear(); - plumber.finishJob(); - - RealtimePlumber restoredPlumber = (RealtimePlumber) realtimePlumberSchool.findPlumber( - schema2, - tuningConfig, - metrics - ); - restoredPlumber.bootstrapSinksFromDisk(); - - sinks = restoredPlumber.getSinks(); - Assert.assertEquals(1, sinks.size()); - List hydrants = Lists.newArrayList(sinks.get(0L)); - - for (int i = 0; i < hydrants.size(); i++) { - hydrant = hydrants.get(i); - ReferenceCountingSegment segment = hydrant.getIncrementedSegment(); - try { - qindex = segment.asQueryableIndex(); - Assert.assertEquals(i, hydrant.getCount()); - Assert.assertEquals(expectedDims.get(i), ImmutableList.copyOf(qindex.getAvailableDimensions())); - } - finally { - segment.decrement(); - } - } - } - - private InputRow getTestInputRow(final String timeStr) - { - return new InputRow() - { - @Override - public List getDimensions() - { - return new ArrayList<>(); - } - - @Override - public long getTimestampFromEpoch() - { - return DateTimes.of(timeStr).getMillis(); - } - - @Override - public DateTime getTimestamp() - { - return DateTimes.of(timeStr); - } - - @Override - public List getDimension(String dimension) - { - return new ArrayList<>(); - } - - @Override - public Number getMetric(String metric) - { - return 0; - } - - @Override - public Object getRaw(String dimension) - { - return null; - } - - @Override - public int compareTo(Row o) - { - return 0; - } - }; - } - - private InputRow getTestInputRowFull(final String timeStr, final List dims, final List dimVals) - { - return new InputRow() - { - @Override - public List getDimensions() - { - return dims; - } - - @Override - public long getTimestampFromEpoch() - { - return DateTimes.of(timeStr).getMillis(); - } - - @Override - public DateTime getTimestamp() - { - return DateTimes.of(timeStr); - } - - @Override - public List getDimension(String dimension) - { - return dimVals; - } - - @Override - public Number getMetric(String metric) - { - return 0; - } - - @Override - public Object getRaw(String dimension) - { - return dimVals; - } - - @Override - public int compareTo(Row o) - { - return 0; - } - }; - } - - private static Supplier supplierFromRunnable(final Runnable runnable) - { - final Committer committer = new Committer() - { - @Override - public Object getMetadata() - { - return null; - } - - @Override - public void run() - { - runnable.run(); - } - }; - return Suppliers.ofInstance(committer); - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/ServerTimeRejectionPolicyFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/ServerTimeRejectionPolicyFactoryTest.java deleted file mode 100644 index 496788e030e..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/ServerTimeRejectionPolicyFactoryTest.java +++ /dev/null @@ -1,47 +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.segment.realtime.plumber; - -import org.apache.druid.java.util.common.DateTimes; -import org.joda.time.DateTime; -import org.joda.time.Period; -import org.junit.Assert; -import org.junit.Test; - -/** - */ -public class ServerTimeRejectionPolicyFactoryTest -{ - @Test - public void testAccept() - { - Period period = new Period("PT10M"); - - RejectionPolicy rejectionPolicy = new ServerTimeRejectionPolicyFactory().create(period); - - DateTime now = DateTimes.nowUtc(); - DateTime past = now.minus(period).minus(100); - DateTime future = now.plus(period).plus(100); - - Assert.assertTrue(rejectionPolicy.accept(now.getMillis())); - Assert.assertFalse(rejectionPolicy.accept(past.getMillis())); - Assert.assertFalse(rejectionPolicy.accept(future.getMillis())); - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java similarity index 74% rename from server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java index 0d3c9c23bff..9d85ec6c8e6 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.realtime.plumber; +package org.apache.druid.segment.realtime.sink; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -43,16 +43,17 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; +import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.FireHydrant; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.utils.CloseableUtils; import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Interval; -import org.joda.time.Period; import org.junit.Assert; import org.junit.Test; @@ -69,6 +70,9 @@ import java.util.function.Function; */ public class SinkTest extends InitializedNullHandlingTest { + private static final ShardSpec SHARD_SPEC = new NumberedShardSpec(0, 1); + private static final int MAX_ROWS_IN_MEMORY = 100; + @Test public void testSwap() throws Exception { @@ -83,39 +87,15 @@ public class SinkTest extends InitializedNullHandlingTest final Interval interval = Intervals.of("2013-01-01/2013-01-02"); final String version = DateTimes.nowUtc().toString(); - RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( - null, - 100, - null, - null, - new Period("P1Y"), - null, - null, - null, - null, - null, - null, - null, - null, - 0, - 0, - null, - null, - null, - null, - null, - null - ); final Sink sink = new Sink( interval, schema, - tuningConfig.getShardSpec(), + SHARD_SPEC, version, - tuningConfig.getAppendableIndexSpec(), - tuningConfig.getMaxRowsInMemory(), - tuningConfig.getMaxBytesInMemoryOrDefault(), - true, - tuningConfig.getDedupColumn() + TuningConfig.DEFAULT_APPENDABLE_INDEX, + MAX_ROWS_IN_MEMORY, + TuningConfig.DEFAULT_APPENDABLE_INDEX.getDefaultMaxBytesInMemory(), + true ); sink.add( @@ -227,93 +207,6 @@ public class SinkTest extends InitializedNullHandlingTest Assert.assertEquals(2, Iterators.size(sink.iterator())); } - @Test - public void testDedup() throws Exception - { - final DataSchema schema = new DataSchema( - "test", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null), - null - ); - - final Interval interval = Intervals.of("2013-01-01/2013-01-02"); - final String version = DateTimes.nowUtc().toString(); - RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( - null, - 100, - null, - null, - new Period("P1Y"), - null, - null, - null, - null, - null, - null, - null, - null, - 0, - 0, - null, - null, - null, - null, - "dedupColumn", - null - ); - final Sink sink = new Sink( - interval, - schema, - tuningConfig.getShardSpec(), - version, - tuningConfig.getAppendableIndexSpec(), - tuningConfig.getMaxRowsInMemory(), - tuningConfig.getMaxBytesInMemoryOrDefault(), - true, - tuningConfig.getDedupColumn() - ); - - int rows = sink.add(new MapBasedInputRow( - DateTimes.of("2013-01-01"), - ImmutableList.of("field", "dedupColumn"), - ImmutableMap.of("field1", "value1", "dedupColumn", "v1") - ), false).getRowCount(); - Assert.assertTrue(rows > 0); - - // dedupColumn is null - rows = sink.add(new MapBasedInputRow( - DateTimes.of("2013-01-01"), - ImmutableList.of("field", "dedupColumn"), - ImmutableMap.of("field1", "value2") - ), false).getRowCount(); - Assert.assertTrue(rows > 0); - - // dedupColumn is null - rows = sink.add(new MapBasedInputRow( - DateTimes.of("2013-01-01"), - ImmutableList.of("field", "dedupColumn"), - ImmutableMap.of("field1", "value3") - ), false).getRowCount(); - Assert.assertTrue(rows > 0); - - rows = sink.add(new MapBasedInputRow( - DateTimes.of("2013-01-01"), - ImmutableList.of("field", "dedupColumn"), - ImmutableMap.of("field1", "value4", "dedupColumn", "v2") - ), false).getRowCount(); - Assert.assertTrue(rows > 0); - - rows = sink.add(new MapBasedInputRow( - DateTimes.of("2013-01-01"), - ImmutableList.of("field", "dedupColumn"), - ImmutableMap.of("field1", "value5", "dedupColumn", "v1") - ), false).getRowCount(); - Assert.assertTrue(rows == -2); - } - @Test public void testAcquireSegmentReferences_empty() { @@ -378,39 +271,15 @@ public class SinkTest extends InitializedNullHandlingTest final Interval interval = Intervals.of("2013-01-01/2013-01-02"); final String version = DateTimes.nowUtc().toString(); - RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( - null, - 2, - null, - null, - new Period("P1Y"), - null, - null, - null, - null, - null, - null, - null, - null, - 0, - 0, - null, - null, - null, - null, - "dedupColumn", - null - ); final Sink sink = new Sink( interval, schema, - tuningConfig.getShardSpec(), + SHARD_SPEC, version, - tuningConfig.getAppendableIndexSpec(), - tuningConfig.getMaxRowsInMemory(), - tuningConfig.getMaxBytesInMemoryOrDefault(), - true, - tuningConfig.getDedupColumn() + TuningConfig.DEFAULT_APPENDABLE_INDEX, + MAX_ROWS_IN_MEMORY, + TuningConfig.DEFAULT_APPENDABLE_INDEX.getDefaultMaxBytesInMemory(), + true ); sink.add(new MapBasedInputRow( diff --git a/server/src/test/java/org/apache/druid/server/QueryStackTests.java b/server/src/test/java/org/apache/druid/server/QueryStackTests.java index 7d3f58ef600..3df1d95b33a 100644 --- a/server/src/test/java/org/apache/druid/server/QueryStackTests.java +++ b/server/src/test/java/org/apache/druid/server/QueryStackTests.java @@ -98,7 +98,6 @@ import org.apache.druid.utils.JvmUtils; import org.junit.Assert; import javax.annotation.Nullable; - import java.util.Collections; import java.util.Map; import java.util.Set; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java index 2455610ceac..9d0f752869e 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java @@ -36,9 +36,11 @@ import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.metadata.SQLMetadataSegmentPublisher; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.SQLMetadataConnector; import org.apache.druid.metadata.SegmentsMetadataManagerConfig; import org.apache.druid.metadata.SqlSegmentsMetadataManager; +import org.apache.druid.metadata.SqlSegmentsMetadataManagerTestBase; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; @@ -98,12 +100,13 @@ public class KillUnusedSegmentsTest @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); private SqlSegmentsMetadataManager sqlSegmentsMetadataManager; - private SQLMetadataSegmentPublisher publisher; + private SQLMetadataConnector connector; + private MetadataStorageTablesConfig config; @Before public void setup() { - final TestDerbyConnector connector = derbyConnectorRule.getConnector(); + connector = derbyConnectorRule.getConnector(); SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(); config.setPollDuration(Period.millis(1)); sqlSegmentsMetadataManager = new SqlSegmentsMetadataManager( @@ -116,11 +119,7 @@ public class KillUnusedSegmentsTest ); sqlSegmentsMetadataManager.start(); - publisher = new SQLMetadataSegmentPublisher( - TestHelper.makeJsonMapper(), - derbyConnectorRule.metadataTablesConfigSupplier().get(), - connector - ); + this.config = derbyConnectorRule.metadataTablesConfigSupplier().get(); connector.createSegmentTable(); overlordClient = new TestOverlordClient(); @@ -733,7 +732,7 @@ public class KillUnusedSegmentsTest { final DataSegment segment = createSegment(dataSource, interval, version); try { - publisher.publishSegment(segment); + SqlSegmentsMetadataManagerTestBase.publishSegment(connector, config, TestHelper.makeJsonMapper(), segment); } catch (IOException e) { throw new RuntimeException(e); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index 5bcec39a1e8..a7d534db700 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -106,7 +106,6 @@ import org.joda.time.Interval; import javax.annotation.Nonnull; import javax.annotation.Nullable; - import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidRel.java index 798b0bb407b..8db108fbb94 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidRel.java @@ -31,7 +31,6 @@ import org.apache.druid.server.QueryResponse; import org.apache.druid.sql.calcite.planner.PlannerContext; import javax.annotation.Nullable; - import java.util.Set; public abstract class DruidRel> extends AbstractRelNode diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 606710ff53b..a2e45c4af8c 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -110,7 +110,6 @@ import org.junit.jupiter.api.Named; import org.junit.jupiter.api.extension.RegisterExtension; import javax.annotation.Nullable; - import java.io.File; import java.io.IOException; import java.io.InputStream; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java index 621c919b0ed..514695c4afe 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java @@ -56,7 +56,6 @@ import org.apache.druid.sql.calcite.util.CalciteTests; import org.junit.jupiter.api.Test; import javax.annotation.Nullable; - import java.util.HashMap; import java.util.Map; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteIngestionDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteIngestionDmlTest.java index 74cd5a07b26..e26803fa498 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteIngestionDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteIngestionDmlTest.java @@ -76,7 +76,6 @@ import org.junit.jupiter.api.AfterEach; import javax.annotation.Nonnull; import javax.annotation.Nullable; - import java.io.File; import java.util.ArrayList; import java.util.Arrays; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteJoinQueryTest.java index d8abfbb2075..19553104a83 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteJoinQueryTest.java @@ -24,6 +24,7 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; + import java.util.Map; import static org.junit.Assert.assertNotNull; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java index 9bd57e10afb..f23197d437a 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java @@ -22,6 +22,7 @@ package org.apache.druid.sql.calcite; import org.apache.calcite.rel.rules.CoreRules; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; + import java.lang.annotation.ElementType; import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java index 5ef280e24b8..ec661f8d315 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java @@ -72,7 +72,6 @@ import org.junit.jupiter.api.extension.RegisterExtension; import javax.annotation.Nonnull; import javax.annotation.Nullable; - import java.io.File; import java.io.IOException; import java.io.InputStream; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestBuilder.java b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestBuilder.java index 9484fd3d892..0c3ee685732 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestBuilder.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestBuilder.java @@ -38,6 +38,7 @@ import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.calcite.util.SqlTestFramework; import org.apache.druid.sql.calcite.util.SqlTestFramework.PlannerFixture; import org.apache.druid.sql.http.SqlParameter; + import java.util.ArrayList; import java.util.Collections; import java.util.List; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfig.java b/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfig.java index a3311d32c87..9eda5849fcf 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfig.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfig.java @@ -41,7 +41,6 @@ import org.junit.jupiter.api.extension.ExtensionContext; import org.reflections.Reflections; import javax.annotation.Nonnull; - import java.io.Closeable; import java.lang.annotation.Annotation; import java.lang.annotation.ElementType; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java index 8ef3ad3106f..43d47733277 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java @@ -60,7 +60,6 @@ import org.junit.jupiter.api.extension.ExtendWith; import javax.validation.Validation; import javax.validation.Validator; - import java.util.Collections; import java.util.Optional; import java.util.Set; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index c2321d52250..992d91dabe8 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -117,6 +117,7 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; + import java.io.File; import java.io.IOException; import java.net.URI; diff --git a/website/.spelling b/website/.spelling index 31b151a921d..5825f8ddb2f 100644 --- a/website/.spelling +++ b/website/.spelling @@ -1665,7 +1665,6 @@ Filesystem JVMMonitor jvmVersion QueryCountStatsMonitor -RealtimeMetricsMonitor Sys SysMonitor TaskCountStatsMonitor