diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java index b23af62f6309..6518dcc368dc 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java @@ -31,6 +31,7 @@ import org.apache.druid.indexer.HadoopIOConfig; import org.apache.druid.indexer.HadoopIngestionSpec; import org.apache.druid.indexer.HadoopTuningConfig; +import org.apache.druid.indexer.granularity.ArbitraryGranularitySpec; import org.apache.druid.indexer.hadoop.DatasourceIngestionSpec; import org.apache.druid.indexing.common.task.HadoopIndexTask; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -46,7 +47,6 @@ import org.apache.druid.metadata.SqlSegmentsMetadataManager; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.realtime.ChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java index b2ac927b17e1..d0a46e25db27 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java @@ -28,6 +28,7 @@ import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; @@ -49,7 +50,6 @@ 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.granularity.UniformGranularitySpec; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.easymock.EasyMockSupport; import org.joda.time.Period; 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 a0eb4e942aa3..b26ecd5d4d69 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 @@ -55,6 +55,7 @@ import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.report.IngestionStatsAndErrors; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TestUtils; @@ -109,7 +110,6 @@ import org.apache.druid.segment.incremental.RowIngestionMetersTotals; import org.apache.druid.segment.incremental.RowMeters; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.security.Action; diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java index 9cdc0ac0edcd..9ed5f068e0bc 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java @@ -35,6 +35,7 @@ import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.data.input.kafkainput.KafkaInputFormat; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; import org.apache.druid.indexing.kafka.test.TestBroker; @@ -50,7 +51,6 @@ import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; 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 1151bec626c7..f330c97f0ea9 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 @@ -37,6 +37,7 @@ import org.apache.druid.data.input.kafka.KafkaTopicPartition; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.task.NoopTask; @@ -87,7 +88,6 @@ 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.granularity.UniformGranularitySpec; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java index 02ce3c1a09c4..c85fdbc92f7b 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java @@ -37,6 +37,7 @@ import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.data.input.kinesis.KinesisRecordEntity; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorIOConfig; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec; import org.apache.druid.indexing.overlord.sampler.InputSourceSampler; @@ -51,7 +52,6 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; 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 4f8b3720a24a..193ad3a34828 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 @@ -34,6 +34,7 @@ import org.apache.druid.data.input.kinesis.KinesisRecordEntity; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.task.NoopTask; @@ -82,7 +83,6 @@ 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.granularity.UniformGranularitySpec; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java index 161e80569521..a86099047eb4 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java @@ -27,6 +27,7 @@ import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.NoopInputFormat; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.NoopTask; @@ -37,7 +38,6 @@ 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.granularity.UniformGranularitySpec; import java.io.File; import java.util.Collections; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 228f184f0ea3..cb021813c5d9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -34,7 +34,6 @@ import it.unimi.dsi.fastutil.ints.IntArraySet; import it.unimi.dsi.fastutil.ints.IntList; import it.unimi.dsi.fastutil.ints.IntSet; -import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.data.input.StringTuple; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -54,6 +53,8 @@ import org.apache.druid.frame.write.InvalidFieldException; import org.apache.druid.frame.write.InvalidNullByteException; import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.granularity.GranularitySpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; @@ -180,9 +181,8 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.DruidNode; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; @@ -1695,12 +1695,10 @@ private static Function, Set> addCompactionStateTo ); DimensionsSpec dimensionsSpec = dataSchema.getDimensionsSpec(); - Map transformSpec = TransformSpec.NONE.equals(dataSchema.getTransformSpec()) - ? null - : new ClientCompactionTaskTransformSpec( - dataSchema.getTransformSpec().getFilter() - ).asMap(jsonMapper); - List metricsSpec = Collections.emptyList(); + CompactionTransformSpec transformSpec = TransformSpec.NONE.equals(dataSchema.getTransformSpec()) + ? null + : CompactionTransformSpec.of(dataSchema.getTransformSpec()); + List metricsSpec = Collections.emptyList(); if (querySpec.getQuery() instanceof GroupByQuery) { // For group-by queries, the aggregators are transformed to their combining factories in the dataschema, resulting @@ -1730,8 +1728,8 @@ private static Function, Set> addCompactionStateTo dimensionsSpec, metricsSpec, transformSpec, - indexSpec.asMap(jsonMapper), - granularitySpec.asMap(jsonMapper) + indexSpec, + granularitySpec ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationUtils.java index b37a29f53e20..f33a92a4bd09 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationUtils.java @@ -29,6 +29,8 @@ import org.apache.druid.error.DruidException; import org.apache.druid.frame.key.ClusterBy; import org.apache.druid.frame.key.KeyColumn; +import org.apache.druid.indexer.granularity.ArbitraryGranularitySpec; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; @@ -51,8 +53,6 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.calcite.rel.DruidQuery; import org.apache.druid.utils.CollectionUtils; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index bf97432d3f8e..941b67a3a8a3 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -32,6 +32,8 @@ import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; +import org.apache.druid.indexer.granularity.GranularitySpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; @@ -52,11 +54,10 @@ import org.apache.druid.msq.test.MSQTestTaskActionClient; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.QueryContext; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; @@ -2780,15 +2781,15 @@ private CompactionState expectedCompactionState( false, Collections.singletonList(interval) ); - List metricsSpec = Collections.emptyList(); + List metricsSpec = Collections.emptyList(); return new CompactionState( partitionsSpec, dimensionsSpec, metricsSpec, null, - indexSpec.asMap(objectMapper), - granularitySpec.asMap(objectMapper) + indexSpec, + granularitySpec ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index 6df649403c2d..1bf0da2a1488 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -26,13 +26,13 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; -import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; @@ -69,7 +69,7 @@ import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.indexing.CombinedDataSchema; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; @@ -100,7 +100,11 @@ public class MSQCompactionRunnerTest private static final GranularityType QUERY_GRANULARITY = GranularityType.HOUR; private static final StringDimensionSchema STRING_DIMENSION = new StringDimensionSchema("string_dim", null, false); - private static final StringDimensionSchema MV_STRING_DIMENSION = new StringDimensionSchema("mv_string_dim", null, null); + private static final StringDimensionSchema MV_STRING_DIMENSION = new StringDimensionSchema( + "mv_string_dim", + null, + null + ); private static final LongDimensionSchema LONG_DIMENSION = new LongDimensionSchema("long_dim"); private static final NestedDataColumnSchema NESTED_DIMENSION = new NestedDataColumnSchema("nested_dim", 5); private static final AutoTypeColumnSchema AUTO_DIMENSION = new AutoTypeColumnSchema("auto_dim", null); @@ -127,7 +131,11 @@ public class MSQCompactionRunnerTest private static final AggregatorFactory AGG1 = new CountAggregatorFactory("agg_0"); private static final AggregatorFactory AGG2 = new LongSumAggregatorFactory("sum_added", "sum_added"); private static final List AGGREGATORS = ImmutableList.of(AGG1, AGG2); - private static final MSQCompactionRunner MSQ_COMPACTION_RUNNER = new MSQCompactionRunner(JSON_MAPPER, TestExprMacroTable.INSTANCE, null); + private static final MSQCompactionRunner MSQ_COMPACTION_RUNNER = new MSQCompactionRunner( + JSON_MAPPER, + TestExprMacroTable.INSTANCE, + null + ); private static final List PARTITION_DIMENSIONS = Collections.singletonList(STRING_DIMENSION.getName()); @Test @@ -191,8 +199,9 @@ public void testLongDimensionInRangePartitionsSpecIsInvalid() null ); - CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, - INTERVAL_DATASCHEMAS + CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask( + compactionTask, + INTERVAL_DATASCHEMAS ); Assert.assertFalse(validationResult.isValid()); Assert.assertEquals( @@ -213,8 +222,9 @@ public void testMultiValuedDimensionInRangePartitionsSpecIsInvalid() null ); - CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, - INTERVAL_DATASCHEMAS + CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask( + compactionTask, + INTERVAL_DATASCHEMAS ); Assert.assertFalse(validationResult.isValid()); Assert.assertEquals( @@ -542,8 +552,8 @@ private CompactionTask createCompactionTask( @Nullable AggregatorFactory[] metricsSpec ) { - ClientCompactionTaskTransformSpec transformSpec = - new ClientCompactionTaskTransformSpec(dimFilter); + CompactionTransformSpec transformSpec = + new CompactionTransformSpec(dimFilter); final CompactionTask.Builder builder = new CompactionTask.Builder( DATA_SOURCE, null diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DetermineHashedPartitionsJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DetermineHashedPartitionsJob.java index 2328552efb7b..18b13630b44e 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DetermineHashedPartitionsJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DetermineHashedPartitionsJob.java @@ -30,6 +30,7 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.Rows; import org.apache.druid.hll.HyperLogLogCollector; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.java.util.common.DateTimes; @@ -37,7 +38,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashPartitionFunction; import org.apache.hadoop.conf.Configurable; diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java index bb830afd244f..0cabc734617f 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java @@ -38,6 +38,7 @@ import org.apache.druid.guice.GuiceInjectors; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.annotations.Self; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.DimensionBasedPartitionsSpec; import org.apache.druid.indexer.path.PathSpec; import org.apache.druid.initialization.Initialization; @@ -51,7 +52,6 @@ import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.server.DruidNode; import org.apache.druid.timeline.DataSegment; diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerMapper.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerMapper.java index 788b41d7e779..4f8e7c8b1383 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerMapper.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerMapper.java @@ -23,6 +23,7 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.RE; @@ -30,7 +31,6 @@ import org.apache.druid.java.util.common.collect.Utils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.ParseException; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Counter; import org.apache.hadoop.mapreduce.Mapper; diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/GranularUnprocessedPathSpec.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/GranularUnprocessedPathSpec.java index 3891851d9ed7..f462c486ddfd 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/GranularUnprocessedPathSpec.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/GranularUnprocessedPathSpec.java @@ -24,12 +24,12 @@ import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import org.apache.druid.indexer.HadoopDruidIndexerConfig; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.hadoop.FSSpideringIterator; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Comparators; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java index 41dae9c9e65e..65192d0a1308 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java @@ -32,6 +32,7 @@ import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.hll.HyperLogLogCollector; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.hadoop.DatasourceRecordReader; import org.apache.druid.indexer.hadoop.WindowedDataSegment; import org.apache.druid.jackson.DefaultObjectMapper; @@ -46,7 +47,6 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.realtime.WindowedCursorFactory; import org.apache.druid.segment.transform.TransformSpec; diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java index dd22a95083ce..fcdda375bc2a 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java @@ -25,6 +25,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.FileUtils; @@ -34,7 +35,6 @@ import org.apache.druid.java.util.common.granularity.PeriodGranularity; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashPartitionFunction; import org.joda.time.Interval; diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java index bfd28d2cfca0..3e604a7ef259 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java @@ -25,13 +25,13 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; 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.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; import org.junit.After; import org.junit.Assert; diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineRangePartitionsJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineRangePartitionsJobTest.java index 3ff525c8b433..881fc3a38330 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineRangePartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineRangePartitionsJobTest.java @@ -25,13 +25,13 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; 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.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.timeline.partition.DimensionRangeShardSpec; import org.junit.After; import org.junit.Assert; diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java index ed47d180b432..3178e21d4cca 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -28,6 +28,7 @@ import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.annotations.Self; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionBasedPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; @@ -38,7 +39,6 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashPartitionFunction; import org.apache.druid.timeline.partition.NoneShardSpec; diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java index da57b8ccf4a3..31c8b587d767 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java @@ -26,6 +26,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.path.StaticPathSpec; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.jackson.JacksonUtils; @@ -35,7 +36,6 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.hadoop.conf.Configuration; diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecTest.java index f2775184311f..693990da0873 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecTest.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Strings; import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; @@ -33,7 +34,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.PeriodGranularity; import org.apache.druid.metadata.MetadataStorageConnectorConfig; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.joda.time.Period; import org.junit.Assert; import org.junit.Test; diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java index afcfb4023595..ec0bb1a3071f 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.hadoop.DatasourceIngestionSpec; import org.apache.druid.indexer.hadoop.WindowedDataSegment; import org.apache.druid.indexer.path.DatasourcePathSpec; @@ -35,7 +36,6 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; import org.apache.druid.timeline.partition.NoneShardSpec; diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java index 37cdbb7300d7..820867134b63 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java @@ -29,6 +29,7 @@ import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimeAndDimsParseSpec; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -37,7 +38,6 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.mapreduce.Reducer; diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java index a53850704e65..33f56f50400e 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java @@ -31,6 +31,7 @@ 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.indexer.granularity.UniformGranularitySpec; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; @@ -40,7 +41,6 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashPartitionFunction; diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java index d7825aa01478..6559cd5faeef 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java @@ -27,6 +27,7 @@ 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.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.path.StaticPathSpec; import org.apache.druid.java.util.common.CompressionUtilsTest; import org.apache.druid.java.util.common.ISE; @@ -38,7 +39,6 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.hadoop.conf.Configuration; diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java index 75a4fe45eee6..51c18c832f52 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java @@ -37,6 +37,7 @@ import org.apache.druid.indexer.HadoopIOConfig; import org.apache.druid.indexer.HadoopIngestionSpec; import org.apache.druid.indexer.HadoopTuningConfig; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.hadoop.DatasourceIngestionSpec; import org.apache.druid.indexer.hadoop.DatasourceInputFormat; import org.apache.druid.indexer.hadoop.WindowedDataSegment; @@ -46,7 +47,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.DruidNode; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/GranularityPathSpecTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/GranularityPathSpecTest.java index 92bd8595560c..0d06d83f32dd 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/GranularityPathSpecTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/GranularityPathSpecTest.java @@ -28,6 +28,7 @@ import org.apache.druid.indexer.HadoopIOConfig; import org.apache.druid.indexer.HadoopIngestionSpec; import org.apache.druid.indexer.HadoopTuningConfig; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -35,7 +36,6 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.PeriodGranularity; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.security.UserGroupInformation; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index 185e6e014c69..38bc961dc47a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -19,18 +19,17 @@ package org.apache.druid.indexing.common.task; -import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.errorprone.annotations.concurrent.GuardedBy; -import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.indexer.IngestionState; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.report.IngestionStatsAndErrors; import org.apache.druid.indexer.report.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexer.report.TaskContextReport; @@ -68,6 +67,7 @@ import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.incremental.ParseExceptionHandler; @@ -75,9 +75,8 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.IngestionSpec; import org.apache.druid.segment.indexing.TuningConfig; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.druid.segment.transform.TransformSpec; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.Partitions; @@ -92,6 +91,7 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -131,7 +131,12 @@ public abstract class AbstractBatchIndexTask extends AbstractTask private final Map intervalToLockVersion = new HashMap<>(); - protected AbstractBatchIndexTask(String id, String dataSource, Map context, IngestionMode ingestionMode) + protected AbstractBatchIndexTask( + String id, + String dataSource, + Map context, + IngestionMode ingestionMode + ) { super(id, dataSource, context, ingestionMode); maxAllowedLockCount = -1; @@ -634,20 +639,20 @@ public static Function, Set> addCompactionStateToS .build(); } // We only need to store filter since that is the only field auto compaction support - Map transformSpec = ingestionSpec.getDataSchema().getTransformSpec() == null || TransformSpec.NONE.equals(ingestionSpec.getDataSchema().getTransformSpec()) - ? null - : new ClientCompactionTaskTransformSpec(ingestionSpec.getDataSchema().getTransformSpec().getFilter()).asMap(toolbox.getJsonMapper()); - List metricsSpec = ingestionSpec.getDataSchema().getAggregators() == null - ? null - : toolbox.getJsonMapper().convertValue(ingestionSpec.getDataSchema().getAggregators(), new TypeReference<>() {}); + CompactionTransformSpec transformSpec = CompactionTransformSpec.of( + ingestionSpec.getDataSchema().getTransformSpec() + ); + List metricsSpec = ingestionSpec.getDataSchema().getAggregators() == null + ? null + : Arrays.asList(ingestionSpec.getDataSchema().getAggregators()); return CompactionState.addCompactionStateToSegments( tuningConfig.getPartitionsSpec(), dimensionsSpec, metricsSpec, transformSpec, - tuningConfig.getIndexSpec().asMap(toolbox.getJsonMapper()), - granularitySpec.asMap(toolbox.getJsonMapper()) + tuningConfig.getIndexSpec(), + granularitySpec ); } else { return Function.identity(); @@ -719,7 +724,10 @@ protected boolean waitForSegmentAvailability( try ( SegmentHandoffNotifier notifier = toolbox.getSegmentHandoffNotifierFactory() - .createSegmentHandoffNotifier(segmentsToWaitFor.get(0).getDataSource(), getId()) + .createSegmentHandoffNotifier( + segmentsToWaitFor.get(0).getDataSource(), + getId() + ) ) { final ExecutorService exec = Execs.directExecutor(); @@ -761,7 +769,7 @@ protected boolean waitForSegmentAvailability( ); } } - + @Nullable public static String findVersion(Map versions, Interval interval) { @@ -936,8 +944,8 @@ protected TaskReport.ReportMap buildLiveIngestionStatsReport( /** * Builds a map with the following keys and values: - * {@link IngestionStatsAndErrorsTaskReport#REPORT_KEY} : {@link IngestionStatsAndErrorsTaskReport}. - * {@link TaskContextReport#REPORT_KEY} : {@link TaskContextReport}. + * {@link IngestionStatsAndErrorsTaskReport#REPORT_KEY} : {@link IngestionStatsAndErrorsTaskReport}. + * {@link TaskContextReport#REPORT_KEY} : {@link TaskContextReport}. */ protected TaskReport.ReportMap buildIngestionStatsAndContextReport( IngestionState ingestionState, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java index 8d722848e2d0..2e618e5d85cb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java @@ -23,6 +23,7 @@ import it.unimi.dsi.fastutil.objects.Object2IntMap; import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import org.apache.druid.data.input.InputRow; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.LockListAction; @@ -32,7 +33,6 @@ import org.apache.druid.indexing.common.task.batch.partition.CompletePartitionAnalysis; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.apache.druid.timeline.partition.ShardSpec; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index dc2eb75e3cc8..2369014bd7a3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -36,7 +36,6 @@ import org.apache.curator.shaded.com.google.common.base.Verify; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; -import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.data.input.SplitHintSpec; import org.apache.druid.data.input.impl.DimensionSchema; @@ -48,6 +47,8 @@ import org.apache.druid.indexer.Checks; import org.apache.druid.indexer.Property; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.GranularitySpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.LockGranularity; @@ -86,10 +87,9 @@ import org.apache.druid.segment.indexing.CombinedDataSchema; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.TuningConfig; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; @@ -153,7 +153,7 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg @Nullable private final DimensionsSpec dimensionsSpec; @Nullable - private final ClientCompactionTaskTransformSpec transformSpec; + private final CompactionTransformSpec transformSpec; @Nullable private final AggregatorFactory[] metricsSpec; @Nullable @@ -177,7 +177,7 @@ public CompactionTask( @JsonProperty("ioConfig") @Nullable CompactionIOConfig ioConfig, @JsonProperty("dimensions") @Nullable final DimensionsSpec dimensions, @JsonProperty("dimensionsSpec") @Nullable final DimensionsSpec dimensionsSpec, - @JsonProperty("transformSpec") @Nullable final ClientCompactionTaskTransformSpec transformSpec, + @JsonProperty("transformSpec") @Nullable final CompactionTransformSpec transformSpec, @JsonProperty("metricsSpec") @Nullable final AggregatorFactory[] metricsSpec, @JsonProperty("segmentGranularity") @Deprecated @Nullable final Granularity segmentGranularity, @JsonProperty("granularitySpec") @Nullable final ClientCompactionTaskGranularitySpec granularitySpec, @@ -359,7 +359,7 @@ public DimensionsSpec getDimensionsSpec() @JsonProperty @Nullable - public ClientCompactionTaskTransformSpec getTransformSpec() + public CompactionTransformSpec getTransformSpec() { return transformSpec; } @@ -538,7 +538,7 @@ static Map createDataSchemasForIntervals( final LockGranularity lockGranularityInUse, final SegmentProvider segmentProvider, @Nullable final DimensionsSpec dimensionsSpec, - @Nullable final ClientCompactionTaskTransformSpec transformSpec, + @Nullable final CompactionTransformSpec transformSpec, @Nullable final AggregatorFactory[] metricsSpec, @Nullable final ClientCompactionTaskGranularitySpec granularitySpec, final ServiceMetricEvent.Builder metricBuilder, @@ -661,7 +661,7 @@ private static DataSchema createDataSchema( Interval totalInterval, Iterable>>> segments, @Nullable DimensionsSpec dimensionsSpec, - @Nullable ClientCompactionTaskTransformSpec transformSpec, + @Nullable CompactionTransformSpec transformSpec, @Nullable AggregatorFactory[] metricsSpec, @Nonnull ClientCompactionTaskGranularitySpec granularitySpec, boolean needMultiValuedColumns @@ -1153,7 +1153,7 @@ public static class Builder @Nullable private DimensionsSpec dimensionsSpec; @Nullable - private ClientCompactionTaskTransformSpec transformSpec; + private CompactionTransformSpec transformSpec; @Nullable private AggregatorFactory[] metricsSpec; @Nullable @@ -1209,7 +1209,7 @@ public Builder dimensionsSpec(DimensionsSpec dimensionsSpec) return this; } - public Builder transformSpec(ClientCompactionTaskTransformSpec transformSpec) + public Builder transformSpec(CompactionTransformSpec transformSpec) { this.transformSpec = transformSpec; return this; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index 6ae833a83c16..7361759ed223 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java @@ -42,6 +42,8 @@ import org.apache.druid.indexer.TaskMetricsGetter; import org.apache.druid.indexer.TaskMetricsUtils; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.ArbitraryGranularitySpec; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.path.SegmentMetadataPublisher; import org.apache.druid.indexer.report.TaskReport; import org.apache.druid.indexing.common.TaskLock; @@ -58,8 +60,6 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.incremental.RowIngestionMeters; -import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.ChatHandler; import org.apache.druid.segment.realtime.ChatHandlerProvider; import org.apache.druid.server.security.Action; 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 9c114d3403f1..eae1f7caf1e6 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 @@ -41,6 +41,8 @@ import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.Property; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.ArbitraryGranularitySpec; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; @@ -82,8 +84,6 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.IngestionSpec; 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.ChatHandler; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java index 9115818dcb36..b8deef48fdb9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java @@ -25,6 +25,7 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.task.batch.parallel.iterator.IndexTaskInputRowIteratorBuilder; @@ -36,7 +37,6 @@ 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.granularity.GranularitySpec; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java index c2488e6244ba..911856d88d84 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java @@ -23,11 +23,11 @@ import com.google.common.collect.Maps; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.druid.data.input.InputRow; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.partition.BuildingNumberedShardSpec; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/OverlordCoordinatingSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/OverlordCoordinatingSegmentAllocator.java index bd9c3c8b4502..fd1092fc80a7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/OverlordCoordinatingSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/OverlordCoordinatingSegmentAllocator.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.common.task; import org.apache.druid.data.input.InputRow; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SecondaryPartitionType; import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; @@ -31,7 +32,6 @@ import org.apache.druid.indexing.common.task.batch.parallel.SupervisorTaskAccess; import org.apache.druid.java.util.common.ISE; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.partition.NumberedOverwritePartialShardSpec; import org.apache.druid.timeline.partition.NumberedPartialShardSpec; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocators.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocators.java index 1e0216d81059..bc360546042c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocators.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocators.java @@ -20,12 +20,12 @@ package org.apache.druid.indexing.common.task; import com.google.common.base.Preconditions; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.batch.parallel.SupervisorTaskAccess; import org.apache.druid.indexing.common.task.batch.partition.CompletePartitionAnalysis; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import javax.annotation.Nullable; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 594816974929..86e31c74a72f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -37,6 +37,8 @@ import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.ArbitraryGranularitySpec; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; @@ -73,8 +75,6 @@ import org.apache.druid.segment.incremental.RowIngestionMetersTotals; import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; 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.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.ChatHandler; import org.apache.druid.segment.realtime.ChatHandlers; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java index d75304f38c08..6a47af05adbc 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java @@ -33,6 +33,7 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputSource; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SurrogateTaskActionClient; @@ -44,7 +45,6 @@ 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.granularity.GranularitySpec; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java index 4bb395420b3f..f206c9850801 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java @@ -36,6 +36,7 @@ import org.apache.druid.data.input.Rows; import org.apache.druid.data.input.StringTuple; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SurrogateTaskActionClient; @@ -50,7 +51,6 @@ 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.granularity.GranularitySpec; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java index 4c224e396d18..92bba5b3862b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableSet; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.report.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; @@ -34,7 +35,6 @@ import org.apache.druid.indexing.common.task.batch.parallel.iterator.DefaultIndexTaskInputRowIteratorBuilder; import org.apache.druid.indexing.common.task.batch.partition.HashPartitionAnalysis; import org.apache.druid.indexing.worker.shuffle.ShuffleDataSegmentPusher; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTask.java index 3b00f0fedf67..023d2b39cc2b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTask.java @@ -20,13 +20,13 @@ package org.apache.druid.indexing.common.task.batch.parallel; import com.google.common.base.Preconditions; +import org.apache.druid.indexer.granularity.ArbitraryGranularitySpec; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; 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 8d49a7f3dbe7..0b459913de46 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 @@ -31,6 +31,8 @@ import org.apache.druid.data.input.InputSource; import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.ArbitraryGranularitySpec; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.report.TaskReport; import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; @@ -57,8 +59,6 @@ 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.granularity.ArbitraryGranularitySpec; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.ChatHandler; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TombstoneHelper.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TombstoneHelper.java index 15ba6788307e..e30e0ec7c3a4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TombstoneHelper.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TombstoneHelper.java @@ -21,6 +21,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; @@ -32,7 +33,6 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.IntervalsByGranularity; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.ShardSpec; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java index aa74f1199e6b..c081118976c6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java @@ -23,9 +23,9 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.HandlingInputRowIterator; import org.apache.druid.data.input.InputRow; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import java.util.ArrayList; import java.util.List; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java index 3e3d653e3c32..977bd17bebc7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java @@ -21,8 +21,8 @@ import org.apache.druid.data.input.HandlingInputRowIterator; import org.apache.druid.data.input.InputRow; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; public interface IndexTaskInputRowIteratorBuilder { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/RangePartitionIndexTaskInputRowIteratorBuilder.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/RangePartitionIndexTaskInputRowIteratorBuilder.java index c377d371c9af..c8c7ad101c16 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/RangePartitionIndexTaskInputRowIteratorBuilder.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/RangePartitionIndexTaskInputRowIteratorBuilder.java @@ -21,10 +21,10 @@ import org.apache.druid.data.input.HandlingInputRowIterator; import org.apache.druid.data.input.InputRow; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import java.util.List; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java index b81d6964e42d..8eb738bf9d0e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.java.util.common.HumanReadableBytes; import javax.annotation.Nullable; @@ -57,7 +58,7 @@ public SamplerConfig( * The maximum number of rows to return in a response. The actual number of returned rows may be less if: * - The sampled source contains fewer rows * - {@link SamplerConfig#timeoutMs} elapses before this value is reached - * - {@link org.apache.druid.segment.indexing.granularity.GranularitySpec#isRollup()} is true and input rows get + * - {@link GranularitySpec#isRollup()} is true and input rows get * rolled-up into fewer indexed rows. * - The incremental index performing the sampling reaches {@link SamplerConfig#getMaxBytesInMemory()} before this * value is reached diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index a354cb20db28..21d043728ccb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -33,7 +33,6 @@ import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; -import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.client.indexing.ClientTaskQuery; import org.apache.druid.data.input.SegmentsSplitHintSpec; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -63,6 +62,7 @@ import org.apache.druid.segment.realtime.ChatHandlerProvider; import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.server.security.AuthTestUtils; @@ -96,8 +96,8 @@ public class ClientCompactionTaskQuerySerdeTest private static final ClientCompactionTaskGranularitySpec CLIENT_COMPACTION_TASK_GRANULARITY_SPEC = new ClientCompactionTaskGranularitySpec(Granularities.DAY, Granularities.HOUR, true); private static final AggregatorFactory[] METRICS_SPEC = new AggregatorFactory[] {new CountAggregatorFactory("cnt")}; - private static final ClientCompactionTaskTransformSpec CLIENT_COMPACTION_TASK_TRANSFORM_SPEC = - new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)); + private static final CompactionTransformSpec CLIENT_COMPACTION_TASK_TRANSFORM_SPEC = + new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null)); private static final DynamicPartitionsSpec DYNAMIC_PARTITIONS_SPEC = new DynamicPartitionsSpec(100, 30000L); private static final SegmentsSplitHintSpec SEGMENTS_SPLIT_HINT_SPEC = new SegmentsSplitHintSpec(new HumanReadableBytes(100000L), 10); @@ -293,7 +293,7 @@ private void assertQueryToTask(ClientCompactionTaskQuery query, CompactionTask t ); } - private ClientCompactionTaskQuery createCompactionTaskQuery(String id, ClientCompactionTaskTransformSpec transformSpec) + private ClientCompactionTaskQuery createCompactionTaskQuery(String id, CompactionTransformSpec transformSpec) { Map context = new HashMap<>(); context.put("key", "value"); @@ -334,7 +334,7 @@ private ClientCompactionTaskQuery createCompactionTaskQuery(String id, ClientCom ); } - private CompactionTask createCompactionTask(ClientCompactionTaskTransformSpec transformSpec) + private CompactionTask createCompactionTask(CompactionTransformSpec transformSpec) { CompactionTask.Builder compactionTaskBuilder = new CompactionTask.Builder( "datasource", diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java index a8684eba731c..29d076ad8f83 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java @@ -25,7 +25,6 @@ import com.google.common.collect.Lists; import com.google.common.io.Files; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; -import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.SegmentsSplitHintSpec; import org.apache.druid.data.input.impl.CsvInputFormat; @@ -36,6 +35,7 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; @@ -62,8 +62,8 @@ import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.NoopSegmentCacheManager; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; @@ -94,7 +94,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -176,10 +175,7 @@ public void testRunParallelWithDynamicPartitioningMatchCompactionState() throws segment.getShardSpec().getClass() ); // Expect compaction state to exist as store compaction state by default - Map expectedLongSumMetric = new HashMap<>(); - expectedLongSumMetric.put("type", "longSum"); - expectedLongSumMetric.put("name", "val"); - expectedLongSumMetric.put("fieldName", "val"); + LongSumAggregatorFactory expectedLongSumMetric = new LongSumAggregatorFactory("val", "val"); CompactionState expectedState = new CompactionState( new DynamicPartitionsSpec(null, Long.MAX_VALUE), new DimensionsSpec( @@ -190,17 +186,12 @@ public void testRunParallelWithDynamicPartitioningMatchCompactionState() throws ), ImmutableList.of(expectedLongSumMetric), null, - compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), - getObjectMapper().readValue( - getObjectMapper().writeValueAsString( - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.MINUTE, - true, - ImmutableList.of(segment.getInterval()) - ) - ), - Map.class + compactionTask.getTuningConfig().getIndexSpec(), + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + true, + ImmutableList.of(segment.getInterval()) ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); @@ -230,10 +221,7 @@ public void testRunParallelWithHashPartitioningMatchCompactionState() throws Exc final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); for (DataSegment segment : compactedSegments) { // Expect compaction state to exist as store compaction state by default - Map expectedLongSumMetric = new HashMap<>(); - expectedLongSumMetric.put("type", "longSum"); - expectedLongSumMetric.put("name", "val"); - expectedLongSumMetric.put("fieldName", "val"); + LongSumAggregatorFactory expectedLongSumMetric = new LongSumAggregatorFactory("val", "val"); Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass()); CompactionState expectedState = new CompactionState( new HashedPartitionsSpec(null, 3, null), @@ -245,17 +233,12 @@ public void testRunParallelWithHashPartitioningMatchCompactionState() throws Exc ), ImmutableList.of(expectedLongSumMetric), null, - compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), - getObjectMapper().readValue( - getObjectMapper().writeValueAsString( - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.MINUTE, - true, - ImmutableList.of(segment.getInterval()) - ) - ), - Map.class + compactionTask.getTuningConfig().getIndexSpec(), + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + true, + ImmutableList.of(segment.getInterval()) ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); @@ -300,10 +283,7 @@ public void testRunParallelWithRangePartitioning() throws Exception final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); for (DataSegment segment : compactedSegments) { // Expect compaction state to exist as store compaction state by default - Map expectedLongSumMetric = new HashMap<>(); - expectedLongSumMetric.put("type", "longSum"); - expectedLongSumMetric.put("name", "val"); - expectedLongSumMetric.put("fieldName", "val"); + LongSumAggregatorFactory expectedLongSumMetric = new LongSumAggregatorFactory("val", "val"); Assert.assertSame(SingleDimensionShardSpec.class, segment.getShardSpec().getClass()); CompactionState expectedState = new CompactionState( new SingleDimensionPartitionsSpec(7, null, "dim", false), @@ -315,17 +295,12 @@ public void testRunParallelWithRangePartitioning() throws Exception ), ImmutableList.of(expectedLongSumMetric), null, - compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), - getObjectMapper().readValue( - getObjectMapper().writeValueAsString( - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.MINUTE, - true, - ImmutableList.of(segment.getInterval()) - ) - ), - Map.class + compactionTask.getTuningConfig().getIndexSpec(), + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + true, + ImmutableList.of(segment.getInterval()) ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); @@ -365,29 +340,21 @@ public void testRunParallelWithRangePartitioningAndNoUpfrontSegmentFetching() th final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); for (DataSegment segment : compactedSegments) { // Expect compaction state to exist as store compaction state by default - Map expectedLongSumMetric = new HashMap<>(); - expectedLongSumMetric.put("type", "longSum"); - expectedLongSumMetric.put("name", "val"); - expectedLongSumMetric.put("fieldName", "val"); + LongSumAggregatorFactory expectedLongSumMetric = new LongSumAggregatorFactory("val", "val"); Assert.assertSame(SingleDimensionShardSpec.class, segment.getShardSpec().getClass()); CompactionState expectedState = new CompactionState( new SingleDimensionPartitionsSpec(7, null, "dim", false), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), ImmutableList.of(expectedLongSumMetric), null, - compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), - getObjectMapper().readValue( - getObjectMapper().writeValueAsString( - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.MINUTE, - true, - - // Umbrella interval for all segments, since CompactionTasks generated a single granularitySpec. - ImmutableList.of(Intervals.of("2014-01-01/2014-01-01T03:00:00")) - ) - ), - Map.class + compactionTask.getTuningConfig().getIndexSpec(), + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + true, + + // Umbrella interval for all segments, since CompactionTasks generated a single granularitySpec. + ImmutableList.of(Intervals.of("2014-01-01/2014-01-01T03:00:00")) ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); @@ -420,10 +387,7 @@ public void testRunParallelWithMultiDimensionRangePartitioning() throws Exceptio final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); for (DataSegment segment : compactedSegments) { // Expect compaction state to exist as store compaction state by default - Map expectedLongSumMetric = new HashMap<>(); - expectedLongSumMetric.put("type", "longSum"); - expectedLongSumMetric.put("name", "val"); - expectedLongSumMetric.put("fieldName", "val"); + LongSumAggregatorFactory expectedLongSumMetric = new LongSumAggregatorFactory("val", "val"); Assert.assertSame(DimensionRangeShardSpec.class, segment.getShardSpec().getClass()); CompactionState expectedState = new CompactionState( new DimensionRangePartitionsSpec(7, null, Arrays.asList("dim1", "dim2"), false), @@ -435,17 +399,12 @@ public void testRunParallelWithMultiDimensionRangePartitioning() throws Exceptio ), ImmutableList.of(expectedLongSumMetric), null, - compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), - getObjectMapper().readValue( - getObjectMapper().writeValueAsString( - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.MINUTE, - true, - ImmutableList.of(segment.getInterval()) - ) - ), - Map.class + compactionTask.getTuningConfig().getIndexSpec(), + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + true, + ImmutableList.of(segment.getInterval()) ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); @@ -475,10 +434,7 @@ public void testRunParallelWithRangePartitioningWithSingleTask() throws Exceptio final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); for (DataSegment segment : compactedSegments) { // Expect compaction state to exist as store compaction state by default - Map expectedLongSumMetric = new HashMap<>(); - expectedLongSumMetric.put("type", "longSum"); - expectedLongSumMetric.put("name", "val"); - expectedLongSumMetric.put("fieldName", "val"); + LongSumAggregatorFactory expectedLongSumMetric = new LongSumAggregatorFactory("val", "val"); Assert.assertSame(SingleDimensionShardSpec.class, segment.getShardSpec().getClass()); CompactionState expectedState = new CompactionState( new SingleDimensionPartitionsSpec(7, null, "dim", false), @@ -490,17 +446,12 @@ public void testRunParallelWithRangePartitioningWithSingleTask() throws Exceptio ), ImmutableList.of(expectedLongSumMetric), null, - compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), - getObjectMapper().readValue( - getObjectMapper().writeValueAsString( - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.MINUTE, - true, - ImmutableList.of(segment.getInterval()) - ) - ), - Map.class + compactionTask.getTuningConfig().getIndexSpec(), + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + true, + ImmutableList.of(segment.getInterval()) ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); @@ -533,10 +484,7 @@ public void testRunParallelWithMultiDimensionRangePartitioningWithSingleTask() t final Set compactedSegments = dataSegmentsWithSchemas.getSegments(); for (DataSegment segment : compactedSegments) { // Expect compaction state to exist as store compaction state by default - Map expectedLongSumMetric = new HashMap<>(); - expectedLongSumMetric.put("type", "longSum"); - expectedLongSumMetric.put("name", "val"); - expectedLongSumMetric.put("fieldName", "val"); + LongSumAggregatorFactory expectedLongSumMetric = new LongSumAggregatorFactory("val", "val"); Assert.assertSame(DimensionRangeShardSpec.class, segment.getShardSpec().getClass()); CompactionState expectedState = new CompactionState( new DimensionRangePartitionsSpec(7, null, Arrays.asList("dim1", "dim2"), false), @@ -548,17 +496,12 @@ public void testRunParallelWithMultiDimensionRangePartitioningWithSingleTask() t ), ImmutableList.of(expectedLongSumMetric), null, - compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), - getObjectMapper().readValue( - getObjectMapper().writeValueAsString( - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.MINUTE, - true, - ImmutableList.of(segment.getInterval()) - ) - ), - Map.class + compactionTask.getTuningConfig().getIndexSpec(), + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + true, + ImmutableList.of(segment.getInterval()) ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); @@ -608,7 +551,7 @@ public void testRunCompactionWithFilterShouldStoreInState() throws Exception final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) .tuningConfig(AbstractParallelIndexSupervisorTaskTest.DEFAULT_TUNING_CONFIG_FOR_PARALLEL_INDEXING) - .transformSpec(new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim", "a", null))) + .transformSpec(new CompactionTransformSpec(new SelectorDimFilter("dim", "a", null))) .build(); final DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(compactionTask); @@ -622,10 +565,7 @@ public void testRunCompactionWithFilterShouldStoreInState() throws Exception lockGranularity == LockGranularity.TIME_CHUNK ? NumberedShardSpec.class : NumberedOverwriteShardSpec.class, segment.getShardSpec().getClass() ); - Map expectedLongSumMetric = new HashMap<>(); - expectedLongSumMetric.put("type", "longSum"); - expectedLongSumMetric.put("name", "val"); - expectedLongSumMetric.put("fieldName", "val"); + LongSumAggregatorFactory expectedLongSumMetric = new LongSumAggregatorFactory("val", "val"); CompactionState expectedState = new CompactionState( new DynamicPartitionsSpec(null, Long.MAX_VALUE), new DimensionsSpec( @@ -635,21 +575,13 @@ public void testRunCompactionWithFilterShouldStoreInState() throws Exception ) ), ImmutableList.of(expectedLongSumMetric), - getObjectMapper().readValue( - getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), - Map.class - ), - compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), - getObjectMapper().readValue( - getObjectMapper().writeValueAsString( - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.MINUTE, - true, - ImmutableList.of(segment.getInterval()) - ) - ), - Map.class + compactionTask.getTransformSpec(), + compactionTask.getTuningConfig().getIndexSpec(), + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + true, + ImmutableList.of(segment.getInterval()) ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); @@ -686,13 +618,8 @@ public void testRunCompactionWithNewMetricsShouldStoreInState() throws Exception lockGranularity == LockGranularity.TIME_CHUNK ? NumberedShardSpec.class : NumberedOverwriteShardSpec.class, segment.getShardSpec().getClass() ); - Map expectedCountMetric = new HashMap<>(); - expectedCountMetric.put("type", "count"); - expectedCountMetric.put("name", "cnt"); - Map expectedLongSumMetric = new HashMap<>(); - expectedLongSumMetric.put("type", "longSum"); - expectedLongSumMetric.put("name", "val"); - expectedLongSumMetric.put("fieldName", "val"); + CountAggregatorFactory expectedCountMetric = new CountAggregatorFactory("cnt"); + LongSumAggregatorFactory expectedLongSumMetric = new LongSumAggregatorFactory("val", "val"); CompactionState expectedState = new CompactionState( new DynamicPartitionsSpec(null, Long.MAX_VALUE), new DimensionsSpec( @@ -702,21 +629,13 @@ public void testRunCompactionWithNewMetricsShouldStoreInState() throws Exception ) ), ImmutableList.of(expectedCountMetric, expectedLongSumMetric), - getObjectMapper().readValue( - getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), - Map.class - ), - compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), - getObjectMapper().readValue( - getObjectMapper().writeValueAsString( - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.MINUTE, - true, - ImmutableList.of(segment.getInterval()) - ) - ), - Map.class + compactionTask.getTransformSpec(), + compactionTask.getTuningConfig().getIndexSpec(), + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + true, + ImmutableList.of(segment.getInterval()) ) ); Assert.assertEquals("Compaction state for " + segment.getId(), expectedState, segment.getLastCompactionState()); @@ -1003,10 +922,12 @@ private void runIndexTask(@Nullable PartitionsSpec partitionsSpec, boolean appen Assert.assertEquals( Collections.singleton( - new ResourceAction(new Resource( - LocalInputSource.TYPE_KEY, - ResourceType.EXTERNAL - ), Action.READ)), + new ResourceAction( + new Resource( + LocalInputSource.TYPE_KEY, + ResourceType.EXTERNAL + ), Action.READ + )), indexTask.getInputSourceResources() ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 76068d5de00f..613a8277530c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.common.task; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.collect.ImmutableList; @@ -31,7 +30,6 @@ import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; -import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -42,6 +40,7 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.report.IngestionStatsAndErrors; @@ -54,7 +53,6 @@ import org.apache.druid.indexing.common.config.TaskConfigBuilder; import org.apache.druid.indexing.common.task.CompactionTask.Builder; import org.apache.druid.indexing.overlord.Segments; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; @@ -84,7 +82,6 @@ import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy; import org.apache.druid.segment.loading.LocalDataSegmentPuller; @@ -101,6 +98,7 @@ import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.WindowedCursorFactory; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; @@ -130,10 +128,8 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.TreeSet; import java.util.concurrent.CountDownLatch; @@ -220,12 +216,9 @@ public static CompactionState getDefaultCompactionState( Granularity segmentGranularity, Granularity queryGranularity, List intervals - ) throws JsonProcessingException + ) { - Map expectedLongSumMetric = new HashMap<>(); - expectedLongSumMetric.put("type", "longSum"); - expectedLongSumMetric.put("name", "val"); - expectedLongSumMetric.put("fieldName", "val"); + AggregatorFactory expectedLongSumMetric = new LongSumAggregatorFactory("val", "val"); return getDefaultCompactionState( segmentGranularity, queryGranularity, @@ -245,27 +238,21 @@ public static CompactionState getDefaultCompactionState( Granularity queryGranularity, List intervals, DimensionsSpec expectedDims, - Map expectedMetric - ) throws JsonProcessingException + AggregatorFactory expectedMetric + ) { - ObjectMapper mapper = new DefaultObjectMapper(); // Expected compaction state to exist after compaction as we store compaction state by default return new CompactionState( new DynamicPartitionsSpec(5000000, Long.MAX_VALUE), expectedDims, ImmutableList.of(expectedMetric), null, - IndexSpec.DEFAULT.asMap(mapper), - mapper.readValue( - mapper.writeValueAsString( - new UniformGranularitySpec( - segmentGranularity, - queryGranularity, - true, - intervals - ) - ), - Map.class + IndexSpec.DEFAULT, + new UniformGranularitySpec( + segmentGranularity, + queryGranularity, + true, + intervals ) ); } @@ -316,9 +303,10 @@ public void testRunWithDynamicPartitioning() throws Exception getDefaultCompactionState( Granularities.HOUR, Granularities.MINUTE, - ImmutableList.of(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", - i, - i + 1 + ImmutableList.of(Intervals.of( + "2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", + i, + i + 1 )) ), segments.get(i).getLastCompactionState() @@ -388,10 +376,7 @@ public void testRunWithHashPartitioning() throws Exception interval, segments.get(segmentIdx).getInterval() ); - Map expectedLongSumMetric = new HashMap<>(); - expectedLongSumMetric.put("type", "longSum"); - expectedLongSumMetric.put("name", "val"); - expectedLongSumMetric.put("fieldName", "val"); + AggregatorFactory expectedLongSumMetric = new LongSumAggregatorFactory("val", "val"); CompactionState expectedState = new CompactionState( new HashedPartitionsSpec(null, 3, null), new DimensionsSpec( @@ -402,17 +387,12 @@ public void testRunWithHashPartitioning() throws Exception ), ImmutableList.of(expectedLongSumMetric), null, - compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), - getObjectMapper().readValue( - getObjectMapper().writeValueAsString( - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.MINUTE, - true, - ImmutableList.of(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1)) - ) - ), - Map.class + compactionTask.getTuningConfig().getIndexSpec(), + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + true, + ImmutableList.of(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1)) ) ); Assert.assertEquals(expectedState, segments.get(segmentIdx).getLastCompactionState()); @@ -457,9 +437,10 @@ public void testRunCompactionTwice() throws Exception getDefaultCompactionState( Granularities.HOUR, Granularities.MINUTE, - ImmutableList.of(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", - i, - i + 1 + ImmutableList.of(Intervals.of( + "2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", + i, + i + 1 )) ), segments.get(i).getLastCompactionState() @@ -495,9 +476,10 @@ public void testRunCompactionTwice() throws Exception getDefaultCompactionState( Granularities.HOUR, Granularities.MINUTE, - ImmutableList.of(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", - i, - i + 1 + ImmutableList.of(Intervals.of( + "2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", + i, + i + 1 )) ), segments.get(i).getLastCompactionState() @@ -612,9 +594,10 @@ public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exc getDefaultCompactionState( Granularities.HOUR, Granularities.MINUTE, - ImmutableList.of(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", - i, - i + 1 + ImmutableList.of(Intervals.of( + "2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", + i, + i + 1 )) ), segments.get(i).getLastCompactionState() @@ -792,7 +775,7 @@ public void testCompactionWithFilterInTransformSpec() throws Exception final CompactionTask compactionTask = builder .interval(Intervals.of("2014-01-01/2014-01-02")) .granularitySpec(new ClientCompactionTaskGranularitySpec(Granularities.DAY, null, null)) - .transformSpec(new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim", "a", null))) + .transformSpec(new CompactionTransformSpec(new SelectorDimFilter("dim", "a", null))) .build(); Pair resultPair = runTask(compactionTask); @@ -807,11 +790,7 @@ public void testCompactionWithFilterInTransformSpec() throws Exception Assert.assertEquals(Intervals.of("2014-01-01/2014-01-02"), segments.get(0).getInterval()); Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(0).getShardSpec()); - ObjectMapper mapper = new DefaultObjectMapper(); - Map expectedLongSumMetric = new HashMap<>(); - expectedLongSumMetric.put("type", "longSum"); - expectedLongSumMetric.put("name", "val"); - expectedLongSumMetric.put("fieldName", "val"); + AggregatorFactory expectedLongSumMetric = new LongSumAggregatorFactory("val", "val"); CompactionState expectedCompactionState = new CompactionState( new DynamicPartitionsSpec(5000000, Long.MAX_VALUE), new DimensionsSpec( @@ -821,18 +800,13 @@ public void testCompactionWithFilterInTransformSpec() throws Exception ) ), ImmutableList.of(expectedLongSumMetric), - getObjectMapper().readValue(getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), Map.class), - IndexSpec.DEFAULT.asMap(mapper), - mapper.readValue( - mapper.writeValueAsString( - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - true, - ImmutableList.of(Intervals.of("2014-01-01T00:00:00/2014-01-01T03:00:00")) - ) - ), - Map.class + compactionTask.getTransformSpec(), + IndexSpec.DEFAULT, + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + true, + ImmutableList.of(Intervals.of("2014-01-01T00:00:00/2014-01-01T03:00:00")) ) ); Assert.assertEquals( @@ -874,14 +848,8 @@ public void testCompactionWithNewMetricInMetricsSpec() throws Exception Assert.assertEquals(Intervals.of("2014-01-01/2014-01-02"), segments.get(0).getInterval()); Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(0).getShardSpec()); - ObjectMapper mapper = new DefaultObjectMapper(); - Map expectedCountMetric = new HashMap<>(); - expectedCountMetric.put("type", "count"); - expectedCountMetric.put("name", "cnt"); - Map expectedLongSumMetric = new HashMap<>(); - expectedLongSumMetric.put("type", "longSum"); - expectedLongSumMetric.put("name", "val"); - expectedLongSumMetric.put("fieldName", "val"); + AggregatorFactory expectedCountMetric = new CountAggregatorFactory("cnt"); + AggregatorFactory expectedLongSumMetric = new LongSumAggregatorFactory("val", "val"); CompactionState expectedCompactionState = new CompactionState( new DynamicPartitionsSpec(5000000, Long.MAX_VALUE), new DimensionsSpec( @@ -891,18 +859,13 @@ public void testCompactionWithNewMetricInMetricsSpec() throws Exception ) ), ImmutableList.of(expectedCountMetric, expectedLongSumMetric), - getObjectMapper().readValue(getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), Map.class), - IndexSpec.DEFAULT.asMap(mapper), - mapper.readValue( - mapper.writeValueAsString( - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - true, - ImmutableList.of(Intervals.of("2014-01-01T00:00:00/2014-01-01T03:00:00")) - ) - ), - Map.class + compactionTask.getTransformSpec(), + IndexSpec.DEFAULT, + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + true, + ImmutableList.of(Intervals.of("2014-01-01T00:00:00/2014-01-01T03:00:00")) ) ); Assert.assertEquals( @@ -1014,9 +977,10 @@ public void testWithGranularitySpecNonNullQueryGranularityAndNullSegmentGranular getDefaultCompactionState( Granularities.HOUR, Granularities.SECOND, - ImmutableList.of(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", - i, - i + 1 + ImmutableList.of(Intervals.of( + "2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", + i, + i + 1 )) ), segments.get(i).getLastCompactionState() @@ -1104,9 +1068,10 @@ public void testWithGranularitySpecNullQueryGranularityAndNullSegmentGranularity getDefaultCompactionState( Granularities.HOUR, Granularities.MINUTE, - ImmutableList.of(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", - i, - i + 1 + ImmutableList.of(Intervals.of( + "2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", + i, + i + 1 )) ), segments.get(i).getLastCompactionState() @@ -1663,7 +1628,13 @@ public void testRunWithSpatialDimensions() throws Exception false, 0 ); - Pair indexTaskResult = runIndexTask(null, null, spatialSpec, spatialrows, false); + Pair indexTaskResult = runIndexTask( + null, + null, + spatialSpec, + spatialrows, + false + ); verifySchema(indexTaskResult.rhs); final Builder builder = new Builder( @@ -1688,10 +1659,7 @@ public void testRunWithSpatialDimensions() throws Exception Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval() ); - Map expectedLongSumMetric = new HashMap<>(); - expectedLongSumMetric.put("name", "val"); - expectedLongSumMetric.put("type", "longSum"); - expectedLongSumMetric.put("fieldName", "val"); + AggregatorFactory expectedLongSumMetric = new LongSumAggregatorFactory("val", "val"); Assert.assertEquals( getDefaultCompactionState( Granularities.HOUR, @@ -1814,10 +1782,7 @@ public void testRunWithAutoCastDimensions() throws Exception Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval() ); - Map expectedLongSumMetric = new HashMap<>(); - expectedLongSumMetric.put("name", "val"); - expectedLongSumMetric.put("type", "longSum"); - expectedLongSumMetric.put("fieldName", "val"); + AggregatorFactory expectedLongSumMetric = new LongSumAggregatorFactory("val", "val"); Assert.assertEquals( getDefaultCompactionState( Granularities.HOUR, @@ -1947,10 +1912,7 @@ public void testRunWithAutoCastDimensionsSortByDimension() throws Exception Intervals.of("2014-01-01T00:00:00/2014-01-01T01:00:00"), compactSegment.getInterval() ); - Map expectedLongSumMetric = new HashMap<>(); - expectedLongSumMetric.put("name", "val"); - expectedLongSumMetric.put("type", "longSum"); - expectedLongSumMetric.put("fieldName", "val"); + AggregatorFactory expectedLongSumMetric = new LongSumAggregatorFactory("val", "val"); Assert.assertEquals( getDefaultCompactionState( Granularities.HOUR, @@ -2004,7 +1966,7 @@ public void testRunWithAutoCastDimensionsSortByDimension() throws Exception try (final CursorHolder cursorHolder = windowed.getCursorFactory() - .makeCursorHolder(CursorBuildSpec.builder().setInterval(compactSegment.getInterval()).build())) { + .makeCursorHolder(CursorBuildSpec.builder().setInterval(compactSegment.getInterval()).build())) { final Cursor cursor = cursorHolder.asCursor(); cursor.reset(); final ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -2170,7 +2132,8 @@ private Pair runTask( status, new DataSegmentsWithSchemas( new TreeSet<>(((TestLocalTaskActionClient) box.getTaskActionClient()).getPublishedSegments()), - ((TestLocalTaskActionClient) box.getTaskActionClient()).getSegmentSchemas()) + ((TestLocalTaskActionClient) box.getTaskActionClient()).getSegmentSchemas() + ) ); } else { throw new ISE("task[%s] is not ready", task.getId()); @@ -2197,7 +2160,7 @@ public List getLocations() ); final TaskConfig config = new TaskConfigBuilder() - .build(); + .build(); CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig = CentralizedDatasourceSchemaConfig.create(); centralizedDatasourceSchemaConfig.setEnabled(true); return new TaskToolbox.Builder() 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 668884706908..02c344b0ea8b 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 @@ -40,7 +40,6 @@ import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; -import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.common.guava.SettableSupplier; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.DimensionSchema; @@ -55,6 +54,7 @@ import org.apache.druid.guice.GuiceInjectors; import org.apache.druid.guice.IndexingServiceTuningConfigModule; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexing.common.LockGranularity; @@ -120,7 +120,6 @@ import org.apache.druid.segment.indexing.CombinedDataSchema; 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.join.NoopJoinableFactory; import org.apache.druid.segment.loading.NoopSegmentCacheManager; import org.apache.druid.segment.loading.SegmentCacheManager; @@ -129,6 +128,7 @@ import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.selector.settable.SettableColumnValueSelector; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; @@ -419,8 +419,8 @@ public void testCreateCompactionTaskWithConflictingGranularitySpecAndSegmentGran @Test public void testCreateCompactionTaskWithTransformSpec() { - ClientCompactionTaskTransformSpec transformSpec = - new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)); + CompactionTransformSpec transformSpec = + new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null)); final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory @@ -1686,7 +1686,7 @@ public void testGetDefaultLookupLoadingSpecWithTransformSpec() ); final CompactionTask task = builder .interval(Intervals.of("2000-01-01/2000-01-02")) - .transformSpec(new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null))) + .transformSpec(new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null))) .build(); Assert.assertEquals(LookupLoadingSpec.ALL, task.getLookupLoadingSpec()); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopIndexTaskTest.java index 0a72b77e1db7..91b94b2f3cd5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopIndexTaskTest.java @@ -24,11 +24,11 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.indexer.HadoopIOConfig; import org.apache.druid.indexer.HadoopIngestionSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.server.security.Resource; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java index d84aa154fd26..8076fc926156 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java @@ -22,11 +22,11 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.impl.NoopInputFormat; import org.apache.druid.data.input.impl.NoopInputSource; +import org.apache.druid.indexer.granularity.ArbitraryGranularitySpec; import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig; import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; 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 d1e54665f2d7..bef25dec434a 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 @@ -41,6 +41,9 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.ArbitraryGranularitySpec; +import org.apache.druid.indexer.granularity.GranularitySpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; @@ -84,9 +87,6 @@ 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.granularity.ArbitraryGranularitySpec; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java index dc3911f0bb07..8b5fe58a04c2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.StringTuple; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; @@ -33,7 +34,6 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.SegmentId; 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 99b0f8e7a759..651c1ba60087 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 @@ -29,6 +29,7 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.HadoopIOConfig; import org.apache.druid.indexer.HadoopIngestionSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.TestUtils; @@ -41,7 +42,6 @@ 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.granularity.UniformGranularitySpec; import org.apache.druid.server.security.AuthTestUtils; import org.hamcrest.CoreMatchers; import org.joda.time.Period; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java index 6caab0a0652e..672670e38c8e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java @@ -28,6 +28,8 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.GranularitySpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.report.TaskReport; import org.apache.druid.indexing.common.LockGranularity; @@ -53,8 +55,6 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.segment.loading.TombstoneLoadSpec; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java index ef8f09562311..c0d1561287a3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; @@ -37,7 +38,6 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.SegmentId; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java index 3af5952abeaa..9712ca602936 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java @@ -29,6 +29,8 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.GranularitySpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.LockGranularity; @@ -43,8 +45,6 @@ import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java index 3adc154bb226..4f621e17712d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java @@ -41,6 +41,7 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.data.input.impl.systemfield.SystemFieldDecoratorFactory; import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; @@ -54,7 +55,6 @@ import org.apache.druid.java.util.common.parsers.JSONPathFieldType; import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; import org.junit.Assert; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java index 2ea9385a0145..8f5d5450dcf2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java @@ -29,6 +29,7 @@ import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.TaskResource; @@ -38,7 +39,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.joda.time.Interval; import org.junit.After; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index 4587ef6ce7e3..d7ec15deff77 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -33,6 +33,7 @@ import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.report.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; @@ -47,7 +48,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java index 7e328d42f781..d31bc195959f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java @@ -24,6 +24,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; @@ -35,7 +36,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Rule; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java index f93ea8d0a8b8..091e7b2d7cef 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java @@ -27,13 +27,13 @@ import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.ArbitraryGranularitySpec; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashPartitionFunction; import org.joda.time.Interval; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTaskTest.java index 0c33f002a613..d531cf928f8c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTaskTest.java @@ -32,6 +32,7 @@ import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; @@ -46,7 +47,6 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java index a0b59d8c0757..f5b9e952df2e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java @@ -24,13 +24,13 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.indexing.common.task.batch.partition.HashPartitionAnalysis; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTaskTest.java index b95fd53c74f7..cf80bce60f99 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTaskTest.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.common.task.batch.parallel; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; @@ -27,7 +28,6 @@ import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.Rule; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java index 269ab6aa0be0..13155d9d3f40 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java @@ -29,6 +29,8 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.GranularitySpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.indexing.common.LockGranularity; @@ -41,8 +43,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.timeline.partition.PartitionBoundaries; import org.joda.time.Interval; import org.junit.Assert; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index 790a57786e13..f933ad97d543 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -29,6 +29,7 @@ import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.report.IngestionStatsAndErrors; import org.apache.druid.indexer.report.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexer.report.TaskReport; @@ -50,7 +51,6 @@ import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TombstoneHelperTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TombstoneHelperTest.java index fcb92543914a..9898dda0b3bd 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TombstoneHelperTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TombstoneHelperTest.java @@ -21,6 +21,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import org.apache.druid.indexer.granularity.GranularitySpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.actions.TaskAction; @@ -32,8 +34,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.TombstoneShardSpec; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java index 31f310e77ba2..35f9e8f36202 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java @@ -21,9 +21,9 @@ import org.apache.druid.data.input.HandlingInputRowIterator; import org.apache.druid.data.input.InputRow; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.easymock.EasyMock; import org.joda.time.DateTime; import org.junit.Assert; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilderTestingFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilderTestingFactory.java index dab24fbb3a03..941ab3025c91 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilderTestingFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilderTestingFactory.java @@ -23,10 +23,10 @@ import org.apache.druid.data.input.HandlingInputRowIterator; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.indexer.granularity.GranularitySpec; 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.segment.indexing.granularity.GranularitySpec; import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/RangePartitionIndexTaskInputRowIteratorBuilderTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/RangePartitionIndexTaskInputRowIteratorBuilderTest.java index 20065b597fa8..f933f5b3c383 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/RangePartitionIndexTaskInputRowIteratorBuilderTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/RangePartitionIndexTaskInputRowIteratorBuilderTest.java @@ -20,8 +20,8 @@ package org.apache.druid.indexing.common.task.batch.parallel.iterator; import org.apache.druid.data.input.InputRow; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.hamcrest.Matchers; import org.joda.time.DateTime; import org.junit.Assert; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java index 35aeef9715d3..5815f043c51d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java @@ -29,6 +29,7 @@ import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -36,7 +37,6 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.testing.InitializedNullHandlingTest; 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 8400e27d2359..e0a88f500b82 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 @@ -51,6 +51,7 @@ import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; @@ -118,7 +119,6 @@ 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.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.JoinableFactoryWrapperTest; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.DataSegmentPusher; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java index 8f1393f2c675..58ef76eb7d6f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java @@ -41,6 +41,7 @@ import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; @@ -81,7 +82,6 @@ import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import org.apache.druid.metadata.SQLMetadataConnector; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.initialization.IndexerZkConfig; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java index 694053a4e490..83339f1a3572 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java @@ -41,6 +41,8 @@ import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.GranularitySpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexing.seekablestream.RecordSupplierInputSource; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; @@ -58,8 +60,6 @@ import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.testing.InitializedNullHandlingTest; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java index d3b69d438d5f..e30502254bba 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java @@ -25,6 +25,7 @@ import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.ArbitraryGranularitySpec; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; @@ -34,7 +35,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthConfig; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java index db724931b391..edc58fb44146 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java @@ -28,6 +28,7 @@ import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; @@ -36,7 +37,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.security.AuthorizerMapper; import org.joda.time.DateTime; import org.junit.Assert; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index f75f14f86770..1144a77de36d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -49,6 +49,7 @@ import org.apache.druid.discovery.LookupNodeService; import org.apache.druid.error.DruidException; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.report.IngestionStatsAndErrors; import org.apache.druid.indexer.report.SingleFileTaskReportFileWriter; import org.apache.druid.indexer.report.TaskReport; @@ -110,7 +111,6 @@ import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; import org.apache.druid.segment.indexing.DataSchema; -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.loading.LocalDataSegmentPusher; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java index 7a9873ddce9c..740953dc0761 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java @@ -33,6 +33,7 @@ import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexing.overlord.sampler.InputSourceSampler; import org.apache.druid.indexing.overlord.sampler.SamplerConfig; import org.apache.druid.indexing.overlord.sampler.SamplerTestUtils; @@ -50,7 +51,6 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; import org.joda.time.DateTime; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java index 425b38045970..1a704729dafd 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java @@ -28,6 +28,7 @@ import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -62,7 +63,6 @@ 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.granularity.UniformGranularitySpec; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; import org.joda.time.DateTime; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java index 58c838615ac5..245aa5aba3df 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -38,6 +38,7 @@ import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.task.Task; @@ -86,7 +87,6 @@ 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.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.easymock.Capture; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java index 8218328c9221..baf7177b373d 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java @@ -30,6 +30,7 @@ import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; @@ -56,7 +57,7 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; @@ -64,7 +65,6 @@ import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; -import org.apache.druid.server.coordinator.UserCompactionTaskTransformConfig; import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.clients.CompactionResourceTestClient; import org.apache.druid.testing.clients.TaskResponseObject; @@ -1610,7 +1610,7 @@ public void testAutoCompactionDutyWithFilter() throws Exception NO_SKIP_OFFSET, null, null, - new UserCompactionTaskTransformConfig(new SelectorDimFilter("page", "Striker Eureka", null)), + new CompactionTransformSpec(new SelectorDimFilter("page", "Striker Eureka", null)), null, false, CompactionEngine.NATIVE @@ -1868,7 +1868,7 @@ private void submitCompactionConfig( Period skipOffsetFromLatest, UserCompactionTaskGranularityConfig granularitySpec, UserCompactionTaskDimensionsConfig dimensionsSpec, - UserCompactionTaskTransformConfig transformSpec, + CompactionTransformSpec transformSpec, AggregatorFactory[] metricsSpec, boolean dropExisting, CompactionEngine engine @@ -1893,7 +1893,7 @@ private void submitCompactionConfig( int maxNumConcurrentSubTasks, UserCompactionTaskGranularityConfig granularitySpec, UserCompactionTaskDimensionsConfig dimensionsSpec, - UserCompactionTaskTransformConfig transformSpec, + CompactionTransformSpec transformSpec, AggregatorFactory[] metricsSpec, boolean dropExisting, CompactionEngine engine diff --git a/server/src/main/java/org/apache/druid/segment/indexing/granularity/ArbitraryGranularitySpec.java b/processing/src/main/java/org/apache/druid/indexer/granularity/ArbitraryGranularitySpec.java similarity index 98% rename from server/src/main/java/org/apache/druid/segment/indexing/granularity/ArbitraryGranularitySpec.java rename to processing/src/main/java/org/apache/druid/indexer/granularity/ArbitraryGranularitySpec.java index 8bf6396bf3ea..ef78c2c798e0 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/granularity/ArbitraryGranularitySpec.java +++ b/processing/src/main/java/org/apache/druid/indexer/granularity/ArbitraryGranularitySpec.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.indexing.granularity; +package org.apache.druid.indexer.granularity; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/org/apache/druid/segment/indexing/granularity/BaseGranularitySpec.java b/processing/src/main/java/org/apache/druid/indexer/granularity/BaseGranularitySpec.java similarity index 93% rename from server/src/main/java/org/apache/druid/segment/indexing/granularity/BaseGranularitySpec.java rename to processing/src/main/java/org/apache/druid/indexer/granularity/BaseGranularitySpec.java index 17185bac7f49..f2ddc9998297 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/granularity/BaseGranularitySpec.java +++ b/processing/src/main/java/org/apache/druid/indexer/granularity/BaseGranularitySpec.java @@ -17,11 +17,9 @@ * under the License. */ -package org.apache.druid.segment.indexing.granularity; +package org.apache.druid.indexer.granularity; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.collect.Iterators; import org.apache.druid.java.util.common.DateTimes; @@ -34,7 +32,6 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.TreeSet; public abstract class BaseGranularitySpec implements GranularitySpec @@ -80,15 +77,6 @@ public TreeSet materializedBucketIntervals() protected abstract LookupIntervalBuckets getLookupTableBuckets(); - @Override - public Map asMap(ObjectMapper objectMapper) - { - return objectMapper.convertValue( - this, - new TypeReference<>() {} - ); - } - /** * This is a helper class to facilitate sharing the code for sortedBucketIntervals among * the various GranularitySpec implementations. In particular, the UniformGranularitySpec diff --git a/server/src/main/java/org/apache/druid/segment/indexing/granularity/GranularitySpec.java b/processing/src/main/java/org/apache/druid/indexer/granularity/GranularitySpec.java similarity index 93% rename from server/src/main/java/org/apache/druid/segment/indexing/granularity/GranularitySpec.java rename to processing/src/main/java/org/apache/druid/indexer/granularity/GranularitySpec.java index 75e9ac12ab1b..f79557bc4dee 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/granularity/GranularitySpec.java +++ b/processing/src/main/java/org/apache/druid/indexer/granularity/GranularitySpec.java @@ -17,18 +17,16 @@ * under the License. */ -package org.apache.druid.segment.indexing.granularity; +package org.apache.druid.indexer.granularity; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import org.apache.druid.java.util.common.granularity.Granularity; import org.joda.time.DateTime; import org.joda.time.Interval; import java.util.List; -import java.util.Map; import java.util.TreeSet; /** @@ -79,6 +77,4 @@ public interface GranularitySpec Granularity getQueryGranularity(); GranularitySpec withIntervals(List inputIntervals); - - Map asMap(ObjectMapper objectMapper); } diff --git a/server/src/main/java/org/apache/druid/segment/indexing/granularity/UniformGranularitySpec.java b/processing/src/main/java/org/apache/druid/indexer/granularity/UniformGranularitySpec.java similarity index 98% rename from server/src/main/java/org/apache/druid/segment/indexing/granularity/UniformGranularitySpec.java rename to processing/src/main/java/org/apache/druid/indexer/granularity/UniformGranularitySpec.java index a68d51725f37..ef8fa2073a02 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/granularity/UniformGranularitySpec.java +++ b/processing/src/main/java/org/apache/druid/indexer/granularity/UniformGranularitySpec.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.indexing.granularity; +package org.apache.druid.indexer.granularity; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java index cf9527f43342..b111975b3945 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java @@ -22,8 +22,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.segment.column.StringEncodingStrategy; import org.apache.druid.segment.data.BitmapSerde; import org.apache.druid.segment.data.BitmapSerdeFactory; @@ -32,7 +30,6 @@ import org.apache.druid.segment.loading.SegmentizerFactory; import javax.annotation.Nullable; -import java.util.Map; import java.util.Objects; /** @@ -165,14 +162,6 @@ public CompressionStrategy getJsonCompression() return jsonCompression; } - public Map asMap(ObjectMapper objectMapper) - { - return objectMapper.convertValue( - this, - new TypeReference<>() {} - ); - } - @Override public boolean equals(Object o) { diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskTransformSpec.java b/processing/src/main/java/org/apache/druid/segment/transform/CompactionTransformSpec.java similarity index 75% rename from server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskTransformSpec.java rename to processing/src/main/java/org/apache/druid/segment/transform/CompactionTransformSpec.java index c8e3281956ac..ac62a6f4e43e 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskTransformSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/CompactionTransformSpec.java @@ -17,31 +17,42 @@ * under the License. */ -package org.apache.druid.client.indexing; +package org.apache.druid.segment.transform; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.query.filter.DimFilter; import javax.annotation.Nullable; -import java.util.Map; import java.util.Objects; /** * Spec containing transform configs for Compaction Task. * This class mimics JSON field names for fields supported in compaction task with - * the corresponding fields in {@link org.apache.druid.segment.transform.TransformSpec}. + * the corresponding fields in {@link TransformSpec}, but omits actual transforms since compaction may only apply + * filtering transforms. * This is done for end-user ease of use. Basically, end-user will use the same syntax / JSON structure to set * transform configs for Compaction task as they would for any other ingestion task. */ -public class ClientCompactionTaskTransformSpec +public class CompactionTransformSpec { + @Nullable + public static CompactionTransformSpec of(@Nullable TransformSpec transformSpec) + { + if (transformSpec == null) { + return null; + } + if (TransformSpec.NONE.equals(transformSpec)) { + return null; + } + + return new CompactionTransformSpec(transformSpec.getFilter()); + } + @Nullable private final DimFilter filter; @JsonCreator - public ClientCompactionTaskTransformSpec( + public CompactionTransformSpec( @JsonProperty("filter") final DimFilter filter ) { @@ -55,14 +66,6 @@ public DimFilter getFilter() return filter; } - public Map asMap(ObjectMapper objectMapper) - { - return objectMapper.convertValue( - this, - new TypeReference<>() {} - ); - } - @Override public boolean equals(Object o) { @@ -72,7 +75,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - ClientCompactionTaskTransformSpec that = (ClientCompactionTaskTransformSpec) o; + CompactionTransformSpec that = (CompactionTransformSpec) o; return Objects.equals(filter, that.filter); } @@ -85,7 +88,7 @@ public int hashCode() @Override public String toString() { - return "ClientCompactionTaskTransformSpec{" + + return "CompactionTransformSpec{" + "filter=" + filter + '}'; } diff --git a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java index 2c6e0d96c397..e36952249f0d 100644 --- a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java +++ b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java @@ -22,10 +22,13 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.transform.CompactionTransformSpec; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.function.Function; @@ -46,27 +49,19 @@ public class CompactionState { private final PartitionsSpec partitionsSpec; private final DimensionsSpec dimensionsSpec; - // org.apache.druid.segment.transform.TransformSpec cannot be used here because it's in the 'processing' module which - // has a dependency on the 'core' module where this class is. - private final Map transformSpec; - // org.apache.druid.segment.IndexSpec cannot be used here because it's in the 'processing' module which - // has a dependency on the 'core' module where this class is. - private final Map indexSpec; - // org.apache.druid.segment.indexing.granularity.GranularitySpec cannot be used here because it's in the - // 'server' module which has a dependency on the 'core' module where this class is. - private final Map granularitySpec; - // org.apache.druid.query.aggregation.AggregatorFactory cannot be used here because it's in the 'processing' module which - // has a dependency on the 'core' module where this class is. - private final List metricsSpec; + private final CompactionTransformSpec transformSpec; + private final IndexSpec indexSpec; + private final GranularitySpec granularitySpec; + private final List metricsSpec; @JsonCreator public CompactionState( @JsonProperty("partitionsSpec") PartitionsSpec partitionsSpec, @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec, - @JsonProperty("metricsSpec") List metricsSpec, - @JsonProperty("transformSpec") Map transformSpec, - @JsonProperty("indexSpec") Map indexSpec, - @JsonProperty("granularitySpec") Map granularitySpec + @JsonProperty("metricsSpec") List metricsSpec, + @JsonProperty("transformSpec") CompactionTransformSpec transformSpec, + @JsonProperty("indexSpec") IndexSpec indexSpec, + @JsonProperty("granularitySpec") GranularitySpec granularitySpec ) { this.partitionsSpec = partitionsSpec; @@ -90,25 +85,25 @@ public DimensionsSpec getDimensionsSpec() } @JsonProperty - public List getMetricsSpec() + public List getMetricsSpec() { return metricsSpec; } @JsonProperty - public Map getTransformSpec() + public CompactionTransformSpec getTransformSpec() { return transformSpec; } @JsonProperty - public Map getIndexSpec() + public IndexSpec getIndexSpec() { return indexSpec; } @JsonProperty - public Map getGranularitySpec() + public GranularitySpec getGranularitySpec() { return granularitySpec; } @@ -153,10 +148,10 @@ public String toString() public static Function, Set> addCompactionStateToSegments( PartitionsSpec partitionsSpec, DimensionsSpec dimensionsSpec, - List metricsSpec, - Map transformSpec, - Map indexSpec, - Map granularitySpec + List metricsSpec, + CompactionTransformSpec transformSpec, + IndexSpec indexSpec, + GranularitySpec granularitySpec ) { CompactionState compactionState = new CompactionState( @@ -173,5 +168,4 @@ public static Function, Set> addCompactionStateToS .map(s -> s.withLastCompactionState(compactionState)) .collect(Collectors.toSet()); } - } diff --git a/server/src/test/java/org/apache/druid/segment/indexing/granularity/ArbitraryGranularityTest.java b/processing/src/test/java/org/apache/druid/indexer/granularity/ArbitraryGranularityTest.java similarity index 87% rename from server/src/test/java/org/apache/druid/segment/indexing/granularity/ArbitraryGranularityTest.java rename to processing/src/test/java/org/apache/druid/indexer/granularity/ArbitraryGranularityTest.java index 25bf848d8b1c..470ecb6a5408 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/granularity/ArbitraryGranularityTest.java +++ b/processing/src/test/java/org/apache/druid/indexer/granularity/ArbitraryGranularityTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.indexing.granularity; +package org.apache.druid.indexer.granularity; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; @@ -32,7 +32,6 @@ import org.junit.Test; import java.util.List; -import java.util.Map; public class ArbitraryGranularityTest { @@ -218,32 +217,6 @@ public void testJson() } } - @Test - public void testAsMap() - { - final GranularitySpec spec = new ArbitraryGranularitySpec(Granularities.NONE, Lists.newArrayList( - Intervals.of("2012-01-08T00Z/2012-01-11T00Z"), - Intervals.of("2012-02-01T00Z/2012-03-01T00Z"), - Intervals.of("2012-01-07T00Z/2012-01-08T00Z"), - Intervals.of("2012-01-03T00Z/2012-01-04T00Z"), - Intervals.of("2012-01-01T00Z/2012-01-03T00Z") - )); - - Map map = spec.asMap(JSON_MAPPER); - final GranularitySpec rtSpec = JSON_MAPPER.convertValue(map, GranularitySpec.class); - Assert.assertEquals( - "Round-trip", - ImmutableList.copyOf(spec.sortedBucketIntervals()), - ImmutableList.copyOf(rtSpec.sortedBucketIntervals()) - ); - Assert.assertEquals( - "Round-trip", - ImmutableList.copyOf(spec.inputIntervals()), - ImmutableList.copyOf(rtSpec.inputIntervals()) - ); - Assert.assertEquals(spec, rtSpec); - } - @Test public void testNullInputIntervals() { diff --git a/server/src/test/java/org/apache/druid/segment/indexing/granularity/UniformGranularityTest.java b/processing/src/test/java/org/apache/druid/indexer/granularity/UniformGranularityTest.java similarity index 91% rename from server/src/test/java/org/apache/druid/segment/indexing/granularity/UniformGranularityTest.java rename to processing/src/test/java/org/apache/druid/indexer/granularity/UniformGranularityTest.java index 5ec957747295..e74751773f27 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/granularity/UniformGranularityTest.java +++ b/processing/src/test/java/org/apache/druid/indexer/granularity/UniformGranularityTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.indexing.granularity; +package org.apache.druid.indexer.granularity; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; @@ -39,7 +39,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Map; public class UniformGranularityTest { @@ -180,34 +179,6 @@ public void testJson() } } - @Test - public void testAsMap() - { - final GranularitySpec spec = new UniformGranularitySpec( - Granularities.DAY, - null, - Lists.newArrayList( - Intervals.of("2012-01-08T00Z/2012-01-11T00Z"), - Intervals.of("2012-01-07T00Z/2012-01-08T00Z"), - Intervals.of("2012-01-03T00Z/2012-01-04T00Z"), - Intervals.of("2012-01-01T00Z/2012-01-03T00Z") - ) - ); - Map map = spec.asMap(JSON_MAPPER); - final GranularitySpec rtSpec = JSON_MAPPER.convertValue(map, GranularitySpec.class); - Assert.assertEquals( - "Round-trip sortedBucketIntervals", - ImmutableList.copyOf(spec.sortedBucketIntervals()), - ImmutableList.copyOf(rtSpec.sortedBucketIntervals().iterator()) - ); - Assert.assertEquals( - "Round-trip granularity", - spec.getSegmentGranularity(), - rtSpec.getSegmentGranularity() - ); - Assert.assertEquals(spec, rtSpec); - } - @Test public void testEquals() { diff --git a/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java b/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java index 261281cec0b1..960f7fb96d54 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionFactory.LongEncodingStrategy; import org.apache.druid.segment.data.CompressionStrategy; @@ -30,8 +29,6 @@ import org.junit.Assert; import org.junit.Test; -import java.util.Map; - public class IndexSpecTest { @Test @@ -72,30 +69,6 @@ public void testDefaults() Assert.assertEquals(LongEncodingStrategy.LONGS, spec.getLongEncoding()); } - @Test - public void testAsMap() - { - final ObjectMapper objectMapper = new DefaultObjectMapper(); - final IndexSpec spec = IndexSpec.DEFAULT; - final Map map = spec.asMap(objectMapper); - Assert.assertEquals( - spec.getBitmapSerdeFactory(), - objectMapper.convertValue(map.get("bitmap"), BitmapSerdeFactory.class) - ); - Assert.assertEquals( - spec.getDimensionCompression(), - objectMapper.convertValue(map.get("dimensionCompression"), CompressionStrategy.class) - ); - Assert.assertEquals( - spec.getMetricCompression(), - objectMapper.convertValue(map.get("metricCompression"), CompressionStrategy.class) - ); - Assert.assertEquals( - spec.getLongEncoding(), - objectMapper.convertValue(map.get("longEncoding"), LongEncodingStrategy.class) - ); - } - @Test public void testEquals() { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/UserCompactionTaskTransformConfigTest.java b/processing/src/test/java/org/apache/druid/segment/transform/CompactionTransformSpecTest.java similarity index 78% rename from server/src/test/java/org/apache/druid/server/coordinator/UserCompactionTaskTransformConfigTest.java rename to processing/src/test/java/org/apache/druid/segment/transform/CompactionTransformSpecTest.java index 5bf08abe6e75..1d7a09d245f4 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/UserCompactionTaskTransformConfigTest.java +++ b/processing/src/test/java/org/apache/druid/segment/transform/CompactionTransformSpecTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator; +package org.apache.druid.segment.transform; import com.fasterxml.jackson.databind.ObjectMapper; import nl.jqno.equalsverifier.EqualsVerifier; @@ -28,12 +28,12 @@ import java.io.IOException; -public class UserCompactionTaskTransformConfigTest +public class CompactionTransformSpecTest { @Test public void testEquals() { - EqualsVerifier.forClass(UserCompactionTaskTransformConfig.class) + EqualsVerifier.forClass(CompactionTransformSpec.class) .withNonnullFields("filter") .usingGetClass() .verify(); @@ -42,14 +42,14 @@ public void testEquals() @Test public void testSerde() throws IOException { - final UserCompactionTaskTransformConfig expected = new UserCompactionTaskTransformConfig( + final CompactionTransformSpec expected = new CompactionTransformSpec( new SelectorDimFilter("dim1", "foo", null) ); final ObjectMapper mapper = new DefaultObjectMapper(); final byte[] json = mapper.writeValueAsBytes(expected); - final UserCompactionTaskTransformConfig fromJson = (UserCompactionTaskTransformConfig) mapper.readValue( + final CompactionTransformSpec fromJson = (CompactionTransformSpec) mapper.readValue( json, - UserCompactionTaskTransformConfig.class + CompactionTransformSpec.class ); Assert.assertEquals(expected, fromJson); } diff --git a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java index 3f0667b870c9..2bd5de0fe622 100644 --- a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -26,12 +26,18 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.RangeSet; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; @@ -45,7 +51,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -129,10 +134,10 @@ public void testV1Serialization() throws Exception new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "bar", "foo")) ), - ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), - ImmutableMap.of("filter", ImmutableMap.of("type", "selector", "dimension", "dim1", "value", "foo")), - ImmutableMap.of(), - ImmutableMap.of() + ImmutableList.of(new CountAggregatorFactory("count")), + new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null)), + MAPPER.convertValue(ImmutableMap.of(), IndexSpec.class), + MAPPER.convertValue(ImmutableMap.of(), GranularitySpec.class) ), TEST_VERSION, 1 @@ -196,8 +201,8 @@ public void testDeserializationDataSegmentLastCompactionStateWithNullSpecs() thr null, null, null, - ImmutableMap.of(), - ImmutableMap.of() + MAPPER.convertValue(ImmutableMap.of(), IndexSpec.class), + MAPPER.convertValue(ImmutableMap.of(), GranularitySpec.class) ), TEST_VERSION, 1 @@ -345,10 +350,10 @@ public void testWithLastCompactionState() final CompactionState compactionState = new CompactionState( new DynamicPartitionsSpec(null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))), - ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), - ImmutableMap.of("filter", ImmutableMap.of("type", "selector", "dimension", "dim1", "value", "foo")), - Collections.singletonMap("test", "map"), - Collections.singletonMap("test2", "map2") + ImmutableList.of(new CountAggregatorFactory("count")), + new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null)), + MAPPER.convertValue(Map.of("test", "map"), IndexSpec.class), + MAPPER.convertValue(Map.of("test2", "map2"), GranularitySpec.class) ); final DataSegment segment1 = DataSegment.builder() .dataSource("foo") @@ -376,13 +381,12 @@ public void testAnnotateWithLastCompactionState() "bar", "foo" ))); - List metricsSpec = ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")); - Map transformSpec = ImmutableMap.of( - "filter", - ImmutableMap.of("type", "selector", "dimension", "dim1", "value", "foo") + List metricsSpec = ImmutableList.of(new CountAggregatorFactory("count")); + CompactionTransformSpec transformSpec = new CompactionTransformSpec( + new SelectorDimFilter("dim1", "foo", null) ); - Map indexSpec = Collections.singletonMap("test", "map"); - Map granularitySpec = Collections.singletonMap("test2", "map"); + IndexSpec indexSpec = MAPPER.convertValue(Map.of("test", "map"), IndexSpec.class); + GranularitySpec granularitySpec = MAPPER.convertValue(Map.of("test2", "map"), GranularitySpec.class); final CompactionState compactionState = new CompactionState( dynamicPartitionsSpec, @@ -430,7 +434,7 @@ public void testTombstoneType() .interval(Intervals.of("2012-01-01/2012-01-02")) .version(DateTimes.of("2012-01-01T11:22:33.444Z").toString()) .shardSpec(new TombstoneShardSpec()) - .loadSpec(Collections.singletonMap( + .loadSpec(Map.of( "type", DataSegment.TOMBSTONE_LOADSPEC_TYPE )) @@ -444,7 +448,7 @@ public void testTombstoneType() .interval(Intervals.of("2012-01-01/2012-01-02")) .version(DateTimes.of("2012-01-01T11:22:33.444Z").toString()) .shardSpec(getShardSpec(7)) - .loadSpec(Collections.singletonMap( + .loadSpec(Map.of( "type", "foo" )) diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskGranularitySpec.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskGranularitySpec.java index 27d5b21f43dc..f5be9706caea 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskGranularitySpec.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskGranularitySpec.java @@ -21,8 +21,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import javax.annotation.Nullable; import java.util.Objects; diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java index d8efe4397ac5..a3a4a6653a09 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.transform.CompactionTransformSpec; import javax.annotation.Nullable; import java.util.Arrays; @@ -44,7 +45,7 @@ public class ClientCompactionTaskQuery implements ClientTaskQuery private final ClientCompactionTaskGranularitySpec granularitySpec; private final ClientCompactionTaskDimensionsSpec dimensionsSpec; private final AggregatorFactory[] metricsSpec; - private final ClientCompactionTaskTransformSpec transformSpec; + private final CompactionTransformSpec transformSpec; private final Map context; private final ClientCompactionRunnerInfo compactionRunner; @@ -57,7 +58,7 @@ public ClientCompactionTaskQuery( @JsonProperty("granularitySpec") ClientCompactionTaskGranularitySpec granularitySpec, @JsonProperty("dimensionsSpec") ClientCompactionTaskDimensionsSpec dimensionsSpec, @JsonProperty("metricsSpec") AggregatorFactory[] metrics, - @JsonProperty("transformSpec") ClientCompactionTaskTransformSpec transformSpec, + @JsonProperty("transformSpec") CompactionTransformSpec transformSpec, @JsonProperty("context") Map context, @JsonProperty("compactionRunner") @Nullable ClientCompactionRunnerInfo compactionRunner ) @@ -127,7 +128,7 @@ public AggregatorFactory[] getMetricsSpec() } @JsonProperty - public ClientCompactionTaskTransformSpec getTransformSpec() + public CompactionTransformSpec getTransformSpec() { return transformSpec; } diff --git a/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java index 2e4cf91252fe..c51f154e4edd 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java @@ -21,8 +21,8 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.transform.TransformSpec; import javax.annotation.Nullable; diff --git a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java index 76a8c16802cb..6c35cb8d3919 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java @@ -37,14 +37,14 @@ import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.error.DruidException; +import org.apache.druid.indexer.granularity.GranularitySpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.transform.TransformSpec; import javax.annotation.Nullable; diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java index fd53ed38c257..77786993ac99 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java @@ -23,7 +23,6 @@ import org.apache.commons.lang3.ArrayUtils; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; -import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.common.config.Configs; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; @@ -35,6 +34,7 @@ import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.timeline.CompactionState; @@ -443,10 +443,10 @@ private CompactionStatus metricsSpecIsUpToDate() return COMPLETE; } - final List metricSpecList = lastCompactionState.getMetricsSpec(); + final List metricSpecList = lastCompactionState.getMetricsSpec(); final AggregatorFactory[] existingMetricsSpec = CollectionUtils.isNullOrEmpty(metricSpecList) - ? null : objectMapper.convertValue(metricSpecList, AggregatorFactory[].class); + ? null : metricSpecList.toArray(new AggregatorFactory[0]); if (existingMetricsSpec == null || !Arrays.deepEquals(configuredMetricsSpec, existingMetricsSpec)) { return CompactionStatus.configChanged( @@ -466,9 +466,9 @@ private CompactionStatus transformSpecFilterIsUpToDate() return COMPLETE; } - ClientCompactionTaskTransformSpec existingTransformSpec = convertIfNotNull( + CompactionTransformSpec existingTransformSpec = convertIfNotNull( lastCompactionState.getTransformSpec(), - ClientCompactionTaskTransformSpec.class + CompactionTransformSpec.class ); return CompactionStatus.completeIfEqual( "transformSpec filter", diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index b4b2be780328..9a9eac5ddb13 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -26,6 +26,7 @@ import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.joda.time.Period; import javax.annotation.Nullable; @@ -61,7 +62,7 @@ public static Builder builder() private final UserCompactionTaskGranularityConfig granularitySpec; private final UserCompactionTaskDimensionsConfig dimensionsSpec; private final AggregatorFactory[] metricsSpec; - private final UserCompactionTaskTransformConfig transformSpec; + private final CompactionTransformSpec transformSpec; private final UserCompactionTaskIOConfig ioConfig; private final Map taskContext; private final CompactionEngine engine; @@ -77,7 +78,7 @@ public DataSourceCompactionConfig( @JsonProperty("granularitySpec") @Nullable UserCompactionTaskGranularityConfig granularitySpec, @JsonProperty("dimensionsSpec") @Nullable UserCompactionTaskDimensionsConfig dimensionsSpec, @JsonProperty("metricsSpec") @Nullable AggregatorFactory[] metricsSpec, - @JsonProperty("transformSpec") @Nullable UserCompactionTaskTransformConfig transformSpec, + @JsonProperty("transformSpec") @Nullable CompactionTransformSpec transformSpec, @JsonProperty("ioConfig") @Nullable UserCompactionTaskIOConfig ioConfig, @JsonProperty("engine") @Nullable CompactionEngine engine, @JsonProperty("taskContext") @Nullable Map taskContext @@ -164,7 +165,7 @@ public UserCompactionTaskDimensionsConfig getDimensionsSpec() @JsonProperty @Nullable - public UserCompactionTaskTransformConfig getTransformSpec() + public CompactionTransformSpec getTransformSpec() { return transformSpec; } @@ -254,7 +255,7 @@ public static class Builder private UserCompactionTaskGranularityConfig granularitySpec; private UserCompactionTaskDimensionsConfig dimensionsSpec; private AggregatorFactory[] metricsSpec; - private UserCompactionTaskTransformConfig transformSpec; + private CompactionTransformSpec transformSpec; private UserCompactionTaskIOConfig ioConfig; private CompactionEngine engine; private Map taskContext; @@ -340,7 +341,7 @@ public Builder withMetricsSpec(AggregatorFactory[] metricsSpec) } public Builder withTransformSpec( - UserCompactionTaskTransformConfig transformSpec + CompactionTransformSpec transformSpec ) { this.transformSpec = transformSpec; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java index d6b320a24838..be3c7e1af965 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java @@ -21,8 +21,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import java.util.Objects; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskTransformConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskTransformConfig.java deleted file mode 100644 index 85aabb507c6d..000000000000 --- a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskTransformConfig.java +++ /dev/null @@ -1,81 +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.server.coordinator; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.query.filter.DimFilter; - -import javax.annotation.Nullable; -import java.util.Objects; - -/** - * Spec containing transform configs for Compaction Task. - * This class mimics JSON field names for fields supported in compaction task with - * the corresponding fields in {@link org.apache.druid.segment.transform.TransformSpec}. - * This is done for end-user ease of use. Basically, end-user will use the same syntax / JSON structure to set - * transform configs for Compaction task as they would for any other ingestion task. - */ -public class UserCompactionTaskTransformConfig -{ - @Nullable private final DimFilter filter; - - @JsonCreator - public UserCompactionTaskTransformConfig( - @JsonProperty("filter") final DimFilter filter - ) - { - this.filter = filter; - } - - @JsonProperty - @Nullable - public DimFilter getFilter() - { - return filter; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - UserCompactionTaskTransformConfig that = (UserCompactionTaskTransformConfig) o; - return Objects.equals(filter, that.filter); - } - - @Override - public int hashCode() - { - return Objects.hash(filter); - } - - @Override - public String toString() - { - return "UserCompactionTaskTransformConfig{" + - "filter=" + filter + - '}'; - } -} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index ca80c7efe30e..bb559769d9d4 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -32,7 +32,6 @@ import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; -import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.client.indexing.ClientMSQContext; import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.common.guava.FutureUtils; @@ -48,6 +47,7 @@ import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSegmentIterator; @@ -513,14 +513,6 @@ private int submitCompactionTasks( dimensionsSpec = null; } - // Create transformSpec to send to compaction task - ClientCompactionTaskTransformSpec transformSpec = null; - if (config.getTransformSpec() != null) { - transformSpec = new ClientCompactionTaskTransformSpec( - config.getTransformSpec().getFilter() - ); - } - Boolean dropExisting = null; if (config.getIoConfig() != null) { dropExisting = config.getIoConfig().isDropExisting(); @@ -584,7 +576,7 @@ private int submitCompactionTasks( granularitySpec, dimensionsSpec, config.getMetricsSpec(), - transformSpec, + config.getTransformSpec(), dropExisting, autoCompactionContext, new ClientCompactionRunnerInfo(compactionEngine) @@ -687,7 +679,7 @@ private String compactSegments( ClientCompactionTaskGranularitySpec granularitySpec, @Nullable ClientCompactionTaskDimensionsSpec dimensionsSpec, @Nullable AggregatorFactory[] metricsSpec, - @Nullable ClientCompactionTaskTransformSpec transformSpec, + @Nullable CompactionTransformSpec transformSpec, @Nullable Boolean dropExisting, @Nullable Map context, ClientCompactionRunnerInfo compactionRunner diff --git a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionTaskTransformSpecTest.java b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionTaskTransformSpecTest.java deleted file mode 100644 index 77593a8d1a6b..000000000000 --- a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionTaskTransformSpecTest.java +++ /dev/null @@ -1,74 +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.client.indexing; - -import com.fasterxml.jackson.databind.ObjectMapper; -import nl.jqno.equalsverifier.EqualsVerifier; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.query.filter.SelectorDimFilter; -import org.junit.Assert; -import org.junit.Test; - -import java.io.IOException; -import java.util.Map; - -public class ClientCompactionTaskTransformSpecTest -{ - @Test - public void testEquals() - { - EqualsVerifier.forClass(ClientCompactionTaskTransformSpec.class) - .withNonnullFields("filter") - .usingGetClass() - .verify(); - } - - @Test - public void testSerde() throws IOException - { - final ClientCompactionTaskTransformSpec expected = new ClientCompactionTaskTransformSpec( - new SelectorDimFilter("dim1", "foo", null) - ); - final ObjectMapper mapper = new DefaultObjectMapper(); - final byte[] json = mapper.writeValueAsBytes(expected); - final ClientCompactionTaskTransformSpec fromJson = (ClientCompactionTaskTransformSpec) mapper.readValue( - json, - ClientCompactionTaskTransformSpec.class - ); - Assert.assertEquals(expected, fromJson); - } - - @SuppressWarnings("unchecked") - @Test - public void testAsMap() - { - final ObjectMapper objectMapper = new DefaultObjectMapper(); - String dimension = "dim1"; - String value = "foo"; - final ClientCompactionTaskTransformSpec spec = new ClientCompactionTaskTransformSpec(new SelectorDimFilter(dimension, value, null)); - final Map map = spec.asMap(objectMapper); - Assert.assertNotNull(map); - Assert.assertEquals(3, ((Map) map.get("filter")).size()); - Assert.assertEquals(dimension, ((Map) map.get("filter")).get("dimension")); - Assert.assertEquals(value, ((Map) map.get("filter")).get("value")); - ClientCompactionTaskTransformSpec actual = objectMapper.convertValue(map, ClientCompactionTaskTransformSpec.class); - Assert.assertEquals(spec, actual); - } -} diff --git a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index 94bd77e810df..a4e4579c7d7b 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java @@ -36,6 +36,8 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; +import org.apache.druid.indexer.granularity.ArbitraryGranularitySpec; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -47,8 +49,6 @@ import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.testing.InitializedNullHandlingTest; diff --git a/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java b/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java index 326ad6dbed7c..bae5140392be 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java @@ -25,8 +25,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.transform.TransformSpec; import javax.annotation.Nullable; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java index 5f43236e075b..6b19aaca3ca1 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java @@ -24,6 +24,7 @@ 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.granularity.UniformGranularitySpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.granularity.Granularities; @@ -42,7 +43,6 @@ 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.SegmentGenerationMetrics; 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 88cd05376aef..6f9bf400b0fd 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 @@ -31,6 +31,7 @@ import org.apache.druid.data.input.impl.MapInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.BuiltInTypesModule; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.jackson.AggregatorsModule; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.FileUtils; @@ -66,7 +67,6 @@ 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.loading.SegmentLoaderConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; 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 66ac774e1e9e..e92df0f65183 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 @@ -26,6 +26,7 @@ import org.apache.druid.client.cache.MapCache; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -45,7 +46,6 @@ import org.apache.druid.segment.incremental.NoopRowIngestionMeters; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.indexing.DataSchema; -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; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java b/server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java index 750ea06c6534..510901cd7353 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java @@ -30,6 +30,7 @@ import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -43,7 +44,6 @@ import org.apache.druid.segment.incremental.IndexSizeExceededException; 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.realtime.FireHydrant; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java index 41d789734cae..0b8c04a552c4 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java @@ -21,6 +21,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; +import org.apache.druid.indexer.granularity.GranularitySpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; @@ -31,8 +33,6 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.data.CompressionStrategy; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; @@ -237,7 +237,7 @@ public void testStatusOnIndexSpecMismatch() null, null, null, - currentIndexSpec.asMap(OBJECT_MAPPER), + currentIndexSpec, null ); final DataSourceCompactionConfig compactionConfig = DataSourceCompactionConfig @@ -275,8 +275,8 @@ public void testStatusOnSegmentGranularityMismatch() null, null, null, - currentIndexSpec.asMap(OBJECT_MAPPER), - currentGranularitySpec.asMap(OBJECT_MAPPER) + currentIndexSpec, + currentGranularitySpec ); final DataSourceCompactionConfig compactionConfig = DataSourceCompactionConfig .builder() @@ -305,8 +305,8 @@ public void testStatusWhenLastCompactionStateSameAsRequired() null, null, null, - currentIndexSpec.asMap(OBJECT_MAPPER), - currentGranularitySpec.asMap(OBJECT_MAPPER) + currentIndexSpec, + currentGranularitySpec ); final DataSourceCompactionConfig compactionConfig = DataSourceCompactionConfig .builder() diff --git a/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java index 73fdd28fb239..4aee0457bed5 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java @@ -19,7 +19,6 @@ package org.apache.druid.server.compaction; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; @@ -27,6 +26,8 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.granularity.GranularitySpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; @@ -46,13 +47,12 @@ import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.data.ConciseBitmapSerdeFactory; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; -import org.apache.druid.segment.transform.TransformSpec; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskDimensionsConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; -import org.apache.druid.server.coordinator.UserCompactionTaskTransformConfig; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.Partitions; @@ -72,7 +72,6 @@ import java.util.Comparator; import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.TimeZone; import java.util.stream.Collectors; @@ -476,7 +475,10 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularitySmaller() // We should only get segments in Oct final List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-14T00:00:00/2017-10-15T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-14T00:00:00/2017-10-15T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertTrue(iterator.hasNext()); @@ -604,7 +606,10 @@ public void testIteratorReturnsSegmentsInConfiguredSegmentGranularity() // Month of Dec Assert.assertTrue(iterator.hasNext()); List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-12-01T00:00:00/2017-12-31T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-12-01T00:00:00/2017-12-31T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -613,7 +618,10 @@ public void testIteratorReturnsSegmentsInConfiguredSegmentGranularity() // Month of Nov Assert.assertTrue(iterator.hasNext()); expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-11-01T00:00:00/2017-12-01T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-11-01T00:00:00/2017-12-01T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -622,7 +630,10 @@ public void testIteratorReturnsSegmentsInConfiguredSegmentGranularity() // Month of Oct Assert.assertTrue(iterator.hasNext()); expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-01T00:00:00/2017-11-01T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-01T00:00:00/2017-11-01T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -681,7 +692,10 @@ public void testIteratorDoesNotReturnCompactedInterval() ); final List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-12-01T00:00:00/2017-12-02T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-12-01T00:00:00/2017-12-02T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertTrue(iterator.hasNext()); Assert.assertEquals( @@ -710,7 +724,10 @@ public void testIteratorReturnsAllMixedVersionSegmentsInConfiguredSegmentGranula // We should get all segments in timeline back since skip offset is P0D. Assert.assertTrue(iterator.hasNext()); List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -724,9 +741,10 @@ public void testIteratorReturnsAllMixedVersionSegmentsInConfiguredSegmentGranula public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGranularityAndSameTimezone() { // Same indexSpec as what is set in the auto compaction config - Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); + IndexSpec indexSpec = IndexSpec.DEFAULT; // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( + null)); // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final SegmentTimeline timeline = createTimeline( @@ -752,13 +770,21 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGranularityInLastCompactionState() { // Same indexSpec as what is set in the auto compaction config - Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); + IndexSpec indexSpec = IndexSpec.DEFAULT; // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( + null)); // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final CompactionState compactionState - = new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day")); + = new CompactionState( + partitionsSpec, + null, + null, + null, + indexSpec, + mapper.convertValue(ImmutableMap.of("segmentGranularity", "day"), GranularitySpec.class) + ); final SegmentTimeline timeline = createTimeline( createSegments() .forIntervals(2, Granularities.DAY) @@ -779,9 +805,10 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSegmentGranularity() { // Same indexSpec as what is set in the auto compaction config - Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); + IndexSpec indexSpec = IndexSpec.DEFAULT; // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( + null)); // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final CompactionState compactionState = new CompactionState(partitionsSpec, null, null, null, indexSpec, null); @@ -801,7 +828,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg // We should get all segments in timeline back since skip offset is P0D. Assert.assertTrue(iterator.hasNext()); List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-01T00:00:00/2017-10-03T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-01T00:00:00/2017-10-03T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -815,13 +845,21 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSegmentGranularityInLastCompactionState() { // Same indexSpec as what is set in the auto compaction config - Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); + IndexSpec indexSpec = IndexSpec.DEFAULT; // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( + null)); // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final CompactionState compactionState - = new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day")); + = new CompactionState( + partitionsSpec, + null, + null, + null, + indexSpec, + mapper.convertValue(ImmutableMap.of("segmentGranularity", "day"), GranularitySpec.class) + ); final SegmentTimeline timeline = createTimeline( createSegments() .forIntervals(2, Granularities.DAY) @@ -838,7 +876,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg // We should get all segments in timeline back since skip offset is P0D. Assert.assertTrue(iterator.hasNext()); List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-01T00:00:00/2017-10-03T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-01T00:00:00/2017-10-03T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -852,9 +893,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentTimezone() { // Same indexSpec as what is set in the auto compaction config - Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); + IndexSpec indexSpec = IndexSpec.DEFAULT; // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( + null)); // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final SegmentTimeline timeline = createTimeline( @@ -884,7 +926,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentTim // We should get all segments in timeline back since skip offset is P0D. Assert.assertTrue(iterator.hasNext()); List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-01T00:00:00/2017-10-03T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-01T00:00:00/2017-10-03T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -898,9 +943,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentTim public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentOrigin() { // Same indexSpec as what is set in the auto compaction config - Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); + IndexSpec indexSpec = IndexSpec.DEFAULT; // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( + null)); // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final SegmentTimeline timeline = createTimeline( @@ -929,7 +975,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentOri // We should get all segments in timeline back since skip offset is P0D. Assert.assertTrue(iterator.hasNext()); List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-01T00:00:00/2017-10-03T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-01T00:00:00/2017-10-03T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -943,9 +992,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentOri public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentRollup() { // Same indexSpec as what is set in the auto compaction config - Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); + IndexSpec indexSpec = IndexSpec.DEFAULT; // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( + null)); // Create segments that were compacted (CompactionState != null) and have // rollup=false for interval 2017-10-01T00:00:00/2017-10-02T00:00:00, @@ -956,17 +1006,38 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentRol .forIntervals(1, Granularities.DAY) .startingAt("2017-10-01") .withNumPartitions(4) - .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("rollup", "false"))), + .withCompactionState(new CompactionState( + partitionsSpec, + null, + null, + null, + indexSpec, + new UniformGranularitySpec(null, null, false, null) + )), createSegments() .forIntervals(1, Granularities.DAY) .startingAt("2017-10-02") .withNumPartitions(4) - .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("rollup", "true"))), + .withCompactionState(new CompactionState( + partitionsSpec, + null, + null, + null, + indexSpec, + new UniformGranularitySpec(null, null, true, null) + )), createSegments() .forIntervals(1, Granularities.DAY) .startingAt("2017-10-03") .withNumPartitions(4) - .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of())) + .withCompactionState(new CompactionState( + partitionsSpec, + null, + null, + null, + indexSpec, + new UniformGranularitySpec(null, null, false, null) + )) ); // Auto compaction config sets rollup=true @@ -979,7 +1050,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentRol // We should get interval 2017-10-01T00:00:00/2017-10-02T00:00:00 and interval 2017-10-03T00:00:00/2017-10-04T00:00:00. Assert.assertTrue(iterator.hasNext()); List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -987,7 +1061,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentRol ); Assert.assertTrue(iterator.hasNext()); expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -1001,9 +1078,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentRol public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentQueryGranularity() { // Same indexSpec as what is set in the auto compaction config - Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); + IndexSpec indexSpec = IndexSpec.DEFAULT; // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( + null)); // Create segments that were compacted (CompactionState != null) and have // queryGranularity=DAY for interval 2017-10-01T00:00:00/2017-10-02T00:00:00, @@ -1014,17 +1092,44 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentQue .forIntervals(1, Granularities.DAY) .startingAt("2017-10-01") .withNumPartitions(4) - .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("queryGranularity", "day"))), + .withCompactionState(new CompactionState( + partitionsSpec, + null, + null, + null, + indexSpec, + mapper.convertValue( + ImmutableMap.of("queryGranularity", "day"), + GranularitySpec.class + ) + )), createSegments() .forIntervals(1, Granularities.DAY) .startingAt("2017-10-02") .withNumPartitions(4) - .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("queryGranularity", "minute"))), + .withCompactionState(new CompactionState( + partitionsSpec, + null, + null, + null, + indexSpec, + mapper.convertValue( + ImmutableMap.of("queryGranularity", "minute"), + GranularitySpec.class + ) + )), createSegments() .forIntervals(1, Granularities.DAY) .startingAt("2017-10-03") .withNumPartitions(4) - .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of())) + .withCompactionState(new CompactionState( + partitionsSpec, + null, + null, + null, + indexSpec, + mapper.convertValue(ImmutableMap.of(), GranularitySpec.class) + )) ); // Auto compaction config sets queryGranularity=MINUTE @@ -1037,7 +1142,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentQue // We should get interval 2017-10-01T00:00:00/2017-10-02T00:00:00 and interval 2017-10-03T00:00:00/2017-10-04T00:00:00. Assert.assertTrue(iterator.hasNext()); List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -1045,7 +1153,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentQue ); Assert.assertTrue(iterator.hasNext()); expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -1059,9 +1170,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentQue public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDimensions() { // Same indexSpec as what is set in the auto compaction config - Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); + IndexSpec indexSpec = IndexSpec.DEFAULT; // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( + null)); // Create segments that were compacted (CompactionState != null) and have // Dimensions=["foo", "bar"] for interval 2017-10-01T00:00:00/2017-10-02T00:00:00, @@ -1073,18 +1185,42 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim .startingAt("2017-10-01") .withNumPartitions(4) .withCompactionState( - new CompactionState(partitionsSpec, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))), null, null, indexSpec, null) + new CompactionState( + partitionsSpec, + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of( + "bar", + "foo" + ))), + null, + null, + indexSpec, + null + ) ), createSegments() .startingAt("2017-10-02") .withNumPartitions(4) .withCompactionState( - new CompactionState(partitionsSpec, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))), null, null, indexSpec, null) + new CompactionState( + partitionsSpec, + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))), + null, + null, + indexSpec, + null + ) ), createSegments() .startingAt("2017-10-03") .withNumPartitions(4) - .withCompactionState(new CompactionState(partitionsSpec, DimensionsSpec.EMPTY, null, null, indexSpec, null)), + .withCompactionState(new CompactionState( + partitionsSpec, + DimensionsSpec.EMPTY, + null, + null, + indexSpec, + null + )), createSegments() .startingAt("2017-10-04") .withNumPartitions(4) @@ -1101,7 +1237,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim // We should get interval 2017-10-01T00:00:00/2017-10-02T00:00:00, interval 2017-10-04T00:00:00/2017-10-05T00:00:00, and interval 2017-10-03T00:00:00/2017-10-04T00:00:00. Assert.assertTrue(iterator.hasNext()); List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -1109,7 +1248,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim ); Assert.assertTrue(iterator.hasNext()); expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -1117,7 +1259,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim ); Assert.assertTrue(iterator.hasNext()); expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -1141,7 +1286,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim public void testIteratorDoesNotReturnsSegmentsWhenPartitionDimensionsPrefixed() { // Same indexSpec as what is set in the auto compaction config - Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); + IndexSpec indexSpec = IndexSpec.DEFAULT; // Set range partitions spec with dimensions ["dim2", "dim4"] -- the same as what is set in the auto compaction config PartitionsSpec partitionsSpec = new DimensionRangePartitionsSpec( null, @@ -1158,20 +1303,49 @@ public void testIteratorDoesNotReturnsSegmentsWhenPartitionDimensionsPrefixed() .startingAt("2017-10-01") .withNumPartitions(4) .withCompactionState( - new CompactionState(partitionsSpec, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim2", "dim4", "dim3", "dim1"))), null, null, indexSpec, null) + new CompactionState( + partitionsSpec, + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of( + "dim2", + "dim4", + "dim3", + "dim1" + ))), + null, + null, + indexSpec, + null + ) ), createSegments() .startingAt("2017-10-02") .withNumPartitions(4) .withCompactionState( - new CompactionState(partitionsSpec, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim2", "dim4", "dim1", "dim3"))), null, null, indexSpec, null) + new CompactionState( + partitionsSpec, + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of( + "dim2", + "dim4", + "dim1", + "dim3" + ))), + null, + null, + indexSpec, + null + ) ) ); // Auto compaction config sets Dimensions=["dim1", "dim2", "dim3", "dim4"] and partition dimensions as ["dim2", "dim4"] CompactionSegmentIterator iterator = createIterator( configBuilder().withDimensionsSpec( - new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim3", "dim4"))) + new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of( + "dim1", + "dim2", + "dim3", + "dim4" + ))) ) .withTuningConfig( new UserCompactionTaskQueryTuningConfig( @@ -1203,7 +1377,10 @@ public void testIteratorDoesNotReturnsSegmentsWhenPartitionDimensionsPrefixed() // has dimension order as expected post reordering of partition dimensions. Assert.assertTrue(iterator.hasNext()); List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -1217,9 +1394,10 @@ public void testIteratorDoesNotReturnsSegmentsWhenPartitionDimensionsPrefixed() public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFilter() throws Exception { // Same indexSpec as what is set in the auto compaction config - Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); + IndexSpec indexSpec = IndexSpec.DEFAULT; // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( + null)); // Create segments that were compacted (CompactionState != null) and have // filter=SelectorDimFilter("dim1", "foo", null) for interval 2017-10-01T00:00:00/2017-10-02T00:00:00, @@ -1235,7 +1413,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil partitionsSpec, null, null, - mapper.readValue(mapper.writeValueAsString(new TransformSpec(new SelectorDimFilter("dim1", "foo", null), null)), new TypeReference<>() {}), + new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null)), indexSpec, null ) @@ -1248,7 +1426,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil partitionsSpec, null, null, - mapper.readValue(mapper.writeValueAsString(new TransformSpec(new SelectorDimFilter("dim1", "bar", null), null)), new TypeReference<>() {}), + new CompactionTransformSpec(new SelectorDimFilter("dim1", "bar", null)), indexSpec, null ) @@ -1261,7 +1439,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil partitionsSpec, null, null, - mapper.readValue(mapper.writeValueAsString(new TransformSpec(null, null)), new TypeReference<>() {}), + new CompactionTransformSpec(null), indexSpec, null ) @@ -1275,14 +1453,17 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil // Auto compaction config sets filter=SelectorDimFilter("dim1", "bar", null) CompactionSegmentIterator iterator = createIterator( configBuilder().withTransformSpec( - new UserCompactionTaskTransformConfig(new SelectorDimFilter("dim1", "bar", null)) + new CompactionTransformSpec(new SelectorDimFilter("dim1", "bar", null)) ).build(), timeline ); // We should get interval 2017-10-01T00:00:00/2017-10-02T00:00:00, interval 2017-10-04T00:00:00/2017-10-05T00:00:00, and interval 2017-10-03T00:00:00/2017-10-04T00:00:00. Assert.assertTrue(iterator.hasNext()); List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -1290,7 +1471,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil ); Assert.assertTrue(iterator.hasNext()); expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -1298,7 +1482,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil ); Assert.assertTrue(iterator.hasNext()); expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -1310,7 +1497,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil // Auto compaction config sets filter=null iterator = createIterator( configBuilder().withTransformSpec( - new UserCompactionTaskTransformConfig(null) + new CompactionTransformSpec(null) ).build(), timeline ); @@ -1326,9 +1513,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) ); // Same indexSpec as what is set in the auto compaction config - Map indexSpec = IndexSpec.DEFAULT.asMap(mapper); + IndexSpec indexSpec = IndexSpec.DEFAULT; // Same partitionsSpec as what is set in the auto compaction config - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( + null)); // Create segments that were compacted (CompactionState != null) and have // metricsSpec={CountAggregatorFactory("cnt")} for interval 2017-10-01T00:00:00/2017-10-02T00:00:00, @@ -1343,7 +1531,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet new CompactionState( partitionsSpec, null, - mapper.convertValue(new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, new TypeReference<>() {}), + List.of(new CountAggregatorFactory("cnt")), null, indexSpec, null @@ -1356,7 +1544,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet new CompactionState( partitionsSpec, null, - mapper.convertValue(new AggregatorFactory[] {new CountAggregatorFactory("cnt"), new LongSumAggregatorFactory("val", "val")}, new TypeReference<>() {}), + List.of( + new CountAggregatorFactory("cnt"), + new LongSumAggregatorFactory("val", "val") + ), null, indexSpec, null @@ -1369,7 +1560,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet new CompactionState( partitionsSpec, null, - mapper.convertValue(new AggregatorFactory[] {}, new TypeReference<>() {}), + List.of(), null, indexSpec, null @@ -1391,7 +1582,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet // We should get interval 2017-10-01T00:00:00/2017-10-02T00:00:00, interval 2017-10-04T00:00:00/2017-10-05T00:00:00, and interval 2017-10-03T00:00:00/2017-10-04T00:00:00. Assert.assertTrue(iterator.hasNext()); List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -1399,7 +1593,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet ); Assert.assertTrue(iterator.hasNext()); expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -1407,7 +1604,10 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet ); Assert.assertTrue(iterator.hasNext()); expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -1446,7 +1646,10 @@ public void testIteratorReturnsSegmentsSmallerSegmentGranularityCoveringMultiple // also convers the HOUR segment (2017-10-01T01:00:00/2017-10-01T02:00:00) Assert.assertTrue(iterator.hasNext()); List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -1461,8 +1664,8 @@ public void testIteratorReturnsSegmentsAsCompactionStateChangedWithCompactedStat { // Different indexSpec as what is set in the auto compaction config IndexSpec newIndexSpec = IndexSpec.builder().withBitmapSerdeFactory(new ConciseBitmapSerdeFactory()).build(); - Map newIndexSpecMap = mapper.convertValue(newIndexSpec, new TypeReference<>() {}); - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( + null)); // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final SegmentTimeline timeline = createTimeline( @@ -1470,7 +1673,7 @@ public void testIteratorReturnsSegmentsAsCompactionStateChangedWithCompactedStat .forIntervals(1, Granularities.DAY) .startingAt("2017-10-02") .withNumPartitions(4) - .withCompactionState(new CompactionState(partitionsSpec, null, null, null, newIndexSpecMap, null)) + .withCompactionState(new CompactionState(partitionsSpec, null, null, null, newIndexSpec, null)) ); // Duration of new segmentGranularity is the same as before (P1D) @@ -1491,7 +1694,10 @@ public void testIteratorReturnsSegmentsAsCompactionStateChangedWithCompactedStat // We should get all segments in timeline back since indexSpec changed Assert.assertTrue(iterator.hasNext()); List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-01T00:00:00/2017-10-03T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval( + Intervals.of("2017-10-01T00:00:00/2017-10-03T00:00:00"), + Partitions.ONLY_COMPLETE + ) ); Assert.assertEquals( ImmutableSet.copyOf(expectedSegmentsToCompact), @@ -1504,14 +1710,15 @@ public void testIteratorReturnsSegmentsAsCompactionStateChangedWithCompactedStat @Test public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() { - PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from( + null)); final SegmentTimeline timeline = createTimeline( createSegments() .forIntervals(1, Granularities.DAY) .startingAt("2017-10-01") .withNumPartitions(4) .withCompactionState( - new CompactionState(partitionsSpec, null, null, null, IndexSpec.DEFAULT.asMap(mapper), null) + new CompactionState(partitionsSpec, null, null, null, IndexSpec.DEFAULT, null) ) ); @@ -1708,7 +1915,8 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() new ArrayList<>(), TombstoneShardSpec.INSTANCE, 0, - 1); + 1 + ); final DataSegment dataSegment2023 = new DataSegment( TestDataSource.WIKI, Intervals.of("2023/2024"), @@ -1718,7 +1926,8 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() new ArrayList<>(), new NumberedShardSpec(1, 0), 0, - 100); + 100 + ); final DataSegment tombstone2024 = new DataSegment( TestDataSource.WIKI, Intervals.of("2024/2025"), @@ -1728,7 +1937,8 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() new ArrayList<>(), TombstoneShardSpec.INSTANCE, 0, - 1); + 1 + ); CompactionSegmentIterator iterator = createIterator( configBuilder().withGranularitySpec( @@ -1753,7 +1963,8 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() new ArrayList<>(), TombstoneShardSpec.INSTANCE, 0, - 1); + 1 + ); final DataSegment tombstone2025Feb = new DataSegment( TestDataSource.WIKI, Intervals.of("2025-02-01/2025-03-01"), @@ -1763,7 +1974,8 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() new ArrayList<>(), TombstoneShardSpec.INSTANCE, 0, - 1); + 1 + ); final DataSegment tombstone2025Mar = new DataSegment( TestDataSource.WIKI, Intervals.of("2025-03-01/2025-04-01"), @@ -1773,7 +1985,8 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() new ArrayList<>(), TombstoneShardSpec.INSTANCE, 0, - 1); + 1 + ); iterator = createIterator( configBuilder().withGranularitySpec( new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null) @@ -1912,7 +2125,7 @@ private static SegmentTimeline createTimeline( } private static DataSourceCompactionConfig createConfigWithSegmentGranularity( - Granularity segmentGranularity + Granularity segmentGranularity ) { return configBuilder().withGranularitySpec( diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java index b3c9d4f5afdc..8783312d1a7d 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java @@ -36,6 +36,7 @@ import org.apache.druid.segment.data.CompressionFactory.LongEncodingStrategy; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.Duration; @@ -435,7 +436,7 @@ public void testSerdeTransformSpec() throws IOException .forDataSource("dataSource") .withInputSegmentSizeBytes(500L) .withSkipOffsetFromLatest(new Period(3600)) - .withTransformSpec(new UserCompactionTaskTransformConfig(new SelectorDimFilter("dim1", "foo", null))) + .withTransformSpec(new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null))) .withTaskContext(ImmutableMap.of("key", "val")) .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index d8bcf4837e49..e1a5ed52e335 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -19,8 +19,6 @@ package org.apache.druid.server.coordinator.duty; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; @@ -47,6 +45,7 @@ import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; @@ -66,9 +65,10 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.rpc.indexing.NoopOverlordClient; import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.indexing.BatchIOConfig; -import org.apache.druid.segment.transform.TransformSpec; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -78,7 +78,6 @@ import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; -import org.apache.druid.server.coordinator.UserCompactionTaskTransformConfig; import org.apache.druid.server.coordinator.config.DruidCoordinatorConfig; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Stats; @@ -105,6 +104,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -422,12 +422,12 @@ public void testMakeStatsForDataSourceWithCompactedIntervalBetweenNonCompactedIn DataSegment afterNoon = createSegment(dataSourceName, j, false, k); if (j == 3) { // Make two intervals on this day compacted (two compacted intervals back-to-back) - beforeNoon = beforeNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, null, null, ImmutableMap.of(), ImmutableMap.of())); - afterNoon = afterNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, null, null, ImmutableMap.of(), ImmutableMap.of())); + beforeNoon = beforeNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, null, null, JSON_MAPPER.convertValue(ImmutableMap.of(), IndexSpec.class), JSON_MAPPER.convertValue(ImmutableMap.of(), GranularitySpec.class))); + afterNoon = afterNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, null, null, JSON_MAPPER.convertValue(ImmutableMap.of(), IndexSpec.class), JSON_MAPPER.convertValue(ImmutableMap.of(), GranularitySpec.class))); } if (j == 1) { // Make one interval on this day compacted - afterNoon = afterNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, null, null, ImmutableMap.of(), ImmutableMap.of())); + afterNoon = afterNoon.withLastCompactionState(new CompactionState(partitionsSpec, null, null, null, JSON_MAPPER.convertValue(ImmutableMap.of(), IndexSpec.class), JSON_MAPPER.convertValue(ImmutableMap.of(), GranularitySpec.class))); } segments.add(beforeNoon); segments.add(afterNoon); @@ -1347,7 +1347,7 @@ public void testCompactWithTransformSpec() null, null, null, - new UserCompactionTaskTransformConfig(new SelectorDimFilter("dim1", "foo", null)), + new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null)), null, engine, null @@ -2159,23 +2159,9 @@ private void compactSegments( compactionPartitionsSpec = clientCompactionTaskQuery.getTuningConfig().getPartitionsSpec(); } - Map transformSpec = null; - try { - if (clientCompactionTaskQuery.getTransformSpec() != null) { - transformSpec = jsonMapper.readValue( - jsonMapper.writeValueAsString(new TransformSpec(clientCompactionTaskQuery.getTransformSpec() - .getFilter(), null)), - new TypeReference<>() {} - ); - } - } - catch (JsonProcessingException e) { - throw new IAE("Invalid Json payload"); - } - - List metricsSpec = null; + List metricsSpec = null; if (clientCompactionTaskQuery.getMetricsSpec() != null) { - metricsSpec = jsonMapper.convertValue(clientCompactionTaskQuery.getMetricsSpec(), new TypeReference<>() {}); + metricsSpec = Arrays.asList(clientCompactionTaskQuery.getMetricsSpec()); } for (int i = 0; i < 2; i++) { @@ -2193,18 +2179,21 @@ private void compactSegments( clientCompactionTaskQuery.getDimensionsSpec().getDimensions() ), metricsSpec, - transformSpec, - ImmutableMap.of( - "bitmap", - ImmutableMap.of("type", "roaring"), - "dimensionCompression", - "lz4", - "metricCompression", - "lz4", - "longEncoding", - "longs" + clientCompactionTaskQuery.getTransformSpec(), + jsonMapper.convertValue( + ImmutableMap.of( + "bitmap", + ImmutableMap.of("type", "roaring"), + "dimensionCompression", + "lz4", + "metricCompression", + "lz4", + "longEncoding", + "longs" + ), + IndexSpec.class ), - ImmutableMap.of() + jsonMapper.convertValue(ImmutableMap.of(), GranularitySpec.class) ), 1, segmentSize diff --git a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java index b963f4337081..459b952cc78b 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java @@ -26,11 +26,16 @@ import com.google.common.collect.ImmutableMap; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpec; @@ -88,10 +93,10 @@ public void testSerde() throws JsonProcessingException new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "bar", "foo")) ), - ImmutableList.of(ImmutableMap.of("type", "count", "name", "count")), - ImmutableMap.of("filter", ImmutableMap.of("type", "selector", "dimension", "dim1", "value", "foo")), - ImmutableMap.of(), - ImmutableMap.of() + ImmutableList.of(new CountAggregatorFactory("cnt")), + new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null)), + MAPPER.convertValue(ImmutableMap.of(), IndexSpec.class), + MAPPER.convertValue(ImmutableMap.of(), GranularitySpec.class) ), TEST_VERSION, 1 diff --git a/services/src/test/java/org/apache/druid/cli/CliPeonTest.java b/services/src/test/java/org/apache/druid/cli/CliPeonTest.java index 456f70e7012b..576298921628 100644 --- a/services/src/test/java/org/apache/druid/cli/CliPeonTest.java +++ b/services/src/test/java/org/apache/druid/cli/CliPeonTest.java @@ -28,6 +28,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.GuiceInjectors; +import org.apache.druid.indexer.granularity.ArbitraryGranularitySpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.TaskResource; @@ -42,7 +43,6 @@ import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.storage.local.LocalTmpStorageConfig; import org.joda.time.Duration; import org.junit.Assert; diff --git a/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java b/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java index c0634ed403c0..5ced6c377579 100644 --- a/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java +++ b/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java @@ -25,6 +25,7 @@ import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.guice.GuiceInjectors; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.TaskResource; @@ -32,7 +33,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.junit.After; import org.junit.Assert; import org.junit.Before; 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 7a42b60f4611..ab9652f3f314 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 @@ -52,6 +52,7 @@ import org.apache.druid.discovery.DruidNodeDiscoveryProvider; import org.apache.druid.discovery.NodeRole; import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; import org.apache.druid.java.util.common.CloseableIterators; @@ -72,8 +73,8 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.IndexBuilder; +import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexSchema; @@ -133,6 +134,8 @@ public class SystemSchemaTest extends CalciteTestBase { + private static final ObjectMapper MAPPER = CalciteTests.getJsonMapper(); + private static final BrokerSegmentMetadataCacheConfig SEGMENT_CACHE_CONFIG_DEFAULT = BrokerSegmentMetadataCacheConfig.create(); private static final List ROWS1 = ImmutableList.of( @@ -158,7 +161,6 @@ public class SystemSchemaTest extends CalciteTestBase private DruidLeaderClient coordinatorClient; private OverlordClient overlordClient; private TimelineServerView serverView; - private ObjectMapper mapper; private StringFullResponseHolder responseHolder; private BytesAccumulatingResponseHandler responseHandler; private Request request; @@ -190,7 +192,6 @@ public void setUp(@TempDir File tmpDir) throws Exception client = EasyMock.createMock(DruidLeaderClient.class); coordinatorClient = EasyMock.createMock(DruidLeaderClient.class); overlordClient = EasyMock.createMock(OverlordClient.class); - mapper = TestHelper.makeJsonMapper(); responseHolder = EasyMock.createMock(StringFullResponseHolder.class); responseHandler = EasyMock.createMockBuilder(BytesAccumulatingResponseHandler.class) .withConstructor() @@ -277,7 +278,7 @@ public void setUp(@TempDir File tmpDir) throws Exception client, overlordClient, druidNodeDiscoveryProvider, - mapper + MAPPER ); } @@ -286,8 +287,8 @@ public void setUp(@TempDir File tmpDir) throws Exception null, null, null, - Collections.singletonMap("test", "map"), - Collections.singletonMap("test2", "map2") + MAPPER.convertValue(Collections.singletonMap("test", "map"), IndexSpec.class), + MAPPER.convertValue(Collections.singletonMap("test2", "map2"), GranularitySpec.class) ); private final DataSegment publishedCompactedSegment1 = new DataSegment( @@ -566,7 +567,7 @@ public void testGetTableMap() @Test public void testSegmentsTable() throws Exception { - final SegmentsTable segmentsTable = new SegmentsTable(druidSchema, metadataView, new ObjectMapper(), authMapper); + final SegmentsTable segmentsTable = new SegmentsTable(druidSchema, metadataView, MAPPER, authMapper); final Set publishedSegments = new HashSet<>(Arrays.asList( new SegmentStatusInCluster(publishedCompactedSegment1, true, 2, null, false), new SegmentStatusInCluster(publishedCompactedSegment2, false, 0, null, false), @@ -720,7 +721,7 @@ public void testSegmentsTable() throws Exception @Test public void testSegmentsTableWithProjection() throws JsonProcessingException { - final SegmentsTable segmentsTable = new SegmentsTable(druidSchema, metadataView, new ObjectMapper(), authMapper); + final SegmentsTable segmentsTable = new SegmentsTable(druidSchema, metadataView, MAPPER, authMapper); final Set publishedSegments = new HashSet<>(Arrays.asList( new SegmentStatusInCluster(publishedCompactedSegment1, true, 2, null, false), new SegmentStatusInCluster(publishedCompactedSegment2, false, 0, null, false), @@ -766,10 +767,10 @@ public void testSegmentsTableWithProjection() throws JsonProcessingException Assert.assertNull(null, rows.get(4)[0]); Assert.assertEquals("test5_2015-01-01T00:00:00.000Z_2016-01-01T00:00:00.000Z_version5", rows.get(4)[1]); - Assert.assertEquals(mapper.writeValueAsString(expectedCompactionState), rows.get(5)[0]); + Assert.assertEquals(MAPPER.writeValueAsString(expectedCompactionState), rows.get(5)[0]); Assert.assertEquals("wikipedia1_2007-01-01T00:00:00.000Z_2008-01-01T00:00:00.000Z_version1", rows.get(5)[1]); - Assert.assertEquals(mapper.writeValueAsString(expectedCompactionState), rows.get(6)[0]); + Assert.assertEquals(MAPPER.writeValueAsString(expectedCompactionState), rows.get(6)[0]); Assert.assertEquals("wikipedia2_2008-01-01T00:00:00.000Z_2009-01-01T00:00:00.000Z_version2", rows.get(6)[1]); Assert.assertNull(null, rows.get(7)[0]); @@ -818,7 +819,7 @@ private void verifyRow( if (compactionState == null) { Assert.assertNull(row[17]); } else { - Assert.assertEquals(mapper.writeValueAsString(compactionState), row[17]); + Assert.assertEquals(MAPPER.writeValueAsString(compactionState), row[17]); } Assert.assertEquals(replicationFactor, row[18]); } @@ -1266,7 +1267,7 @@ public void testTasksTable() throws Exception EasyMock.expect(overlordClient.taskStatuses(null, null, null)).andReturn( Futures.immediateFuture( CloseableIterators.withEmptyBaggage( - mapper.readValue(json, new TypeReference>() {}).iterator() + MAPPER.readValue(json, new TypeReference>() {}).iterator() ) ) ); @@ -1353,7 +1354,7 @@ public void testTasksTableAuth() EasyMock.expect(overlordClient.taskStatuses(null, null, null)).andAnswer( () -> Futures.immediateFuture( CloseableIterators.withEmptyBaggage( - mapper.readValue(json, new TypeReference>() {}).iterator() + MAPPER.readValue(json, new TypeReference>() {}).iterator() ) ) ).anyTimes(); @@ -1403,7 +1404,7 @@ public void testSupervisorTable() throws Exception EasyMock.expect(overlordClient.supervisorStatuses()).andReturn( Futures.immediateFuture( CloseableIterators.withEmptyBaggage( - mapper.readValue(json, new TypeReference>() {}).iterator() + MAPPER.readValue(json, new TypeReference>() {}).iterator() ) ) ); @@ -1450,7 +1451,7 @@ public void testSupervisorTableAuth() EasyMock.expect(overlordClient.supervisorStatuses()).andAnswer( () -> Futures.immediateFuture( CloseableIterators.withEmptyBaggage( - mapper.readValue(json, new TypeReference>() {}).iterator() + MAPPER.readValue(json, new TypeReference>() {}).iterator() ) ) ).anyTimes();