Fix bug in auto compaction preserveExistingMetrics feature (#12438)

* fix bug

* fix test

* fix IT
This commit is contained in:
Maytas Monsereenusorn 2022-04-15 15:47:47 -07:00 committed by GitHub
parent 0460d45e92
commit c25a556827
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 323 additions and 32 deletions

View File

@ -45,6 +45,7 @@ import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.query.aggregation.FloatSumAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildAggregatorFactory; import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildAggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory; import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory;
@ -121,6 +122,101 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
fullDatasourceName = "wikipedia_index_test_" + UUID.randomUUID() + config.getExtraDatasourceNameSuffix(); fullDatasourceName = "wikipedia_index_test_" + UUID.randomUUID() + config.getExtraDatasourceNameSuffix();
} }
@Test
public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExistingMetricsUsingAggregatorWithDifferentReturnType() throws Exception
{
// added = null, count = 2, sum_added = 62, quantilesDoublesSketch = 2, thetaSketch = 2, HLLSketchBuild = 2
loadData(INDEX_TASK_WITH_ROLLUP_FOR_PRESERVE_METRICS);
// added = 31, count = null, sum_added = null, quantilesDoublesSketch = null, thetaSketch = null, HLLSketchBuild = null
loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS);
try (final Closeable ignored = unloader(fullDatasourceName)) {
final List<String> intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName);
intervalsBeforeCompaction.sort(null);
// 2 segments across 1 days...
verifySegmentsCount(2);
ArrayList<Object> nullList = new ArrayList<Object>();
nullList.add(null);
Map<String, Object> queryAndResultFields = ImmutableMap.of(
"%%FIELD_TO_QUERY%%", "added",
"%%EXPECTED_COUNT_RESULT%%", 2,
"%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(nullList)), ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(31))))
);
verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields);
queryAndResultFields = ImmutableMap.of(
"%%FIELD_TO_QUERY%%", "count",
"%%EXPECTED_COUNT_RESULT%%", 2,
"%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(2))), ImmutableMap.of("events", ImmutableList.of(nullList)))
);
verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields);
queryAndResultFields = ImmutableMap.of(
"%%FIELD_TO_QUERY%%", "sum_added",
"%%EXPECTED_COUNT_RESULT%%", 2,
"%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(62))), ImmutableMap.of("events", ImmutableList.of(nullList)))
);
verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields);
queryAndResultFields = ImmutableMap.of(
"%%QUANTILESRESULT%%", 2,
"%%THETARESULT%%", 2.0,
"%%HLLRESULT%%", 2
);
verifyQuery(INDEX_ROLLUP_SKETCH_QUERIES_RESOURCE, queryAndResultFields);
submitCompactionConfig(
MAX_ROWS_PER_SEGMENT_COMPACTED,
NO_SKIP_OFFSET,
new UserCompactionTaskGranularityConfig(null, null, true),
new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("language"))),
null,
new AggregatorFactory[]{
new CountAggregatorFactory("count"),
// FloatSumAggregator combine method takes in two Float but return Double
new FloatSumAggregatorFactory("sum_added", "added"),
new SketchMergeAggregatorFactory("thetaSketch", "user", 16384, true, false, null),
new HllSketchBuildAggregatorFactory("HLLSketchBuild", "user", 12, TgtHllType.HLL_4.name(), false),
new DoublesSketchAggregatorFactory("quantilesDoublesSketch", "delta", 128, 1000000000L)
},
false
);
// should now only have 1 row after compaction
// added = null, count = 3, sum_added = 93.0
forceTriggerAutoCompaction(1);
queryAndResultFields = ImmutableMap.of(
"%%FIELD_TO_QUERY%%", "added",
"%%EXPECTED_COUNT_RESULT%%", 1,
"%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(nullList)))
);
verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields);
queryAndResultFields = ImmutableMap.of(
"%%FIELD_TO_QUERY%%", "count",
"%%EXPECTED_COUNT_RESULT%%", 1,
"%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(3))))
);
verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields);
queryAndResultFields = ImmutableMap.of(
"%%FIELD_TO_QUERY%%", "sum_added",
"%%EXPECTED_COUNT_RESULT%%", 1,
"%%EXPECTED_SCAN_RESULT%%", ImmutableList.of(ImmutableMap.of("events", ImmutableList.of(ImmutableList.of(93.0f))))
);
verifyQuery(INDEX_ROLLUP_QUERIES_RESOURCE, queryAndResultFields);
queryAndResultFields = ImmutableMap.of(
"%%QUANTILESRESULT%%", 3,
"%%THETARESULT%%", 3.0,
"%%HLLRESULT%%", 3
);
verifyQuery(INDEX_ROLLUP_SKETCH_QUERIES_RESOURCE, queryAndResultFields);
verifySegmentsCompacted(1, MAX_ROWS_PER_SEGMENT_COMPACTED);
checkCompactionIntervals(intervalsBeforeCompaction);
List<TaskResponseObject> compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName);
// Verify rollup segments does not get compacted again
forceTriggerAutoCompaction(1);
List<TaskResponseObject> compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName);
Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size());
}
}
@Test @Test
public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExistingMetrics() throws Exception public void testAutoCompactionRowWithMetricAndRowWithoutMetricShouldPreserveExistingMetrics() throws Exception
{ {

View File

@ -452,13 +452,13 @@ public class OnheapIncrementalIndex extends IncrementalIndex
@Override @Override
public float getMetricFloatValue(int rowOffset, int aggOffset) public float getMetricFloatValue(int rowOffset, int aggOffset)
{ {
return getMetricHelper(getMetricAggs(), concurrentGet(rowOffset), aggOffset, Aggregator::getFloat); return ((Number) getMetricHelper(getMetricAggs(), concurrentGet(rowOffset), aggOffset, Aggregator::getFloat)).floatValue();
} }
@Override @Override
public long getMetricLongValue(int rowOffset, int aggOffset) public long getMetricLongValue(int rowOffset, int aggOffset)
{ {
return getMetricHelper(getMetricAggs(), concurrentGet(rowOffset), aggOffset, Aggregator::getLong); return ((Number) getMetricHelper(getMetricAggs(), concurrentGet(rowOffset), aggOffset, Aggregator::getLong)).longValue();
} }
@Override @Override
@ -470,7 +470,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex
@Override @Override
protected double getMetricDoubleValue(int rowOffset, int aggOffset) protected double getMetricDoubleValue(int rowOffset, int aggOffset)
{ {
return getMetricHelper(getMetricAggs(), concurrentGet(rowOffset), aggOffset, Aggregator::getDouble); return ((Number) getMetricHelper(getMetricAggs(), concurrentGet(rowOffset), aggOffset, Aggregator::getDouble)).doubleValue();
} }
@Override @Override
@ -544,7 +544,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex
* If preserveExistingMetrics flag is set, then this method will combine values from two aggregators, the aggregator * If preserveExistingMetrics flag is set, then this method will combine values from two aggregators, the aggregator
* for aggregating from input into output field and the aggregator for combining already aggregated field, as needed * for aggregating from input into output field and the aggregator for combining already aggregated field, as needed
*/ */
private <T> T getMetricHelper(AggregatorFactory[] metrics, Aggregator[] aggs, int aggOffset, Function<Aggregator, T> getMetricTypeFunction) private <T> Object getMetricHelper(AggregatorFactory[] metrics, Aggregator[] aggs, int aggOffset, Function<Aggregator, T> getMetricTypeFunction)
{ {
if (preserveExistingMetrics) { if (preserveExistingMetrics) {
// Since the preserveExistingMetrics flag is set, we will have to check and possibly retrieve the aggregated values // Since the preserveExistingMetrics flag is set, we will have to check and possibly retrieve the aggregated values
@ -564,7 +564,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex
AggregatorFactory aggregatorFactory = metrics[aggOffset]; AggregatorFactory aggregatorFactory = metrics[aggOffset];
T aggregatedFromSource = getMetricTypeFunction.apply(aggs[aggOffset]); T aggregatedFromSource = getMetricTypeFunction.apply(aggs[aggOffset]);
T aggregatedFromCombined = getMetricTypeFunction.apply(aggs[aggOffset + metrics.length]); T aggregatedFromCombined = getMetricTypeFunction.apply(aggs[aggOffset + metrics.length]);
return (T) aggregatorFactory.combine(aggregatedFromSource, aggregatedFromCombined); return aggregatorFactory.combine(aggregatedFromSource, aggregatedFromCombined);
} }
} else { } else {
// If preserveExistingMetrics flag is not set then we simply get metrics from the list of Aggregator, aggs, using the // If preserveExistingMetrics flag is not set then we simply get metrics from the list of Aggregator, aggs, using the

View File

@ -50,6 +50,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.query.aggregation.FilteredAggregatorFactory; import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
import org.apache.druid.query.aggregation.FloatSumAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.filter.BoundDimFilter; import org.apache.druid.query.filter.BoundDimFilter;
import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.filter.SelectorDimFilter;
@ -799,6 +800,74 @@ public class IncrementalIndexTest extends InitializedNullHandlingTest
} }
} }
@Test
public void testSchemaRollupWithRowWithExistingMetricsAndWithoutMetricUsingAggregatorWithDifferentReturnType() throws IndexSizeExceededException
{
AggregatorFactory[] aggregatorFactories = new AggregatorFactory[]{
new CountAggregatorFactory("count"),
// FloatSumAggregator combine method takes in two Float but return Double
new FloatSumAggregatorFactory("sum_of_x", "x")
};
final IncrementalIndex index = indexCreator.createIndex((Object) aggregatorFactories);
index.add(
new MapBasedInputRow(
1481871600000L,
Arrays.asList("name", "host"),
ImmutableMap.of("name", "name1", "host", "host", "x", 2)
)
);
index.add(
new MapBasedInputRow(
1481871600000L,
Arrays.asList("name", "host"),
ImmutableMap.of("name", "name1", "host", "host", "x", 3)
)
);
index.add(
new MapBasedInputRow(
1481871600000L,
Arrays.asList("name", "host"),
ImmutableMap.of("name", "name1", "host", "host", "count", 2, "sum_of_x", 4)
)
);
index.add(
new MapBasedInputRow(
1481871600000L,
Arrays.asList("name", "host"),
ImmutableMap.of("name", "name1", "host", "host", "count", 3, "sum_of_x", 5)
)
);
Assert.assertEquals(index.isRollup() ? 1 : 4, index.size());
Iterator<Row> iterator = index.iterator();
int rowCount = 0;
while (iterator.hasNext()) {
rowCount++;
Row row = iterator.next();
Assert.assertEquals(1481871600000L, row.getTimestampFromEpoch());
if (index.isRollup()) {
// All rows are rollup into one row
Assert.assertEquals(isPreserveExistingMetrics ? 7 : 4, row.getMetric("count").intValue());
Assert.assertEquals(isPreserveExistingMetrics ? 14 : 5, row.getMetric("sum_of_x").intValue());
} else {
// We still have 4 rows
if (rowCount == 1 || rowCount == 2) {
Assert.assertEquals(1, row.getMetric("count").intValue());
Assert.assertEquals(1 + rowCount, row.getMetric("sum_of_x").intValue());
} else {
if (isPreserveExistingMetrics) {
Assert.assertEquals(rowCount - 1, row.getMetric("count").intValue());
Assert.assertEquals(1 + rowCount, row.getMetric("sum_of_x").intValue());
} else {
Assert.assertEquals(1, row.getMetric("count").intValue());
// The rows does not have the dim "x", hence metric is null (useDefaultValueForNull=false) or 0 (useDefaultValueForNull=true)
Assert.assertEquals(NullHandling.sqlCompatible() ? null : 0.0f, row.getMetric("sum_of_x"));
}
}
}
}
}
@Test @Test
public void testSchemaRollupWithRowWithOnlyExistingMetrics() throws IndexSizeExceededException public void testSchemaRollupWithRowWithOnlyExistingMetrics() throws IndexSizeExceededException
{ {

View File

@ -79,13 +79,14 @@ public class ClientCompactionTaskQueryTuningConfig
public static ClientCompactionTaskQueryTuningConfig from( public static ClientCompactionTaskQueryTuningConfig from(
@Nullable UserCompactionTaskQueryTuningConfig userCompactionTaskQueryTuningConfig, @Nullable UserCompactionTaskQueryTuningConfig userCompactionTaskQueryTuningConfig,
@Nullable Integer maxRowsPerSegment @Nullable Integer maxRowsPerSegment,
@Nullable Boolean preserveExistingMetrics
) )
{ {
if (userCompactionTaskQueryTuningConfig == null) { if (userCompactionTaskQueryTuningConfig == null) {
return new ClientCompactionTaskQueryTuningConfig( return new ClientCompactionTaskQueryTuningConfig(
maxRowsPerSegment, maxRowsPerSegment,
new OnheapIncrementalIndex.Spec(true), new OnheapIncrementalIndex.Spec(preserveExistingMetrics),
null, null,
null, null,
null, null,
@ -107,7 +108,7 @@ public class ClientCompactionTaskQueryTuningConfig
} else { } else {
AppendableIndexSpec appendableIndexSpecToUse = userCompactionTaskQueryTuningConfig.getAppendableIndexSpec() != null AppendableIndexSpec appendableIndexSpecToUse = userCompactionTaskQueryTuningConfig.getAppendableIndexSpec() != null
? userCompactionTaskQueryTuningConfig.getAppendableIndexSpec() ? userCompactionTaskQueryTuningConfig.getAppendableIndexSpec()
: new OnheapIncrementalIndex.Spec(true); : new OnheapIncrementalIndex.Spec(preserveExistingMetrics);
return new ClientCompactionTaskQueryTuningConfig( return new ClientCompactionTaskQueryTuningConfig(
maxRowsPerSegment, maxRowsPerSegment,
appendableIndexSpecToUse, appendableIndexSpecToUse,

View File

@ -452,7 +452,7 @@ public class CompactSegments implements CoordinatorCustomDuty
"coordinator-issued", "coordinator-issued",
segmentsToCompact, segmentsToCompact,
config.getTaskPriority(), config.getTaskPriority(),
ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment()), ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), config.getMetricsSpec() != null),
granularitySpec, granularitySpec,
dimensionsSpec, dimensionsSpec,
config.getMetricsSpec(), config.getMetricsSpec(),

View File

@ -338,7 +338,7 @@ public class NewestSegmentFirstIterator implements CompactionSegmentIterator
{ {
Preconditions.checkState(!candidates.isEmpty(), "Empty candidates"); Preconditions.checkState(!candidates.isEmpty(), "Empty candidates");
final ClientCompactionTaskQueryTuningConfig tuningConfig = final ClientCompactionTaskQueryTuningConfig tuningConfig =
ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment()); ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null);
final PartitionsSpec partitionsSpecFromConfig = findPartitionsSpecFromConfig(tuningConfig); final PartitionsSpec partitionsSpecFromConfig = findPartitionsSpecFromConfig(tuningConfig);
final CompactionState lastCompactionState = candidates.segments.get(0).getLastCompactionState(); final CompactionState lastCompactionState = candidates.segments.get(0).getLastCompactionState();
if (lastCompactionState == null) { if (lastCompactionState == null) {

View File

@ -69,6 +69,7 @@ import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.filter.SelectorDimFilter;
import org.apache.druid.segment.incremental.OnheapIncrementalIndex;
import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.server.DruidNode; import org.apache.druid.server.DruidNode;
import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.AutoCompactionSnapshot;
@ -1741,6 +1742,130 @@ public class CompactSegmentsTest
Assert.assertEquals(expected, actual); Assert.assertEquals(expected, actual);
} }
@Test
public void testCompactWithMetricsSpecShouldSetPreserveExistingMetricsTrue()
{
final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class);
final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, JSON_MAPPER, mockIndexingServiceClient);
final List<DataSourceCompactionConfig> compactionConfigs = new ArrayList<>();
final String dataSource = DATA_SOURCE_PREFIX + 0;
compactionConfigs.add(
new DataSourceCompactionConfig(
dataSource,
0,
500L,
null,
new Period("PT0H"), // smaller than segment interval
new UserCompactionTaskQueryTuningConfig(
null,
null,
null,
null,
null,
partitionsSpec,
null,
null,
null,
null,
null,
3,
null,
null,
null,
null,
null,
null
),
null,
null,
new AggregatorFactory[] {new CountAggregatorFactory("cnt")},
null,
null,
null
)
);
doCompactSegments(compactSegments, compactionConfigs);
ArgumentCaptor<ClientCompactionTaskQueryTuningConfig> clientCompactionTaskQueryTuningConfigArgumentCaptor = ArgumentCaptor.forClass(
ClientCompactionTaskQueryTuningConfig.class);
Mockito.verify(mockIndexingServiceClient).compactSegments(
ArgumentMatchers.anyString(),
ArgumentMatchers.any(),
ArgumentMatchers.anyInt(),
clientCompactionTaskQueryTuningConfigArgumentCaptor.capture(),
ArgumentMatchers.any(),
ArgumentMatchers.any(),
ArgumentMatchers.any(),
ArgumentMatchers.any(),
ArgumentMatchers.any(),
ArgumentMatchers.any()
);
Assert.assertNotNull(clientCompactionTaskQueryTuningConfigArgumentCaptor.getValue());
Assert.assertNotNull(clientCompactionTaskQueryTuningConfigArgumentCaptor.getValue().getAppendableIndexSpec());
Assert.assertTrue(((OnheapIncrementalIndex.Spec) clientCompactionTaskQueryTuningConfigArgumentCaptor.getValue().getAppendableIndexSpec()).isPreserveExistingMetrics());
}
@Test
public void testCompactWithoutMetricsSpecShouldSetPreserveExistingMetricsFalse()
{
final HttpIndexingServiceClient mockIndexingServiceClient = Mockito.mock(HttpIndexingServiceClient.class);
final CompactSegments compactSegments = new CompactSegments(COORDINATOR_CONFIG, JSON_MAPPER, mockIndexingServiceClient);
final List<DataSourceCompactionConfig> compactionConfigs = new ArrayList<>();
final String dataSource = DATA_SOURCE_PREFIX + 0;
compactionConfigs.add(
new DataSourceCompactionConfig(
dataSource,
0,
500L,
null,
new Period("PT0H"), // smaller than segment interval
new UserCompactionTaskQueryTuningConfig(
null,
null,
null,
null,
null,
partitionsSpec,
null,
null,
null,
null,
null,
3,
null,
null,
null,
null,
null,
null
),
null,
null,
null,
null,
null,
null
)
);
doCompactSegments(compactSegments, compactionConfigs);
ArgumentCaptor<ClientCompactionTaskQueryTuningConfig> clientCompactionTaskQueryTuningConfigArgumentCaptor = ArgumentCaptor.forClass(
ClientCompactionTaskQueryTuningConfig.class);
Mockito.verify(mockIndexingServiceClient).compactSegments(
ArgumentMatchers.anyString(),
ArgumentMatchers.any(),
ArgumentMatchers.anyInt(),
clientCompactionTaskQueryTuningConfigArgumentCaptor.capture(),
ArgumentMatchers.any(),
ArgumentMatchers.any(),
ArgumentMatchers.any(),
ArgumentMatchers.any(),
ArgumentMatchers.any(),
ArgumentMatchers.any()
);
Assert.assertNotNull(clientCompactionTaskQueryTuningConfigArgumentCaptor.getValue());
Assert.assertNotNull(clientCompactionTaskQueryTuningConfigArgumentCaptor.getValue().getAppendableIndexSpec());
Assert.assertFalse(((OnheapIncrementalIndex.Spec) clientCompactionTaskQueryTuningConfigArgumentCaptor.getValue().getAppendableIndexSpec()).isPreserveExistingMetrics());
}
private void verifySnapshot( private void verifySnapshot(
CompactSegments compactSegments, CompactSegments compactSegments,
AutoCompactionSnapshot.AutoCompactionScheduleStatus scheduleStatus, AutoCompactionSnapshot.AutoCompactionScheduleStatus scheduleStatus,

View File

@ -101,7 +101,7 @@ public class NewestSegmentFirstIteratorTest
Assert.assertEquals( Assert.assertEquals(
new DynamicPartitionsSpec(null, Long.MAX_VALUE), new DynamicPartitionsSpec(null, Long.MAX_VALUE),
NewestSegmentFirstIterator.findPartitionsSpecFromConfig( NewestSegmentFirstIterator.findPartitionsSpecFromConfig(
ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment()) ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null)
) )
); );
} }
@ -145,7 +145,7 @@ public class NewestSegmentFirstIteratorTest
Assert.assertEquals( Assert.assertEquals(
new DynamicPartitionsSpec(null, Long.MAX_VALUE), new DynamicPartitionsSpec(null, Long.MAX_VALUE),
NewestSegmentFirstIterator.findPartitionsSpecFromConfig( NewestSegmentFirstIterator.findPartitionsSpecFromConfig(
ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment()) ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null)
) )
); );
} }
@ -189,7 +189,7 @@ public class NewestSegmentFirstIteratorTest
Assert.assertEquals( Assert.assertEquals(
new DynamicPartitionsSpec(null, 1000L), new DynamicPartitionsSpec(null, 1000L),
NewestSegmentFirstIterator.findPartitionsSpecFromConfig( NewestSegmentFirstIterator.findPartitionsSpecFromConfig(
ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment()) ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null)
) )
); );
} }
@ -233,7 +233,7 @@ public class NewestSegmentFirstIteratorTest
Assert.assertEquals( Assert.assertEquals(
new DynamicPartitionsSpec(100, 1000L), new DynamicPartitionsSpec(100, 1000L),
NewestSegmentFirstIterator.findPartitionsSpecFromConfig( NewestSegmentFirstIterator.findPartitionsSpecFromConfig(
ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment()) ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null)
) )
); );
} }
@ -277,7 +277,7 @@ public class NewestSegmentFirstIteratorTest
Assert.assertEquals( Assert.assertEquals(
new DynamicPartitionsSpec(100, 1000L), new DynamicPartitionsSpec(100, 1000L),
NewestSegmentFirstIterator.findPartitionsSpecFromConfig( NewestSegmentFirstIterator.findPartitionsSpecFromConfig(
ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment()) ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null)
) )
); );
} }
@ -321,7 +321,7 @@ public class NewestSegmentFirstIteratorTest
Assert.assertEquals( Assert.assertEquals(
new DynamicPartitionsSpec(null, Long.MAX_VALUE), new DynamicPartitionsSpec(null, Long.MAX_VALUE),
NewestSegmentFirstIterator.findPartitionsSpecFromConfig( NewestSegmentFirstIterator.findPartitionsSpecFromConfig(
ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment()) ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null)
) )
); );
} }
@ -365,7 +365,7 @@ public class NewestSegmentFirstIteratorTest
Assert.assertEquals( Assert.assertEquals(
new DynamicPartitionsSpec(null, Long.MAX_VALUE), new DynamicPartitionsSpec(null, Long.MAX_VALUE),
NewestSegmentFirstIterator.findPartitionsSpecFromConfig( NewestSegmentFirstIterator.findPartitionsSpecFromConfig(
ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment()) ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null)
) )
); );
} }
@ -409,7 +409,7 @@ public class NewestSegmentFirstIteratorTest
Assert.assertEquals( Assert.assertEquals(
new HashedPartitionsSpec(null, 10, ImmutableList.of("dim")), new HashedPartitionsSpec(null, 10, ImmutableList.of("dim")),
NewestSegmentFirstIterator.findPartitionsSpecFromConfig( NewestSegmentFirstIterator.findPartitionsSpecFromConfig(
ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment()) ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null)
) )
); );
} }
@ -453,7 +453,7 @@ public class NewestSegmentFirstIteratorTest
Assert.assertEquals( Assert.assertEquals(
new SingleDimensionPartitionsSpec(10000, null, "dim", false), new SingleDimensionPartitionsSpec(10000, null, "dim", false),
NewestSegmentFirstIterator.findPartitionsSpecFromConfig( NewestSegmentFirstIterator.findPartitionsSpecFromConfig(
ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment()) ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment(), null)
) )
); );
} }

View File

@ -700,7 +700,7 @@ public class NewestSegmentFirstPolicyTest
// Same indexSpec as what is set in the auto compaction config // Same indexSpec as what is set in the auto compaction config
Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {}); Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {});
// Same partitionsSpec as what is set in the auto compaction config // Same partitionsSpec as what is set in the auto compaction config
PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null)); PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null));
// Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY
final VersionedIntervalTimeline<String, DataSegment> timeline = createTimeline( final VersionedIntervalTimeline<String, DataSegment> timeline = createTimeline(
@ -733,7 +733,7 @@ public class NewestSegmentFirstPolicyTest
// Same indexSpec as what is set in the auto compaction config // Same indexSpec as what is set in the auto compaction config
Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {}); Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {});
// Same partitionsSpec as what is set in the auto compaction config // Same partitionsSpec as what is set in the auto compaction config
PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null)); PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null));
// Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY
final VersionedIntervalTimeline<String, DataSegment> timeline = createTimeline( final VersionedIntervalTimeline<String, DataSegment> timeline = createTimeline(
@ -766,7 +766,7 @@ public class NewestSegmentFirstPolicyTest
// Same indexSpec as what is set in the auto compaction config // Same indexSpec as what is set in the auto compaction config
Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {}); Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {});
// Same partitionsSpec as what is set in the auto compaction config // Same partitionsSpec as what is set in the auto compaction config
PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null)); PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null));
// Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY
final VersionedIntervalTimeline<String, DataSegment> timeline = createTimeline( final VersionedIntervalTimeline<String, DataSegment> timeline = createTimeline(
@ -809,7 +809,7 @@ public class NewestSegmentFirstPolicyTest
// Same indexSpec as what is set in the auto compaction config // Same indexSpec as what is set in the auto compaction config
Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {}); Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {});
// Same partitionsSpec as what is set in the auto compaction config // Same partitionsSpec as what is set in the auto compaction config
PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null)); PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null));
// Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY
final VersionedIntervalTimeline<String, DataSegment> timeline = createTimeline( final VersionedIntervalTimeline<String, DataSegment> timeline = createTimeline(
@ -852,7 +852,7 @@ public class NewestSegmentFirstPolicyTest
// Same indexSpec as what is set in the auto compaction config // Same indexSpec as what is set in the auto compaction config
Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {}); Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {});
// Same partitionsSpec as what is set in the auto compaction config // Same partitionsSpec as what is set in the auto compaction config
PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null)); PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null));
// Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY
final VersionedIntervalTimeline<String, DataSegment> timeline = createTimeline( final VersionedIntervalTimeline<String, DataSegment> timeline = createTimeline(
@ -904,7 +904,7 @@ public class NewestSegmentFirstPolicyTest
// Same indexSpec as what is set in the auto compaction config // Same indexSpec as what is set in the auto compaction config
Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {}); Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {});
// Same partitionsSpec as what is set in the auto compaction config // Same partitionsSpec as what is set in the auto compaction config
PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null)); PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null));
// Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY
final VersionedIntervalTimeline<String, DataSegment> timeline = createTimeline( final VersionedIntervalTimeline<String, DataSegment> timeline = createTimeline(
@ -955,7 +955,7 @@ public class NewestSegmentFirstPolicyTest
// Same indexSpec as what is set in the auto compaction config // Same indexSpec as what is set in the auto compaction config
Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {}); Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {});
// Same partitionsSpec as what is set in the auto compaction config // Same partitionsSpec as what is set in the auto compaction config
PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null)); PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null));
// Create segments that were compacted (CompactionState != null) and have // Create segments that were compacted (CompactionState != null) and have
// rollup=false for interval 2017-10-01T00:00:00/2017-10-02T00:00:00, // rollup=false for interval 2017-10-01T00:00:00/2017-10-02T00:00:00,
@ -1015,7 +1015,7 @@ public class NewestSegmentFirstPolicyTest
// Same indexSpec as what is set in the auto compaction config // Same indexSpec as what is set in the auto compaction config
Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {}); Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {});
// Same partitionsSpec as what is set in the auto compaction config // Same partitionsSpec as what is set in the auto compaction config
PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null)); PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null));
// Create segments that were compacted (CompactionState != null) and have // Create segments that were compacted (CompactionState != null) and have
// queryGranularity=DAY for interval 2017-10-01T00:00:00/2017-10-02T00:00:00, // queryGranularity=DAY for interval 2017-10-01T00:00:00/2017-10-02T00:00:00,
@ -1075,7 +1075,7 @@ public class NewestSegmentFirstPolicyTest
// Same indexSpec as what is set in the auto compaction config // Same indexSpec as what is set in the auto compaction config
Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {}); Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {});
// Same partitionsSpec as what is set in the auto compaction config // Same partitionsSpec as what is set in the auto compaction config
PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null)); PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null));
// Create segments that were compacted (CompactionState != null) and have // Create segments that were compacted (CompactionState != null) and have
// Dimensions=["foo", "bar"] for interval 2017-10-01T00:00:00/2017-10-02T00:00:00, // Dimensions=["foo", "bar"] for interval 2017-10-01T00:00:00/2017-10-02T00:00:00,
@ -1174,7 +1174,7 @@ public class NewestSegmentFirstPolicyTest
// Same indexSpec as what is set in the auto compaction config // Same indexSpec as what is set in the auto compaction config
Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {}); Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {});
// Same partitionsSpec as what is set in the auto compaction config // Same partitionsSpec as what is set in the auto compaction config
PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null)); PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null));
// Create segments that were compacted (CompactionState != null) and have // Create segments that were compacted (CompactionState != null) and have
// filter=SelectorDimFilter("dim1", "foo", null) for interval 2017-10-01T00:00:00/2017-10-02T00:00:00, // filter=SelectorDimFilter("dim1", "foo", null) for interval 2017-10-01T00:00:00/2017-10-02T00:00:00,
@ -1298,7 +1298,7 @@ public class NewestSegmentFirstPolicyTest
// Same indexSpec as what is set in the auto compaction config // Same indexSpec as what is set in the auto compaction config
Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {}); Map<String, Object> indexSpec = mapper.convertValue(new IndexSpec(), new TypeReference<Map<String, Object>>() {});
// Same partitionsSpec as what is set in the auto compaction config // Same partitionsSpec as what is set in the auto compaction config
PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null)); PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null));
// Create segments that were compacted (CompactionState != null) and have // Create segments that were compacted (CompactionState != null) and have
// metricsSpec={CountAggregatorFactory("cnt")} for interval 2017-10-01T00:00:00/2017-10-02T00:00:00, // metricsSpec={CountAggregatorFactory("cnt")} for interval 2017-10-01T00:00:00/2017-10-02T00:00:00,
@ -1447,7 +1447,7 @@ public class NewestSegmentFirstPolicyTest
// Different indexSpec as what is set in the auto compaction config // Different indexSpec as what is set in the auto compaction config
IndexSpec newIndexSpec = new IndexSpec(new ConciseBitmapSerdeFactory(), null, null, null); IndexSpec newIndexSpec = new IndexSpec(new ConciseBitmapSerdeFactory(), null, null, null);
Map<String, Object> newIndexSpecMap = mapper.convertValue(newIndexSpec, new TypeReference<Map<String, Object>>() {}); Map<String, Object> newIndexSpecMap = mapper.convertValue(newIndexSpec, new TypeReference<Map<String, Object>>() {});
PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null)); PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null));
// Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY
final VersionedIntervalTimeline<String, DataSegment> timeline = createTimeline( final VersionedIntervalTimeline<String, DataSegment> timeline = createTimeline(
@ -1496,7 +1496,7 @@ public class NewestSegmentFirstPolicyTest
public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec()
{ {
NullHandling.initializeForTests(); NullHandling.initializeForTests();
PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null)); PartitionsSpec partitionsSpec = NewestSegmentFirstIterator.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null, null, null));
final VersionedIntervalTimeline<String, DataSegment> timeline = createTimeline( final VersionedIntervalTimeline<String, DataSegment> timeline = createTimeline(
new SegmentGenerateSpec( new SegmentGenerateSpec(
Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"),