From 3c7f368c81a22b277fa43726a426dc5dfb094ae7 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Thu, 18 Feb 2021 14:20:51 -0800 Subject: [PATCH 01/14] add query granularity to compaction task --- .../indexing/common/task/CompactionTask.java | 54 ++++++----- .../ClientCompactionTaskQuerySerdeTest.java | 11 +-- .../common/task/CompactionTaskTest.java | 13 +-- ...entCompactionTaskQueryGranularitySpec.java | 39 +++----- .../CompactionGranularitySpec.java | 90 +++++++++++++++++++ .../DataSourceCompactionConfig.java | 23 ++--- .../coordinator/duty/CompactSegments.java | 3 +- .../DataSourceCompactionConfigTest.java | 52 +---------- .../coordinator/duty/CompactSegmentsTest.java | 7 +- .../duty/NewestSegmentFirstPolicyTest.java | 17 ++-- 10 files changed, 168 insertions(+), 141 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/segment/indexing/granularity/CompactionGranularitySpec.java 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 276b1761281a..e149c7310bff 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 @@ -78,6 +78,7 @@ import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.TuningConfig; +import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.SegmentLoadingException; @@ -90,7 +91,9 @@ import org.apache.druid.timeline.partition.PartitionHolder; import org.joda.time.Interval; +import javax.annotation.Nonnull; import javax.annotation.Nullable; +import javax.validation.constraints.NotNull; import java.io.File; import java.io.IOException; import java.util.ArrayList; @@ -141,7 +144,7 @@ public class CompactionTask extends AbstractBatchIndexTask @Nullable private final Granularity segmentGranularity; @Nullable - private final GranularitySpec granularitySpec; + private final CompactionGranularitySpec granularitySpec; @Nullable private final ParallelIndexTuningConfig tuningConfig; @JsonIgnore @@ -175,7 +178,7 @@ public CompactionTask( @JsonProperty("dimensionsSpec") @Nullable final DimensionsSpec dimensionsSpec, @JsonProperty("metricsSpec") @Nullable final AggregatorFactory[] metricsSpec, @JsonProperty("segmentGranularity") @Deprecated @Nullable final Granularity segmentGranularity, - @JsonProperty("granularitySpec") @Nullable final GranularitySpec granularitySpec, + @JsonProperty("granularitySpec") @Nullable final CompactionGranularitySpec granularitySpec, @JsonProperty("tuningConfig") @Nullable final TuningConfig tuningConfig, @JsonProperty("context") @Nullable final Map context, @JacksonInject SegmentLoaderFactory segmentLoaderFactory, @@ -206,12 +209,7 @@ public CompactionTask( this.metricsSpec = metricsSpec; this.segmentGranularity = segmentGranularity; if (granularitySpec == null && segmentGranularity != null) { - this.granularitySpec = new UniformGranularitySpec( - segmentGranularity, - null, - null, - null - ); + this.granularitySpec = new CompactionGranularitySpec(segmentGranularity, null); } else { this.granularitySpec = granularitySpec; } @@ -306,7 +304,7 @@ public Granularity getSegmentGranularity() @JsonProperty @Nullable - public GranularitySpec getGranularitySpec() + public CompactionGranularitySpec getGranularitySpec() { return granularitySpec; } @@ -368,7 +366,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception partitionConfigurationManager, dimensionsSpec, metricsSpec, - getSegmentGranularity(), + granularitySpec, toolbox.getCoordinatorClient(), segmentLoaderFactory, retryPolicyFactory @@ -476,7 +474,7 @@ static List createIngestionSchema( final PartitionConfigurationManager partitionConfigurationManager, @Nullable final DimensionsSpec dimensionsSpec, @Nullable final AggregatorFactory[] metricsSpec, - @Nullable final Granularity segmentGranularity, + @Nullable final CompactionGranularitySpec granularitySpec, final CoordinatorClient coordinatorClient, final SegmentLoaderFactory segmentLoaderFactory, final RetryPolicyFactory retryPolicyFactory @@ -504,7 +502,7 @@ static List createIngestionSchema( final ParallelIndexTuningConfig compactionTuningConfig = partitionConfigurationManager.computeTuningConfig(); - if (segmentGranularity == null) { + if (granularitySpec == null || granularitySpec.getSegmentGranularity() == null) { // original granularity final Map>> intervalToSegments = new TreeMap<>( Comparators.intervalsByStartThenEnd() @@ -539,12 +537,15 @@ static List createIngestionSchema( for (NonnullPair>> entry : intervalToSegmentsUnified) { final Interval interval = entry.lhs; final List> segmentsToCompact = entry.rhs; + // If granularitySpec is not null, then set segmentGranularity. Otherwise, + // creates new granularitySpec and set segmentGranularity + Granularity segmentGranularityToUse = GranularityType.fromPeriod(interval.toPeriod()).getDefaultGranularity(); final DataSchema dataSchema = createDataSchema( segmentProvider.dataSource, segmentsToCompact, dimensionsSpec, metricsSpec, - GranularityType.fromPeriod(interval.toPeriod()).getDefaultGranularity() + granularitySpec == null ? new CompactionGranularitySpec(segmentGranularityToUse, null) : granularitySpec.withSegmentGranularity(segmentGranularityToUse) ); specs.add( @@ -571,7 +572,7 @@ static List createIngestionSchema( queryableIndexAndSegments, dimensionsSpec, metricsSpec, - segmentGranularity + granularitySpec ); return Collections.singletonList( @@ -639,7 +640,7 @@ private static DataSchema createDataSchema( List> queryableIndexAndSegments, @Nullable DimensionsSpec dimensionsSpec, @Nullable AggregatorFactory[] metricsSpec, - Granularity segmentGranularity + @Nonnull CompactionGranularitySpec granularitySpec ) { // check index metadata & @@ -648,15 +649,22 @@ private static DataSchema createDataSchema( final SettableSupplier queryGranularity = new SettableSupplier<>(); decideRollupAndQueryGranularityCarryOver(rollup, queryGranularity, queryableIndexAndSegments); - // find granularity spec - final Interval totalInterval = JodaUtils.umbrellaInterval( queryableIndexAndSegments.stream().map(p -> p.rhs.getInterval()).collect(Collectors.toList()) ); - final GranularitySpec granularitySpec = new UniformGranularitySpec( - Preconditions.checkNotNull(segmentGranularity), - queryGranularity.get(), + final Granularity queryGranularityToUse; + if (granularitySpec.getQueryGranularity() == null) { + queryGranularityToUse = queryGranularity.get(); + log.info("Generate compaction task spec with segments original query granularity [%s]", queryGranularityToUse); + } else { + queryGranularityToUse = granularitySpec.getQueryGranularity(); + log.info("Generate compaction task spec with new query granularity overrided from input [%s]", queryGranularityToUse); + } + + final GranularitySpec uniformGranularitySpec = new UniformGranularitySpec( + Preconditions.checkNotNull(granularitySpec.getSegmentGranularity()), + queryGranularityToUse, rollup.get(), Collections.singletonList(totalInterval) ); @@ -675,7 +683,7 @@ private static DataSchema createDataSchema( new TimestampSpec(null, null, null), finalDimensionsSpec, finalMetricsSpec, - granularitySpec, + uniformGranularitySpec, null ); } @@ -963,7 +971,7 @@ public static class Builder @Nullable private Granularity segmentGranularity; @Nullable - private GranularitySpec granularitySpec; + private CompactionGranularitySpec granularitySpec; @Nullable private TuningConfig tuningConfig; @Nullable @@ -1014,7 +1022,7 @@ public Builder segmentGranularity(Granularity segmentGranularity) return this; } - public Builder granularitySpec(GranularitySpec granularitySpec) + public Builder granularitySpec(CompactionGranularitySpec granularitySpec) { this.granularitySpec = granularitySpec; return this; 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 1e9628f96a71..0a3692e45627 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 @@ -52,6 +52,7 @@ import org.apache.druid.segment.data.CompressionFactory.LongEncodingStrategy; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; +import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; @@ -116,7 +117,7 @@ public void testClientCompactionTaskQueryToCompactionTask() throws IOException 1000, 100 ), - new ClientCompactionTaskQueryGranularitySpec(Granularities.DAY, Granularities.HOUR, true), + new ClientCompactionTaskQueryGranularitySpec(Granularities.DAY, Granularities.HOUR), ImmutableMap.of("key", "value") ); @@ -198,10 +199,6 @@ public void testClientCompactionTaskQueryToCompactionTask() throws IOException query.getGranularitySpec().getSegmentGranularity(), task.getGranularitySpec().getSegmentGranularity() ); - Assert.assertEquals( - query.getGranularitySpec().isRollup(), - task.getGranularitySpec().isRollup() - ); Assert.assertEquals(query.getContext(), task.getContext()); } @@ -259,7 +256,7 @@ public void testCompactionTaskToClientCompactionTaskQuery() throws IOException null ) ) - .granularitySpec(new UniformGranularitySpec(Granularities.DAY, Granularities.HOUR, null)) + .granularitySpec(new CompactionGranularitySpec(Granularities.DAY, Granularities.HOUR)) .build(); final ClientCompactionTaskQuery expected = new ClientCompactionTaskQuery( @@ -301,7 +298,7 @@ public void testCompactionTaskToClientCompactionTaskQuery() throws IOException 1000, 100 ), - new ClientCompactionTaskQueryGranularitySpec(Granularities.DAY, Granularities.HOUR, true), + new ClientCompactionTaskQueryGranularitySpec(Granularities.DAY, Granularities.HOUR), new HashMap<>() ); 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 0cc831a12a94..c2dc6bde171c 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 @@ -108,6 +108,7 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.indexing.TuningConfig; +import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.SegmentLoadingException; @@ -376,7 +377,7 @@ public void testCreateCompactionTaskWithGranularitySpec() ); builder2.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder2.tuningConfig(createTuningConfig()); - builder2.granularitySpec(new UniformGranularitySpec(Granularities.HOUR, Granularities.DAY, null)); + builder2.granularitySpec(new CompactionGranularitySpec(Granularities.HOUR, Granularities.DAY)); final CompactionTask taskCreatedWithGranularitySpec = builder2.build(); Assert.assertEquals(taskCreatedWithGranularitySpec.getSegmentGranularity(), taskCreatedWithSegmentGranularity.getSegmentGranularity()); } @@ -392,7 +393,7 @@ public void testCreateCompactionTaskWithGranularitySpecOverrideSegmentGranularit builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); builder.segmentGranularity(Granularities.HOUR); - builder.granularitySpec(new UniformGranularitySpec(Granularities.MINUTE, Granularities.DAY, null)); + builder.granularitySpec(new CompactionGranularitySpec(Granularities.MINUTE, Granularities.DAY)); final CompactionTask taskCreatedWithSegmentGranularity = builder.build(); Assert.assertEquals(Granularities.MINUTE, taskCreatedWithSegmentGranularity.getSegmentGranularity()); } @@ -1059,10 +1060,10 @@ public void testSegmentGranularity() throws IOException, SegmentLoadingException new PartitionConfigurationManager(TUNING_CONFIG), null, null, - new PeriodGranularity(Period.months(3), null, null), - COORDINATOR_CLIENT, - segmentLoaderFactory, - RETRY_POLICY_FACTORY + new CompactionGranularitySpec(new PeriodGranularity(Period.months(3), null, null), null), + COORDINATOR_CLIENT, + segmentLoaderFactory, + RETRY_POLICY_FACTORY ); final List expectedDimensionsSpec = ImmutableList.of( new DimensionsSpec(getDimensionSchema(new DoubleDimensionSchema("string_to_double"))) diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryGranularitySpec.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryGranularitySpec.java index 6f12ea771331..8ed82271d3cd 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryGranularitySpec.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryGranularitySpec.java @@ -30,18 +30,15 @@ public class ClientCompactionTaskQueryGranularitySpec { private final Granularity segmentGranularity; private final Granularity queryGranularity; - private final boolean rollup; @JsonCreator public ClientCompactionTaskQueryGranularitySpec( @JsonProperty("segmentGranularity") Granularity segmentGranularity, - @JsonProperty("queryGranularity") Granularity queryGranularity, - @JsonProperty("rollup") Boolean rollup + @JsonProperty("queryGranularity") Granularity queryGranularity ) { - this.queryGranularity = queryGranularity == null ? BaseGranularitySpec.DEFAULT_QUERY_GRANULARITY : queryGranularity; - this.rollup = rollup == null ? BaseGranularitySpec.DEFAULT_ROLLUP : rollup; - this.segmentGranularity = segmentGranularity == null ? BaseGranularitySpec.DEFAULT_SEGMENT_GRANULARITY : segmentGranularity; + this.queryGranularity = queryGranularity; + this.segmentGranularity = segmentGranularity; } @JsonProperty @@ -56,22 +53,6 @@ public Granularity getQueryGranularity() return queryGranularity; } - @JsonProperty - public boolean isRollup() - { - return rollup; - } - - @Override - public String toString() - { - return "ClientCompactionTaskQueryGranularitySpec{" + - "segmentGranularity=" + segmentGranularity + - ", queryGranularity=" + queryGranularity + - ", rollup=" + rollup + - '}'; - } - @Override public boolean equals(Object o) { @@ -83,13 +64,21 @@ public boolean equals(Object o) } ClientCompactionTaskQueryGranularitySpec that = (ClientCompactionTaskQueryGranularitySpec) o; return Objects.equals(segmentGranularity, that.segmentGranularity) && - Objects.equals(queryGranularity, that.queryGranularity) && - Objects.equals(rollup, that.rollup); + Objects.equals(queryGranularity, that.queryGranularity); } @Override public int hashCode() { - return Objects.hash(segmentGranularity, queryGranularity, rollup); + return Objects.hash(segmentGranularity, queryGranularity); + } + + @Override + public String toString() + { + return "ClientCompactionTaskQueryGranularitySpec{" + + "segmentGranularity=" + segmentGranularity + + ", queryGranularity=" + queryGranularity + + '}'; } } diff --git a/server/src/main/java/org/apache/druid/segment/indexing/granularity/CompactionGranularitySpec.java b/server/src/main/java/org/apache/druid/segment/indexing/granularity/CompactionGranularitySpec.java new file mode 100644 index 000000000000..ba979b3012ac --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/indexing/granularity/CompactionGranularitySpec.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.indexing.granularity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.joda.time.Interval; + +import java.util.List; +import java.util.Objects; + +public class CompactionGranularitySpec +{ + private final Granularity segmentGranularity; + private final Granularity queryGranularity; + + @JsonCreator + public CompactionGranularitySpec( + @JsonProperty("segmentGranularity") Granularity segmentGranularity, + @JsonProperty("queryGranularity") Granularity queryGranularity + ) + { + this.queryGranularity = queryGranularity; + this.segmentGranularity = segmentGranularity; + } + + @JsonProperty("segmentGranularity") + public Granularity getSegmentGranularity() + { + return segmentGranularity; + } + + @JsonProperty("queryGranularity") + public Granularity getQueryGranularity() + { + return queryGranularity; + } + + public CompactionGranularitySpec withSegmentGranularity(Granularity segmentGranularity) + { + return new CompactionGranularitySpec(segmentGranularity, queryGranularity); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CompactionGranularitySpec that = (CompactionGranularitySpec) o; + return Objects.equals(segmentGranularity, that.segmentGranularity) && + Objects.equals(queryGranularity, that.queryGranularity); + } + + @Override + public int hashCode() + { + return Objects.hash(segmentGranularity, queryGranularity); + } + + @Override + public String toString() + { + return "CompactionGranularitySpec{" + + "segmentGranularity=" + segmentGranularity + + ", queryGranularity=" + queryGranularity + + '}'; + } +} 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 692eb81d23b6..6ecce01d784c 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 @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.segment.indexing.granularity.BaseGranularitySpec; +import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.joda.time.Period; @@ -49,7 +50,7 @@ public class DataSourceCompactionConfig private final Integer maxRowsPerSegment; private final Period skipOffsetFromLatest; private final UserCompactionTaskQueryTuningConfig tuningConfig; - private final GranularitySpec granularitySpec; + private final CompactionGranularitySpec granularitySpec; private final Map taskContext; @JsonCreator @@ -60,7 +61,7 @@ public DataSourceCompactionConfig( @JsonProperty("maxRowsPerSegment") @Deprecated @Nullable Integer maxRowsPerSegment, @JsonProperty("skipOffsetFromLatest") @Nullable Period skipOffsetFromLatest, @JsonProperty("tuningConfig") @Nullable UserCompactionTaskQueryTuningConfig tuningConfig, - @JsonProperty("granularitySpec") @Nullable GranularitySpec granularitySpec, + @JsonProperty("granularitySpec") @Nullable CompactionGranularitySpec granularitySpec, @JsonProperty("taskContext") @Nullable Map taskContext ) { @@ -76,20 +77,8 @@ public DataSourceCompactionConfig( this.tuningConfig = tuningConfig; if (granularitySpec != null) { Preconditions.checkArgument( - granularitySpec instanceof UniformGranularitySpec, - "Auto compaction granularitySpec only supports uniform type" - ); - Preconditions.checkArgument( - granularitySpec.isRollup() == BaseGranularitySpec.DEFAULT_ROLLUP, - "Auto compaction granularitySpec only supports default rollup value" - ); - Preconditions.checkArgument( - granularitySpec.getQueryGranularity().equals(BaseGranularitySpec.DEFAULT_QUERY_GRANULARITY), - "Auto compaction granularitySpec only supports default query granularity value"); - Preconditions.checkArgument( - granularitySpec.inputIntervals().isEmpty(), - "Auto compaction granularitySpec does not supports interval value" - ); + granularitySpec.getQueryGranularity() == null, + "Auto compaction granularitySpec does not support query granularity value"); } this.granularitySpec = granularitySpec; this.taskContext = taskContext; @@ -136,7 +125,7 @@ public UserCompactionTaskQueryTuningConfig getTuningConfig() @JsonProperty @Nullable - public GranularitySpec getGranularitySpec() + public CompactionGranularitySpec getGranularitySpec() { return granularitySpec; } 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 dcd7fab8df41..0f7e9f914de2 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 @@ -314,8 +314,7 @@ private CoordinatorStats doRun( if (config.getGranularitySpec() != null) { queryGranularitySpec = new ClientCompactionTaskQueryGranularitySpec( config.getGranularitySpec().getSegmentGranularity(), - config.getGranularitySpec().getQueryGranularity(), - config.getGranularitySpec().isRollup() + config.getGranularitySpec().getQueryGranularity() ); } else { queryGranularitySpec = null; 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 01cfabb2c40b..c8e95e628f96 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 @@ -33,6 +33,7 @@ import org.apache.druid.segment.data.CompressionFactory.LongEncodingStrategy; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; +import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import org.joda.time.Duration; @@ -238,7 +239,7 @@ public void testSerdeGranularitySpec() throws IOException null, new Period(3600), null, - new UniformGranularitySpec(Granularities.HOUR, null, null), + new CompactionGranularitySpec(Granularities.HOUR, null), ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -264,55 +265,8 @@ public void testFailIfGranularitySpecContainsNonDefaultQueryGranularity() null, new Period(3600), null, - new UniformGranularitySpec(Granularities.HOUR, Granularities.MONTH, null), + new CompactionGranularitySpec(Granularities.HOUR, Granularities.MONTH), ImmutableMap.of("key", "val") ); } - - @Test(expected = IllegalArgumentException.class) - public void testFailIfGranularitySpecContainsNonDefaultRollup() - { - new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UniformGranularitySpec(Granularities.HOUR, Granularities.MONTH, false, null), - ImmutableMap.of("key", "val") - ); - } - - @Test(expected = IllegalArgumentException.class) - public void testFailIfGranularitySpecContainsNonEmptyInterval() - { - new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new UniformGranularitySpec(Granularities.HOUR, Granularities.MONTH, ImmutableList.of(Intervals.of("2012-01-08T00Z/2012-01-11T00Z"))), - ImmutableMap.of("key", "val") - ); - } - - @Test(expected = IllegalArgumentException.class) - public void testFailIfGranularitySpecIsNotUniform() - { - new DataSourceCompactionConfig( - "dataSource", - null, - 500L, - null, - new Period(3600), - null, - new ArbitraryGranularitySpec(null, null, null), - ImmutableMap.of("key", "val") - ); - } - - } 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 c8928103c095..2480b069b536 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 @@ -58,6 +58,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; +import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; @@ -644,7 +645,7 @@ public void testCompactWithGranularitySpec() null, null ), - new UniformGranularitySpec(Granularities.YEAR, null, null), + new CompactionGranularitySpec(Granularities.YEAR, null), null ) ); @@ -696,7 +697,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() ) ), null, - new ClientCompactionTaskQueryGranularitySpec(Granularities.DAY, null, null), + new ClientCompactionTaskQueryGranularitySpec(Granularities.DAY, null), null ) ); @@ -731,7 +732,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() null, null ), - new UniformGranularitySpec(Granularities.YEAR, null, null), + new CompactionGranularitySpec(Granularities.YEAR, null), null ) ); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/NewestSegmentFirstPolicyTest.java index cc23f6a4ab32..a4f3bb6bbc20 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/NewestSegmentFirstPolicyTest.java @@ -29,8 +29,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Comparators; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.Partitions; @@ -383,7 +382,7 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularityEqual() ); final CompactionSegmentIterator iterator = policy.reset( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new UniformGranularitySpec(Granularities.DAY, null, null))), + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new CompactionGranularitySpec(Granularities.DAY, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() ); @@ -410,7 +409,7 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularityLarger() ); final CompactionSegmentIterator iterator = policy.reset( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new UniformGranularitySpec(Granularities.MONTH, null, null))), + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new CompactionGranularitySpec(Granularities.MONTH, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() ); @@ -436,7 +435,7 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularitySmaller() ); final CompactionSegmentIterator iterator = policy.reset( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new UniformGranularitySpec(Granularities.MINUTE, null, null))), + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new CompactionGranularitySpec(Granularities.MINUTE, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() ); @@ -545,7 +544,7 @@ public void testIteratorReturnsSegmentsInConfiguredSegmentGranularity() ); final CompactionSegmentIterator iterator = policy.reset( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UniformGranularitySpec(Granularities.MONTH, null, null))), + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new CompactionGranularitySpec(Granularities.MONTH, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() ); @@ -594,7 +593,7 @@ public void testIteratorReturnsSegmentsInMultipleIntervalIfConfiguredSegmentGran ); final CompactionSegmentIterator iterator = policy.reset( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UniformGranularitySpec(Granularities.MONTH, null, null))), + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new CompactionGranularitySpec(Granularities.MONTH, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() ); @@ -629,7 +628,7 @@ public void testIteratorDoesNotReturnCompactedInterval() ); final CompactionSegmentIterator iterator = policy.reset( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P0D"), new UniformGranularitySpec(Granularities.MINUTE, null, null))), + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P0D"), new CompactionGranularitySpec(Granularities.MINUTE, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() ); @@ -735,7 +734,7 @@ private static VersionedIntervalTimeline createTimeline( private DataSourceCompactionConfig createCompactionConfig( long inputSegmentSizeBytes, Period skipOffsetFromLatest, - GranularitySpec granularitySpec + CompactionGranularitySpec granularitySpec ) { return new DataSourceCompactionConfig( From bf5f4979e443f624389b944ce77b0e9786f426dd Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Thu, 18 Feb 2021 14:25:18 -0800 Subject: [PATCH 02/14] fix checkstyle --- .../indexing/ClientCompactionTaskQueryGranularitySpec.java | 1 - .../indexing/granularity/CompactionGranularitySpec.java | 2 -- .../druid/server/coordinator/DataSourceCompactionConfig.java | 3 --- .../server/coordinator/DataSourceCompactionConfigTest.java | 4 ---- .../druid/server/coordinator/duty/CompactSegmentsTest.java | 1 - 5 files changed, 11 deletions(-) diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryGranularitySpec.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryGranularitySpec.java index 8ed82271d3cd..6ebe9f0f9da9 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryGranularitySpec.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryGranularitySpec.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.segment.indexing.granularity.BaseGranularitySpec; import java.util.Objects; diff --git a/server/src/main/java/org/apache/druid/segment/indexing/granularity/CompactionGranularitySpec.java b/server/src/main/java/org/apache/druid/segment/indexing/granularity/CompactionGranularitySpec.java index ba979b3012ac..1b71f69bf8da 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/granularity/CompactionGranularitySpec.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/granularity/CompactionGranularitySpec.java @@ -22,9 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.java.util.common.granularity.Granularity; -import org.joda.time.Interval; -import java.util.List; import java.util.Objects; public class CompactionGranularitySpec 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 6ecce01d784c..fa550c17d5ce 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 @@ -22,10 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.apache.druid.segment.indexing.granularity.BaseGranularitySpec; import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.joda.time.Period; import javax.annotation.Nullable; 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 c8e95e628f96..e8f00951fcfb 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 @@ -20,21 +20,17 @@ package org.apache.druid.server.coordinator; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.SegmentsSplitHintSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.HumanReadableBytes; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.data.BitmapSerde.DefaultBitmapSerdeFactory; import org.apache.druid.segment.data.CompressionFactory.LongEncodingStrategy; import org.apache.druid.segment.data.CompressionStrategy; -import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import org.joda.time.Duration; import org.joda.time.Period; 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 2480b069b536..0288fc3a0014 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 @@ -59,7 +59,6 @@ import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; From 3188e978d3b9af957ae06d28f1c3e7730118e3a6 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Thu, 18 Feb 2021 14:27:13 -0800 Subject: [PATCH 03/14] fix checkstyle --- .../org/apache/druid/indexing/common/task/CompactionTask.java | 1 - .../indexing/common/task/ClientCompactionTaskQuerySerdeTest.java | 1 - 2 files changed, 2 deletions(-) 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 e149c7310bff..1b1c37416d50 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 @@ -93,7 +93,6 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; -import javax.validation.constraints.NotNull; import java.io.File; import java.io.IOException; import java.util.ArrayList; 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 0a3692e45627..7de5d22d1812 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 @@ -53,7 +53,6 @@ import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; From 40bc1d09f0c983e3d453c5fcb0ab690617a24086 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Thu, 18 Feb 2021 21:55:52 -0800 Subject: [PATCH 04/14] fix test --- .../tests/coordinator/duty/ITAutoCompactionTest.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) 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 1ccfa3f85c9b..ded6ddf1fcd3 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 @@ -31,6 +31,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; 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.CompactionGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; @@ -298,7 +299,7 @@ public void testAutoCompactionDutyWithSegmentGranularity() throws Exception verifyQuery(INDEX_QUERIES_RESOURCE); Granularity newGranularity = Granularities.YEAR; - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UniformGranularitySpec(newGranularity, null, null)); + submitCompactionConfig(1000, NO_SKIP_OFFSET, new CompactionGranularitySpec(newGranularity, null)); LOG.info("Auto compaction test with YEAR segment granularity"); @@ -314,7 +315,7 @@ public void testAutoCompactionDutyWithSegmentGranularity() throws Exception checkCompactionIntervals(expectedIntervalAfterCompaction); newGranularity = Granularities.DAY; - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UniformGranularitySpec(newGranularity, null, null)); + submitCompactionConfig(1000, NO_SKIP_OFFSET, new CompactionGranularitySpec(newGranularity, null)); LOG.info("Auto compaction test with DAY segment granularity"); @@ -374,7 +375,7 @@ private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffset submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, null); } - private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest, GranularitySpec granularitySpec) throws Exception + private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest, CompactionGranularitySpec granularitySpec) throws Exception { submitCompactionConfig(new DynamicPartitionsSpec(maxRowsPerSegment, null), skipOffsetFromLatest, 1, granularitySpec); } @@ -383,7 +384,7 @@ private void submitCompactionConfig( PartitionsSpec partitionsSpec, Period skipOffsetFromLatest, int maxNumConcurrentSubTasks, - GranularitySpec granularitySpec + CompactionGranularitySpec granularitySpec ) throws Exception { DataSourceCompactionConfig compactionConfig = new DataSourceCompactionConfig( From f2dadfb485dbc15a4a2c2fcbe4f74adfb4a3ad38 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Thu, 18 Feb 2021 21:56:24 -0800 Subject: [PATCH 05/14] fix test --- .../druid/tests/coordinator/duty/ITAutoCompactionTest.java | 2 -- 1 file changed, 2 deletions(-) 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 ded6ddf1fcd3..7605b2c63e47 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 @@ -32,8 +32,6 @@ 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.CompactionGranularitySpec; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; From c08a9ed4dbd38c9415ca1710206cc642d72180e3 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Fri, 19 Feb 2021 21:12:02 -0800 Subject: [PATCH 06/14] add tests --- .../common/task/CompactionTaskRunTest.java | 178 ++++++++++++++++++ .../common/task/CompactionTaskTest.java | 153 +++++++++++++-- .../tests/indexer/ITCompactionTaskTest.java | 142 +++++++++++--- .../indexer/segment_metadata_qr2.json | 4 +- .../indexer/segment_metadata_qr4.json | 8 +- ...compaction_task_with_granularity_spec.json | 4 +- ...paction_task_with_segment_granularity.json | 2 +- ..._index_queries_hour_query_granularity.json | 150 +++++++++++++++ ..._index_queries_year_query_granularity.json | 150 +++++++++++++++ .../DataSourceCompactionConfigTest.java | 52 +++++ 10 files changed, 787 insertions(+), 56 deletions(-) create mode 100644 integration-tests/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json create mode 100644 integration-tests/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json 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 0e287721c440..7cb9602f5da2 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 @@ -65,6 +65,7 @@ import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.LocalDataSegmentPuller; @@ -586,6 +587,183 @@ public void testWithSegmentGranularity() throws Exception } } + @Test + public void testWithGranularitySpecNonNullSegmentGranularityAndNullQueryGranularity() throws Exception + { + runIndexTask(); + + final Builder builder = new Builder( + DATA_SOURCE, + segmentLoaderFactory, + RETRY_POLICY_FACTORY + ); + + // day segmentGranularity + final CompactionTask compactionTask1 = builder + .interval(Intervals.of("2014-01-01/2014-01-02")) + .granularitySpec(new CompactionGranularitySpec(Granularities.DAY, null)) + .build(); + + Pair> resultPair = runTask(compactionTask1); + + Assert.assertTrue(resultPair.lhs.isSuccess()); + + List segments = resultPair.rhs; + + Assert.assertEquals(1, segments.size()); + + Assert.assertEquals(Intervals.of("2014-01-01/2014-01-02"), segments.get(0).getInterval()); + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(0).getShardSpec()); + Assert.assertEquals( + getDefaultCompactionState(Granularities.DAY, Granularities.MINUTE, ImmutableList.of(Intervals.of("2014-01-01T00:00:00/2014-01-01T03:00:00"))), + segments.get(0).getLastCompactionState() + ); + + // hour segmentGranularity + final CompactionTask compactionTask2 = builder + .interval(Intervals.of("2014-01-01/2014-01-02")) + .granularitySpec(new CompactionGranularitySpec(Granularities.HOUR, null)) + .build(); + + resultPair = runTask(compactionTask2); + + Assert.assertTrue(resultPair.lhs.isSuccess()); + + segments = resultPair.rhs; + Assert.assertEquals(3, segments.size()); + + for (int i = 0; i < 3; i++) { + Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval()); + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(i).getShardSpec()); + Assert.assertEquals( + getDefaultCompactionState(Granularities.HOUR, Granularities.MINUTE, ImmutableList.of(Intervals.of("2014-01-01/2014-01-02"))), + segments.get(i).getLastCompactionState() + ); + } + } + + @Test + public void testWithGranularitySpecNonNullQueryGranularityAndNullSegmentGranularity() throws Exception + { + runIndexTask(); + + final Builder builder = new Builder( + DATA_SOURCE, + segmentLoaderFactory, + RETRY_POLICY_FACTORY + ); + + // day queryGranularity + final CompactionTask compactionTask1 = builder + .interval(Intervals.of("2014-01-01/2014-01-02")) + .granularitySpec(new CompactionGranularitySpec(null, Granularities.SECOND)) + .build(); + + Pair> resultPair = runTask(compactionTask1); + + Assert.assertTrue(resultPair.lhs.isSuccess()); + + List segments = resultPair.rhs; + + Assert.assertEquals(3, segments.size()); + + for (int i = 0; i < 3; i++) { + Assert.assertEquals( + Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), + segments.get(i).getInterval() + ); + Assert.assertEquals( + getDefaultCompactionState(Granularities.HOUR, Granularities.SECOND, ImmutableList.of(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1))), + segments.get(i).getLastCompactionState() + ); + if (lockGranularity == LockGranularity.SEGMENT) { + Assert.assertEquals( + new NumberedOverwriteShardSpec(32768, 0, 2, (short) 1, (short) 1), + segments.get(i).getShardSpec() + ); + } else { + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(i).getShardSpec()); + } + } + } + + @Test + public void testWithGranularitySpecNonNullQueryGranularityAndNonNullSegmentGranularity() throws Exception + { + runIndexTask(); + + final Builder builder = new Builder( + DATA_SOURCE, + segmentLoaderFactory, + RETRY_POLICY_FACTORY + ); + + // day segmentGranularity and day queryGranularity + final CompactionTask compactionTask1 = builder + .interval(Intervals.of("2014-01-01/2014-01-02")) + .granularitySpec(new CompactionGranularitySpec(Granularities.DAY, Granularities.DAY)) + .build(); + + Pair> resultPair = runTask(compactionTask1); + + Assert.assertTrue(resultPair.lhs.isSuccess()); + + List segments = resultPair.rhs; + + Assert.assertEquals(1, segments.size()); + + Assert.assertEquals(Intervals.of("2014-01-01/2014-01-02"), segments.get(0).getInterval()); + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(0).getShardSpec()); + Assert.assertEquals( + getDefaultCompactionState(Granularities.DAY, Granularities.DAY, ImmutableList.of(Intervals.of("2014-01-01T00:00:00/2014-01-01T03:00:00"))), + segments.get(0).getLastCompactionState() + ); + } + + @Test + public void testWithGranularitySpecNullQueryGranularityAndNullSegmentGranularity() throws Exception + { + runIndexTask(); + + final Builder builder = new Builder( + DATA_SOURCE, + segmentLoaderFactory, + RETRY_POLICY_FACTORY + ); + + final CompactionTask compactionTask1 = builder + .interval(Intervals.of("2014-01-01/2014-01-02")) + .granularitySpec(new CompactionGranularitySpec(null, null)) + .build(); + + Pair> resultPair = runTask(compactionTask1); + + Assert.assertTrue(resultPair.lhs.isSuccess()); + + List segments = resultPair.rhs; + + Assert.assertEquals(3, segments.size()); + + for (int i = 0; i < 3; i++) { + Assert.assertEquals( + Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), + segments.get(i).getInterval() + ); + Assert.assertEquals( + getDefaultCompactionState(Granularities.HOUR, Granularities.MINUTE, ImmutableList.of(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1))), + segments.get(i).getLastCompactionState() + ); + if (lockGranularity == LockGranularity.SEGMENT) { + Assert.assertEquals( + new NumberedOverwriteShardSpec(32768, 0, 2, (short) 1, (short) 1), + segments.get(i).getShardSpec() + ); + } else { + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(i).getShardSpec()); + } + } + } + @Test public void testCompactThenAppend() throws Exception { 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 c2dc6bde171c..e96d56bd3ea0 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 @@ -637,7 +637,8 @@ public void testCreateIngestionSchema() throws IOException, SegmentLoadingExcept expectedDimensionsSpec, AGGREGATORS, SEGMENT_INTERVALS, - Granularities.MONTH + Granularities.MONTH, + Granularities.NONE ); } @@ -707,7 +708,8 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio AGGREGATORS, SEGMENT_INTERVALS, tuningConfig, - Granularities.MONTH + Granularities.MONTH, + Granularities.NONE ); } @@ -777,7 +779,8 @@ public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException, Segm AGGREGATORS, SEGMENT_INTERVALS, tuningConfig, - Granularities.MONTH + Granularities.MONTH, + Granularities.NONE ); } @@ -847,7 +850,8 @@ public void testCreateIngestionSchemaWithNumShards() throws IOException, Segment AGGREGATORS, SEGMENT_INTERVALS, tuningConfig, - Granularities.MONTH + Granularities.MONTH, + Granularities.NONE ); } @@ -908,7 +912,8 @@ public void testCreateIngestionSchemaWithCustomDimensionsSpec() throws IOExcepti dimensionsSpecs, AGGREGATORS, SEGMENT_INTERVALS, - Granularities.MONTH + Granularities.MONTH, + Granularities.NONE ); } @@ -949,7 +954,8 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException, expectedDimensionsSpec, Arrays.asList(customMetricsSpec), SEGMENT_INTERVALS, - Granularities.MONTH + Granularities.MONTH, + Granularities.NONE ); } @@ -982,7 +988,8 @@ public void testCreateIngestionSchemaWithCustomSegments() throws IOException, Se expectedDimensionsSpec, AGGREGATORS, SEGMENT_INTERVALS, - Granularities.MONTH + Granularities.MONTH, + Granularities.NONE ); } @@ -1051,7 +1058,7 @@ public void testEmptyInterval() } @Test - public void testSegmentGranularity() throws IOException, SegmentLoadingException + public void testSegmentGranularityAndNullQueryGranularity() throws IOException, SegmentLoadingException { final List ingestionSpecs = CompactionTask.createIngestionSchema( toolbox, @@ -1061,9 +1068,9 @@ public void testSegmentGranularity() throws IOException, SegmentLoadingException null, null, new CompactionGranularitySpec(new PeriodGranularity(Period.months(3), null, null), null), - COORDINATOR_CLIENT, - segmentLoaderFactory, - RETRY_POLICY_FACTORY + COORDINATOR_CLIENT, + segmentLoaderFactory, + RETRY_POLICY_FACTORY ); final List expectedDimensionsSpec = ImmutableList.of( new DimensionsSpec(getDimensionSchema(new DoubleDimensionSchema("string_to_double"))) @@ -1081,12 +1088,47 @@ public void testSegmentGranularity() throws IOException, SegmentLoadingException expectedDimensionsSpec, AGGREGATORS, Collections.singletonList(COMPACTION_INTERVAL), + new PeriodGranularity(Period.months(3), null, null), + Granularities.NONE + ); + } + + @Test + public void testQueryGranularityAndNullSegmentGranularity() throws IOException, SegmentLoadingException + { + final List ingestionSpecs = CompactionTask.createIngestionSchema( + toolbox, + LockGranularity.TIME_CHUNK, + new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), + new PartitionConfigurationManager(TUNING_CONFIG), + null, + null, + new CompactionGranularitySpec(null, new PeriodGranularity(Period.months(3), null, null)), + COORDINATOR_CLIENT, + segmentLoaderFactory, + RETRY_POLICY_FACTORY + ); + final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); + + ingestionSpecs.sort( + (s1, s2) -> Comparators.intervalsByStartThenEnd().compare( + s1.getDataSchema().getGranularitySpec().inputIntervals().get(0), + s2.getDataSchema().getGranularitySpec().inputIntervals().get(0) + ) + ); + Assert.assertEquals(6, ingestionSpecs.size()); + assertIngestionSchema( + ingestionSpecs, + expectedDimensionsSpec, + AGGREGATORS, + SEGMENT_INTERVALS, + Granularities.MONTH, new PeriodGranularity(Period.months(3), null, null) ); } @Test - public void testNullSegmentGranularityAnd() throws IOException, SegmentLoadingException + public void testQueryGranularityAndSegmentGranularityNonNull() throws IOException, SegmentLoadingException { final List ingestionSpecs = CompactionTask.createIngestionSchema( toolbox, @@ -1095,7 +1137,80 @@ public void testNullSegmentGranularityAnd() throws IOException, SegmentLoadingEx new PartitionConfigurationManager(TUNING_CONFIG), null, null, + new CompactionGranularitySpec( + new PeriodGranularity(Period.months(3), null, null), + new PeriodGranularity(Period.months(3), null, null) + ), + COORDINATOR_CLIENT, + segmentLoaderFactory, + RETRY_POLICY_FACTORY + ); + final List expectedDimensionsSpec = ImmutableList.of( + new DimensionsSpec(getDimensionSchema(new DoubleDimensionSchema("string_to_double"))) + ); + + ingestionSpecs.sort( + (s1, s2) -> Comparators.intervalsByStartThenEnd().compare( + s1.getDataSchema().getGranularitySpec().inputIntervals().get(0), + s2.getDataSchema().getGranularitySpec().inputIntervals().get(0) + ) + ); + Assert.assertEquals(1, ingestionSpecs.size()); + assertIngestionSchema( + ingestionSpecs, + expectedDimensionsSpec, + AGGREGATORS, + Collections.singletonList(COMPACTION_INTERVAL), + new PeriodGranularity(Period.months(3), null, null), + new PeriodGranularity(Period.months(3), null, null) + ); + } + + @Test + public void testNullGranularitySpec() throws IOException, SegmentLoadingException + { + final List ingestionSpecs = CompactionTask.createIngestionSchema( + toolbox, + LockGranularity.TIME_CHUNK, + new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), + new PartitionConfigurationManager(TUNING_CONFIG), + null, + null, + null, + COORDINATOR_CLIENT, + segmentLoaderFactory, + RETRY_POLICY_FACTORY + ); + final List expectedDimensionsSpec = getExpectedDimensionsSpecForAutoGeneration(); + + ingestionSpecs.sort( + (s1, s2) -> Comparators.intervalsByStartThenEnd().compare( + s1.getDataSchema().getGranularitySpec().inputIntervals().get(0), + s2.getDataSchema().getGranularitySpec().inputIntervals().get(0) + ) + ); + Assert.assertEquals(6, ingestionSpecs.size()); + assertIngestionSchema( + ingestionSpecs, + expectedDimensionsSpec, + AGGREGATORS, + SEGMENT_INTERVALS, + Granularities.MONTH, + Granularities.NONE + ); + } + + @Test + public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity() throws IOException, SegmentLoadingException + { + final List ingestionSpecs = CompactionTask.createIngestionSchema( + toolbox, + LockGranularity.TIME_CHUNK, + new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), + new PartitionConfigurationManager(TUNING_CONFIG), + null, null, + new CompactionGranularitySpec(null, null), COORDINATOR_CLIENT, segmentLoaderFactory, RETRY_POLICY_FACTORY @@ -1114,7 +1229,8 @@ public void testNullSegmentGranularityAnd() throws IOException, SegmentLoadingEx expectedDimensionsSpec, AGGREGATORS, SEGMENT_INTERVALS, - Granularities.MONTH + Granularities.MONTH, + Granularities.NONE ); } @@ -1221,7 +1337,8 @@ private void assertIngestionSchema( List expectedDimensionsSpecs, List expectedMetricsSpec, List expectedSegmentIntervals, - Granularity expectedSegmentGranularity + Granularity expectedSegmentGranularity, + Granularity expectedQueryGranularity ) { assertIngestionSchema( @@ -1265,7 +1382,8 @@ private void assertIngestionSchema( null, null ), - expectedSegmentGranularity + expectedSegmentGranularity, + expectedQueryGranularity ); } @@ -1275,7 +1393,8 @@ private void assertIngestionSchema( List expectedMetricsSpec, List expectedSegmentIntervals, ParallelIndexTuningConfig expectedTuningConfig, - Granularity expectedSegmentGranularity + Granularity expectedSegmentGranularity, + Granularity expectedQueryGranularity ) { Preconditions.checkArgument( @@ -1308,7 +1427,7 @@ private void assertIngestionSchema( Assert.assertEquals( new UniformGranularitySpec( expectedSegmentGranularity, - Granularities.NONE, + expectedQueryGranularity, false, Collections.singletonList(expectedSegmentIntervals.get(i)) ), diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java index b68ab77b7659..b050feeeed21 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java @@ -41,8 +41,10 @@ import java.lang.reflect.Method; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; @Test(groups = {TestNGGroup.COMPACTION, TestNGGroup.QUICKSTART_COMPATIBLE}) @@ -52,6 +54,10 @@ public class ITCompactionTaskTest extends AbstractIndexerTest private static final Logger LOG = new Logger(ITCompactionTaskTest.class); private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json"; private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; + + private static final String INDEX_QUERIES_YEAR_RESOURCE = "/indexer/wikipedia_index_queries_year_query_granularity.json"; + private static final String INDEX_QUERIES_HOUR_RESOURCE = "/indexer/wikipedia_index_queries_hour_query_granularity.json"; + private static final String INDEX_DATASOURCE = "wikipedia_index_test"; private static final String SEGMENT_METADATA_QUERY_RESOURCE_QR4 = "/indexer/segment_metadata_qr4.json"; @@ -87,11 +93,68 @@ public void testCompactionWithSegmentGranularity() throws Exception } @Test - public void testCompactionWithGranularitySpec() throws Exception + public void testCompactionWithSegmentGranularityInGranularitySpec() throws Exception { loadDataAndCompact(INDEX_TASK, INDEX_QUERIES_RESOURCE, COMPACTION_TASK_WITH_GRANULARITY_SPEC, GranularityType.MONTH); } + @Test + public void testCompactionWithQueryGranularityInGranularitySpec() throws Exception + { + try (final Closeable ignored = unloader(fullDatasourceName)) { + loadData(INDEX_TASK); + // 4 segments across 2 days + checkNumberOfSegments(4); + List expectedIntervalAfterCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + expectedIntervalAfterCompaction.sort(null); + + checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE_QR4, GranularityType.SECOND.name()); + String queryResponseTemplate = getQueryResponseTemplate(INDEX_QUERIES_RESOURCE); + queryHelper.testQueriesFromString(queryResponseTemplate); + // QueryGranularity was SECOND, now we will change it to HOUR + compactData(COMPACTION_TASK_WITH_GRANULARITY_SPEC, null, GranularityType.HOUR); + + // The original 4 segments should be compacted into 2 new segments + checkNumberOfSegments(2); + queryResponseTemplate = getQueryResponseTemplate(INDEX_QUERIES_HOUR_RESOURCE); + queryHelper.testQueriesFromString(queryResponseTemplate); + checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE_QR2, GranularityType.HOUR.name()); + checkCompactionIntervals(expectedIntervalAfterCompaction); + } + } + + @Test + public void testCompactionWithSegmentGranularityAndQueryGranularityInGranularitySpec() throws Exception + { + try (final Closeable ignored = unloader(fullDatasourceName)) { + loadData(INDEX_TASK); + // 4 segments across 2 days + checkNumberOfSegments(4); + List expectedIntervalAfterCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + expectedIntervalAfterCompaction.sort(null); + + checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE_QR4, GranularityType.SECOND.name()); + String queryResponseTemplate = getQueryResponseTemplate(INDEX_QUERIES_RESOURCE); + queryHelper.testQueriesFromString(queryResponseTemplate); + compactData(COMPACTION_TASK_WITH_GRANULARITY_SPEC, GranularityType.YEAR, GranularityType.YEAR); + + // The original 4 segments should be compacted into 1 new segment + checkNumberOfSegments(1); + queryResponseTemplate = getQueryResponseTemplate(INDEX_QUERIES_YEAR_RESOURCE); + queryHelper.testQueriesFromString(queryResponseTemplate); + checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE_QR2, GranularityType.YEAR.name()); + + List newIntervals = new ArrayList<>(); + for (String interval : expectedIntervalAfterCompaction) { + for (Interval newinterval : GranularityType.YEAR.getDefaultGranularity().getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) { + newIntervals.add(newinterval.toString()); + } + } + expectedIntervalAfterCompaction = newIntervals; + checkCompactionIntervals(expectedIntervalAfterCompaction); + } + } + @Test public void testCompactionWithTimestampDimension() throws Exception { @@ -105,37 +168,22 @@ private void loadDataAndCompact( GranularityType newSegmentGranularity ) throws Exception { - loadData(indexTask); - - // 4 segments across 2 days - checkNumberOfSegments(4); - - List expectedIntervalAfterCompaction = coordinator.getSegmentIntervals(fullDatasourceName); - expectedIntervalAfterCompaction.sort(null); try (final Closeable ignored = unloader(fullDatasourceName)) { - String queryResponseTemplate; - try { - InputStream is = AbstractITBatchIndexTest.class.getResourceAsStream(queriesResource); - queryResponseTemplate = IOUtils.toString(is, StandardCharsets.UTF_8); - } - catch (IOException e) { - throw new ISE(e, "could not read query file: %s", queriesResource); - } - - queryResponseTemplate = StringUtils.replace( - queryResponseTemplate, - "%%DATASOURCE%%", - fullDatasourceName - ); + loadData(indexTask); + // 4 segments across 2 days + checkNumberOfSegments(4); + List expectedIntervalAfterCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + expectedIntervalAfterCompaction.sort(null); - checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE_QR4); + checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE_QR4, GranularityType.SECOND.name()); + String queryResponseTemplate = getQueryResponseTemplate(queriesResource); queryHelper.testQueriesFromString(queryResponseTemplate); - compactData(compactionResource, newSegmentGranularity); + compactData(compactionResource, newSegmentGranularity, null); // The original 4 segments should be compacted into 2 new segments checkNumberOfSegments(2); queryHelper.testQueriesFromString(queryResponseTemplate); - checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE_QR2); + checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE_QR2, GranularityType.SECOND.name()); if (newSegmentGranularity != null) { List newIntervals = new ArrayList<>(); @@ -164,18 +212,31 @@ private void loadData(String indexTask) throws Exception ); } - private void compactData(String compactionResource, GranularityType newSegmentGranularity) throws Exception + private void compactData(String compactionResource, GranularityType newSegmentGranularity, GranularityType newQueryGranularity) throws Exception { String template = getResourceAsString(compactionResource); template = StringUtils.replace(template, "%%DATASOURCE%%", fullDatasourceName); + // For the new granularitySpec map + Map granularityMap = new HashMap<>(); + if (newSegmentGranularity != null) { + granularityMap.put("segmentGranularity", newSegmentGranularity.name()); + } + if (newQueryGranularity != null) { + granularityMap.put("queryGranularity", newQueryGranularity.name()); + } + template = StringUtils.replace( + template, + "%%GRANULARITY_SPEC%%", + jsonMapper.writeValueAsString(granularityMap) + ); + // For the deprecated segment granularity field if (newSegmentGranularity != null) { template = StringUtils.replace( template, - "%%SEGMENTGRANULARITY%%", + "%%SEGMENT_GRANULARITY%%", newSegmentGranularity.name() ); } - final String taskID = indexer.submitTask(template); LOG.info("TaskID for compaction task %s", taskID); indexer.waitUntilTaskCompletes(taskID); @@ -186,7 +247,7 @@ private void compactData(String compactionResource, GranularityType newSegmentGr ); } - private void checkQueryGranularity(String queryResource) throws Exception + private void checkQueryGranularity(String queryResource, String expectedQueryGranularity) throws Exception { String queryResponseTemplate; try { @@ -212,6 +273,11 @@ private void checkQueryGranularity(String queryResource) throws Exception "%%INTERVALS%%", "2013-08-31/2013-09-02" ); + queryResponseTemplate = StringUtils.replace( + queryResponseTemplate, + "%%EXPECTED_QUERY_GRANULARITY%%", + expectedQueryGranularity + ); queryHelper.testQueriesFromString(queryResponseTemplate); } @@ -240,4 +306,22 @@ private void checkCompactionIntervals(List expectedIntervals) "Compaction interval check" ); } + + private String getQueryResponseTemplate(String queryResourcePath) + { + String queryResponseTemplate; + try { + InputStream is = AbstractITBatchIndexTest.class.getResourceAsStream(queryResourcePath); + queryResponseTemplate = IOUtils.toString(is, StandardCharsets.UTF_8); + queryResponseTemplate = StringUtils.replace( + queryResponseTemplate, + "%%DATASOURCE%%", + fullDatasourceName + ); + } + catch (IOException e) { + throw new ISE(e, "could not read query file: %s", queryResourcePath); + } + return queryResponseTemplate; + } } diff --git a/integration-tests/src/test/resources/indexer/segment_metadata_qr2.json b/integration-tests/src/test/resources/indexer/segment_metadata_qr2.json index 948760d8a655..1de34b748850 100644 --- a/integration-tests/src/test/resources/indexer/segment_metadata_qr2.json +++ b/integration-tests/src/test/resources/indexer/segment_metadata_qr2.json @@ -12,10 +12,10 @@ }, "expectedResults": [ { - "queryGranularity": "SECOND" + "queryGranularity": "%%EXPECTED_QUERY_GRANULARITY%%" }, { - "queryGranularity": "SECOND" + "queryGranularity": "%%EXPECTED_QUERY_GRANULARITY%%" } ], "fieldsToTest": ["queryGranularity"] diff --git a/integration-tests/src/test/resources/indexer/segment_metadata_qr4.json b/integration-tests/src/test/resources/indexer/segment_metadata_qr4.json index 180c69230fe6..ac6a03ba8916 100644 --- a/integration-tests/src/test/resources/indexer/segment_metadata_qr4.json +++ b/integration-tests/src/test/resources/indexer/segment_metadata_qr4.json @@ -12,16 +12,16 @@ }, "expectedResults": [ { - "queryGranularity": "SECOND" + "queryGranularity": "%%EXPECTED_QUERY_GRANULARITY%%" }, { - "queryGranularity": "SECOND" + "queryGranularity": "%%EXPECTED_QUERY_GRANULARITY%%" }, { - "queryGranularity": "SECOND" + "queryGranularity": "%%EXPECTED_QUERY_GRANULARITY%%" }, { - "queryGranularity": "SECOND" + "queryGranularity": "%%EXPECTED_QUERY_GRANULARITY%%" } ], "fieldsToTest": ["queryGranularity"] diff --git a/integration-tests/src/test/resources/indexer/wikipedia_compaction_task_with_granularity_spec.json b/integration-tests/src/test/resources/indexer/wikipedia_compaction_task_with_granularity_spec.json index 828579b6e46b..4fddeaa8aa3c 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_compaction_task_with_granularity_spec.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_compaction_task_with_granularity_spec.json @@ -8,9 +8,7 @@ "interval": "2013-08-31/2013-09-02" } }, - "granularitySpec": { - "segmentGranularity": "%%SEGMENTGRANULARITY%%" - }, + "granularitySpec": %%GRANULARITY_SPEC%%, "context" : { "storeCompactionState" : true } diff --git a/integration-tests/src/test/resources/indexer/wikipedia_compaction_task_with_segment_granularity.json b/integration-tests/src/test/resources/indexer/wikipedia_compaction_task_with_segment_granularity.json index 254926f224c5..7a2c016dc9d6 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_compaction_task_with_segment_granularity.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_compaction_task_with_segment_granularity.json @@ -8,7 +8,7 @@ "interval": "2013-08-31/2013-09-02" } }, - "segmentGranularity": "%%SEGMENTGRANULARITY%%", + "segmentGranularity": "%%SEGMENT_GRANULARITY%%", "context" : { "storeCompactionState" : true } diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json b/integration-tests/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json new file mode 100644 index 000000000000..6ed715b89eb5 --- /dev/null +++ b/integration-tests/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json @@ -0,0 +1,150 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:00:00.000Z", + "result" : { + "minTime" : "2013-08-31T01:00:00.000Z", + "maxTime" : "2013-09-01T12:00:00.000Z" + } + } + ] + }, + { + "description": "timeseries, datasketch aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "HLLSketchMerge", + "name": "approxCountHLL", + "fieldName": "HLLSketchBuild", + "lgK": 12, + "tgtHllType": "HLL_4", + "round": true + }, + { + "type":"thetaSketch", + "name":"approxCountTheta", + "fieldName":"thetaSketch", + "size":16384, + "shouldFinalize":true, + "isInputThetaSketch":false, + "errorBoundsStdDev":null + }, + { + "type":"quantilesDoublesSketch", + "name":"quantilesSketch", + "fieldName":"quantilesDoublesSketch", + "k":128 + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "quantilesSketch":5, + "approxCountTheta":5.0, + "approxCountHLL":5 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 9050.0, + "page" : "Crimson Typhoon", + "added_count" : 905, + "rows" : 1 + } + } ] + }, + { + "description": "timeseries, stringFirst/stringLast aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "stringFirst", + "name": "first_user", + "fieldName": "user" + }, + { + "type":"stringLast", + "name":"last_user", + "fieldName":"user" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "first_user":"nuclear", + "last_user":"stringer" + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json b/integration-tests/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json new file mode 100644 index 000000000000..c564cb1f784b --- /dev/null +++ b/integration-tests/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json @@ -0,0 +1,150 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-01-01T00:00:00.000Z", + "result" : { + "minTime" : "2013-01-01T00:00:00.000Z", + "maxTime" : "2013-01-01T00:00:00.000Z" + } + } + ] + }, + { + "description": "timeseries, datasketch aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z" + ], + "filter":null, + "aggregations":[ + { + "type": "HLLSketchMerge", + "name": "approxCountHLL", + "fieldName": "HLLSketchBuild", + "lgK": 12, + "tgtHllType": "HLL_4", + "round": true + }, + { + "type":"thetaSketch", + "name":"approxCountTheta", + "fieldName":"thetaSketch", + "size":16384, + "shouldFinalize":true, + "isInputThetaSketch":false, + "errorBoundsStdDev":null + }, + { + "type":"quantilesDoublesSketch", + "name":"quantilesSketch", + "fieldName":"quantilesDoublesSketch", + "k":128 + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-01-01T00:00:00.000Z", + "result" : { + "quantilesSketch":5, + "approxCountTheta":5.0, + "approxCountHLL":5 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-01-01T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 9050.0, + "page" : "Crimson Typhoon", + "added_count" : 905, + "rows" : 1 + } + } ] + }, + { + "description": "timeseries, stringFirst/stringLast aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z" + ], + "filter":null, + "aggregations":[ + { + "type": "stringFirst", + "name": "first_user", + "fieldName": "user" + }, + { + "type":"stringLast", + "name":"last_user", + "fieldName":"user" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-01-01T00:00:00.000Z", + "result" : { + "first_user":"nuclear", + "last_user":"stringer" + } + } + ] + } +] \ No newline at end of file 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 e8f00951fcfb..b589d2385fcc 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 @@ -265,4 +265,56 @@ public void testFailIfGranularitySpecContainsNonDefaultQueryGranularity() ImmutableMap.of("key", "val") ); } + + @Test + public void testSerdeWithNullGranularitySpec() throws IOException + { + final DataSourceCompactionConfig config = new DataSourceCompactionConfig( + "dataSource", + null, + 500L, + null, + new Period(3600), + null, + null, + ImmutableMap.of("key", "val") + ); + final String json = OBJECT_MAPPER.writeValueAsString(config); + final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); + + Assert.assertEquals(config.getDataSource(), fromJson.getDataSource()); + Assert.assertEquals(25, fromJson.getTaskPriority()); + Assert.assertEquals(config.getInputSegmentSizeBytes(), fromJson.getInputSegmentSizeBytes()); + Assert.assertEquals(config.getMaxRowsPerSegment(), fromJson.getMaxRowsPerSegment()); + Assert.assertEquals(config.getSkipOffsetFromLatest(), fromJson.getSkipOffsetFromLatest()); + Assert.assertEquals(config.getTuningConfig(), fromJson.getTuningConfig()); + Assert.assertEquals(config.getTaskContext(), fromJson.getTaskContext()); + Assert.assertEquals(config.getGranularitySpec(), fromJson.getGranularitySpec()); + } + + @Test + public void testSerdeGranularitySpecWithNullValues() throws IOException + { + final DataSourceCompactionConfig config = new DataSourceCompactionConfig( + "dataSource", + null, + 500L, + null, + new Period(3600), + null, + new CompactionGranularitySpec(null, null), + ImmutableMap.of("key", "val") + ); + final String json = OBJECT_MAPPER.writeValueAsString(config); + final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); + + Assert.assertEquals(config.getDataSource(), fromJson.getDataSource()); + Assert.assertEquals(25, fromJson.getTaskPriority()); + Assert.assertEquals(config.getInputSegmentSizeBytes(), fromJson.getInputSegmentSizeBytes()); + Assert.assertEquals(config.getMaxRowsPerSegment(), fromJson.getMaxRowsPerSegment()); + Assert.assertEquals(config.getSkipOffsetFromLatest(), fromJson.getSkipOffsetFromLatest()); + Assert.assertEquals(config.getTuningConfig(), fromJson.getTuningConfig()); + Assert.assertEquals(config.getTaskContext(), fromJson.getTaskContext()); + Assert.assertEquals(config.getGranularitySpec(), fromJson.getGranularitySpec()); + } } From 844e404c7831cf581bfe25470115c4fe25a0b85f Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Fri, 19 Feb 2021 21:35:20 -0800 Subject: [PATCH 07/14] fix test --- ...dia_index_queries_year_query_granularity.json | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json b/integration-tests/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json index c564cb1f784b..25f9203ac1b8 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json @@ -22,7 +22,7 @@ "dataSource": "%%DATASOURCE%%", "granularity":"day", "intervals":[ - "2013-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z" + "2013-01-01T00:00/2014-01-01T00:00" ], "filter":null, "aggregations":[ @@ -55,7 +55,7 @@ { "timestamp" : "2013-01-01T00:00:00.000Z", "result" : { - "quantilesSketch":5, + "quantilesSketch":10, "approxCountTheta":5.0, "approxCountHLL":5 } @@ -100,16 +100,16 @@ ], "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, "intervals":[ - "2013-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z" + "2013-01-01T00:00/2014-01-01T00:00" ] }, "expectedResults":[ { "version" : "v1", "timestamp" : "2013-01-01T00:00:00.000Z", "event" : { - "added_count_times_ten" : 9050.0, + "added_count_times_ten" : 18100.0, "page" : "Crimson Typhoon", - "added_count" : 905, + "added_count" : 1810, "rows" : 1 } } ] @@ -121,7 +121,7 @@ "dataSource": "%%DATASOURCE%%", "granularity":"day", "intervals":[ - "2013-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z" + "2013-01-01T00:00/2014-01-01T00:00" ], "filter":null, "aggregations":[ @@ -141,8 +141,8 @@ { "timestamp" : "2013-01-01T00:00:00.000Z", "result" : { - "first_user":"nuclear", - "last_user":"stringer" + "first_user":"masterYi", + "last_user":"speed" } } ] From d73a845aee0403273565a8581293df0c98e9f063 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Sat, 20 Feb 2021 00:28:16 -0800 Subject: [PATCH 08/14] fix test --- .../tests/indexer/ITCompactionTaskTest.java | 25 +++++++++------- .../indexer/segment_metadata_qr4.json | 29 ------------------- ...a_qr2.json => segment_metadata_query.json} | 9 +----- 3 files changed, 16 insertions(+), 47 deletions(-) delete mode 100644 integration-tests/src/test/resources/indexer/segment_metadata_qr4.json rename integration-tests/src/test/resources/indexer/{segment_metadata_qr2.json => segment_metadata_query.json} (58%) diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java index b050feeeed21..e63fbb0110d0 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java @@ -60,8 +60,7 @@ public class ITCompactionTaskTest extends AbstractIndexerTest private static final String INDEX_DATASOURCE = "wikipedia_index_test"; - private static final String SEGMENT_METADATA_QUERY_RESOURCE_QR4 = "/indexer/segment_metadata_qr4.json"; - private static final String SEGMENT_METADATA_QUERY_RESOURCE_QR2 = "/indexer/segment_metadata_qr2.json"; + private static final String SEGMENT_METADATA_QUERY_RESOURCE = "/indexer/segment_metadata_query.json"; private static final String COMPACTION_TASK = "/indexer/wikipedia_compaction_task.json"; private static final String COMPACTION_TASK_WITH_SEGMENT_GRANULARITY = "/indexer/wikipedia_compaction_task_with_segment_granularity.json"; @@ -108,7 +107,7 @@ public void testCompactionWithQueryGranularityInGranularitySpec() throws Excepti List expectedIntervalAfterCompaction = coordinator.getSegmentIntervals(fullDatasourceName); expectedIntervalAfterCompaction.sort(null); - checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE_QR4, GranularityType.SECOND.name()); + checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE, GranularityType.SECOND.name(), 4); String queryResponseTemplate = getQueryResponseTemplate(INDEX_QUERIES_RESOURCE); queryHelper.testQueriesFromString(queryResponseTemplate); // QueryGranularity was SECOND, now we will change it to HOUR @@ -118,7 +117,7 @@ public void testCompactionWithQueryGranularityInGranularitySpec() throws Excepti checkNumberOfSegments(2); queryResponseTemplate = getQueryResponseTemplate(INDEX_QUERIES_HOUR_RESOURCE); queryHelper.testQueriesFromString(queryResponseTemplate); - checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE_QR2, GranularityType.HOUR.name()); + checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE, GranularityType.HOUR.name(), 2); checkCompactionIntervals(expectedIntervalAfterCompaction); } } @@ -133,7 +132,7 @@ public void testCompactionWithSegmentGranularityAndQueryGranularityInGranularity List expectedIntervalAfterCompaction = coordinator.getSegmentIntervals(fullDatasourceName); expectedIntervalAfterCompaction.sort(null); - checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE_QR4, GranularityType.SECOND.name()); + checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE, GranularityType.SECOND.name(), 4); String queryResponseTemplate = getQueryResponseTemplate(INDEX_QUERIES_RESOURCE); queryHelper.testQueriesFromString(queryResponseTemplate); compactData(COMPACTION_TASK_WITH_GRANULARITY_SPEC, GranularityType.YEAR, GranularityType.YEAR); @@ -142,7 +141,7 @@ public void testCompactionWithSegmentGranularityAndQueryGranularityInGranularity checkNumberOfSegments(1); queryResponseTemplate = getQueryResponseTemplate(INDEX_QUERIES_YEAR_RESOURCE); queryHelper.testQueriesFromString(queryResponseTemplate); - checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE_QR2, GranularityType.YEAR.name()); + checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE, GranularityType.YEAR.name(), 1); List newIntervals = new ArrayList<>(); for (String interval : expectedIntervalAfterCompaction) { @@ -175,7 +174,7 @@ private void loadDataAndCompact( List expectedIntervalAfterCompaction = coordinator.getSegmentIntervals(fullDatasourceName); expectedIntervalAfterCompaction.sort(null); - checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE_QR4, GranularityType.SECOND.name()); + checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE, GranularityType.SECOND.name(), 4); String queryResponseTemplate = getQueryResponseTemplate(queriesResource); queryHelper.testQueriesFromString(queryResponseTemplate); compactData(compactionResource, newSegmentGranularity, null); @@ -183,7 +182,7 @@ private void loadDataAndCompact( // The original 4 segments should be compacted into 2 new segments checkNumberOfSegments(2); queryHelper.testQueriesFromString(queryResponseTemplate); - checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE_QR2, GranularityType.SECOND.name()); + checkQueryGranularity(SEGMENT_METADATA_QUERY_RESOURCE, GranularityType.SECOND.name(), 2); if (newSegmentGranularity != null) { List newIntervals = new ArrayList<>(); @@ -247,7 +246,7 @@ private void compactData(String compactionResource, GranularityType newSegmentGr ); } - private void checkQueryGranularity(String queryResource, String expectedQueryGranularity) throws Exception + private void checkQueryGranularity(String queryResource, String expectedQueryGranularity, int segmentCount) throws Exception { String queryResponseTemplate; try { @@ -273,10 +272,16 @@ private void checkQueryGranularity(String queryResource, String expectedQueryGra "%%INTERVALS%%", "2013-08-31/2013-09-02" ); + List> expectedResults = new ArrayList<>(); + for (int i = 0; i < segmentCount; i++) { + Map result = new HashMap<>(); + result.put("queryGranularity", expectedQueryGranularity); + expectedResults.add(result); + } queryResponseTemplate = StringUtils.replace( queryResponseTemplate, "%%EXPECTED_QUERY_GRANULARITY%%", - expectedQueryGranularity + jsonMapper.writeValueAsString(expectedResults) ); queryHelper.testQueriesFromString(queryResponseTemplate); } diff --git a/integration-tests/src/test/resources/indexer/segment_metadata_qr4.json b/integration-tests/src/test/resources/indexer/segment_metadata_qr4.json deleted file mode 100644 index ac6a03ba8916..000000000000 --- a/integration-tests/src/test/resources/indexer/segment_metadata_qr4.json +++ /dev/null @@ -1,29 +0,0 @@ -[ - { - "query": { - "queryType": "segmentMetadata", - "dataSource": "%%DATASOURCE%%", - "analysisTypes": [ - "%%ANALYSIS_TYPE%%" - ], - "intervals": [ - "%%INTERVALS%%" - ] - }, - "expectedResults": [ - { - "queryGranularity": "%%EXPECTED_QUERY_GRANULARITY%%" - }, - { - "queryGranularity": "%%EXPECTED_QUERY_GRANULARITY%%" - }, - { - "queryGranularity": "%%EXPECTED_QUERY_GRANULARITY%%" - }, - { - "queryGranularity": "%%EXPECTED_QUERY_GRANULARITY%%" - } - ], - "fieldsToTest": ["queryGranularity"] - } -] diff --git a/integration-tests/src/test/resources/indexer/segment_metadata_qr2.json b/integration-tests/src/test/resources/indexer/segment_metadata_query.json similarity index 58% rename from integration-tests/src/test/resources/indexer/segment_metadata_qr2.json rename to integration-tests/src/test/resources/indexer/segment_metadata_query.json index 1de34b748850..1868c4596d0c 100644 --- a/integration-tests/src/test/resources/indexer/segment_metadata_qr2.json +++ b/integration-tests/src/test/resources/indexer/segment_metadata_query.json @@ -10,14 +10,7 @@ "%%INTERVALS%%" ] }, - "expectedResults": [ - { - "queryGranularity": "%%EXPECTED_QUERY_GRANULARITY%%" - }, - { - "queryGranularity": "%%EXPECTED_QUERY_GRANULARITY%%" - } - ], + "expectedResults": %%EXPECTED_QUERY_GRANULARITY%%, "fieldsToTest": ["queryGranularity"] } ] From 4d0224df017d4f36d54da9fbb41e4d6e59a401ad Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Fri, 26 Feb 2021 13:17:40 -0800 Subject: [PATCH 09/14] cleanup --- .../indexing/common/task/CompactionTask.java | 20 +++++++++---------- .../ClientCompactionTaskQuerySerdeTest.java | 3 +-- .../common/task/CompactionTaskRunTest.java | 12 +++++------ .../common/task/CompactionTaskTest.java | 14 ++++++------- .../duty/ITAutoCompactionTest.java | 10 +++++----- ...entCompactionTaskQueryGranularitySpec.java | 5 +++++ .../DataSourceCompactionConfig.java | 7 +++---- .../UserCompactionTaskGranularityConfig.java} | 15 +++++--------- .../DataSourceCompactionConfigTest.java | 7 +++---- 9 files changed, 45 insertions(+), 48 deletions(-) rename server/src/main/java/org/apache/druid/{segment/indexing/granularity/CompactionGranularitySpec.java => server/coordinator/UserCompactionTaskGranularityConfig.java} (84%) 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 1b1c37416d50..904c99815fb0 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 @@ -32,6 +32,7 @@ import org.apache.curator.shaded.com.google.common.base.Verify; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.client.indexing.ClientCompactionTaskQueryGranularitySpec; import org.apache.druid.common.guava.SettableSupplier; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.DimensionSchema; @@ -78,7 +79,6 @@ import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.TuningConfig; -import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.SegmentLoadingException; @@ -143,7 +143,7 @@ public class CompactionTask extends AbstractBatchIndexTask @Nullable private final Granularity segmentGranularity; @Nullable - private final CompactionGranularitySpec granularitySpec; + private final ClientCompactionTaskQueryGranularitySpec granularitySpec; @Nullable private final ParallelIndexTuningConfig tuningConfig; @JsonIgnore @@ -177,7 +177,7 @@ public CompactionTask( @JsonProperty("dimensionsSpec") @Nullable final DimensionsSpec dimensionsSpec, @JsonProperty("metricsSpec") @Nullable final AggregatorFactory[] metricsSpec, @JsonProperty("segmentGranularity") @Deprecated @Nullable final Granularity segmentGranularity, - @JsonProperty("granularitySpec") @Nullable final CompactionGranularitySpec granularitySpec, + @JsonProperty("granularitySpec") @Nullable final ClientCompactionTaskQueryGranularitySpec granularitySpec, @JsonProperty("tuningConfig") @Nullable final TuningConfig tuningConfig, @JsonProperty("context") @Nullable final Map context, @JacksonInject SegmentLoaderFactory segmentLoaderFactory, @@ -208,7 +208,7 @@ public CompactionTask( this.metricsSpec = metricsSpec; this.segmentGranularity = segmentGranularity; if (granularitySpec == null && segmentGranularity != null) { - this.granularitySpec = new CompactionGranularitySpec(segmentGranularity, null); + this.granularitySpec = new ClientCompactionTaskQueryGranularitySpec(segmentGranularity, null); } else { this.granularitySpec = granularitySpec; } @@ -303,7 +303,7 @@ public Granularity getSegmentGranularity() @JsonProperty @Nullable - public CompactionGranularitySpec getGranularitySpec() + public ClientCompactionTaskQueryGranularitySpec getGranularitySpec() { return granularitySpec; } @@ -473,7 +473,7 @@ static List createIngestionSchema( final PartitionConfigurationManager partitionConfigurationManager, @Nullable final DimensionsSpec dimensionsSpec, @Nullable final AggregatorFactory[] metricsSpec, - @Nullable final CompactionGranularitySpec granularitySpec, + @Nullable final ClientCompactionTaskQueryGranularitySpec granularitySpec, final CoordinatorClient coordinatorClient, final SegmentLoaderFactory segmentLoaderFactory, final RetryPolicyFactory retryPolicyFactory @@ -544,7 +544,7 @@ static List createIngestionSchema( segmentsToCompact, dimensionsSpec, metricsSpec, - granularitySpec == null ? new CompactionGranularitySpec(segmentGranularityToUse, null) : granularitySpec.withSegmentGranularity(segmentGranularityToUse) + granularitySpec == null ? new ClientCompactionTaskQueryGranularitySpec(segmentGranularityToUse, null) : granularitySpec.withSegmentGranularity(segmentGranularityToUse) ); specs.add( @@ -639,7 +639,7 @@ private static DataSchema createDataSchema( List> queryableIndexAndSegments, @Nullable DimensionsSpec dimensionsSpec, @Nullable AggregatorFactory[] metricsSpec, - @Nonnull CompactionGranularitySpec granularitySpec + @Nonnull ClientCompactionTaskQueryGranularitySpec granularitySpec ) { // check index metadata & @@ -970,7 +970,7 @@ public static class Builder @Nullable private Granularity segmentGranularity; @Nullable - private CompactionGranularitySpec granularitySpec; + private ClientCompactionTaskQueryGranularitySpec granularitySpec; @Nullable private TuningConfig tuningConfig; @Nullable @@ -1021,7 +1021,7 @@ public Builder segmentGranularity(Granularity segmentGranularity) return this; } - public Builder granularitySpec(CompactionGranularitySpec granularitySpec) + public Builder granularitySpec(ClientCompactionTaskQueryGranularitySpec granularitySpec) { this.granularitySpec = granularitySpec; return this; 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 7de5d22d1812..4645ceff2577 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 @@ -52,7 +52,6 @@ import org.apache.druid.segment.data.CompressionFactory.LongEncodingStrategy; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; -import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; @@ -255,7 +254,7 @@ public void testCompactionTaskToClientCompactionTaskQuery() throws IOException null ) ) - .granularitySpec(new CompactionGranularitySpec(Granularities.DAY, Granularities.HOUR)) + .granularitySpec(new ClientCompactionTaskQueryGranularitySpec(Granularities.DAY, Granularities.HOUR)) .build(); final ClientCompactionTaskQuery expected = new ClientCompactionTaskQuery( 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 7cb9602f5da2..b571ffac53d8 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 @@ -26,6 +26,7 @@ import com.google.common.collect.Ordering; import com.google.common.io.Files; import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.client.indexing.ClientCompactionTaskQueryGranularitySpec; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.NoopIndexingServiceClient; import org.apache.druid.data.input.impl.CSVParseSpec; @@ -65,7 +66,6 @@ import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.LocalDataSegmentPuller; @@ -601,7 +601,7 @@ public void testWithGranularitySpecNonNullSegmentGranularityAndNullQueryGranular // day segmentGranularity final CompactionTask compactionTask1 = builder .interval(Intervals.of("2014-01-01/2014-01-02")) - .granularitySpec(new CompactionGranularitySpec(Granularities.DAY, null)) + .granularitySpec(new ClientCompactionTaskQueryGranularitySpec(Granularities.DAY, null)) .build(); Pair> resultPair = runTask(compactionTask1); @@ -622,7 +622,7 @@ public void testWithGranularitySpecNonNullSegmentGranularityAndNullQueryGranular // hour segmentGranularity final CompactionTask compactionTask2 = builder .interval(Intervals.of("2014-01-01/2014-01-02")) - .granularitySpec(new CompactionGranularitySpec(Granularities.HOUR, null)) + .granularitySpec(new ClientCompactionTaskQueryGranularitySpec(Granularities.HOUR, null)) .build(); resultPair = runTask(compactionTask2); @@ -656,7 +656,7 @@ public void testWithGranularitySpecNonNullQueryGranularityAndNullSegmentGranular // day queryGranularity final CompactionTask compactionTask1 = builder .interval(Intervals.of("2014-01-01/2014-01-02")) - .granularitySpec(new CompactionGranularitySpec(null, Granularities.SECOND)) + .granularitySpec(new ClientCompactionTaskQueryGranularitySpec(null, Granularities.SECOND)) .build(); Pair> resultPair = runTask(compactionTask1); @@ -701,7 +701,7 @@ public void testWithGranularitySpecNonNullQueryGranularityAndNonNullSegmentGranu // day segmentGranularity and day queryGranularity final CompactionTask compactionTask1 = builder .interval(Intervals.of("2014-01-01/2014-01-02")) - .granularitySpec(new CompactionGranularitySpec(Granularities.DAY, Granularities.DAY)) + .granularitySpec(new ClientCompactionTaskQueryGranularitySpec(Granularities.DAY, Granularities.DAY)) .build(); Pair> resultPair = runTask(compactionTask1); @@ -733,7 +733,7 @@ public void testWithGranularitySpecNullQueryGranularityAndNullSegmentGranularity final CompactionTask compactionTask1 = builder .interval(Intervals.of("2014-01-01/2014-01-02")) - .granularitySpec(new CompactionGranularitySpec(null, null)) + .granularitySpec(new ClientCompactionTaskQueryGranularitySpec(null, null)) .build(); Pair> resultPair = runTask(compactionTask1); 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 e96d56bd3ea0..3aa2ffb7b2eb 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 @@ -36,6 +36,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.client.indexing.ClientCompactionTaskQueryGranularitySpec; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.NoopIndexingServiceClient; import org.apache.druid.common.guava.SettableSupplier; @@ -108,7 +109,6 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.indexing.TuningConfig; -import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.SegmentLoadingException; @@ -377,7 +377,7 @@ public void testCreateCompactionTaskWithGranularitySpec() ); builder2.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder2.tuningConfig(createTuningConfig()); - builder2.granularitySpec(new CompactionGranularitySpec(Granularities.HOUR, Granularities.DAY)); + builder2.granularitySpec(new ClientCompactionTaskQueryGranularitySpec(Granularities.HOUR, Granularities.DAY)); final CompactionTask taskCreatedWithGranularitySpec = builder2.build(); Assert.assertEquals(taskCreatedWithGranularitySpec.getSegmentGranularity(), taskCreatedWithSegmentGranularity.getSegmentGranularity()); } @@ -393,7 +393,7 @@ public void testCreateCompactionTaskWithGranularitySpecOverrideSegmentGranularit builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); builder.segmentGranularity(Granularities.HOUR); - builder.granularitySpec(new CompactionGranularitySpec(Granularities.MINUTE, Granularities.DAY)); + builder.granularitySpec(new ClientCompactionTaskQueryGranularitySpec(Granularities.MINUTE, Granularities.DAY)); final CompactionTask taskCreatedWithSegmentGranularity = builder.build(); Assert.assertEquals(Granularities.MINUTE, taskCreatedWithSegmentGranularity.getSegmentGranularity()); } @@ -1067,7 +1067,7 @@ public void testSegmentGranularityAndNullQueryGranularity() throws IOException, new PartitionConfigurationManager(TUNING_CONFIG), null, null, - new CompactionGranularitySpec(new PeriodGranularity(Period.months(3), null, null), null), + new ClientCompactionTaskQueryGranularitySpec(new PeriodGranularity(Period.months(3), null, null), null), COORDINATOR_CLIENT, segmentLoaderFactory, RETRY_POLICY_FACTORY @@ -1103,7 +1103,7 @@ public void testQueryGranularityAndNullSegmentGranularity() throws IOException, new PartitionConfigurationManager(TUNING_CONFIG), null, null, - new CompactionGranularitySpec(null, new PeriodGranularity(Period.months(3), null, null)), + new ClientCompactionTaskQueryGranularitySpec(null, new PeriodGranularity(Period.months(3), null, null)), COORDINATOR_CLIENT, segmentLoaderFactory, RETRY_POLICY_FACTORY @@ -1137,7 +1137,7 @@ public void testQueryGranularityAndSegmentGranularityNonNull() throws IOExceptio new PartitionConfigurationManager(TUNING_CONFIG), null, null, - new CompactionGranularitySpec( + new ClientCompactionTaskQueryGranularitySpec( new PeriodGranularity(Period.months(3), null, null), new PeriodGranularity(Period.months(3), null, null) ), @@ -1210,7 +1210,7 @@ public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity new PartitionConfigurationManager(TUNING_CONFIG), null, null, - new CompactionGranularitySpec(null, null), + new ClientCompactionTaskQueryGranularitySpec(null, null), COORDINATOR_CLIENT, segmentLoaderFactory, RETRY_POLICY_FACTORY 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 7605b2c63e47..4308f73111ef 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 @@ -31,10 +31,10 @@ import org.apache.druid.java.util.common.granularity.Granularities; 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.CompactionGranularitySpec; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.clients.CompactionResourceTestClient; @@ -297,7 +297,7 @@ public void testAutoCompactionDutyWithSegmentGranularity() throws Exception verifyQuery(INDEX_QUERIES_RESOURCE); Granularity newGranularity = Granularities.YEAR; - submitCompactionConfig(1000, NO_SKIP_OFFSET, new CompactionGranularitySpec(newGranularity, null)); + submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null)); LOG.info("Auto compaction test with YEAR segment granularity"); @@ -313,7 +313,7 @@ public void testAutoCompactionDutyWithSegmentGranularity() throws Exception checkCompactionIntervals(expectedIntervalAfterCompaction); newGranularity = Granularities.DAY; - submitCompactionConfig(1000, NO_SKIP_OFFSET, new CompactionGranularitySpec(newGranularity, null)); + submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null)); LOG.info("Auto compaction test with DAY segment granularity"); @@ -373,7 +373,7 @@ private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffset submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, null); } - private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest, CompactionGranularitySpec granularitySpec) throws Exception + private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest, UserCompactionTaskGranularityConfig granularitySpec) throws Exception { submitCompactionConfig(new DynamicPartitionsSpec(maxRowsPerSegment, null), skipOffsetFromLatest, 1, granularitySpec); } @@ -382,7 +382,7 @@ private void submitCompactionConfig( PartitionsSpec partitionsSpec, Period skipOffsetFromLatest, int maxNumConcurrentSubTasks, - CompactionGranularitySpec granularitySpec + UserCompactionTaskGranularityConfig granularitySpec ) throws Exception { DataSourceCompactionConfig compactionConfig = new DataSourceCompactionConfig( diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryGranularitySpec.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryGranularitySpec.java index 6ebe9f0f9da9..a5736e73773e 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryGranularitySpec.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryGranularitySpec.java @@ -52,6 +52,11 @@ public Granularity getQueryGranularity() return queryGranularity; } + public ClientCompactionTaskQueryGranularitySpec withSegmentGranularity(Granularity segmentGranularity) + { + return new ClientCompactionTaskQueryGranularitySpec(segmentGranularity, queryGranularity); + } + @Override public boolean equals(Object o) { 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 fa550c17d5ce..d64eb08de38a 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 @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; import org.joda.time.Period; import javax.annotation.Nullable; @@ -47,7 +46,7 @@ public class DataSourceCompactionConfig private final Integer maxRowsPerSegment; private final Period skipOffsetFromLatest; private final UserCompactionTaskQueryTuningConfig tuningConfig; - private final CompactionGranularitySpec granularitySpec; + private final UserCompactionTaskGranularityConfig granularitySpec; private final Map taskContext; @JsonCreator @@ -58,7 +57,7 @@ public DataSourceCompactionConfig( @JsonProperty("maxRowsPerSegment") @Deprecated @Nullable Integer maxRowsPerSegment, @JsonProperty("skipOffsetFromLatest") @Nullable Period skipOffsetFromLatest, @JsonProperty("tuningConfig") @Nullable UserCompactionTaskQueryTuningConfig tuningConfig, - @JsonProperty("granularitySpec") @Nullable CompactionGranularitySpec granularitySpec, + @JsonProperty("granularitySpec") @Nullable UserCompactionTaskGranularityConfig granularitySpec, @JsonProperty("taskContext") @Nullable Map taskContext ) { @@ -122,7 +121,7 @@ public UserCompactionTaskQueryTuningConfig getTuningConfig() @JsonProperty @Nullable - public CompactionGranularitySpec getGranularitySpec() + public UserCompactionTaskGranularityConfig getGranularitySpec() { return granularitySpec; } diff --git a/server/src/main/java/org/apache/druid/segment/indexing/granularity/CompactionGranularitySpec.java b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java similarity index 84% rename from server/src/main/java/org/apache/druid/segment/indexing/granularity/CompactionGranularitySpec.java rename to server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java index 1b71f69bf8da..ca3a1763fd59 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/granularity/CompactionGranularitySpec.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.indexing.granularity; +package org.apache.druid.server.coordinator; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -25,13 +25,13 @@ import java.util.Objects; -public class CompactionGranularitySpec +public class UserCompactionTaskGranularityConfig { private final Granularity segmentGranularity; private final Granularity queryGranularity; @JsonCreator - public CompactionGranularitySpec( + public UserCompactionTaskGranularityConfig( @JsonProperty("segmentGranularity") Granularity segmentGranularity, @JsonProperty("queryGranularity") Granularity queryGranularity ) @@ -52,11 +52,6 @@ public Granularity getQueryGranularity() return queryGranularity; } - public CompactionGranularitySpec withSegmentGranularity(Granularity segmentGranularity) - { - return new CompactionGranularitySpec(segmentGranularity, queryGranularity); - } - @Override public boolean equals(Object o) { @@ -66,7 +61,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - CompactionGranularitySpec that = (CompactionGranularitySpec) o; + UserCompactionTaskGranularityConfig that = (UserCompactionTaskGranularityConfig) o; return Objects.equals(segmentGranularity, that.segmentGranularity) && Objects.equals(queryGranularity, that.queryGranularity); } @@ -80,7 +75,7 @@ public int hashCode() @Override public String toString() { - return "CompactionGranularitySpec{" + + return "UserCompactionTaskGranularityConfig{" + "segmentGranularity=" + segmentGranularity + ", queryGranularity=" + queryGranularity + '}'; 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 b589d2385fcc..efe04e37ccbb 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 @@ -30,7 +30,6 @@ import org.apache.druid.segment.data.BitmapSerde.DefaultBitmapSerdeFactory; import org.apache.druid.segment.data.CompressionFactory.LongEncodingStrategy; import org.apache.druid.segment.data.CompressionStrategy; -import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import org.joda.time.Duration; import org.joda.time.Period; @@ -235,7 +234,7 @@ public void testSerdeGranularitySpec() throws IOException null, new Period(3600), null, - new CompactionGranularitySpec(Granularities.HOUR, null), + new UserCompactionTaskGranularityConfig(Granularities.HOUR, null), ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); @@ -261,7 +260,7 @@ public void testFailIfGranularitySpecContainsNonDefaultQueryGranularity() null, new Period(3600), null, - new CompactionGranularitySpec(Granularities.HOUR, Granularities.MONTH), + new UserCompactionTaskGranularityConfig(Granularities.HOUR, Granularities.MONTH), ImmutableMap.of("key", "val") ); } @@ -302,7 +301,7 @@ public void testSerdeGranularitySpecWithNullValues() throws IOException null, new Period(3600), null, - new CompactionGranularitySpec(null, null), + new UserCompactionTaskGranularityConfig(null, null), ImmutableMap.of("key", "val") ); final String json = OBJECT_MAPPER.writeValueAsString(config); From c4668c6e9d0c3e7cdb57cd4f0683cadfbf20ad9c Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Fri, 26 Feb 2021 13:24:24 -0800 Subject: [PATCH 10/14] rename class --- .../indexing/common/task/CompactionTask.java | 20 +++++++++---------- .../ClientCompactionTaskQuerySerdeTest.java | 8 ++++---- .../common/task/CompactionTaskRunTest.java | 12 +++++------ .../common/task/CompactionTaskTest.java | 14 ++++++------- ... ClientCompactionTaskGranularitySpec.java} | 12 +++++------ .../indexing/ClientCompactionTaskQuery.java | 6 +++--- .../indexing/HttpIndexingServiceClient.java | 2 +- .../indexing/IndexingServiceClient.java | 2 +- .../coordinator/duty/CompactSegments.java | 6 +++--- .../indexing/NoopIndexingServiceClient.java | 2 +- .../coordinator/duty/CompactSegmentsTest.java | 13 +++++++----- 11 files changed, 50 insertions(+), 47 deletions(-) rename server/src/main/java/org/apache/druid/client/indexing/{ClientCompactionTaskQueryGranularitySpec.java => ClientCompactionTaskGranularitySpec.java} (82%) 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 904c99815fb0..04c622d3e7cd 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 @@ -31,8 +31,8 @@ import com.google.common.collect.Lists; import org.apache.curator.shaded.com.google.common.base.Verify; import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; -import org.apache.druid.client.indexing.ClientCompactionTaskQueryGranularitySpec; import org.apache.druid.common.guava.SettableSupplier; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.DimensionSchema; @@ -143,7 +143,7 @@ public class CompactionTask extends AbstractBatchIndexTask @Nullable private final Granularity segmentGranularity; @Nullable - private final ClientCompactionTaskQueryGranularitySpec granularitySpec; + private final ClientCompactionTaskGranularitySpec granularitySpec; @Nullable private final ParallelIndexTuningConfig tuningConfig; @JsonIgnore @@ -177,7 +177,7 @@ public CompactionTask( @JsonProperty("dimensionsSpec") @Nullable final DimensionsSpec dimensionsSpec, @JsonProperty("metricsSpec") @Nullable final AggregatorFactory[] metricsSpec, @JsonProperty("segmentGranularity") @Deprecated @Nullable final Granularity segmentGranularity, - @JsonProperty("granularitySpec") @Nullable final ClientCompactionTaskQueryGranularitySpec granularitySpec, + @JsonProperty("granularitySpec") @Nullable final ClientCompactionTaskGranularitySpec granularitySpec, @JsonProperty("tuningConfig") @Nullable final TuningConfig tuningConfig, @JsonProperty("context") @Nullable final Map context, @JacksonInject SegmentLoaderFactory segmentLoaderFactory, @@ -208,7 +208,7 @@ public CompactionTask( this.metricsSpec = metricsSpec; this.segmentGranularity = segmentGranularity; if (granularitySpec == null && segmentGranularity != null) { - this.granularitySpec = new ClientCompactionTaskQueryGranularitySpec(segmentGranularity, null); + this.granularitySpec = new ClientCompactionTaskGranularitySpec(segmentGranularity, null); } else { this.granularitySpec = granularitySpec; } @@ -303,7 +303,7 @@ public Granularity getSegmentGranularity() @JsonProperty @Nullable - public ClientCompactionTaskQueryGranularitySpec getGranularitySpec() + public ClientCompactionTaskGranularitySpec getGranularitySpec() { return granularitySpec; } @@ -473,7 +473,7 @@ static List createIngestionSchema( final PartitionConfigurationManager partitionConfigurationManager, @Nullable final DimensionsSpec dimensionsSpec, @Nullable final AggregatorFactory[] metricsSpec, - @Nullable final ClientCompactionTaskQueryGranularitySpec granularitySpec, + @Nullable final ClientCompactionTaskGranularitySpec granularitySpec, final CoordinatorClient coordinatorClient, final SegmentLoaderFactory segmentLoaderFactory, final RetryPolicyFactory retryPolicyFactory @@ -544,7 +544,7 @@ static List createIngestionSchema( segmentsToCompact, dimensionsSpec, metricsSpec, - granularitySpec == null ? new ClientCompactionTaskQueryGranularitySpec(segmentGranularityToUse, null) : granularitySpec.withSegmentGranularity(segmentGranularityToUse) + granularitySpec == null ? new ClientCompactionTaskGranularitySpec(segmentGranularityToUse, null) : granularitySpec.withSegmentGranularity(segmentGranularityToUse) ); specs.add( @@ -639,7 +639,7 @@ private static DataSchema createDataSchema( List> queryableIndexAndSegments, @Nullable DimensionsSpec dimensionsSpec, @Nullable AggregatorFactory[] metricsSpec, - @Nonnull ClientCompactionTaskQueryGranularitySpec granularitySpec + @Nonnull ClientCompactionTaskGranularitySpec granularitySpec ) { // check index metadata & @@ -970,7 +970,7 @@ public static class Builder @Nullable private Granularity segmentGranularity; @Nullable - private ClientCompactionTaskQueryGranularitySpec granularitySpec; + private ClientCompactionTaskGranularitySpec granularitySpec; @Nullable private TuningConfig tuningConfig; @Nullable @@ -1021,7 +1021,7 @@ public Builder segmentGranularity(Granularity segmentGranularity) return this; } - public Builder granularitySpec(ClientCompactionTaskQueryGranularitySpec granularitySpec) + public Builder granularitySpec(ClientCompactionTaskGranularitySpec granularitySpec) { this.granularitySpec = granularitySpec; return this; 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 4645ceff2577..db39b969507f 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 @@ -27,8 +27,8 @@ import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.client.indexing.ClientCompactionIOConfig; import org.apache.druid.client.indexing.ClientCompactionIntervalSpec; +import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; -import org.apache.druid.client.indexing.ClientCompactionTaskQueryGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.client.indexing.ClientTaskQuery; import org.apache.druid.client.indexing.IndexingServiceClient; @@ -115,7 +115,7 @@ public void testClientCompactionTaskQueryToCompactionTask() throws IOException 1000, 100 ), - new ClientCompactionTaskQueryGranularitySpec(Granularities.DAY, Granularities.HOUR), + new ClientCompactionTaskGranularitySpec(Granularities.DAY, Granularities.HOUR), ImmutableMap.of("key", "value") ); @@ -254,7 +254,7 @@ public void testCompactionTaskToClientCompactionTaskQuery() throws IOException null ) ) - .granularitySpec(new ClientCompactionTaskQueryGranularitySpec(Granularities.DAY, Granularities.HOUR)) + .granularitySpec(new ClientCompactionTaskGranularitySpec(Granularities.DAY, Granularities.HOUR)) .build(); final ClientCompactionTaskQuery expected = new ClientCompactionTaskQuery( @@ -296,7 +296,7 @@ public void testCompactionTaskToClientCompactionTaskQuery() throws IOException 1000, 100 ), - new ClientCompactionTaskQueryGranularitySpec(Granularities.DAY, Granularities.HOUR), + new ClientCompactionTaskGranularitySpec(Granularities.DAY, Granularities.HOUR), new HashMap<>() ); 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 b571ffac53d8..95f0aaf16c73 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 @@ -26,7 +26,7 @@ import com.google.common.collect.Ordering; import com.google.common.io.Files; import org.apache.druid.client.coordinator.CoordinatorClient; -import org.apache.druid.client.indexing.ClientCompactionTaskQueryGranularitySpec; +import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.NoopIndexingServiceClient; import org.apache.druid.data.input.impl.CSVParseSpec; @@ -601,7 +601,7 @@ public void testWithGranularitySpecNonNullSegmentGranularityAndNullQueryGranular // day segmentGranularity final CompactionTask compactionTask1 = builder .interval(Intervals.of("2014-01-01/2014-01-02")) - .granularitySpec(new ClientCompactionTaskQueryGranularitySpec(Granularities.DAY, null)) + .granularitySpec(new ClientCompactionTaskGranularitySpec(Granularities.DAY, null)) .build(); Pair> resultPair = runTask(compactionTask1); @@ -622,7 +622,7 @@ public void testWithGranularitySpecNonNullSegmentGranularityAndNullQueryGranular // hour segmentGranularity final CompactionTask compactionTask2 = builder .interval(Intervals.of("2014-01-01/2014-01-02")) - .granularitySpec(new ClientCompactionTaskQueryGranularitySpec(Granularities.HOUR, null)) + .granularitySpec(new ClientCompactionTaskGranularitySpec(Granularities.HOUR, null)) .build(); resultPair = runTask(compactionTask2); @@ -656,7 +656,7 @@ public void testWithGranularitySpecNonNullQueryGranularityAndNullSegmentGranular // day queryGranularity final CompactionTask compactionTask1 = builder .interval(Intervals.of("2014-01-01/2014-01-02")) - .granularitySpec(new ClientCompactionTaskQueryGranularitySpec(null, Granularities.SECOND)) + .granularitySpec(new ClientCompactionTaskGranularitySpec(null, Granularities.SECOND)) .build(); Pair> resultPair = runTask(compactionTask1); @@ -701,7 +701,7 @@ public void testWithGranularitySpecNonNullQueryGranularityAndNonNullSegmentGranu // day segmentGranularity and day queryGranularity final CompactionTask compactionTask1 = builder .interval(Intervals.of("2014-01-01/2014-01-02")) - .granularitySpec(new ClientCompactionTaskQueryGranularitySpec(Granularities.DAY, Granularities.DAY)) + .granularitySpec(new ClientCompactionTaskGranularitySpec(Granularities.DAY, Granularities.DAY)) .build(); Pair> resultPair = runTask(compactionTask1); @@ -733,7 +733,7 @@ public void testWithGranularitySpecNullQueryGranularityAndNullSegmentGranularity final CompactionTask compactionTask1 = builder .interval(Intervals.of("2014-01-01/2014-01-02")) - .granularitySpec(new ClientCompactionTaskQueryGranularitySpec(null, null)) + .granularitySpec(new ClientCompactionTaskGranularitySpec(null, null)) .build(); Pair> resultPair = runTask(compactionTask1); 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 3aa2ffb7b2eb..b0deee546550 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 @@ -36,7 +36,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.druid.client.coordinator.CoordinatorClient; -import org.apache.druid.client.indexing.ClientCompactionTaskQueryGranularitySpec; +import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.NoopIndexingServiceClient; import org.apache.druid.common.guava.SettableSupplier; @@ -377,7 +377,7 @@ public void testCreateCompactionTaskWithGranularitySpec() ); builder2.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder2.tuningConfig(createTuningConfig()); - builder2.granularitySpec(new ClientCompactionTaskQueryGranularitySpec(Granularities.HOUR, Granularities.DAY)); + builder2.granularitySpec(new ClientCompactionTaskGranularitySpec(Granularities.HOUR, Granularities.DAY)); final CompactionTask taskCreatedWithGranularitySpec = builder2.build(); Assert.assertEquals(taskCreatedWithGranularitySpec.getSegmentGranularity(), taskCreatedWithSegmentGranularity.getSegmentGranularity()); } @@ -393,7 +393,7 @@ public void testCreateCompactionTaskWithGranularitySpecOverrideSegmentGranularit builder.inputSpec(new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS))); builder.tuningConfig(createTuningConfig()); builder.segmentGranularity(Granularities.HOUR); - builder.granularitySpec(new ClientCompactionTaskQueryGranularitySpec(Granularities.MINUTE, Granularities.DAY)); + builder.granularitySpec(new ClientCompactionTaskGranularitySpec(Granularities.MINUTE, Granularities.DAY)); final CompactionTask taskCreatedWithSegmentGranularity = builder.build(); Assert.assertEquals(Granularities.MINUTE, taskCreatedWithSegmentGranularity.getSegmentGranularity()); } @@ -1067,7 +1067,7 @@ public void testSegmentGranularityAndNullQueryGranularity() throws IOException, new PartitionConfigurationManager(TUNING_CONFIG), null, null, - new ClientCompactionTaskQueryGranularitySpec(new PeriodGranularity(Period.months(3), null, null), null), + new ClientCompactionTaskGranularitySpec(new PeriodGranularity(Period.months(3), null, null), null), COORDINATOR_CLIENT, segmentLoaderFactory, RETRY_POLICY_FACTORY @@ -1103,7 +1103,7 @@ public void testQueryGranularityAndNullSegmentGranularity() throws IOException, new PartitionConfigurationManager(TUNING_CONFIG), null, null, - new ClientCompactionTaskQueryGranularitySpec(null, new PeriodGranularity(Period.months(3), null, null)), + new ClientCompactionTaskGranularitySpec(null, new PeriodGranularity(Period.months(3), null, null)), COORDINATOR_CLIENT, segmentLoaderFactory, RETRY_POLICY_FACTORY @@ -1137,7 +1137,7 @@ public void testQueryGranularityAndSegmentGranularityNonNull() throws IOExceptio new PartitionConfigurationManager(TUNING_CONFIG), null, null, - new ClientCompactionTaskQueryGranularitySpec( + new ClientCompactionTaskGranularitySpec( new PeriodGranularity(Period.months(3), null, null), new PeriodGranularity(Period.months(3), null, null) ), @@ -1210,7 +1210,7 @@ public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity new PartitionConfigurationManager(TUNING_CONFIG), null, null, - new ClientCompactionTaskQueryGranularitySpec(null, null), + new ClientCompactionTaskGranularitySpec(null, null), COORDINATOR_CLIENT, segmentLoaderFactory, RETRY_POLICY_FACTORY diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryGranularitySpec.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskGranularitySpec.java similarity index 82% rename from server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryGranularitySpec.java rename to server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskGranularitySpec.java index a5736e73773e..6a1072b0a858 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryGranularitySpec.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskGranularitySpec.java @@ -25,13 +25,13 @@ import java.util.Objects; -public class ClientCompactionTaskQueryGranularitySpec +public class ClientCompactionTaskGranularitySpec { private final Granularity segmentGranularity; private final Granularity queryGranularity; @JsonCreator - public ClientCompactionTaskQueryGranularitySpec( + public ClientCompactionTaskGranularitySpec( @JsonProperty("segmentGranularity") Granularity segmentGranularity, @JsonProperty("queryGranularity") Granularity queryGranularity ) @@ -52,9 +52,9 @@ public Granularity getQueryGranularity() return queryGranularity; } - public ClientCompactionTaskQueryGranularitySpec withSegmentGranularity(Granularity segmentGranularity) + public ClientCompactionTaskGranularitySpec withSegmentGranularity(Granularity segmentGranularity) { - return new ClientCompactionTaskQueryGranularitySpec(segmentGranularity, queryGranularity); + return new ClientCompactionTaskGranularitySpec(segmentGranularity, queryGranularity); } @Override @@ -66,7 +66,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - ClientCompactionTaskQueryGranularitySpec that = (ClientCompactionTaskQueryGranularitySpec) o; + ClientCompactionTaskGranularitySpec that = (ClientCompactionTaskGranularitySpec) o; return Objects.equals(segmentGranularity, that.segmentGranularity) && Objects.equals(queryGranularity, that.queryGranularity); } @@ -80,7 +80,7 @@ public int hashCode() @Override public String toString() { - return "ClientCompactionTaskQueryGranularitySpec{" + + return "ClientCompactionTaskGranularitySpec{" + "segmentGranularity=" + segmentGranularity + ", queryGranularity=" + queryGranularity + '}'; 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 ea4155525db3..62e39d62610f 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 @@ -38,7 +38,7 @@ public class ClientCompactionTaskQuery implements ClientTaskQuery private final String dataSource; private final ClientCompactionIOConfig ioConfig; private final ClientCompactionTaskQueryTuningConfig tuningConfig; - private final ClientCompactionTaskQueryGranularitySpec granularitySpec; + private final ClientCompactionTaskGranularitySpec granularitySpec; private final Map context; @JsonCreator @@ -47,7 +47,7 @@ public ClientCompactionTaskQuery( @JsonProperty("dataSource") String dataSource, @JsonProperty("ioConfig") ClientCompactionIOConfig ioConfig, @JsonProperty("tuningConfig") ClientCompactionTaskQueryTuningConfig tuningConfig, - @JsonProperty("granularitySpec") ClientCompactionTaskQueryGranularitySpec granularitySpec, + @JsonProperty("granularitySpec") ClientCompactionTaskGranularitySpec granularitySpec, @JsonProperty("context") Map context ) { @@ -93,7 +93,7 @@ public ClientCompactionTaskQueryTuningConfig getTuningConfig() } @JsonProperty - public ClientCompactionTaskQueryGranularitySpec getGranularitySpec() + public ClientCompactionTaskGranularitySpec getGranularitySpec() { return granularitySpec; } diff --git a/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java b/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java index 321831feb7cb..bc9f9c56da9c 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java +++ b/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java @@ -79,7 +79,7 @@ public String compactSegments( List segments, int compactionTaskPriority, @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig, - @Nullable ClientCompactionTaskQueryGranularitySpec granularitySpec, + @Nullable ClientCompactionTaskGranularitySpec granularitySpec, @Nullable Map context ) { diff --git a/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java b/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java index 8eb803434678..f7c9712a2250 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java +++ b/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java @@ -40,7 +40,7 @@ String compactSegments( List segments, int compactionTaskPriority, @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig, - @Nullable ClientCompactionTaskQueryGranularitySpec granularitySpec, + @Nullable ClientCompactionTaskGranularitySpec granularitySpec, @Nullable Map context ); 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 0f7e9f914de2..b0477586e627 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 @@ -23,8 +23,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Maps; import com.google.inject.Inject; +import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; -import org.apache.druid.client.indexing.ClientCompactionTaskQueryGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.TaskPayloadResponse; @@ -310,9 +310,9 @@ private CoordinatorStats doRun( snapshotBuilder.incrementSegmentCountCompacted(segmentsToCompact.size()); final DataSourceCompactionConfig config = compactionConfigs.get(dataSourceName); - ClientCompactionTaskQueryGranularitySpec queryGranularitySpec; + ClientCompactionTaskGranularitySpec queryGranularitySpec; if (config.getGranularitySpec() != null) { - queryGranularitySpec = new ClientCompactionTaskQueryGranularitySpec( + queryGranularitySpec = new ClientCompactionTaskGranularitySpec( config.getGranularitySpec().getSegmentGranularity(), config.getGranularitySpec().getQueryGranularity() ); diff --git a/server/src/test/java/org/apache/druid/client/indexing/NoopIndexingServiceClient.java b/server/src/test/java/org/apache/druid/client/indexing/NoopIndexingServiceClient.java index 09f4c693fb1f..c5a85fc4dcd3 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/NoopIndexingServiceClient.java +++ b/server/src/test/java/org/apache/druid/client/indexing/NoopIndexingServiceClient.java @@ -50,7 +50,7 @@ public String compactSegments( List segments, int compactionTaskPriority, @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig, - @Nullable ClientCompactionTaskQueryGranularitySpec granularitySpec, + @Nullable ClientCompactionTaskGranularitySpec granularitySpec, @Nullable Map context ) { 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 0288fc3a0014..915f73d339e2 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 @@ -30,8 +30,8 @@ import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.indexing.ClientCompactionIOConfig; import org.apache.druid.client.indexing.ClientCompactionIntervalSpec; +import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; -import org.apache.druid.client.indexing.ClientCompactionTaskQueryGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.client.indexing.ClientTaskQuery; import org.apache.druid.client.indexing.HttpIndexingServiceClient; @@ -597,7 +597,8 @@ public void testCompactWithoutGranularitySpec() ); doCompactSegments(compactSegments, compactionConfigs); ArgumentCaptor> segmentsCaptor = ArgumentCaptor.forClass(List.class); - ArgumentCaptor granularitySpecArgumentCaptor = ArgumentCaptor.forClass(ClientCompactionTaskQueryGranularitySpec.class); + ArgumentCaptor granularitySpecArgumentCaptor = ArgumentCaptor.forClass( + ClientCompactionTaskGranularitySpec.class); Mockito.verify(mockIndexingServiceClient).compactSegments( ArgumentMatchers.anyString(), segmentsCaptor.capture(), @@ -650,7 +651,8 @@ public void testCompactWithGranularitySpec() ); doCompactSegments(compactSegments, compactionConfigs); ArgumentCaptor> segmentsCaptor = ArgumentCaptor.forClass(List.class); - ArgumentCaptor granularitySpecArgumentCaptor = ArgumentCaptor.forClass(ClientCompactionTaskQueryGranularitySpec.class); + ArgumentCaptor granularitySpecArgumentCaptor = ArgumentCaptor.forClass( + ClientCompactionTaskGranularitySpec.class); Mockito.verify(mockIndexingServiceClient).compactSegments( ArgumentMatchers.anyString(), segmentsCaptor.capture(), @@ -696,7 +698,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() ) ), null, - new ClientCompactionTaskQueryGranularitySpec(Granularities.DAY, null), + new ClientCompactionTaskGranularitySpec(Granularities.DAY, null), null ) ); @@ -742,7 +744,8 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() // Make sure that we do not skip interval of conflict task. // Since we cancel the task and will have to compact those intervals with the new segmentGranulartity ArgumentCaptor> segmentsCaptor = ArgumentCaptor.forClass(List.class); - ArgumentCaptor granularitySpecArgumentCaptor = ArgumentCaptor.forClass(ClientCompactionTaskQueryGranularitySpec.class); + ArgumentCaptor granularitySpecArgumentCaptor = ArgumentCaptor.forClass( + ClientCompactionTaskGranularitySpec.class); Mockito.verify(mockIndexingServiceClient).compactSegments( ArgumentMatchers.anyString(), segmentsCaptor.capture(), From ecd418971dec46f548726884a0741ced3b256a48 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Mon, 1 Mar 2021 13:25:03 -0800 Subject: [PATCH 11/14] fix test --- .../coordinator/duty/CompactSegmentsTest.java | 1 - .../duty/NewestSegmentFirstPolicyTest.java | 16 ++++++++-------- 2 files changed, 8 insertions(+), 9 deletions(-) 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 915f73d339e2..e5c950a0e7a7 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 @@ -58,7 +58,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; -import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/NewestSegmentFirstPolicyTest.java index a4f3bb6bbc20..5f7682f5d949 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/NewestSegmentFirstPolicyTest.java @@ -29,8 +29,8 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Comparators; -import org.apache.druid.segment.indexing.granularity.CompactionGranularitySpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.Partitions; import org.apache.druid.timeline.VersionedIntervalTimeline; @@ -382,7 +382,7 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularityEqual() ); final CompactionSegmentIterator iterator = policy.reset( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new CompactionGranularitySpec(Granularities.DAY, null))), + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new UserCompactionTaskGranularityConfig(Granularities.DAY, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() ); @@ -409,7 +409,7 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularityLarger() ); final CompactionSegmentIterator iterator = policy.reset( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new CompactionGranularitySpec(Granularities.MONTH, null))), + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new UserCompactionTaskGranularityConfig(Granularities.MONTH, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() ); @@ -435,7 +435,7 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularitySmaller() ); final CompactionSegmentIterator iterator = policy.reset( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new CompactionGranularitySpec(Granularities.MINUTE, null))), + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new UserCompactionTaskGranularityConfig(Granularities.MINUTE, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() ); @@ -544,7 +544,7 @@ public void testIteratorReturnsSegmentsInConfiguredSegmentGranularity() ); final CompactionSegmentIterator iterator = policy.reset( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new CompactionGranularitySpec(Granularities.MONTH, null))), + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.MONTH, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() ); @@ -593,7 +593,7 @@ public void testIteratorReturnsSegmentsInMultipleIntervalIfConfiguredSegmentGran ); final CompactionSegmentIterator iterator = policy.reset( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new CompactionGranularitySpec(Granularities.MONTH, null))), + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.MONTH, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() ); @@ -628,7 +628,7 @@ public void testIteratorDoesNotReturnCompactedInterval() ); final CompactionSegmentIterator iterator = policy.reset( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P0D"), new CompactionGranularitySpec(Granularities.MINUTE, null))), + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.MINUTE, null))), ImmutableMap.of(DATA_SOURCE, timeline), Collections.emptyMap() ); @@ -734,7 +734,7 @@ private static VersionedIntervalTimeline createTimeline( private DataSourceCompactionConfig createCompactionConfig( long inputSegmentSizeBytes, Period skipOffsetFromLatest, - CompactionGranularitySpec granularitySpec + UserCompactionTaskGranularityConfig granularitySpec ) { return new DataSourceCompactionConfig( From 74c4f0e4db7c8dd7a0234820467511d6079360bb Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Mon, 1 Mar 2021 13:38:13 -0800 Subject: [PATCH 12/14] fix test --- .../druid/server/coordinator/duty/CompactSegmentsTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 e5c950a0e7a7..7363ce94a87b 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 @@ -65,6 +65,7 @@ import org.apache.druid.server.coordinator.CoordinatorStats; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; @@ -644,7 +645,7 @@ public void testCompactWithGranularitySpec() null, null ), - new CompactionGranularitySpec(Granularities.YEAR, null), + new UserCompactionTaskGranularityConfig(Granularities.YEAR, null), null ) ); @@ -732,7 +733,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() null, null ), - new CompactionGranularitySpec(Granularities.YEAR, null), + new UserCompactionTaskGranularityConfig(Granularities.YEAR, null), null ) ); @@ -812,7 +813,6 @@ private void assertCompactSegmentStatistics(CompactSegments compactSegments, int 1, stats.getGlobalStat(CompactSegments.COMPACTION_TASK_COUNT) ); - Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); // Note: Subsequent compaction run after the dataSource was compacted will show different numbers than // on the run it was compacted. For example, in a compaction run, if a dataSource had 4 segments compacted, // on the same compaction run the segment compressed count will be 4 but on subsequent run it might be 2 From da3cf03a1be84ae7d7bebbc618d971ca88f73867 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Mon, 1 Mar 2021 23:40:35 -0800 Subject: [PATCH 13/14] add test --- .../common/task/ClientCompactionTaskQuerySerdeTest.java | 4 ++++ 1 file changed, 4 insertions(+) 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 db39b969507f..a73e44b7dc36 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 @@ -189,6 +189,10 @@ public void testClientCompactionTaskQueryToCompactionTask() throws IOException query.getTuningConfig().getTotalNumMergeTasks().intValue(), task.getTuningConfig().getTotalNumMergeTasks() ); + Assert.assertEquals( + query.getGranularitySpec(), + task.getGranularitySpec() + ); Assert.assertEquals( query.getGranularitySpec().getQueryGranularity(), task.getGranularitySpec().getQueryGranularity() From e8aacf347145e45e4bd63ad30ebd453a3b7caa16 Mon Sep 17 00:00:00 2001 From: Maytas Monsereenusorn Date: Tue, 2 Mar 2021 01:13:20 -0800 Subject: [PATCH 14/14] fix test --- .../server/coordinator/duty/CompactSegmentsTest.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) 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 7363ce94a87b..4bec733940cf 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 @@ -664,7 +664,10 @@ public void testCompactWithGranularitySpec() // All segments is compact at the same time since we changed the segment granularity to YEAR and all segment // are within the same year Assert.assertEquals(datasourceToSegments.get(dataSource).size(), segmentsCaptor.getValue().size()); - Assert.assertEquals(Granularities.YEAR, granularitySpecArgumentCaptor.getValue().getSegmentGranularity()); + ClientCompactionTaskGranularitySpec actual = granularitySpecArgumentCaptor.getValue(); + Assert.assertNotNull(actual); + ClientCompactionTaskGranularitySpec expected = new ClientCompactionTaskGranularitySpec(Granularities.YEAR, null); + Assert.assertEquals(expected, actual); } @Test @@ -757,7 +760,10 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() // All segments is compact at the same time since we changed the segment granularity to YEAR and all segment // are within the same year Assert.assertEquals(datasourceToSegments.get(dataSource).size(), segmentsCaptor.getValue().size()); - Assert.assertEquals(Granularities.YEAR, granularitySpecArgumentCaptor.getValue().getSegmentGranularity()); + ClientCompactionTaskGranularitySpec actual = granularitySpecArgumentCaptor.getValue(); + Assert.assertNotNull(actual); + ClientCompactionTaskGranularitySpec expected = new ClientCompactionTaskGranularitySpec(Granularities.YEAR, null); + Assert.assertEquals(expected, actual); } @Test