diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java index 4bc72870a1e5..ba8df1b802cd 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java @@ -1528,7 +1528,7 @@ public void testAutoCompactionDutyWithFilter(boolean useSupervisors) throws Exce NO_SKIP_OFFSET, null, null, - new CompactionTransformSpec(new SelectorDimFilter("page", "Striker Eureka", null)), + new CompactionTransformSpec(new SelectorDimFilter("page", "Striker Eureka", null), null), null, false, CompactionEngine.NATIVE diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 9aba7f75bacd..e09be28eb517 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -19,26 +19,45 @@ package org.apache.druid.testing.embedded.compact; +import com.fasterxml.jackson.core.type.TypeReference; import org.apache.druid.catalog.guice.CatalogClientModule; import org.apache.druid.catalog.guice.CatalogCoordinatorModule; import org.apache.druid.common.utils.IdUtils; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.indexing.common.task.TaskBuilder; +import org.apache.druid.indexing.compact.CascadingReindexingTemplate; import org.apache.druid.indexing.compact.CompactionSupervisorSpec; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; 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.jackson.JacksonUtils; import org.apache.druid.query.DruidMetrics; +import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.NotDimFilter; -import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.http.ClientSqlQuery; import org.apache.druid.rpc.UpdateResponse; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.metadata.DefaultIndexingStateFingerprintMapper; import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.server.compaction.InlineReindexingRuleProvider; +import org.apache.druid.server.compaction.ReindexingDeletionRule; +import org.apache.druid.server.compaction.ReindexingIOConfigRule; +import org.apache.druid.server.compaction.ReindexingSegmentGranularityRule; +import org.apache.druid.server.compaction.ReindexingTuningConfigRule; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; @@ -56,11 +75,16 @@ import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; import org.hamcrest.Matcher; import org.hamcrest.Matchers; +import org.joda.time.DateTime; +import org.joda.time.Interval; import org.joda.time.Period; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; +import java.nio.charset.StandardCharsets; +import java.time.Duration; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -138,26 +162,8 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity_andCompactToY new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null) ) .withTuningConfig( - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - null, - new DimensionRangePartitionsSpec(null, 5000, List.of("item"), false), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null + createTuningConfigWithPartitionsSpec( + new DimensionRangePartitionsSpec(null, 5000, List.of("item"), false) ) ) .build(); @@ -179,26 +185,8 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity_andCompactToY new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null) ) .withTuningConfig( - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - null, - new DimensionRangePartitionsSpec(null, 5000, List.of("item"), false), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null + createTuningConfigWithPartitionsSpec( + new DimensionRangePartitionsSpec(null, 5000, List.of("item"), false) ) ) .build(); @@ -244,26 +232,8 @@ public void test_compaction_withPersistLastCompactionStateFalse_storesOnlyFinger new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null) ) .withTuningConfig( - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - null, - new DimensionRangePartitionsSpec(1000, null, List.of("item"), false), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null + createTuningConfigWithPartitionsSpec( + new DimensionRangePartitionsSpec(1000, null, List.of("item"), false) ) ) .build(); @@ -274,6 +244,189 @@ public void test_compaction_withPersistLastCompactionStateFalse_storesOnlyFinger verifySegmentsHaveNullLastCompactionStateAndNonNullFingerprint(); } + @MethodSource("getEngine") + @ParameterizedTest(name = "compactionEngine={0}") + public void test_cascadingCompactionTemplate_multiplePeriodsApplyDifferentCompactionRules(CompactionEngine compactionEngine) + { + // Configure cluster with storeCompactionStatePerSegment=false + final UpdateResponse updateResponse = cluster.callApi().onLeaderOverlord( + o -> o.updateClusterCompactionConfig( + new ClusterCompactionConfig(1.0, 100, null, true, compactionEngine, false) + ) + ); + Assertions.assertTrue(updateResponse.isSuccess()); + + + DateTime now = DateTimes.nowUtc(); + + // Note that we are purposely creating events in intervals like this to make the test deterministic regardless of when it is run. + // The supervisor will use the current time as reference time to determine which rules apply to which segments so we take extra + // care to create segments that fall cleanly into the different rule periods that we are testing. + String freshEvents = generateEventsInInterval( + new Interval(now.minusHours(4), now), + 4, + Duration.ofMinutes(30).toMillis() + ); + String hourRuleEvents = generateEventsInInterval( + new Interval(now.minusDays(3), now.minusDays(2)), + 5, + Duration.ofMinutes(90).toMillis() + ); + String dayRuleEvents = generateEventsInInterval( + new Interval(now.minusDays(31), now.minusDays(14)), + 7, + Duration.ofHours(25).toMillis() + ); + + String allData = freshEvents + "\n" + hourRuleEvents + "\n" + dayRuleEvents; + + runIngestionAtGranularity( + "FIFTEEN_MINUTE", + allData + ); + Assertions.assertEquals(16, getNumSegmentsWith(Granularities.FIFTEEN_MINUTE)); + + ReindexingSegmentGranularityRule hourRule = new ReindexingSegmentGranularityRule( + "hourRule", + "Compact to HOUR granularity for data older than 1 days", + Period.days(1), + Granularities.HOUR + ); + ReindexingSegmentGranularityRule dayRule = new ReindexingSegmentGranularityRule( + "dayRule", + "Compact to DAY granularity for data older than 7 days", + Period.days(7), + Granularities.DAY + ); + + ReindexingTuningConfigRule tuningConfigRule = new ReindexingTuningConfigRule( + "tuningConfigRule", + "Use dimension range partitioning with max 1000 rows per segment", + Period.days(1), + createTuningConfigWithPartitionsSpec(new DimensionRangePartitionsSpec(1000, null, List.of("item"), false)) + ); + + ReindexingDeletionRule deletionRule = new ReindexingDeletionRule( + "deletionRule", + "Drop rows where item is 'hat'", + Period.days(7), + new EqualityFilter("item", ColumnType.STRING, "hat", null), + null + ); + + InlineReindexingRuleProvider.Builder ruleProvider = InlineReindexingRuleProvider.builder() + .segmentGranularityRules(List.of(hourRule, dayRule)) + .tuningConfigRules(List.of(tuningConfigRule)) + .deletionRules(List.of(deletionRule)); + + if (compactionEngine == CompactionEngine.NATIVE) { + ruleProvider = ruleProvider.ioConfigRules( + List.of(new ReindexingIOConfigRule("dropExisting", null, Period.days(7), new UserCompactionTaskIOConfig(true))) + ); + } + + CascadingReindexingTemplate cascadingReindexingTemplate = new CascadingReindexingTemplate( + dataSource, + null, + null, + ruleProvider.build(), + compactionEngine, + null, + null, + null, + Granularities.HOUR + ); + runCompactionWithSpec(cascadingReindexingTemplate); + waitForAllCompactionTasksToFinish(); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); + + Assertions.assertEquals(4, getNumSegmentsWith(Granularities.FIFTEEN_MINUTE)); + Assertions.assertEquals(5, getNumSegmentsWith(Granularities.HOUR)); + Assertions.assertEquals(4, getNumSegmentsWith(Granularities.DAY)); + verifyEventCountOlderThan(Period.days(7), "item", "hat", 0); + } + + @Test + public void test_cascadingReindexing_withVirtualColumnOnNestedData_filtersCorrectly() + { + // Virtual Columns on nested data is only supported with MSQ compaction engine right now. + CompactionEngine compactionEngine = CompactionEngine.MSQ; + configureCompaction(compactionEngine); + + String jsonDataWithNestedColumn = + "{\"timestamp\":\"2025-06-01T00:00:00.000Z\",\"item\":\"shirt\",\"value\":105," + + "\"extraInfo\":{\"fieldA\":\"valueA\",\"fieldB\":\"valueB\"}}\n" + + "{\"timestamp\":\"2025-06-02T00:00:00.000Z\",\"item\":\"trousers\",\"value\":210," + + "\"extraInfo\":{\"fieldA\":\"valueC\",\"fieldB\":\"valueD\"}}\n" + + "{\"timestamp\":\"2025-06-03T00:00:00.000Z\",\"item\":\"jeans\",\"value\":150," + + "\"extraInfo\":{\"fieldA\":\"valueA\",\"fieldB\":\"valueE\"}}\n" + + "{\"timestamp\":\"2025-06-04T00:00:00.000Z\",\"item\":\"hat\",\"value\":50," + + "\"extraInfo\":{\"fieldA\":\"valueF\",\"fieldB\":\"valueG\"}}"; + + final TaskBuilder.Index task = TaskBuilder + .ofTypeIndex() + .dataSource(dataSource) + .jsonInputFormat() + .inlineInputSourceWithData(jsonDataWithNestedColumn) + .isoTimestampColumn("timestamp") + .schemaDiscovery() + .segmentGranularity("DAY"); + + cluster.callApi().runTask(task.withId(IdUtils.getRandomId()), overlord); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); + + Assertions.assertEquals(4, getTotalRowCount()); + + VirtualColumns virtualColumns = VirtualColumns.create( + new ExpressionVirtualColumn( + "extractedFieldA", + "json_value(extraInfo, '$.fieldA')", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ) + ); + + ReindexingDeletionRule deletionRule = new ReindexingDeletionRule( + "deleteByNestedField", + "Remove rows where extraInfo.fieldA = 'valueA'", + Period.days(7), + new EqualityFilter("extractedFieldA", ColumnType.STRING, "valueA", null), + virtualColumns + ); + + ReindexingTuningConfigRule tuningConfigRule = new ReindexingTuningConfigRule( + "tuningConfigRule", + null, + Period.days(7), + createTuningConfigWithPartitionsSpec(new DynamicPartitionsSpec(null, null)) + ); + + CascadingReindexingTemplate cascadingTemplate = new CascadingReindexingTemplate( + dataSource, + null, + null, + InlineReindexingRuleProvider.builder() + .deletionRules(List.of(deletionRule)) + .tuningConfigRules(List.of(tuningConfigRule)) + .build(), + compactionEngine, + null, + null, + null, + Granularities.DAY + ); + + runCompactionWithSpec(cascadingTemplate); + waitForAllCompactionTasksToFinish(); + + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); + + // Verify: Should have 2 rows left (valueA appeared in 2 rows, both filtered out) + Assertions.assertEquals(2, getTotalRowCount()); + + // Verify the correct rows were filtered + verifyNoRowsWithNestedValue("extraInfo", "fieldA", "valueA"); + } /** * Tests that when a compaction task filters out all rows using a transform spec, @@ -313,7 +466,8 @@ public void test_compactionWithTransformFilteringAllRows_createsTombstones( .withTransformSpec( // This filter drops all rows: expression "false" always evaluates to false new CompactionTransformSpec( - new NotDimFilter(new SelectorDimFilter("item", "shirt", null)) + new NotDimFilter(new EqualityFilter("item", ColumnType.STRING, "shirt", null)), + null ) ); @@ -380,6 +534,7 @@ public void test_compactionWithTransformFilteringAllRows_createsTombstones( runCompactionWithSpec(compactionConfig); waitForAllCompactionTasksToFinish(); + cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); int finalSegmentCount = getNumSegmentsWith(Granularities.DAY); Assertions.assertEquals( @@ -389,6 +544,58 @@ public void test_compactionWithTransformFilteringAllRows_createsTombstones( ); } + private int getTotalRowCount() + { + String sql = StringUtils.format("SELECT COUNT(*) as cnt FROM \"%s\"", dataSource); + String result = cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(new ClientSqlQuery(sql, null, false, false, false, null, null))); + List> rows = JacksonUtils.readValue( + new DefaultObjectMapper(), + result.getBytes(StandardCharsets.UTF_8), + new TypeReference<>() {} + ); + return ((Number) rows.get(0).get("cnt")).intValue(); + } + + private void verifyNoRowsWithNestedValue(String nestedColumn, String field, String value) + { + String sql = StringUtils.format( + "SELECT COUNT(*) as cnt FROM \"%s\" WHERE json_value(%s, '$.%s') = '%s'", + dataSource, + nestedColumn, + field, + value + ); + String result = cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(new ClientSqlQuery(sql, null, false, false, false, null, null))); + List> rows = JacksonUtils.readValue( + new DefaultObjectMapper(), + result.getBytes(StandardCharsets.UTF_8), + new TypeReference<>() {} + ); + Assertions.assertEquals( + 0, + ((Number) rows.get(0).get("cnt")).intValue(), + StringUtils.format("Expected no rows where %s.%s = '%s'", nestedColumn, field, value) + ); + } + + + private String generateEventsInInterval(Interval interval, int numEvents, long spacingMillis) + { + List events = new ArrayList<>(); + + for (int i = 1; i <= numEvents; i++) { + DateTime eventTime = interval.getStart().plus(spacingMillis * i); + if (eventTime.isAfter(interval.getEnd())) { + throw new IAE("Interval cannot fit [%d] events with spacing of [%d] millis", numEvents, spacingMillis); + } + String item = i % 2 == 0 ? "hat" : "shirt"; + int metricValue = 100 + i * 5; + events.add(eventTime + "," + item + "," + metricValue); + } + + return String.join("\n", events); + } + private void verifySegmentsHaveNullLastCompactionStateAndNonNullFingerprint() { overlord @@ -514,4 +721,73 @@ public static List getEngineAndPartitionType() } return params; } + + private void verifyEventCountOlderThan(Period period, String dimension, String value, int expectedCount) + { + DateTime now = DateTimes.nowUtc(); + DateTime threshold = now.minus(period); + + ClientSqlQuery query = new ClientSqlQuery( + StringUtils.format( + "SELECT COUNT(*) as cnt FROM \"%s\" WHERE %s = '%s' AND __time < MILLIS_TO_TIMESTAMP(%d)", + dataSource, + dimension, + value, + threshold.getMillis() + ), + null, + false, + false, + false, + null, + null + ); + + final String resultAsJson = cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(query)); + + List> result = JacksonUtils.readValue( + new DefaultObjectMapper(), + resultAsJson.getBytes(StandardCharsets.UTF_8), + new TypeReference<>() {} + ); + + Assertions.assertEquals(1, result.size()); + Assertions.assertEquals( + expectedCount, + result.get(0).get("cnt"), + StringUtils.format( + "Expected %d events where %s='%s' older than %s", + expectedCount, + dimension, + value, + period + ) + ); + } + + private UserCompactionTaskQueryTuningConfig createTuningConfigWithPartitionsSpec(PartitionsSpec partitionsSpec) + { + return new UserCompactionTaskQueryTuningConfig( + null, + null, + null, + null, + null, + partitionsSpec, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + } + } diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 3b21a710fb95..3b2b06fc6123 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -198,6 +198,26 @@ junit test + + org.junit.jupiter + junit-jupiter-api + test + + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.jupiter + junit-jupiter-migrationsupport + test + + + org.junit.vintage + junit-vintage-engine + test + org.easymock easymock diff --git a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java index 73e8e06e8964..12730c25b950 100644 --- a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java +++ b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; +import org.apache.druid.indexing.compact.CascadingReindexingTemplate; import org.apache.druid.indexing.compact.CompactionSupervisorSpec; import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.indexing.scheduledbatch.ScheduledBatchSupervisorSpec; @@ -47,7 +48,8 @@ public List getJacksonModules() new SimpleModule(getClass().getSimpleName()) .registerSubtypes( new NamedType(CompactionSupervisorSpec.class, CompactionSupervisorSpec.TYPE), - new NamedType(ScheduledBatchSupervisorSpec.class, ScheduledBatchSupervisorSpec.TYPE) + new NamedType(ScheduledBatchSupervisorSpec.class, ScheduledBatchSupervisorSpec.TYPE), + new NamedType(CascadingReindexingTemplate.class, CascadingReindexingTemplate.TYPE) ) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index 89eb331df49f..559cdc1c6403 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -89,6 +89,14 @@ public CompactionConfigValidationResult validateCompactionTask( Map intervalDataSchemaMap ) { + // Virtual columns in filter rules are not supported by native compaction + if (compactionTask.getTransformSpec() != null + && compactionTask.getTransformSpec().getVirtualColumns() != null + && compactionTask.getTransformSpec().getVirtualColumns().getVirtualColumns().length > 0) { + return CompactionConfigValidationResult.failure( + "Virtual columns in filter rules are not supported by the Native compaction engine. Use MSQ compaction engine instead." + ); + } return CompactionConfigValidationResult.success(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java new file mode 100644 index 000000000000..16307fa15399 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java @@ -0,0 +1,806 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.indexing.input.DruidInputSource; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.server.compaction.IntervalGranularityInfo; +import org.apache.druid.server.compaction.ReindexingRule; +import org.apache.druid.server.compaction.ReindexingRuleProvider; +import org.apache.druid.server.compaction.ReindexingSegmentGranularityRule; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskDimensionsConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; +import org.apache.druid.timeline.CompactionState; +import org.apache.druid.timeline.SegmentTimeline; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * Template to perform period-based cascading reindexing. {@link ReindexingRule} are provided by a {@link ReindexingRuleProvider} + * Each rule specifies a period relative to the current time which is used to determine its applicable interval. + * A timeline is constructed from a condensed set of these periods and tasks are created for each search interval in + * the timeline with the applicable rules for said interval. + *

+ * For example if you had the following rules: + *

    + *
  • Rule A: period = 1 day
  • + *
  • Rule B: period = 7 days
  • + *
  • Rule C: period = 30 days
  • + *
  • Rule D: period = 7 days
  • + *
+ * + * You would end up with the following search intervals (assuming current time is T): + *
    + *
  • Interval 1: [T-7days, T-1day)
  • + *
  • Interval 2: [T-30days, T-7days)
  • + *
  • Interval 3: [-inf, T-30days)
  • + *
+ *

+ * This template never needs to be deserialized as a {@code BatchIndexingJobTemplate} + */ +public class CascadingReindexingTemplate implements CompactionJobTemplate, DataSourceCompactionConfig +{ + private static final Logger LOG = new Logger(CascadingReindexingTemplate.class); + private static final ReindexingConfigOptimizer DELETION_RULE_OPTIMIZER = new ReindexingDeletionRuleOptimizer(); + + + public static final String TYPE = "reindexCascade"; + + private final String dataSource; + private final ReindexingRuleProvider ruleProvider; + @Nullable + private final Map taskContext; + @Nullable + private final CompactionEngine engine; + private final int taskPriority; + private final long inputSegmentSizeBytes; + private final Period skipOffsetFromLatest; + private final Period skipOffsetFromNow; + private final Granularity defaultSegmentGranularity; + + @JsonCreator + public CascadingReindexingTemplate( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("taskPriority") @Nullable Integer taskPriority, + @JsonProperty("inputSegmentSizeBytes") @Nullable Long inputSegmentSizeBytes, + @JsonProperty("ruleProvider") ReindexingRuleProvider ruleProvider, + @JsonProperty("engine") @Nullable CompactionEngine engine, + @JsonProperty("taskContext") @Nullable Map taskContext, + @JsonProperty("skipOffsetFromLatest") @Nullable Period skipOffsetFromLatest, + @JsonProperty("skipOffsetFromNow") @Nullable Period skipOffsetFromNow, + @JsonProperty("defaultSegmentGranularity") Granularity defaultSegmentGranularity + ) + { + InvalidInput.conditionalException(dataSource != null, "'dataSource' cannot be null"); + this.dataSource = dataSource; + + InvalidInput.conditionalException(ruleProvider != null, "'ruleProvider' cannot be null"); + this.ruleProvider = ruleProvider; + + this.engine = engine; + this.taskContext = taskContext; + this.taskPriority = Objects.requireNonNullElse(taskPriority, DEFAULT_COMPACTION_TASK_PRIORITY); + this.inputSegmentSizeBytes = Objects.requireNonNullElse(inputSegmentSizeBytes, DEFAULT_INPUT_SEGMENT_SIZE_BYTES); + + InvalidInput.conditionalException(defaultSegmentGranularity != null, "'defaultSegmentGranularity' cannot be null"); + this.defaultSegmentGranularity = defaultSegmentGranularity; + + if (skipOffsetFromNow != null && skipOffsetFromLatest != null) { + throw InvalidInput.exception("Cannot set both skipOffsetFromNow and skipOffsetFromLatest"); + } + this.skipOffsetFromNow = skipOffsetFromNow; + this.skipOffsetFromLatest = skipOffsetFromLatest; + } + + @Override + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + @Nullable + @Override + public Map getTaskContext() + { + return taskContext; + } + + @JsonProperty + @Nullable + @Override + public CompactionEngine getEngine() + { + return engine; + } + + @JsonProperty + @Override + public int getTaskPriority() + { + return taskPriority; + } + + @JsonProperty + @Override + public long getInputSegmentSizeBytes() + { + return inputSegmentSizeBytes; + } + + @Override + public String getType() + { + return TYPE; + } + + @JsonProperty + private ReindexingRuleProvider getRuleProvider() + { + return ruleProvider; + } + + @Override + @JsonProperty + @Nullable + public Period getSkipOffsetFromLatest() + { + return skipOffsetFromLatest; + } + + @JsonProperty + @Nullable + private Period getSkipOffsetFromNow() + { + return skipOffsetFromNow; + } + + @JsonProperty + public Granularity getDefaultSegmentGranularity() + { + return defaultSegmentGranularity; + } + + /** + * Checks if the given interval's end time is after the specified boundary. + * Used to determine if intervals should be skipped based on skip offset configuration. + * + * @param interval the interval to check + * @param boundary the boundary time to compare against + * @return true if the interval ends after the boundary + */ + private static boolean intervalEndsAfter(Interval interval, DateTime boundary) + { + return interval.getEnd().isAfter(boundary); + } + + @Override + public List createCompactionJobs( + DruidInputSource source, + CompactionJobParams jobParams + ) + { + // Check if the rule provider is ready before attempting to create jobs + if (!ruleProvider.isReady()) { + LOG.info( + "Rule provider [%s] is not ready, skipping reindexing job creation for dataSource[%s]", + ruleProvider.getType(), + dataSource + ); + return Collections.emptyList(); + } + + final List allJobs = new ArrayList<>(); + final DateTime currentTime = jobParams.getScheduleStartTime(); + + SegmentTimeline timeline = jobParams.getTimeline(dataSource); + + if (timeline == null || timeline.isEmpty()) { + LOG.warn("Segment timeline null or empty for [%s] skipping creating compaction jobs.", dataSource); + return Collections.emptyList(); + } + + List searchIntervals = generateAlignedSearchIntervals(currentTime); + if (searchIntervals.isEmpty()) { + LOG.warn("No search intervals generated for dataSource[%s], no reindexing jobs will be created", dataSource); + return Collections.emptyList(); + } + + // Adjust timeline interval by applying user defined skip offset (if any exists) + Interval adjustedTimelineInterval = applySkipOffset( + new Interval(timeline.first().getInterval().getStart(), timeline.last().getInterval().getEnd()), + jobParams.getScheduleStartTime() + ); + if (adjustedTimelineInterval == null) { + LOG.warn("All data for dataSource[%s] is within skip offsets, no reindexing jobs will be created", dataSource); + return Collections.emptyList(); + } + + for (IntervalGranularityInfo intervalInfo : searchIntervals) { + Interval reindexingInterval = intervalInfo.getInterval(); + + if (!reindexingInterval.overlaps(adjustedTimelineInterval)) { + // No underlying data exists to reindex for this interval + LOG.debug("Search interval[%s] does not overlap with data range[%s], skipping", reindexingInterval, adjustedTimelineInterval); + continue; + } + + // Skip intervals that extend past the skip offset boundary (not just data boundary) + // This preserves granularity alignment and ensures intervals exist in synthetic timeline + // Only apply this when a skip offset is actually configured + if ((skipOffsetFromNow != null || skipOffsetFromLatest != null) && + intervalEndsAfter(reindexingInterval, adjustedTimelineInterval.getEnd())) { + LOG.debug("Search interval[%s] extends past skip offset boundary[%s], skipping to preserve alignment", + reindexingInterval, adjustedTimelineInterval.getEnd()); + continue; + } + + InlineSchemaDataSourceCompactionConfig.Builder builder = createBaseBuilder(); + + ReindexingConfigBuilder configBuilder = new ReindexingConfigBuilder( + ruleProvider, + reindexingInterval, + currentTime, + searchIntervals + ); + int ruleCount = configBuilder.applyTo(builder); + + if (ruleCount > 0) { + LOG.debug("Creating reindexing jobs for interval[%s] with [%d] rules selected", reindexingInterval, ruleCount); + allJobs.addAll( + createJobsForSearchInterval( + createJobTemplateForInterval(builder.build()), + reindexingInterval, + source, + jobParams + ) + ); + } else { + LOG.debug("No applicable reindexing rules found for interval[%s]", reindexingInterval); + } + } + return allJobs; + } + + @VisibleForTesting + protected CompactionJobTemplate createJobTemplateForInterval( + InlineSchemaDataSourceCompactionConfig config + ) + { + return new CompactionConfigBasedJobTemplate(config, DELETION_RULE_OPTIMIZER); + } + + /** + * Applies the configured skip offset to an interval by adjusting its end time. Uses either + * skipOffsetFromNow (relative to reference time) or skipOffsetFromLatest (relative to interval end). + * Returns null if the adjusted end would be before the interval start. + * + * @param interval the interval to adjust + * @param skipFromNowReferenceTime the reference time for skipOffsetFromNow calculation + * @return the interval with adjusted end time, or null if the result would be invalid + */ + @Nullable + private Interval applySkipOffset( + Interval interval, + DateTime skipFromNowReferenceTime + ) + { + DateTime maybeAdjustedEnd = interval.getEnd(); + if (skipOffsetFromNow != null) { + maybeAdjustedEnd = skipFromNowReferenceTime.minus(skipOffsetFromNow); + } else if (skipOffsetFromLatest != null) { + maybeAdjustedEnd = maybeAdjustedEnd.minus(skipOffsetFromLatest); + } + if (maybeAdjustedEnd.isBefore(interval.getStart())) { + return null; + } else { + return new Interval(interval.getStart(), maybeAdjustedEnd); + } + } + + private InlineSchemaDataSourceCompactionConfig.Builder createBaseBuilder() + { + return InlineSchemaDataSourceCompactionConfig.builder() + .forDataSource(dataSource) + .withTaskPriority(taskPriority) + .withInputSegmentSizeBytes(inputSegmentSizeBytes) + .withEngine(engine) + .withTaskContext(taskContext) + .withSkipOffsetFromLatest(Period.ZERO); // We handle skip offsets at the timeline level, we know we want to cover the entirety of the interval + } + + /** + * Generates granularity-aligned search intervals based on segment granularity rules, + * then splits them at non-segment-granularity rule thresholds where safe to do so. + *

+ * Algorithm: + *

    + *
  1. Generate base timeline from segment granularity rules with interval boundaries aligned with segment granularity of the underlying rules
  2. + *
  3. Collect olderThan application thresholds from all non-segment-granularity rules
  4. + *
  5. For each interval in the base timeline: + *
      + *
    • find olderThan thresholds for non-segment granularity rules that fall within it
    • + *
    • Align those thresholds to the interval's targeted segment granularity using bucketStart on the threshold date
    • + *
    • Split base intervals at the granularity aligned thresholds that were found inside of them
    • + *
    + *
  6. Return the timeline of non-overlapping intervals split for most precise possible rule application (due to segment gran alignment, sometimes rules will be applied later than their explicitly defined period)
  7. + *
+ * + * @param referenceTime the reference time for calculating period thresholds + * @return list of split and aligned intervals with their granularities and source rules, ordered from oldest to newest + * @throws IAE if no reindexing rules are configured + * @throws SegmentGranularityTimelineValidationException if granularities become coarser over time + */ + List generateAlignedSearchIntervals(DateTime referenceTime) + { + List baseTimeline = generateBaseSegmentGranularityAlignedTimeline(referenceTime); + List nonSegmentGranThresholds = collectNonSegmentGranularityThresholds(referenceTime); + + List finalIntervals = new ArrayList<>(); + for (IntervalGranularityInfo baseInterval : baseTimeline) { + List splitPoints = findGranularityAlignedSplitPoints(baseInterval, nonSegmentGranThresholds); + finalIntervals.addAll(splitIntervalAtPoints(baseInterval, splitPoints)); + } + + return finalIntervals; + } + + /** + * Finds split points within a base interval by aligning non-segment-granularity thresholds + * to the interval's segment granularity. Only includes thresholds that fall strictly inside + * the interval (not at boundaries, which would create zero-length intervals). + * + * @param baseInterval the interval to find split points for + * @param nonSegmentGranThresholds thresholds from non-segment-granularity rules + * @return sorted, distinct list of aligned split points that fall inside the interval + */ + private List findGranularityAlignedSplitPoints( + IntervalGranularityInfo baseInterval, + List nonSegmentGranThresholds + ) + { + List splitPoints = new ArrayList<>(); + + for (DateTime threshold : nonSegmentGranThresholds) { + // Check if threshold falls inside this interval + if (threshold.isAfter(baseInterval.getInterval().getStart()) && + threshold.isBefore(baseInterval.getInterval().getEnd())) { + + // Align threshold to this interval's segment granularity + DateTime alignedThreshold = baseInterval.getGranularity().bucketStart(threshold); + + // Only add if it's not at the boundaries (would create zero-length interval) + if (alignedThreshold.isAfter(baseInterval.getInterval().getStart()) && + alignedThreshold.isBefore(baseInterval.getInterval().getEnd())) { + splitPoints.add(alignedThreshold); + } + } + } + + // Remove duplicates and sort + return splitPoints.stream() + .distinct() + .sorted() + .collect(Collectors.toList()); + } + + /** + * Splits a base interval at the given split points, preserving the interval's granularity + * and source rule. If no split points exist, returns the original interval unchanged. + * + * @param baseInterval the interval to split + * @param splitPoints sorted list of points to split at (must be inside the interval) + * @return list of split intervals, or singleton list with original interval if no splits + */ + private List splitIntervalAtPoints( + IntervalGranularityInfo baseInterval, + List splitPoints + ) + { + if (splitPoints.isEmpty()) { + LOG.debug("No splits for interval [%s]", baseInterval.getInterval()); + return Collections.singletonList(baseInterval); + } + + LOG.debug("Splitting interval [%s] at [%d] points", baseInterval.getInterval(), splitPoints.size()); + + List result = new ArrayList<>(); + DateTime start = baseInterval.getInterval().getStart(); + + for (DateTime splitPoint : splitPoints) { + result.add(new IntervalGranularityInfo( + new Interval(start, splitPoint), + baseInterval.getGranularity(), + baseInterval.getSourceRule() // Preserve source rule from base interval + )); + start = splitPoint; + } + + // Add final interval from last split point to end + result.add(new IntervalGranularityInfo( + new Interval(start, baseInterval.getInterval().getEnd()), + baseInterval.getGranularity(), + baseInterval.getSourceRule() // Preserve source rule from base interval + )); + + return result; + } + + /** + * Generates a base timeline aligned to segment granularities found in segment granularity rules and if necessary, + * the default granularity for the supervisor. + *

+ * Algorithm: + *

    + *
  1. If no segment granularity rules exist: + *
      + *
    1. Find the most recent threshold from non-segment-granularity rules
    2. + *
    3. Use the default granularity to granularity align an interval from [-inf, most recent threshold)
    4. + *
    + *
  2. + *
  3. If segment granularity rules exist: + *
      + *
    1. Sort rules by period from longest to shortest (oldest to most recent threshold)
    2. + *
    3. Create intervals for each rule, adjusting the interval end to be aligned to the rule's segment granularity
    4. + *
    5. If non-segment-granularity thresholds exist that are more recent than the most recent segment granularity rule's end: + *
        + *
      1. Prepend an interval from [most recent segment granularity rule interval end, most recent non-segment-granularity threshold)
      2. + *
      + *
    6. + *
    + *
  4. + *
+ * + * @param referenceTime the reference time for calculating period thresholds + * @return base timeline with granularity-aligned intervals, ordered from oldest to newest + * @throws IAE if no reindexing rules are configured + * @throws SegmentGranularityTimelineValidationException if granularities become coarser over time + */ + private List generateBaseSegmentGranularityAlignedTimeline(DateTime referenceTime) + { + List segmentGranRules = ruleProvider.getSegmentGranularityRules(); + List nonSegmentGranThresholds = collectNonSegmentGranularityThresholds(referenceTime); + + List baseTimeline; + + if (segmentGranRules.isEmpty()) { + baseTimeline = createDefaultGranularityTimeline(nonSegmentGranThresholds); + } else { + baseTimeline = createSegmentGranularityTimeline(segmentGranRules, referenceTime); + baseTimeline = maybePrependRecentInterval(baseTimeline, nonSegmentGranThresholds); + } + + validateSegmentGranularityTimeline(baseTimeline); + return baseTimeline; + } + + /** + * Creates a timeline using the default segment granularity when no segment granularity rules exist. + * Uses the most recent threshold from non-segment-granularity rules to determine the end boundary. + * + * @param nonSegmentGranThresholds thresholds from non-segment-granularity rules + * @return single-interval timeline from MIN to most recent threshold, aligned to default granularity + * @throws IAE if no non-segment-granularity rules exist either + */ + private List createDefaultGranularityTimeline(List nonSegmentGranThresholds) + { + if (nonSegmentGranThresholds.isEmpty()) { + throw InvalidInput.exception( + "CascadingReindexingTemplate requires at least one reindexing rule (segment granularity or other type)" + ); + } + + // Find the smallest period (most recent threshold = largest DateTime value) + DateTime mostRecentThreshold = Collections.max(nonSegmentGranThresholds); + DateTime alignedEnd = defaultSegmentGranularity.bucketStart(mostRecentThreshold); + + LOG.debug( + "No segment granularity rules found for cascading supervisor[%s]. Creating base interval with " + + "default granularity [%s] and threshold [%s] (aligned: [%s])", + dataSource, + defaultSegmentGranularity, + mostRecentThreshold, + alignedEnd + ); + + return Collections.singletonList(new IntervalGranularityInfo( + new Interval(DateTimes.MIN, alignedEnd), + defaultSegmentGranularity, + null // No source rule when using default granularity + )); + } + + /** + * Creates a timeline by processing segment granularity rules in chronological order (oldest to newest). + * Each rule defines an interval with its specific segment granularity, with boundaries aligned to that granularity. + * + * @param segmentGranRules segment granularity rules to process + * @param referenceTime reference time for computing rule thresholds + * @return timeline of intervals with their granularities, ordered from oldest to newest + */ + private List createSegmentGranularityTimeline( + List segmentGranRules, + DateTime referenceTime + ) + { + // Sort rules by period from longest to shortest (oldest to most recent threshold) + List sortedRules = segmentGranRules.stream() + .sorted(Comparator.comparingLong(rule -> { + DateTime threshold = referenceTime.minus(rule.getOlderThan()); + return threshold.getMillis(); + })) + .collect(Collectors.toList()); + + // Build base timeline with granularities tracked + List baseTimeline = new ArrayList<>(); + DateTime previousAlignedEnd = null; + + for (ReindexingSegmentGranularityRule rule : sortedRules) { + DateTime rawEnd = referenceTime.minus(rule.getOlderThan()); + DateTime alignedEnd = rule.getSegmentGranularity().bucketStart(rawEnd); + DateTime alignedStart = (previousAlignedEnd != null) ? previousAlignedEnd : DateTimes.MIN; + + LOG.debug( + "Base interval for rule [%s]: raw end [%s] -> aligned [%s/%s) with granularity [%s]", + rule.getId(), + rawEnd, + alignedStart, + alignedEnd, + rule.getSegmentGranularity() + ); + + baseTimeline.add(new IntervalGranularityInfo( + new Interval(alignedStart, alignedEnd), + rule.getSegmentGranularity(), + rule // Track the source rule + )); + + previousAlignedEnd = alignedEnd; + } + + return baseTimeline; + } + + /** + * Checks if non-segment-granularity rules have more recent thresholds than the most recent + * segment granularity rule, and if so, prepends an interval with the default granularity. + * This ensures that all rules (not just segment granularity rules) are represented in the timeline. + * + * @param baseTimeline existing timeline built from segment granularity rules + * @param nonSegmentGranThresholds thresholds from non-segment-granularity rules + * @return updated timeline with prepended interval if needed, otherwise original timeline + */ + private List maybePrependRecentInterval( + List baseTimeline, + List nonSegmentGranThresholds + ) + { + if (nonSegmentGranThresholds.isEmpty()) { + return baseTimeline; + } + + DateTime mostRecentNonSegmentGranThreshold = Collections.max(nonSegmentGranThresholds); + DateTime mostRecentSegmentGranEnd = baseTimeline.get(baseTimeline.size() - 1).getInterval().getEnd(); + + if (!mostRecentNonSegmentGranThreshold.isAfter(mostRecentSegmentGranEnd)) { + return baseTimeline; + } + + DateTime alignedEnd = defaultSegmentGranularity.bucketStart(mostRecentNonSegmentGranThreshold); + + if (alignedEnd.isBefore(mostRecentSegmentGranEnd) || alignedEnd.isEqual(mostRecentSegmentGranEnd)) { + LOG.debug( + "Most recent non-segment-gran threshold [%s] aligns to [%s], which is not after " + + "most recent segment granularity rule interval end [%s]. No prepended interval needed.", + mostRecentNonSegmentGranThreshold, + alignedEnd, + mostRecentSegmentGranEnd + ); + return baseTimeline; + } + + LOG.debug( + "Most recent non-segment-gran threshold [%s] is after most recent segment gran interval end [%s]. " + + "Prepending interval with default granularity [%s] (aligned end: [%s])", + mostRecentNonSegmentGranThreshold, + mostRecentSegmentGranEnd, + defaultSegmentGranularity, + alignedEnd + ); + + // Create new list with prepended interval (don't modify original) + List updatedTimeline = new ArrayList<>(baseTimeline); + updatedTimeline.add(new IntervalGranularityInfo( + new Interval(mostRecentSegmentGranEnd, alignedEnd), + defaultSegmentGranularity, + null // No source rule when using default granularity + )); + + return updatedTimeline; + } + + /** + * Validates that the completed segment granularity timeline follows the constraint that + * granularity must stay the same or become finer as we move from past to present. + *

+ * This ensures that operators cannot misconfigure rules that would cause data to be + * recompacted from coarse to fine granularity as it ages (e.g., DAY -> HOUR), + * which is typically undesirable and inefficient. + * + * @param timeline the completed base timeline with granularity information + * @throws SegmentGranularityTimelineValidationException if granularity becomes coarser as we move toward present + */ + private void validateSegmentGranularityTimeline(List timeline) + { + if (timeline.size() <= 1) { + return; // Nothing to validate + } + + for (int i = 1; i < timeline.size(); i++) { + IntervalGranularityInfo olderInterval = timeline.get(i - 1); + IntervalGranularityInfo newerInterval = timeline.get(i); + + Granularity olderGran = olderInterval.getGranularity(); + Granularity newerGran = newerInterval.getGranularity(); + + // As we move from past (older intervals) to present (newer intervals), + // granularity should stay the same or get finer. + // If the older interval's granularity is finer than the newer interval's granularity, + // that means we're getting coarser as we move toward present, which is invalid. + if (olderGran.isFinerThan(newerGran)) { + throw new SegmentGranularityTimelineValidationException( + dataSource, + olderInterval.getInterval(), + olderGran, + newerInterval.getInterval(), + newerGran + ); + } + } + + LOG.debug( + "Segment granularity timeline validation passed for dataSource[%s] with [%d] intervals", + dataSource, + timeline.size() + ); + } + + /** + * Collects thresholds from all non-segment-granularity rules. + */ + private List collectNonSegmentGranularityThresholds(DateTime referenceTime) + { + return ruleProvider.streamAllRules() + .filter(rule -> !(rule instanceof ReindexingSegmentGranularityRule)) + .map(rule -> referenceTime.minus(rule.getOlderThan())) + .collect(Collectors.toList()); + } + + private List createJobsForSearchInterval( + CompactionJobTemplate template, + Interval searchInterval, + DruidInputSource inputSource, + CompactionJobParams jobParams + ) + { + return template.createCompactionJobs( + inputSource.withInterval(searchInterval), + jobParams + ); + } + + @Override + public CompactionState toCompactionState() + { + throw new UnsupportedOperationException("CascadingReindexingTemplate cannot be transformed to a CompactionState object"); + } + + + // Legacy fields from DataSourceCompactionConfig that are not used by this template + + @Nullable + @Override + public Integer getMaxRowsPerSegment() + { + return 0; + } + + @Nullable + @Override + public UserCompactionTaskQueryTuningConfig getTuningConfig() + { + return null; + } + + @Nullable + @Override + public UserCompactionTaskIOConfig getIoConfig() + { + return null; + } + + @Nullable + @Override + public Granularity getSegmentGranularity() + { + return null; + } + + @Nullable + @Override + public UserCompactionTaskGranularityConfig getGranularitySpec() + { + return null; + } + + @Nullable + @Override + public List getProjections() + { + return List.of(); + } + + @Nullable + @Override + public CompactionTransformSpec getTransformSpec() + { + return null; + } + + @Nullable + @Override + public UserCompactionTaskDimensionsConfig getDimensionsSpec() + { + return null; + } + + @Nullable + @Override + public AggregatorFactory[] getMetricsSpec() + { + return new AggregatorFactory[0]; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java index 24b6e1f6af40..25643231b2c8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java @@ -48,10 +48,20 @@ public class CompactionConfigBasedJobTemplate implements CompactionJobTemplate { private final DataSourceCompactionConfig config; + private final ReindexingConfigOptimizer configOptimizer; public CompactionConfigBasedJobTemplate(DataSourceCompactionConfig config) + { + this(config, ReindexingConfigOptimizer.IDENTITY); + } + + public CompactionConfigBasedJobTemplate( + DataSourceCompactionConfig config, + ReindexingConfigOptimizer configOptimizer + ) { this.config = config; + this.configOptimizer = configOptimizer; } @Nullable @@ -82,9 +92,12 @@ public List createCompactionJobs( while (segmentIterator.hasNext()) { final CompactionCandidate candidate = segmentIterator.next(); + // Allow template-specific customization of the config per candidate + DataSourceCompactionConfig finalConfig = configOptimizer.optimizeConfig(config, candidate, params); + ClientCompactionTaskQuery taskPayload = CompactSegments.createCompactionTask( candidate, - config, + finalConfig, params.getClusterCompactionConfig().getEngine(), indexingStateFingerprint, params.getClusterCompactionConfig().isStoreCompactionStatePerSegment() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java index cf12be8ce90e..d9002655f742 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java @@ -98,7 +98,11 @@ public CompactionSupervisor createSupervisor() */ public CompactionJobTemplate getTemplate() { - return new CompactionConfigBasedJobTemplate(spec); + if (spec instanceof CascadingReindexingTemplate) { + return (CascadingReindexingTemplate) spec; + } else { + return new CompactionConfigBasedJobTemplate(spec); + } } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/ReindexingConfigBuilder.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/ReindexingConfigBuilder.java new file mode 100644 index 000000000000..8fec45532105 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/ReindexingConfigBuilder.java @@ -0,0 +1,280 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.query.filter.OrDimFilter; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.server.compaction.IntervalGranularityInfo; +import org.apache.druid.server.compaction.ReindexingDataSchemaRule; +import org.apache.druid.server.compaction.ReindexingDeletionRule; +import org.apache.druid.server.compaction.ReindexingIOConfigRule; +import org.apache.druid.server.compaction.ReindexingRule; +import org.apache.druid.server.compaction.ReindexingRuleProvider; +import org.apache.druid.server.compaction.ReindexingTuningConfigRule; +import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Builds compaction configs for cascading reindexing by applying reindexing rules. + * This is an implementation detail of {@link CascadingReindexingTemplate} and encapsulates + * the logic for combining additive rules and applying all rule types. + *

+ * Package-private as this is only used internally by CascadingReindexingTemplate. + */ +class ReindexingConfigBuilder +{ + private static final Logger LOG = new Logger(ReindexingConfigBuilder.class); + + private final ReindexingRuleProvider provider; + private final Interval interval; + private final DateTime referenceTime; + private final List syntheticTimeline; + + /** + * Result of applying reindexing rules to a config builder. + * Contains both the count of rules applied and the actual rules that were applied. + */ + static class BuildResult + { + private final int ruleCount; + private final List appliedRules; + + BuildResult(int ruleCount, List appliedRules) + { + this.ruleCount = ruleCount; + this.appliedRules = appliedRules; + } + + /** + * Returns the count of rules that were actually applied to this specific interval. + * This is NOT the total number of rules in the provider, but rather the count + * of rules that matched and were applied during config building. + * + * @return the number of rules that were applied to the builder + */ + int getRuleCount() + { + return ruleCount; + } + + /** + * @return immutable list of the actual rules that were applied, in application order + */ + List getAppliedRules() + { + return appliedRules; + } + } + + ReindexingConfigBuilder( + ReindexingRuleProvider provider, + Interval interval, + DateTime referenceTime, + List syntheticTimeline + ) + { + this.provider = provider; + this.interval = interval; + this.referenceTime = referenceTime; + this.syntheticTimeline = syntheticTimeline; + } + + /** + * Applies all applicable rules to the builder. + * + * @return number of rules applied + */ + int applyTo(InlineSchemaDataSourceCompactionConfig.Builder builder) + { + return applyToWithDetails(builder).getRuleCount(); + } + + /** + * Applies all applicable rules to the builder and returns detailed information about + * which rules were applied. + * + * @return BuildResult containing the count and list of applied rules + */ + BuildResult applyToWithDetails(InlineSchemaDataSourceCompactionConfig.Builder builder) + { + int count = 0; + List appliedRules = new ArrayList<>(); + + // Apply tuning config rule + ReindexingTuningConfigRule tuningRule = provider.getTuningConfigRule(interval, referenceTime); + if (tuningRule != null) { + builder.withTuningConfig(tuningRule.getTuningConfig()); + appliedRules.add(tuningRule); + count++; + } + + // Apply IO config rule + ReindexingIOConfigRule ioConfigRule = provider.getIOConfigRule(interval, referenceTime); + if (ioConfigRule != null) { + builder.withIoConfig(ioConfigRule.getIoConfig()); + appliedRules.add(ioConfigRule); + count++; + } + + // Apply data schema rules + ReindexingDataSchemaRule dataSchemaRule = provider.getDataSchemaRule(interval, referenceTime); + if (dataSchemaRule != null) { + applyDataSchemaRule(builder, dataSchemaRule); + appliedRules.add(dataSchemaRule); + count++; + } + + // Apply deletion rules (additive) + List deletionRules = provider.getDeletionRules(interval, referenceTime); + if (!deletionRules.isEmpty()) { + applyDeletionRulesList(builder, deletionRules); + appliedRules.addAll(deletionRules); + count += deletionRules.size(); + } + + // Apply segment granularity rule + // Use granularity from synthetic timeline + IntervalGranularityInfo granularityInfo = findMatchingInterval(interval); + if (granularityInfo == null) { + throw DruidException.defensive( + "No matching interval found in synthetic timeline for interval[%s]. This should never happen.", + interval + ); + } + + builder.withSegmentGranularity(granularityInfo.getGranularity()); + if (granularityInfo.getSourceRule() != null) { + // Only count and track the rule if it came from an actual rule (not default) + appliedRules.add(granularityInfo.getSourceRule()); + count++; + } + + return new BuildResult(count, count == 0 ? List.of() : appliedRules); + } + + /** + * Finds the matching interval granularity info from the synthetic timeline. + * Returns null if no synthetic timeline was provided or no match is found. + */ + @Nullable + private IntervalGranularityInfo findMatchingInterval(Interval interval) + { + for (IntervalGranularityInfo candidate : syntheticTimeline) { + if (candidate.getInterval().equals(interval)) { + return candidate; + } + } + + return null; + } + + private void applyDataSchemaRule( + InlineSchemaDataSourceCompactionConfig.Builder builder, + ReindexingDataSchemaRule dataSchemaRule + ) + { + if (dataSchemaRule.getDimensionsSpec() != null) { + builder.withDimensionsSpec(dataSchemaRule.getDimensionsSpec()); + } + + if (dataSchemaRule.getMetricsSpec() != null) { + builder.withMetricsSpec(dataSchemaRule.getMetricsSpec()); + } + + if (dataSchemaRule.getProjections() != null) { + builder.withProjections(dataSchemaRule.getProjections()); + } + + if (dataSchemaRule.getQueryGranularity() != null || dataSchemaRule.getRollup() != null) { + builder.withQueryGranularityAndRollup( + dataSchemaRule.getQueryGranularity(), + dataSchemaRule.getRollup() + ); + } + } + + /** + * Applies deletion rules by combining their filters into a single transform filter. + *

+ * Each deletion rule specifies rows that should be deleted. To implement deletion during + * compaction, we need to keep only rows that do NOT match any deletion rule. + *

+ * Filter construction logic: + *

    + *
  • Collect all deletion filters (one per rule)
  • + *
  • OR them together: (filter1 OR filter2 OR ...)
  • + *
  • Wrap in NOT: NOT(filter1 OR filter2 OR ...)
  • + *
+ *

+ * Result: Rows matching ANY deletion rule are filtered out, all other rows are kept. + *

+ * Example: With rules "delete country=US" and "delete device=mobile": + * Final filter: NOT((country=US) OR (device=mobile)) + * This keeps all rows except those where country=US OR device=mobile. + * + * @param builder the config builder to apply the deletion filter to + * @param rules the deletion rules to combine + */ + private void applyDeletionRulesList( + InlineSchemaDataSourceCompactionConfig.Builder builder, + List rules + ) + { + + // Collect filters and virtual columns in a single pass + List removeConditions = new ArrayList<>(); + List allVirtualColumns = new ArrayList<>(); + + for (ReindexingDeletionRule rule : rules) { + removeConditions.add(rule.getDeleteWhere()); + + if (rule.getVirtualColumns() != null) { + allVirtualColumns.addAll(Arrays.asList(rule.getVirtualColumns().getVirtualColumns())); + } + } + + // Combine filters: OR all filters together, wrap in NOT + DimFilter removeFilter = removeConditions.size() == 1 + ? removeConditions.get(0) + : new OrDimFilter(removeConditions); + DimFilter finalFilter = new NotDimFilter(removeFilter); + + // Create VirtualColumns if any exist + VirtualColumns virtualColumns = allVirtualColumns.isEmpty() + ? null + : VirtualColumns.create(allVirtualColumns); + + builder.withTransformSpec(new CompactionTransformSpec(finalFilter, virtualColumns)); + + LOG.debug("Applied [%d] filter rules for interval %s", rules.size(), interval); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/ReindexingConfigOptimizer.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/ReindexingConfigOptimizer.java new file mode 100644 index 000000000000..d091d5433f22 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/ReindexingConfigOptimizer.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import org.apache.druid.server.compaction.CompactionCandidate; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; + +/** + * Functional interface for customizing a {@link DataSourceCompactionConfig} for a specific + * {@link CompactionCandidate} before creating a reindexing job. This allows template-specific + * logic to be injected without hardcoding behavior in {@link CompactionConfigBasedJobTemplate}. + *

+ * For example, cascading reindexing templates can use this to optimize filter rules based on + * the candidate's indexing state, while simpler templates can use the identity finalizer. + */ +@FunctionalInterface +public interface ReindexingConfigOptimizer +{ + /** + * Customize the reindexing config for a specific candidate. + * + * @param config the base reindexing config + * @param candidate the segment candidate being reindexed + * @param params the reindexing job parameters + * @return the finalized config to use for this candidate (this may be the same as input or a modified version) + */ + DataSourceCompactionConfig optimizeConfig( + DataSourceCompactionConfig config, + CompactionCandidate candidate, + CompactionJobParams params + ); + + /** + * Identity finalizer that returns the config unchanged. + * Use this for templates that don't need per-candidate customization. + */ + ReindexingConfigOptimizer IDENTITY = (config, candidate, params) -> config; +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/ReindexingDeletionRuleOptimizer.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/ReindexingDeletionRuleOptimizer.java new file mode 100644 index 000000000000..9b4bbb597977 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/ReindexingDeletionRuleOptimizer.java @@ -0,0 +1,251 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.query.filter.OrDimFilter; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; +import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.server.compaction.CompactionCandidate; +import org.apache.druid.server.compaction.CompactionStatus; +import org.apache.druid.server.compaction.ReindexingDeletionRule; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; +import org.apache.druid.timeline.CompactionState; +import org.apache.druid.timeline.DataSegment; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Optimization utilities for applying {@link ReindexingDeletionRule}s during reindexing + *

+ * When reindexing with {@link ReindexingDeletionRule}s, it is possible that candidate + * segments have already applied some or all of the deletion rules in previous reindexing runs. Reapplying such rules would + * be wasteful and redundant. This class provides funcionality to optimize the set of rules to be applied by + * any given reindexing task. + */ +public class ReindexingDeletionRuleOptimizer implements ReindexingConfigOptimizer +{ + private static final Logger LOG = new Logger(ReindexingDeletionRuleOptimizer.class); + + @Override + public DataSourceCompactionConfig optimizeConfig( + DataSourceCompactionConfig config, + CompactionCandidate candidate, + CompactionJobParams params + ) + { + if (!shouldOptimizeFilterRules(candidate, config)) { + return config; + } + + NotDimFilter reducedFilter = computeRequiredSetOfFilterRulesForCandidate( + candidate, + (NotDimFilter) config.getTransformSpec().getFilter(), + params.getFingerprintMapper() + ); + + VirtualColumns reducedVirtualColumns = filterVirtualColumnsForFilter( + reducedFilter, + config.getTransformSpec().getVirtualColumns() + ); + + CompactionTransformSpec transformSpec = (reducedFilter == null && reducedVirtualColumns == null) + ? null + : new CompactionTransformSpec(reducedFilter, reducedVirtualColumns); + + return ((InlineSchemaDataSourceCompactionConfig) config) + .toBuilder() + .withTransformSpec(transformSpec) + .build(); + } + + /** + * Computes the required set of deletion rules to be applied for the given {@link CompactionCandidate}. + *

+ * We only want to apply the rules that have not yet been applied to all segments in the candidate. This reduces + * the amount of work the task needs to do while processing rows during reindexing. + *

+ * + * @param candidateSegments the {@link CompactionCandidate} + * @param expectedFilter the expected filter (as a NotDimFilter wrapping an OrDimFilter) + * @param fingerprintMapper the fingerprint mapper to retrieve applied rules from segment fingerprints + * @return the set of unapplied deletion rules wrapped in a NotDimFilter, or null if all rules have been applied + */ + @Nullable + private NotDimFilter computeRequiredSetOfFilterRulesForCandidate( + CompactionCandidate candidateSegments, + NotDimFilter expectedFilter, + IndexingStateFingerprintMapper fingerprintMapper + ) + { + List expectedFilters; + if (!(expectedFilter.getField() instanceof OrDimFilter)) { + expectedFilters = Collections.singletonList(expectedFilter.getField()); + } else { + expectedFilters = ((OrDimFilter) expectedFilter.getField()).getFields(); + } + + Set uniqueFingerprints = candidateSegments.getSegments().stream() + .map(DataSegment::getIndexingStateFingerprint) + .filter(Objects::nonNull) + .collect(Collectors.toSet()); + + if (uniqueFingerprints.isEmpty()) { + // no fingerprints means that no candidate segments have transforms to compare against. Return all filters eagerly. + return expectedFilter; + } + + Set unappliedRules = new HashSet<>(); + + for (String fingerprint : uniqueFingerprints) { + CompactionState state = fingerprintMapper.getStateForFingerprint(fingerprint).orElse(null); + + if (state == null) { + // Safety: if state is missing, return all filters eagerly since we can't determine applied filters + return expectedFilter; + } + + Set appliedFilters = extractAppliedFilters(state); + + if (appliedFilters == null) { + return expectedFilter; + } + + for (DimFilter expected : expectedFilters) { + if (!appliedFilters.contains(expected)) { + unappliedRules.add(expected); + } + } + } + + LOG.debug( + "Computed [%d] unapplied rules out of [%d] possible rules for candidate", + unappliedRules.size(), + expectedFilters.size() + ); + + if (unappliedRules.isEmpty()) { + return null; + } + + return new NotDimFilter(new OrDimFilter(new ArrayList<>(unappliedRules))); + } + + /** + * Filters virtual columns to only include ones referenced by the given {@link DimFilter}. + * This removes virtual columns that were used by deletion rules that have been optimized away. + * + * @param filter the reduced filter to check for column references + * @param virtualColumns the original set of virtual columns + * @return filtered VirtualColumns with only referenced columns, or null if none are referenced + */ + @Nullable + private VirtualColumns filterVirtualColumnsForFilter( + @Nullable DimFilter filter, + @Nullable VirtualColumns virtualColumns + ) + { + if (virtualColumns == null || filter == null) { + return null; + } + + // Get the set of columns required by the filter + Set requiredColumns = filter.getRequiredColumns(); + + // Filter virtual columns to only include ones whose output name is required + List referencedColumns = new ArrayList<>(); + for (VirtualColumn vc : virtualColumns.getVirtualColumns()) { + if (requiredColumns.contains(vc.getOutputName())) { + referencedColumns.add(vc); + } + } + + // Return null if no virtual columns are referenced, otherwise create new VirtualColumns + return referencedColumns.isEmpty() ? null : VirtualColumns.create(referencedColumns); + } + + /** + * Extracts the set of applied filters from a {@link CompactionState}. + * + * @param state the {@link CompactionState} to extract applied filters from + * @return the set of applied filters, or null if transform spec or filter is null (indicating 0 applied filters) + */ + @Nullable + private static Set extractAppliedFilters(CompactionState state) + { + if (state.getTransformSpec() == null) { + return null; + } + + DimFilter filter = state.getTransformSpec().getFilter(); + if (filter == null) { + return null; + } + + if (!(filter instanceof NotDimFilter)) { + return Collections.emptySet(); + } + + DimFilter inner = ((NotDimFilter) filter).getField(); + + if (inner instanceof OrDimFilter) { + return new HashSet<>(((OrDimFilter) inner).getFields()); + } else { + return Collections.singleton(inner); + } + } + + /** + * Determines if we should optimize filter rules for this candidate. + * Returns true only if the candidate has been compacted before and has a NotDimFilter. + */ + private boolean shouldOptimizeFilterRules( + CompactionCandidate candidate, + DataSourceCompactionConfig config + ) + { + if (candidate.getCurrentStatus() == null) { + return false; + } + + if (candidate.getCurrentStatus().getReason().equals(CompactionStatus.NEVER_COMPACTED_REASON)) { + return false; + } + + if (config.getTransformSpec() == null) { + return false; + } + + DimFilter filter = config.getTransformSpec().getFilter(); + return filter instanceof NotDimFilter; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/SegmentGranularityTimelineValidationException.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/SegmentGranularityTimelineValidationException.java new file mode 100644 index 000000000000..bfb38295d1c0 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/SegmentGranularityTimelineValidationException.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.joda.time.Interval; + +/** + * Exception thrown when segment granularity timeline validation fails when using {@link CascadingReindexingTemplate}. + * Contains structured information about the conflicting intervals. + */ +public class SegmentGranularityTimelineValidationException extends IAE +{ + private final Interval olderInterval; + private final Granularity olderGranularity; + private final Interval newerInterval; + private final Granularity newerGranularity; + + public SegmentGranularityTimelineValidationException( + String dataSource, + Interval olderInterval, + Granularity olderGranularity, + Interval newerInterval, + Granularity newerGranularity + ) + { + super( + "Invalid segment granularity timeline for dataSource[%s]: " + + "Interval[%s] with granularity[%s] is more recent than " + + "interval[%s] with granularity[%s], but has a coarser granularity. " + + "Segment granularity must stay the same or become coarser as data ages from present to past.", + dataSource, + newerInterval, + newerGranularity, + olderInterval, + olderGranularity + ); + this.olderInterval = olderInterval; + this.olderGranularity = olderGranularity; + this.newerInterval = newerInterval; + this.newerGranularity = newerGranularity; + } + + public Interval getOlderInterval() + { + return olderInterval; + } + + public Granularity getOlderGranularity() + { + return olderGranularity; + } + + public Interval getNewerInterval() + { + return newerInterval; + } + + public Granularity getNewerGranularity() + { + return newerGranularity; + } +} 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 1c04c7b5bd2b..4136d082aa51 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 @@ -97,7 +97,7 @@ public class ClientCompactionTaskQuerySerdeTest new ClientCompactionTaskGranularitySpec(Granularities.DAY, Granularities.HOUR, true); private static final AggregatorFactory[] METRICS_SPEC = new AggregatorFactory[] {new CountAggregatorFactory("cnt")}; private static final CompactionTransformSpec CLIENT_COMPACTION_TASK_TRANSFORM_SPEC = - new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null)); + new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null), null); private static final DynamicPartitionsSpec DYNAMIC_PARTITIONS_SPEC = new DynamicPartitionsSpec(100, 30000L); private static final SegmentsSplitHintSpec SEGMENTS_SPLIT_HINT_SPEC = new SegmentsSplitHintSpec(new HumanReadableBytes(100000L), 10); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java index b59a1625d49a..75d046593883 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java @@ -573,7 +573,7 @@ public void testRunCompactionWithFilterShouldStoreInState() throws Exception final CompactionTask compactionTask = builder .inputSpec(new CompactionIntervalSpec(INTERVAL_TO_INDEX, null)) .tuningConfig(AbstractParallelIndexSupervisorTaskTest.DEFAULT_TUNING_CONFIG_FOR_PARALLEL_INDEXING) - .transformSpec(new CompactionTransformSpec(new SelectorDimFilter("dim", "a", null))) + .transformSpec(new CompactionTransformSpec(new SelectorDimFilter("dim", "a", null), null)) .build(); final DataSegmentsWithSchemas dataSegmentsWithSchemas = runTask(compactionTask); @@ -1027,7 +1027,8 @@ public void testRunParallelWithRangePartitioningFilteringAllRows() throws Except true )) .transformSpec(new CompactionTransformSpec( - new SelectorDimFilter("dim", "nonexistent_value", null) // Filters out all rows + new SelectorDimFilter("dim", "nonexistent_value", null), // Filters out all rows + null )) .build(); @@ -1074,7 +1075,8 @@ public void testRunParallelRangePartitioningFilterAllRowsReplaceLegacyMode() thr true )) .transformSpec(new CompactionTransformSpec( - new SelectorDimFilter("dim", "nonexistent_value", null) // Filters all rows + new SelectorDimFilter("dim", "nonexistent_value", null), // Filters all rows + null )) .build(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunBase.java index f6dece2ba9da..50b53e614886 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunBase.java @@ -592,7 +592,7 @@ public void testCompactionWithFilterInTransformSpec() throws Exception final CompactionTask compactionTask = compactionTaskBuilder(segmentGranularity) .interval(inputInterval, true) - .transformSpec(new CompactionTransformSpec(new SelectorDimFilter("dim", "a", null))) + .transformSpec(new CompactionTransformSpec(new SelectorDimFilter("dim", "a", null), null)) .build(); Pair resultPair = runTask(compactionTask); @@ -608,7 +608,7 @@ public void testCompactionWithFilterInTransformSpec() throws Exception segmentGranularity, DEFAULT_QUERY_GRAN, List.of(inputInterval) - ).toBuilder().transformSpec(new CompactionTransformSpec(new SelectorDimFilter("dim", "a", null))).build(); + ).toBuilder().transformSpec(new CompactionTransformSpec(new SelectorDimFilter("dim", "a", null), null)).build(); Assert.assertEquals(expectedCompactionState, segments.get(0).getLastCompactionState()); } 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 acb0dba5027b..aeb5006a249b 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 @@ -430,7 +430,7 @@ public void testCreateCompactionTaskWithConflictingGranularitySpecAndSegmentGran public void testCreateCompactionTaskWithTransformSpec() { CompactionTransformSpec transformSpec = - new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null)); + new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null), null); final Builder builder = new Builder( DATA_SOURCE, segmentCacheManagerFactory @@ -1773,7 +1773,7 @@ public void testGetDefaultLookupLoadingSpecWithTransformSpec() ); final CompactionTask task = builder .interval(Intervals.of("2000-01-01/2000-01-02")) - .transformSpec(new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null))) + .transformSpec(new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null), null)) .build(); Assert.assertEquals(LookupLoadingSpec.ALL, task.getLookupLoadingSpec()); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NativeCompactionRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NativeCompactionRunnerTest.java new file mode 100644 index 000000000000..dcb801ab4ecf --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NativeCompactionRunnerTest.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.indexing.common.SegmentCacheManagerFactory; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.server.coordinator.CompactionConfigValidationResult; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.Collections; +import java.util.Map; + +public class NativeCompactionRunnerTest +{ + private static final NativeCompactionRunner NATIVE_COMPACTION_RUNNER = new NativeCompactionRunner( + Mockito.mock(SegmentCacheManagerFactory.class) + ); + + @Test + public void testVirtualColumnsInTransformSpecAreNotSupported() + { + VirtualColumns virtualColumns = VirtualColumns.create( + ImmutableList.of( + new ExpressionVirtualColumn( + "extractedField", + "json_value(metadata, '$.category')", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ) + ) + ); + + CompactionTransformSpec transformSpec = new CompactionTransformSpec(null, virtualColumns); + + CompactionTask compactionTask = createCompactionTask(transformSpec); + Map intervalDataschemas = Collections.emptyMap(); + + CompactionConfigValidationResult validationResult = NATIVE_COMPACTION_RUNNER.validateCompactionTask( + compactionTask, + intervalDataschemas + ); + + Assert.assertFalse(validationResult.isValid()); + Assert.assertEquals( + "Virtual columns in filter rules are not supported by the Native compaction engine. Use MSQ compaction engine instead.", + validationResult.getReason() + ); + } + + @Test + public void testNoVirtualColumnsIsValid() + { + CompactionTask compactionTask = createCompactionTask(null); + Map intervalDataschemas = Collections.emptyMap(); + + CompactionConfigValidationResult validationResult = NATIVE_COMPACTION_RUNNER.validateCompactionTask( + compactionTask, + intervalDataschemas + ); + + Assert.assertTrue(validationResult.isValid()); + } + + @Test + public void testEmptyVirtualColumnsIsValid() + { + CompactionTransformSpec transformSpec = new CompactionTransformSpec(null, VirtualColumns.EMPTY); + + CompactionTask compactionTask = createCompactionTask(transformSpec); + Map intervalDataschemas = Collections.emptyMap(); + + CompactionConfigValidationResult validationResult = NATIVE_COMPACTION_RUNNER.validateCompactionTask( + compactionTask, + intervalDataschemas + ); + + Assert.assertTrue(validationResult.isValid()); + } + + private CompactionTask createCompactionTask(CompactionTransformSpec transformSpec) + { + SegmentCacheManagerFactory segmentCacheManagerFactory = Mockito.mock(SegmentCacheManagerFactory.class); + CompactionTask.Builder builder = new CompactionTask.Builder( + "dataSource", + segmentCacheManagerFactory + ); + builder.inputSpec(new CompactionIntervalSpec(Intervals.of("2020-01-01/2020-01-02"), null)); + builder.transformSpec(transformSpec); + return builder.build(); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CascadingReindexingTemplateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CascadingReindexingTemplateTest.java new file mode 100644 index 000000000000..01b361004445 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CascadingReindexingTemplateTest.java @@ -0,0 +1,1653 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.error.DruidException; +import org.apache.druid.guice.SupervisorModule; +import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.indexing.input.DruidInputSource; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.server.compaction.InlineReindexingRuleProvider; +import org.apache.druid.server.compaction.IntervalGranularityInfo; +import org.apache.druid.server.compaction.ReindexingDataSchemaRule; +import org.apache.druid.server.compaction.ReindexingRule; +import org.apache.druid.server.compaction.ReindexingRuleProvider; +import org.apache.druid.server.compaction.ReindexingSegmentGranularityRule; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentTimeline; +import org.easymock.EasyMock; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class CascadingReindexingTemplateTest extends InitializedNullHandlingTest +{ + private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); + + @BeforeEach + public void setUp() + { + OBJECT_MAPPER.registerModules(new SupervisorModule().getJacksonModules()); + } + + @Test + public void test_serde() throws Exception + { + final CascadingReindexingTemplate template = new CascadingReindexingTemplate( + "testDataSource", + 50, + 1000000L, + InlineReindexingRuleProvider.builder() + .segmentGranularityRules(List.of( + new ReindexingSegmentGranularityRule( + "hourRule", + null, + Period.days(7), + Granularities.HOUR + ), + new ReindexingSegmentGranularityRule( + "dayRule", + null, + Period.days(30), + Granularities.DAY + ) + )) + .build(), + CompactionEngine.NATIVE, + ImmutableMap.of("context_key", "context_value"), + null, + null, + Granularities.DAY + ); + + final String json = OBJECT_MAPPER.writeValueAsString(template); + final CascadingReindexingTemplate fromJson = OBJECT_MAPPER.readValue(json, CascadingReindexingTemplate.class); + + Assertions.assertEquals(template.getDataSource(), fromJson.getDataSource()); + Assertions.assertEquals(template.getTaskPriority(), fromJson.getTaskPriority()); + Assertions.assertEquals(template.getInputSegmentSizeBytes(), fromJson.getInputSegmentSizeBytes()); + Assertions.assertEquals(template.getEngine(), fromJson.getEngine()); + Assertions.assertEquals(template.getTaskContext(), fromJson.getTaskContext()); + Assertions.assertEquals(template.getType(), fromJson.getType()); + } + + @Test + public void test_serde_asDataSourceCompactionConfig() throws Exception + { + final CascadingReindexingTemplate template = new CascadingReindexingTemplate( + "testDataSource", + 30, + 500000L, + InlineReindexingRuleProvider.builder() + .segmentGranularityRules(List.of( + new ReindexingSegmentGranularityRule( + "rule1", + null, + Period.days(7), + Granularities.HOUR + ) + )) + .build(), + CompactionEngine.MSQ, + ImmutableMap.of("key", "value"), + null, + null, + Granularities.HOUR + ); + + // Serialize and deserialize as DataSourceCompactionConfig interface + final String json = OBJECT_MAPPER.writeValueAsString(template); + final DataSourceCompactionConfig fromJson = OBJECT_MAPPER.readValue(json, DataSourceCompactionConfig.class); + + Assertions.assertTrue(fromJson instanceof CascadingReindexingTemplate); + final CascadingReindexingTemplate cascadingFromJson = (CascadingReindexingTemplate) fromJson; + + Assertions.assertEquals("testDataSource", cascadingFromJson.getDataSource()); + Assertions.assertEquals(30, cascadingFromJson.getTaskPriority()); + Assertions.assertEquals(500000L, cascadingFromJson.getInputSegmentSizeBytes()); + Assertions.assertEquals(CompactionEngine.MSQ, cascadingFromJson.getEngine()); + Assertions.assertEquals(ImmutableMap.of("key", "value"), cascadingFromJson.getTaskContext()); + Assertions.assertEquals(CascadingReindexingTemplate.TYPE, cascadingFromJson.getType()); + } + + @Test + public void test_createCompactionJobs_ruleProviderNotReady() + { + final ReindexingRuleProvider notReadyProvider = EasyMock.createMock(ReindexingRuleProvider.class); + EasyMock.expect(notReadyProvider.isReady()).andReturn(false); + EasyMock.expect(notReadyProvider.getType()).andReturn("mock-provider"); + EasyMock.replay(notReadyProvider); + + final CascadingReindexingTemplate template = new CascadingReindexingTemplate( + "testDataSource", + null, + null, + notReadyProvider, + null, + null, + null, + null, + Granularities.DAY + ); + + // Call createCompactionJobs - should return empty list without processing + final List jobs = template.createCompactionJobs(null, null); + + Assertions.assertTrue(jobs.isEmpty()); + EasyMock.verify(notReadyProvider); + } + + @Test + public void test_constructor_setBothSkipOffsetStrategiesThrowsException() + { + final ReindexingRuleProvider mockProvider = EasyMock.createMock(ReindexingRuleProvider.class); + EasyMock.replay(mockProvider); + + DruidException exception = Assertions.assertThrows( + DruidException.class, + () -> new CascadingReindexingTemplate( + "testDataSource", + null, + null, + mockProvider, + null, + null, + Period.days(7), // skipOffsetFromLatest + Period.days(3), // skipOffsetFromNow + Granularities.DAY + ) + ); + + Assertions.assertEquals("Cannot set both skipOffsetFromNow and skipOffsetFromLatest", exception.getMessage()); + EasyMock.verify(mockProvider); + } + + @Test + public void test_constructor_nullDataSourceThrowsException() + { + final ReindexingRuleProvider mockProvider = EasyMock.createMock(ReindexingRuleProvider.class); + EasyMock.replay(mockProvider); + + DruidException exception = Assertions.assertThrows( + DruidException.class, + () -> new CascadingReindexingTemplate( + null, // null dataSource + null, + null, + mockProvider, + null, + null, + null, + null, + Granularities.DAY + ) + ); + + Assertions.assertTrue(exception.getMessage().contains("'dataSource' cannot be null")); + EasyMock.verify(mockProvider); + } + + @Test + public void test_constructor_nullRuleProviderThrowsException() + { + DruidException exception = Assertions.assertThrows( + DruidException.class, + () -> new CascadingReindexingTemplate( + "testDataSource", + null, + null, + null, // null ruleProvider + null, + null, + null, + null, + Granularities.DAY + ) + ); + + Assertions.assertTrue(exception.getMessage().contains("'ruleProvider' cannot be null")); + } + + @Test + public void test_constructor_nullDefaultSegmentGranularityThrowsException() + { + final ReindexingRuleProvider mockProvider = EasyMock.createMock(ReindexingRuleProvider.class); + EasyMock.replay(mockProvider); + + DruidException exception = Assertions.assertThrows( + DruidException.class, + () -> new CascadingReindexingTemplate( + "testDataSource", + null, + null, + mockProvider, + null, + null, + null, + null, + null // null defaultSegmentGranularity + ) + ); + + Assertions.assertTrue(exception.getMessage().contains("'defaultSegmentGranularity' cannot be null")); + EasyMock.verify(mockProvider); + } + + @Test + public void test_createCompactionJobs_simple() + { + DateTime referenceTime = DateTimes.of("2024-01-15T00:00:00Z"); + SegmentTimeline timeline = createTestTimeline(referenceTime.minusDays(90), referenceTime.minusDays(10)); + ReindexingRuleProvider mockProvider = createMockProvider(List.of(Period.days(7), Period.days(30))); + CompactionJobParams mockParams = createMockParams(referenceTime, timeline); + DruidInputSource mockSource = createMockSource(); + + TestCascadingReindexingTemplate template = new TestCascadingReindexingTemplate( + "testDS", null, null, mockProvider, null, null, null, null + ); + + template.createCompactionJobs(mockSource, mockParams); + List processedIntervals = template.getProcessedIntervals(); + + Assertions.assertEquals(2, processedIntervals.size()); + // Intervals are now in chronological order (oldest first) + Assertions.assertEquals(DateTimes.MIN, processedIntervals.get(0).getStart()); + Assertions.assertEquals(referenceTime.minusDays(30), processedIntervals.get(0).getEnd()); + Assertions.assertEquals(referenceTime.minusDays(30), processedIntervals.get(1).getStart()); + Assertions.assertEquals(referenceTime.minusDays(7), processedIntervals.get(1).getEnd()); + + EasyMock.verify(mockProvider, mockParams, mockSource); + } + + @Test + public void test_createCompactionJobs_withSkipOffsetFromLatest_skipAllOfTime() + { + DateTime referenceTime = DateTimes.of("2024-01-15T00:00:00Z"); + SegmentTimeline timeline = createTestTimeline(referenceTime.minusDays(90), referenceTime.minusDays(10)); + ReindexingRuleProvider mockProvider = createMockProvider(List.of(Period.days(7), Period.days(30))); + CompactionJobParams mockParams = createMockParams(referenceTime, timeline); + DruidInputSource mockSource = createMockSource(); + + TestCascadingReindexingTemplate template = new TestCascadingReindexingTemplate( + "testDS", null, null, mockProvider, null, null, Period.days(100), null + ); + + List jobs = template.createCompactionJobs(mockSource, mockParams); + + Assertions.assertTrue(jobs.isEmpty()); + Assertions.assertTrue(template.getProcessedIntervals().isEmpty()); + + EasyMock.verify(mockProvider, mockParams, mockSource); + } + + @Test + public void test_createCompactionJobs_withSkipOffsetFromLatest_skipsIntervalsExtendingPastOffset() + { + DateTime referenceTime = DateTimes.of("2024-01-15T00:00:00Z"); + SegmentTimeline timeline = createTestTimeline(referenceTime.minusDays(90), referenceTime.minusDays(10)); + ReindexingRuleProvider mockProvider = createMockProvider(List.of(Period.days(7), Period.days(30))); + CompactionJobParams mockParams = createMockParams(referenceTime, timeline); + DruidInputSource mockSource = createMockSource(); + + TestCascadingReindexingTemplate template = new TestCascadingReindexingTemplate( + "testDS", null, null, mockProvider, null, null, Period.days(5), null + ); + + template.createCompactionJobs(mockSource, mockParams); + List processedIntervals = template.getProcessedIntervals(); + + Assertions.assertEquals(1, processedIntervals.size()); + Assertions.assertEquals(DateTimes.MIN, processedIntervals.get(0).getStart()); + Assertions.assertEquals(referenceTime.minusDays(30), processedIntervals.get(0).getEnd()); + + EasyMock.verify(mockProvider, mockParams, mockSource); + } + + @Test + public void test_createCompactionJobs_withSkipOffsetFromLatest_eliminatesInterval() + { + DateTime referenceTime = DateTimes.of("2024-01-15T00:00:00Z"); + SegmentTimeline timeline = createTestTimeline(referenceTime.minusDays(90), referenceTime.minusDays(10)); + ReindexingRuleProvider mockProvider = createMockProvider(List.of(Period.days(7), Period.days(30))); + CompactionJobParams mockParams = createMockParams(referenceTime, timeline); + DruidInputSource mockSource = createMockSource(); + + TestCascadingReindexingTemplate template = new TestCascadingReindexingTemplate( + "testDS", null, null, mockProvider, null, null, Period.days(15), null + ); + + template.createCompactionJobs(mockSource, mockParams); + List processedIntervals = template.getProcessedIntervals(); + + Assertions.assertEquals(1, processedIntervals.size()); + Assertions.assertEquals(DateTimes.MIN, processedIntervals.get(0).getStart()); + Assertions.assertEquals(referenceTime.minusDays(30), processedIntervals.get(0).getEnd()); + + EasyMock.verify(mockProvider, mockParams, mockSource); + } + + @Test + public void test_createCompactionJobs_withSkipOffsetFromNow_skipAllOfTime() + { + DateTime referenceTime = DateTimes.of("2024-01-15T00:00:00Z"); + SegmentTimeline timeline = createTestTimeline(referenceTime.minusDays(90), referenceTime.minusDays(10)); + ReindexingRuleProvider mockProvider = createMockProvider(List.of(Period.days(7), Period.days(30))); + CompactionJobParams mockParams = createMockParams(referenceTime, timeline); + DruidInputSource mockSource = createMockSource(); + + TestCascadingReindexingTemplate template = new TestCascadingReindexingTemplate( + "testDS", null, null, mockProvider, null, null, null, Period.days(100) + ); + + List jobs = template.createCompactionJobs(mockSource, mockParams); + + Assertions.assertTrue(jobs.isEmpty()); + Assertions.assertTrue(template.getProcessedIntervals().isEmpty()); + + EasyMock.verify(mockProvider, mockParams, mockSource); + } + + @Test + public void test_createCompactionJobs_withSkipOffsetFromNow_skipsIntervalsExtendingPastOffset() + { + DateTime referenceTime = DateTimes.of("2024-01-15T00:00:00Z"); + SegmentTimeline timeline = createTestTimeline(referenceTime.minusDays(90), referenceTime.minusDays(10)); + ReindexingRuleProvider mockProvider = createMockProvider(List.of(Period.days(7), Period.days(30))); + CompactionJobParams mockParams = createMockParams(referenceTime, timeline); + DruidInputSource mockSource = createMockSource(); + + TestCascadingReindexingTemplate template = new TestCascadingReindexingTemplate( + "testDS", null, null, mockProvider, null, null, null, Period.days(20) + ); + + template.createCompactionJobs(mockSource, mockParams); + List processedIntervals = template.getProcessedIntervals(); + + Assertions.assertEquals(1, processedIntervals.size()); + Assertions.assertEquals(DateTimes.MIN, processedIntervals.get(0).getStart()); + Assertions.assertEquals(referenceTime.minusDays(30), processedIntervals.get(0).getEnd()); + + EasyMock.verify(mockProvider, mockParams, mockSource); + } + + @Test + public void test_createCompactionJobs_withSkipOffsetFromNow_eliminatesInterval() + { + DateTime referenceTime = DateTimes.of("2024-01-15T00:00:00Z"); + SegmentTimeline timeline = createTestTimeline(referenceTime.minusDays(90), referenceTime.minusDays(10)); + ReindexingRuleProvider mockProvider = createMockProvider(List.of(Period.days(7), Period.days(30))); + CompactionJobParams mockParams = createMockParams(referenceTime, timeline); + DruidInputSource mockSource = createMockSource(); + + TestCascadingReindexingTemplate template = new TestCascadingReindexingTemplate( + "testDS", null, null, mockProvider, null, null, null, Period.days(20) + ); + + template.createCompactionJobs(mockSource, mockParams); + List processedIntervals = template.getProcessedIntervals(); + + Assertions.assertEquals(1, processedIntervals.size()); + Assertions.assertEquals(DateTimes.MIN, processedIntervals.get(0).getStart()); + Assertions.assertEquals(referenceTime.minusDays(30), processedIntervals.get(0).getEnd()); + + EasyMock.verify(mockProvider, mockParams, mockSource); + } + + /** + * TEST: Basic timeline construction with multiple segment granularity rules + *

+ * REFERENCE TIME: 2025-01-29T16:15:00Z + *

+ * INPUT RULES: + *

    + *
  • Segment Granularity Rules: P7D→HOUR, P1M→DAY, P3M→MONTH
  • + *
  • Other Rules: None
  • + *
  • Default Segment Granularity: DAY
  • + *
+ *

+ * PROCESSING: + *

    + *
  1. Synthetic Rules: None created
  2. + *
  3. Initial Timeline: + *
      + *
    • P3M → MONTH: Raw 2024-10-29T16:15 → Aligned 2024-10-01T00:00
    • + *
    • P1M → DAY: Raw 2024-12-29T16:15 → Aligned 2024-12-29T00:00
    • + *
    • P7D → HOUR: Raw 2025-01-22T16:15 → Aligned 2025-01-22T16:00
    • + *
    + *
  4. + *
  5. Timeline Splits: None (no non-segment-gran rules)
  6. + *
+ *

+ * EXPECTED OUTPUT: 3 intervals + *

    + *
  1. [-∞, 2024-10-01T00:00:00) - MONTH
  2. + *
  3. [2024-10-01T00:00:00, 2024-12-29T00:00:00) - DAY
  4. + *
  5. [2024-12-29T00:00:00, 2025-01-22T16:00:00) - HOUR
  6. + *
+ */ + @Test + public void test_generateAlignedSearchIntervals_withGranularityAlignment() + { + DateTime referenceTime = DateTimes.of("2025-01-29T16:15:00Z"); + + ReindexingSegmentGranularityRule hourRule = new ReindexingSegmentGranularityRule("hour-rule", null, Period.days(7), Granularities.HOUR); + ReindexingSegmentGranularityRule dayRule = new ReindexingSegmentGranularityRule("day-rule", null, Period.months(1), Granularities.DAY); + ReindexingSegmentGranularityRule monthRule = new ReindexingSegmentGranularityRule("month-rule", null, Period.months(3), Granularities.MONTH); + + ReindexingRuleProvider provider = InlineReindexingRuleProvider.builder() + .segmentGranularityRules(List.of(hourRule, dayRule, monthRule)) + .build(); + + CascadingReindexingTemplate template = new CascadingReindexingTemplate( + "testDS", + null, + null, + provider, + null, + null, + null, + null, + Granularities.DAY + ); + + List expected = List.of( + new IntervalGranularityInfo( + new Interval(DateTimes.MIN, DateTimes.of("2024-10-01T00:00:00Z")), + Granularities.MONTH, + monthRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2024-10-01T00:00:00Z"), DateTimes.of("2024-12-29T00:00:00Z")), + Granularities.DAY, + dayRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2024-12-29T00:00:00Z"), DateTimes.of("2025-01-22T16:00:00Z")), + Granularities.HOUR, + hourRule + ) + ); + + List actual = template.generateAlignedSearchIntervals(referenceTime); + Assertions.assertEquals(expected, actual); + } + + /** + * TEST: Timeline splitting by non-segment-granularity rules (metrics rules) + *

+ * REFERENCE TIME: 2025-01-29T16:15:00Z + *

+ * INPUT RULES: + *

    + *
  • Segment Granularity Rules: P7D→HOUR, P1M→DAY, P3M→MONTH
  • + *
  • Other Rules: P8D-metrics, P14D-metrics, P45D-metrics, P100D-metrics
  • + *
  • Default Segment Granularity: DAY
  • + *
+ *

+ * PROCESSING: + *

    + *
  1. Synthetic Rules: None (smallest segment gran rule P7D is finer than all metrics rules)
  2. + *
  3. Initial Timeline: [-∞, 2024-10-01) MONTH, [2024-10-01, 2024-12-29) DAY, [2024-12-29, 2025-01-22T16:00) HOUR
  4. + *
  5. Timeline Splits: + *
      + *
    • P100D → Raw 2024-10-21T16:15 → Falls in DAY interval → Aligned 2024-10-21T00:00 → CREATES SPLIT
    • + *
    • P45D → Raw 2024-12-15T16:15 → Falls in DAY interval → Aligned 2024-12-15T00:00 → CREATES SPLIT
    • + *
    • P14D → Raw 2025-01-15T16:15 → Falls in HOUR interval → Aligned 2025-01-15T16:00 → CREATES SPLIT
    • + *
    • P8D → Raw 2025-01-21T16:15 → Falls in HOUR interval → Aligned 2025-01-21T16:00 → CREATES SPLIT
    • + *
    + *
  6. + *
+ *

+ * EXPECTED OUTPUT: 7 intervals + *

    + *
  1. [-∞, 2024-10-01T00:00:00) - MONTH
  2. + *
  3. [2024-10-01T00:00:00, 2024-10-21T00:00:00) - DAY
  4. + *
  5. [2024-10-21T00:00:00, 2024-12-15T00:00:00) - DAY
  6. + *
  7. [2024-12-15T00:00:00, 2024-12-29T00:00:00) - DAY
  8. + *
  9. [2024-12-29T00:00:00, 2025-01-15T16:00:00) - HOUR
  10. + *
  11. [2025-01-15T16:00:00, 2025-01-21T16:00:00) - HOUR
  12. + *
  13. [2025-01-21T16:00:00, 2025-01-22T16:00:00) - HOUR
  14. + *
+ */ + @Test + public void test_generateAlignedSearchIntervals_withNonSegmentGranularityRuleSplits() + { + DateTime referenceTime = DateTimes.of("2025-01-29T16:15:00Z"); + + ReindexingSegmentGranularityRule hourRule = new ReindexingSegmentGranularityRule("hour-rule", null, Period.days(7), Granularities.HOUR); + ReindexingSegmentGranularityRule dayRule = new ReindexingSegmentGranularityRule("day-rule", null, Period.months(1), Granularities.DAY); + ReindexingSegmentGranularityRule monthRule = new ReindexingSegmentGranularityRule("month-rule", null, Period.months(3), Granularities.MONTH); + + // The data schema rules are here to trigger splits in the base timeline for granularity rules. + ReindexingRuleProvider provider = InlineReindexingRuleProvider.builder() + .segmentGranularityRules(List.of(hourRule, dayRule, monthRule)) + .dataSchemaRules(List.of( + createReindexingDataSchemaRule("metrics-8d", Period.days(8)), + createReindexingDataSchemaRule("metrics-14d", Period.days(14)), + createReindexingDataSchemaRule("metrics-45d", Period.days(45)), + createReindexingDataSchemaRule("metrics-100d", Period.days(100)) + )) + .build(); + + CascadingReindexingTemplate template = new CascadingReindexingTemplate( + "testDS", + null, + null, + provider, + null, + null, + null, + null, + Granularities.DAY + ); + + List expected = List.of( + new IntervalGranularityInfo( + new Interval(DateTimes.MIN, DateTimes.of("2024-10-01T00:00:00Z")), + Granularities.MONTH, + monthRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2024-10-01T00:00:00Z"), DateTimes.of("2024-10-21T00:00:00Z")), + Granularities.DAY, + dayRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2024-10-21T00:00:00Z"), DateTimes.of("2024-12-15T00:00:00Z")), + Granularities.DAY, + dayRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2024-12-15T00:00:00Z"), DateTimes.of("2024-12-29T00:00:00Z")), + Granularities.DAY, + dayRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2024-12-29T00:00:00Z"), DateTimes.of("2025-01-15T16:00:00Z")), + Granularities.HOUR, + hourRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2025-01-15T16:00:00Z"), DateTimes.of("2025-01-21T16:00:00Z")), + Granularities.HOUR, + hourRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2025-01-21T16:00:00Z"), DateTimes.of("2025-01-22T16:00:00Z")), + Granularities.HOUR, + hourRule + ) + ); + + List actual = template.generateAlignedSearchIntervals(referenceTime); + Assertions.assertEquals(expected, actual); + } + + /** + * TEST: Timeline construction when NO segment granularity rules exist (Case A: default usage) + *

+ * REFERENCE TIME: 2025-01-29T16:15:00Z + *

+ * INPUT RULES: + *

    + *
  • Segment Granularity Rules: None
  • + *
  • Other Rules: P8D-metrics, P14D-metrics, P45D-metrics
  • + *
  • Default Segment Granularity: DAY
  • + *
+ *

+ * PROCESSING: + *

    + *
  1. Synthetic Rules: Created P8D→DAY (Case A: no segment gran rules exist, use smallest rule period with default gran)
  2. + *
  3. Initial Timeline: [-∞, 2025-01-21T00:00) - DAY (from synthetic P8D rule)
  4. + *
  5. Timeline Splits: + *
      + *
    • P45D → Raw 2024-12-15T16:15 → Falls in DAY interval → Aligned 2024-12-15T00:00 → CREATES SPLIT
    • + *
    • P14D → Raw 2025-01-15T16:15 → Falls in DAY interval → Aligned 2025-01-15T00:00 → CREATES SPLIT
    • + *
    • P8D is now a segment gran rule (not processed as split)
    • + *
    + *
  6. + *
+ *

+ * EXPECTED OUTPUT: 3 intervals + *

    + *
  1. [-∞, 2024-12-15T00:00:00) - DAY
  2. + *
  3. [2024-12-15T00:00:00, 2025-01-15T00:00:00) - DAY
  4. + *
  5. [2025-01-15T00:00:00, 2025-01-21T00:00:00) - DAY
  6. + *
+ */ + @Test + public void test_generateAlignedSearchIntervals_withNoSegmentGranularityRules() + { + DateTime referenceTime = DateTimes.of("2025-01-29T16:15:00Z"); + + ReindexingRuleProvider provider = InlineReindexingRuleProvider.builder() + .dataSchemaRules(List.of( + new ReindexingDataSchemaRule("metrics-8d", null, Period.days(8), null, new AggregatorFactory[0], null, null, null), + createReindexingDataSchemaRule("metrics-8d", Period.days(8)), + createReindexingDataSchemaRule("metrics-14d", Period.days(14)), + createReindexingDataSchemaRule("metrics-45d", Period.days(45)) + )) + .build(); + + CascadingReindexingTemplate template = new CascadingReindexingTemplate( + "testDS", + null, + null, + provider, + null, + null, + null, + null, + Granularities.DAY + ); + + // When no segment granularity rules exist, a synthetic rule is created with the smallest period + List expected = List.of( + new IntervalGranularityInfo( + new Interval(DateTimes.MIN, DateTimes.of("2024-12-15T00:00:00Z")), + Granularities.DAY, + null // Synthetic rule has no source + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2024-12-15T00:00:00Z"), DateTimes.of("2025-01-15T00:00:00Z")), + Granularities.DAY, + null // Synthetic rule has no source + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2025-01-15T00:00:00Z"), DateTimes.of("2025-01-21T00:00:00Z")), + Granularities.DAY, + null // Synthetic rule has no source + ) + ); + + List actual = template.generateAlignedSearchIntervals(referenceTime); + Assertions.assertEquals(expected, actual); + } + + /** + * TEST: Synthetic segment gran rule creation when rules are finer than smallest segment gran rule (Case B) + *

+ * REFERENCE TIME: 2025-01-29T16:15:00Z + *

+ * INPUT RULES: + *

    + *
  • Segment Granularity Rules: P1M→DAY, P3M→MONTH
  • + *
  • Other Rules: P7D-metrics, P14D-metrics, P21D-metrics (all finer than P1M!)
  • + *
  • Default Segment Granularity: HOUR
  • + *
+ *

+ * PROCESSING: + *

    + *
  1. Synthetic Rules: Created P7D→HOUR (Case B: P7D/P14D/P21D are finer than smallest segment gran rule P1M, use finest with default gran)
  2. + *
  3. Initial Timeline: + *
      + *
    • P3M → MONTH: Raw 2024-10-29T16:15 → Aligned 2024-10-01T00:00
    • + *
    • P1M → DAY: Raw 2024-12-29T16:15 → Aligned 2024-12-29T00:00
    • + *
    • P7D → HOUR (synthetic): Raw 2025-01-22T16:15 → Aligned 2025-01-22T16:00 (PREPENDED interval!)
    • + *
    + *
  4. + *
  5. Timeline Splits: + *
      + *
    • P21D → Raw 2025-01-08T16:15 → Falls in HOUR interval → Aligned 2025-01-08T16:00 → CREATES SPLIT
    • + *
    • P14D → Raw 2025-01-15T16:15 → Falls in HOUR interval → Aligned 2025-01-15T16:00 → CREATES SPLIT
    • + *
    • P7D is now a segment gran rule (not processed as split)
    • + *
    + *
  6. + *
+ *

+ * EXPECTED OUTPUT: 5 intervals + *

    + *
  1. [-∞, 2024-10-01T00:00:00) - MONTH
  2. + *
  3. [2024-10-01T00:00:00, 2024-12-29T00:00:00) - DAY
  4. + *
  5. [2024-12-29T00:00:00, 2025-01-08T16:00:00) - HOUR (prepended)
  6. + *
  7. [2025-01-08T16:00:00, 2025-01-15T16:00:00) - HOUR (prepended)
  8. + *
  9. [2025-01-15T16:00:00, 2025-01-22T16:00:00) - HOUR (prepended)
  10. + *
+ */ + @Test + public void test_generateAlignedSearchIntervals_prependIntervalForShortNonSegmentGranRules() + { + DateTime referenceTime = DateTimes.of("2025-01-29T16:15:00Z"); + + ReindexingSegmentGranularityRule monthRule = new ReindexingSegmentGranularityRule("month-rule", null, Period.months(3), Granularities.MONTH); + ReindexingSegmentGranularityRule dayRule = new ReindexingSegmentGranularityRule("day-rule", null, Period.months(1), Granularities.DAY); + + ReindexingRuleProvider provider = InlineReindexingRuleProvider.builder() + .segmentGranularityRules(List.of(monthRule, dayRule)) + .dataSchemaRules(List.of( + new ReindexingDataSchemaRule("metrics-7d", null, Period.days(7), null, new AggregatorFactory[0], null, null, null), + new ReindexingDataSchemaRule("metrics-14d", null, Period.days(14), null, new AggregatorFactory[0], null, null, null), + new ReindexingDataSchemaRule("metrics-21d", null, Period.days(21), null, new AggregatorFactory[0], null, null, null) + )) + .build(); + + CascadingReindexingTemplate template = new CascadingReindexingTemplate( + "testDS", + null, + null, + provider, + null, + null, + null, + null, + Granularities.HOUR + ); + + List expected = List.of( + new IntervalGranularityInfo( + new Interval(DateTimes.MIN, DateTimes.of("2024-10-01T00:00:00Z")), + Granularities.MONTH, + monthRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2024-10-01T00:00:00Z"), DateTimes.of("2024-12-29T00:00:00Z")), + Granularities.DAY, + dayRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2024-12-29T00:00:00Z"), DateTimes.of("2025-01-08T16:00:00Z")), + Granularities.HOUR, + null // Synthetic prepended rule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2025-01-08T16:00:00Z"), DateTimes.of("2025-01-15T16:00:00Z")), + Granularities.HOUR, + null // Synthetic prepended rule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2025-01-15T16:00:00Z"), DateTimes.of("2025-01-22T16:00:00Z")), + Granularities.HOUR, + null // Synthetic prepended rule + ) + ); + + List actual = template.generateAlignedSearchIntervals(referenceTime); + Assertions.assertEquals(expected, actual); + } + + /** + * TEST: Comprehensive example demonstrating Case B, multiple segment gran rules, and timeline splits + *

+ * REFERENCE TIME: 2024-02-04T22:12:04.873Z (realistic messy timestamp) + *

+ * INPUT RULES: + *

    + *
  • Segment Granularity Rules: P1Y→YEAR, P1M→MONTH, P7D→DAY
  • + *
  • Other Rules: P1D-metrics, P14D-metrics, P45D-metrics (P1D is finer than P7D!)
  • + *
  • Default Segment Granularity: HOUR
  • + *
+ *

+ * PROCESSING: + *

    + *
  1. Synthetic Rules: Created P1D→HOUR (Case B: P1D is finer than smallest segment gran rule P7D)
  2. + *
  3. Initial Timeline: + *
      + *
    • P1Y → YEAR: Raw 2023-02-04T22:12:04.873 → Aligned 2023-01-01T00:00:00
    • + *
    • P1M → MONTH: Raw 2024-01-04T22:12:04.873 → Aligned 2024-01-01T00:00:00
    • + *
    • P7D → DAY: Raw 2024-01-28T22:12:04.873 → Aligned 2024-01-28T00:00:00
    • + *
    • P1D → HOUR (synthetic): Raw 2024-02-03T22:12:04.873 → Aligned 2024-02-03T22:00:00 (PREPENDED!)
    • + *
    + *
  4. + *
  5. Timeline Splits: + *
      + *
    • P45D → Raw 2023-12-21T22:12:04.873 → Falls in MONTH interval → Aligned 2023-12-01T00:00:00 → CREATES SPLIT
    • + *
    • P14D → Raw 2024-01-21T22:12:04.873 → Falls in DAY interval → Aligned 2024-01-21T00:00:00 → CREATES SPLIT
    • + *
    • P1D is now a segment gran rule (not processed as split)
    • + *
    + *
  6. + *
+ *

+ * EXPECTED OUTPUT: 6 intervals + *

    + *
  1. [-∞, 2023-01-01T00:00:00) - YEAR
  2. + *
  3. [2023-01-01T00:00:00, 2023-12-01T00:00:00) - MONTH
  4. + *
  5. [2023-12-01T00:00:00, 2024-01-01T00:00:00) - MONTH
  6. + *
  7. [2024-01-01T00:00:00, 2024-01-21T00:00:00) - DAY
  8. + *
  9. [2024-01-21T00:00:00, 2024-01-28T00:00:00) - DAY
  10. + *
  11. [2024-01-28T00:00:00, 2024-02-03T22:00:00) - HOUR (prepended, note non-midnight end)
  12. + *
+ */ + @Test + public void test_generateAlignedSearchIntervals() + { + DateTime referenceTime = DateTimes.of("2024-02-04T22:12:04.873Z"); + + ReindexingSegmentGranularityRule yearRule = new ReindexingSegmentGranularityRule("year-rule", null, Period.years(1), Granularities.YEAR); + ReindexingSegmentGranularityRule monthRule = new ReindexingSegmentGranularityRule("month-rule", null, Period.months(1), Granularities.MONTH); + ReindexingSegmentGranularityRule dayRule = new ReindexingSegmentGranularityRule("day-rule", null, Period.days(7), Granularities.DAY); + + ReindexingRuleProvider provider = + InlineReindexingRuleProvider + .builder() + .segmentGranularityRules(List.of(yearRule, monthRule, dayRule)) + .dataSchemaRules(List.of( + new ReindexingDataSchemaRule("metrics-1d", null, Period.days(1), null, new AggregatorFactory[0], null, null, null), + new ReindexingDataSchemaRule("metrics-14d", null, Period.days(14), null, new AggregatorFactory[0], null, null, null), + new ReindexingDataSchemaRule("metrics-45d", null, Period.days(45), null, new AggregatorFactory[0], null, null, null) + )) + .build(); + + CascadingReindexingTemplate template = new CascadingReindexingTemplate( + "testDS", + null, + null, + provider, + null, + null, + null, + null, + Granularities.HOUR + ); + + List expected = List.of( + new IntervalGranularityInfo( + new Interval(DateTimes.MIN, DateTimes.of("2023-01-01T00:00:00Z")), + Granularities.YEAR, + yearRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2023-01-01T00:00:00Z"), DateTimes.of("2023-12-01T00:00:00Z")), + Granularities.MONTH, + monthRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2023-12-01T00:00:00Z"), DateTimes.of("2024-01-01T00:00:00Z")), + Granularities.MONTH, + monthRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2024-01-01T00:00:00Z"), DateTimes.of("2024-01-21T00:00:00Z")), + Granularities.DAY, + dayRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2024-01-21T00:00:00Z"), DateTimes.of("2024-01-28T00:00:00Z")), + Granularities.DAY, + dayRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2024-01-28T00:00:00"), DateTimes.of("2024-02-03T22:00:00")), + Granularities.HOUR, + null // Synthetic prepended rule + ) + ); + + List actual = template.generateAlignedSearchIntervals(referenceTime); + Assertions.assertEquals(expected, actual); + } + + /** + * TEST: No rules at all - should throw IAE + *

+ * REFERENCE TIME: 2025-01-29T16:15:00Z + *

+ * INPUT RULES: + *

    + *
  • Segment Granularity Rules: None
  • + *
  • Other Rules: None
  • + *
  • Default Segment Granularity: DAY
  • + *
+ *

+ * EXPECTED: IllegalArgumentException with message "requires at least one reindexing rule" + */ + @Test + public void test_generateAlignedSearchIntervals_noRulesThrowsException() + { + DateTime referenceTime = DateTimes.of("2025-01-29T16:15:00Z"); + + ReindexingRuleProvider provider = InlineReindexingRuleProvider.builder().build(); + + CascadingReindexingTemplate template = new CascadingReindexingTemplate( + "testDS", + null, + null, + provider, + null, + null, + null, + null, + Granularities.DAY + ); + + DruidException exception = Assertions.assertThrows( + DruidException.class, + () -> template.generateAlignedSearchIntervals(referenceTime) + ); + + Assertions.assertTrue( + exception.getMessage().contains("requires at least one reindexing rule") + ); + } + + /** + * TEST: Split point aligns exactly to existing boundary (boundary snapping, no split created) + *

+ * REFERENCE TIME: 2025-02-01T00:00:00Z (carefully chosen for alignment) + *

+ * INPUT RULES: + *

    + *
  • Segment Granularity Rules: P1M→MONTH
  • + *
  • Other Rules: P1M-metrics (same period as segment gran rule!)
  • + *
  • Default Segment Granularity: DAY
  • + *
+ *

+ * PROCESSING: + *

    + *
  1. Synthetic Rules: None
  2. + *
  3. Initial Timeline: [-∞, 2025-01-01T00:00:00) - MONTH
  4. + *
  5. Timeline Splits: + *
      + *
    • P1M metrics → Raw 2025-01-01T00:00:00 → Aligned to MONTH: 2025-01-01T00:00:00
    • + *
    • This aligns EXACTLY to the existing boundary → no split created
    • + *
    + *
  6. + *
+ *

+ * EXPECTED OUTPUT: 1 interval (no split despite having a non-segment-gran rule) + *

    + *
  1. [-∞, 2025-01-01T00:00:00) - MONTH
  2. + *
+ */ + @Test + public void test_generateAlignedSearchIntervals_splitPointSnapsToExistingBoundary() + { + DateTime referenceTime = DateTimes.of("2025-02-01T00:00:00Z"); + + ReindexingSegmentGranularityRule monthRule = new ReindexingSegmentGranularityRule("month-rule", null, Period.months(1), Granularities.MONTH); + + ReindexingRuleProvider provider = InlineReindexingRuleProvider.builder() + .segmentGranularityRules(List.of(monthRule)) + .dataSchemaRules(List.of( + new ReindexingDataSchemaRule("metrics-1m", null, Period.months(1), null, new AggregatorFactory[0], null, null, null) + )) + .build(); + + CascadingReindexingTemplate template = new CascadingReindexingTemplate( + "testDS", + null, + null, + provider, + null, + null, + null, + null, + Granularities.DAY + ); + + List expected = List.of( + new IntervalGranularityInfo( + new Interval(DateTimes.MIN, DateTimes.of("2025-01-01T00:00:00Z")), + Granularities.MONTH, + monthRule + ) + ); + + List actual = template.generateAlignedSearchIntervals(referenceTime); + Assertions.assertEquals(expected, actual); + } + + /** + * TEST: Prepending that aligns back to last segment gran rule interval end (no prepend actually created) + *

+ * REFERENCE TIME: 2025-01-01T01:00:00Z + *

+ * INPUT RULES: + *

    + *
  • Segment Granularity Rules: P1D→DAY
  • + *
  • Other Rules: PT12H-metrics (finer than P1D, but aligns back to same interval end as the P1D rule so no prepend is done)
  • + *
  • Default Segment Granularity: DAY
  • + *
+ *

+ * PROCESSING: + *

    + *
  1. Initial Timeline: [-∞, 2024-12-31T00:00:00) - DAY
  2. + *
  3. Check for prepending: + *
      + *
    • PT12H threshold: 2024-12-31T13:00:00
    • + *
    • Align to DAY (default gran): 2024-12-31T00:00:00
    • + *
    • This EQUALS the most recent segment gran rule end (2024-12-31T00:00:00) → NO PREPEND
    • + *
    + *
  4. + *
+ *

+ * EXPECTED OUTPUT: 1 interval (no split prepend despite having a finer non-segment-gran rule) + *

    + *
  1. [-∞, 2024-12-31T00:00:00) - DAY
  2. + *
+ */ + @Test + public void test_generateAlignedSearchIntervals_prependAlignmentDoesNotExtendTimeline() + { + DateTime referenceTime = DateTimes.of("2025-01-01T01:00:00Z"); + + ReindexingSegmentGranularityRule dayRule = new ReindexingSegmentGranularityRule("day-rule", null, Period.days(1), Granularities.DAY); + + ReindexingRuleProvider provider = InlineReindexingRuleProvider.builder() + .segmentGranularityRules(List.of(dayRule)) + .dataSchemaRules(List.of( + new ReindexingDataSchemaRule("metrics-12h", null, Period.hours(12), null, new AggregatorFactory[0], null, null, null) + )) + .build(); + + CascadingReindexingTemplate template = new CascadingReindexingTemplate( + "testDS", + null, + null, + provider, + null, + null, + null, + null, + Granularities.DAY + ); + + List expected = List.of( + new IntervalGranularityInfo( + new Interval(DateTimes.MIN, DateTimes.of("2024-12-31T00:00:00Z")), + Granularities.DAY, + dayRule + ) + ); + + List actual = template.generateAlignedSearchIntervals(referenceTime); + Assertions.assertEquals(expected, actual); + } + + /** + * TEST: Multiple split points align to same timestamp (distinct() filtering removes duplicates) + *

+ * REFERENCE TIME: 2025-01-15T00:00:00Z + *

+ * INPUT RULES: + *

    + *
  • Segment Granularity Rules: P1M→DAY
  • + *
  • Other Rules: P23D+6h-metrics, P23D+18h-metrics (both align to same DAY boundary in DAY interval)
  • + *
  • Default Segment Granularity: DAY
  • + *
+ *

+ * PROCESSING: + *

    + *
  1. Synthetic Rules: None
  2. + *
  3. Initial Timeline: [-∞, 2024-12-15T00:00:00) - DAY
  4. + *
  5. Timeline Splits: + *
      + *
    • P33D+6h → Raw: 2024-12-12T18:00:00 → Falls in DAY interval → Align to DAY: 2024-12-12T00:00:00
    • + *
    • P33D+18h → Raw: 2024-12-12T06:00:00 → Falls in DAY interval → Align to DAY: 2024-12-12T00:00:00
    • + *
    • Both create split point 2024-12-12T00:00:00 → distinct() removes duplicate!
    • + *
    • Only ONE split created at 2024-12-12T00:00:00
    • + *
    + *
  6. + *
+ *

+ * EXPECTED OUTPUT: 2 intervals (not 3, because duplicate split point was filtered) + *

    + *
  1. [-∞, 2024-12-12T00:00:00) - DAY
  2. + *
  3. [2024-12-12T00:00:00, 2024-12-15T00:00:00) - DAY
  4. + *
+ */ + @Test + public void test_generateAlignedSearchIntervals_duplicateSplitPointsFiltered() + { + DateTime referenceTime = DateTimes.of("2025-01-15T00:00:00Z"); + + ReindexingSegmentGranularityRule dayRule = new ReindexingSegmentGranularityRule("day-rule", null, Period.months(1), Granularities.DAY); + + ReindexingRuleProvider provider = InlineReindexingRuleProvider.builder() + .segmentGranularityRules(List.of(dayRule)) + .dataSchemaRules(List.of( + new ReindexingDataSchemaRule("metrics-33d-6h", null, Period.hours(33 * 24 + 6), null, new AggregatorFactory[0], null, null, null), + new ReindexingDataSchemaRule("metrics-33d-18h", null, Period.hours(33 * 24 + 18), null, new AggregatorFactory[0], null, null, null) + )) + .build(); + + CascadingReindexingTemplate template = new CascadingReindexingTemplate( + "testDS", + null, + null, + provider, + null, + null, + null, + null, + Granularities.DAY + ); + + List expected = List.of( + new IntervalGranularityInfo( + new Interval(DateTimes.MIN, DateTimes.of("2024-12-12T00:00:00Z")), + Granularities.DAY, + dayRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2024-12-12T00:00:00Z"), DateTimes.of("2024-12-15T00:00:00Z")), + Granularities.DAY, + dayRule + ) + ); + + List actual = template.generateAlignedSearchIntervals(referenceTime); + Assertions.assertEquals(expected, actual); + } + + /** + * TEST: Single rule only (minimal valid case) + *

+ * REFERENCE TIME: 2025-01-29T16:15:00Z + *

+ * INPUT RULES: + *

    + *
  • Segment Granularity Rules: P1M→MONTH
  • + *
  • Other Rules: None
  • + *
  • Default Segment Granularity: DAY
  • + *
+ *

+ * PROCESSING: + *

    + *
  1. Synthetic Rules: None
  2. + *
  3. Initial Timeline: [-∞, 2024-12-01T00:00:00) - MONTH
  4. + *
  5. Timeline Splits: None (no non-segment-gran rules)
  6. + *
+ *

+ * EXPECTED OUTPUT: 1 interval + *

    + *
  1. [-∞, 2024-12-01T00:00:00) - MONTH
  2. + *
+ */ + @Test + public void test_generateAlignedSearchIntervals_singleRuleOnly() + { + DateTime referenceTime = DateTimes.of("2025-01-29T16:15:00Z"); + + ReindexingSegmentGranularityRule monthRule = new ReindexingSegmentGranularityRule("month-rule", null, Period.months(1), Granularities.MONTH); + + ReindexingRuleProvider provider = InlineReindexingRuleProvider.builder() + .segmentGranularityRules(List.of(monthRule)) + .build(); + + CascadingReindexingTemplate template = new CascadingReindexingTemplate( + "testDS", + null, + null, + provider, + null, + null, + null, + null, + Granularities.DAY + ); + + List expected = List.of( + new IntervalGranularityInfo( + new Interval(DateTimes.MIN, DateTimes.of("2024-12-01T00:00:00Z")), + Granularities.MONTH, + monthRule + ) + ); + + List actual = template.generateAlignedSearchIntervals(referenceTime); + Assertions.assertEquals(expected, actual); + } + + /** + * TEST: Zero period rule (P0D) applies immediately to all data + *

+ * REFERENCE TIME: 2025-01-29T16:15:00Z + *

+ * INPUT RULES: + *

    + *
  • Segment Granularity Rules: P0D→HOUR (applies to all data immediately)
  • + *
  • Other Rules: None
  • + *
  • Default Segment Granularity: DAY
  • + *
+ *

+ * PROCESSING: + *

    + *
  1. Synthetic Rules: None
  2. + *
  3. Initial Timeline: [-∞, 2025-01-29T16:00:00) - HOUR (P0D means threshold equals reference time)
  4. + *
  5. Timeline Splits: None (no non-segment-gran rules)
  6. + *
+ *

+ * EXPECTED OUTPUT: 1 interval + *

    + *
  1. [-∞, 2025-01-29T16:00:00) - HOUR (aligned to hour boundary at reference time)
  2. + *
+ */ + @Test + public void test_generateAlignedSearchIntervals_zeroPeriodRuleAppliesImmediately() + { + DateTime referenceTime = DateTimes.of("2025-01-29T16:15:00Z"); + + ReindexingSegmentGranularityRule hourRule = new ReindexingSegmentGranularityRule( + "immediate-hour-rule", + "Apply HOUR granularity to all data immediately", + Period.days(0), + Granularities.HOUR + ); + + ReindexingRuleProvider provider = InlineReindexingRuleProvider.builder() + .segmentGranularityRules(List.of(hourRule)) + .build(); + + CascadingReindexingTemplate template = new CascadingReindexingTemplate( + "testDS", + null, + null, + provider, + null, + null, + null, + null, + Granularities.DAY + ); + + List expected = List.of( + new IntervalGranularityInfo( + new Interval(DateTimes.MIN, DateTimes.of("2025-01-29T16:00:00Z")), + Granularities.HOUR, + hourRule + ) + ); + + List actual = template.generateAlignedSearchIntervals(referenceTime); + Assertions.assertEquals(expected, actual); + } + + /** + * TEST: Zero period rule (P0D) with other rules creates proper cascading timeline + *

+ * REFERENCE TIME: 2025-01-29T16:15:00Z + *

+ * INPUT RULES: + *

    + *
  • Segment Granularity Rules: P0D→HOUR, P30D→DAY, P90D→MONTH
  • + *
  • Other Rules: None
  • + *
  • Default Segment Granularity: DAY
  • + *
+ *

+ * PROCESSING: + *

    + *
  1. Synthetic Rules: None
  2. + *
  3. Sort rules by period: P90D (oldest), P30D (middle), P0D (newest/applies immediately)
  4. + *
  5. Initial Timeline: + *
      + *
    • P90D → MONTH: Raw 2024-10-31T16:15 → Aligned 2024-10-01T00:00
    • + *
    • P30D → DAY: Raw 2024-12-30T16:15 → Aligned 2024-12-30T00:00
    • + *
    • P0D → HOUR: Raw 2025-01-29T16:15 → Aligned 2025-01-29T16:00
    • + *
    + *
  6. + *
  7. Timeline Splits: None (no non-segment-gran rules)
  8. + *
+ *

+ * EXPECTED OUTPUT: 3 intervals + *

    + *
  1. [-∞, 2024-10-01T00:00:00) - MONTH
  2. + *
  3. [2024-10-01T00:00:00, 2024-12-30T00:00:00) - DAY
  4. + *
  5. [2024-12-30T00:00:00, 2025-01-29T16:00:00) - HOUR
  6. + *
+ */ + @Test + public void test_generateAlignedSearchIntervals_zeroPeriodRuleWithOtherRules() + { + DateTime referenceTime = DateTimes.of("2025-01-29T16:15:00Z"); + + ReindexingSegmentGranularityRule monthRule = new ReindexingSegmentGranularityRule( + "month-rule", + null, + Period.days(90), + Granularities.MONTH + ); + ReindexingSegmentGranularityRule dayRule = new ReindexingSegmentGranularityRule( + "day-rule", + null, + Period.days(30), + Granularities.DAY + ); + ReindexingSegmentGranularityRule hourRule = new ReindexingSegmentGranularityRule( + "immediate-hour-rule", + "Apply HOUR granularity immediately", + Period.days(0), + Granularities.HOUR + ); + + ReindexingRuleProvider provider = InlineReindexingRuleProvider.builder() + .segmentGranularityRules(List.of(hourRule, dayRule, monthRule)) + .build(); + + CascadingReindexingTemplate template = new CascadingReindexingTemplate( + "testDS", + null, + null, + provider, + null, + null, + null, + null, + Granularities.DAY + ); + + List expected = List.of( + new IntervalGranularityInfo( + new Interval(DateTimes.MIN, DateTimes.of("2024-10-01T00:00:00Z")), + Granularities.MONTH, + monthRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2024-10-01T00:00:00Z"), DateTimes.of("2024-12-30T00:00:00Z")), + Granularities.DAY, + dayRule + ), + new IntervalGranularityInfo( + new Interval(DateTimes.of("2024-12-30T00:00:00Z"), DateTimes.of("2025-01-29T16:00:00Z")), + Granularities.HOUR, + hourRule + ) + ); + + List actual = template.generateAlignedSearchIntervals(referenceTime); + Assertions.assertEquals(expected, actual); + } + + /** + * TEST: Validation failure - default granularity is coarser than most recent segment granularity rule + *

+ * REFERENCE TIME: 2025-01-29T16:15:00Z + *

+ * INPUT RULES: + *

    + *
  • Segment Granularity Rules: P30D→HOUR, P90D→DAY
  • + *
  • Other Rules: P7D-metrics (finer than P30D, triggers prepending with default granularity)
  • + *
  • Default Segment Granularity: MONTH (COARSER than HOUR!)
  • + *
+ *

+ * PROCESSING: + *

    + *
  1. Sort rules by period: P90D→DAY (oldest), P30D→HOUR (newest)
  2. + *
  3. P7D metrics is finer than P30D, so prepend interval with default MONTH granularity
  4. + *
  5. Timeline would be: [-∞, DAY_boundary) DAY, [DAY_boundary, HOUR_boundary) HOUR, [HOUR_boundary, MONTH_boundary) MONTH
  6. + *
  7. Validation: HOUR → MONTH progression means granularity is getting COARSER toward present
  8. + *
+ *

+ * EXPECTED: IllegalArgumentException with message about invalid granularity timeline + */ + @Test + public void test_generateAlignedSearchIntervals_failsWhenDefaultGranularityIsCoarserThanMostRecentSegmentGranRule() + { + DateTime referenceTime = DateTimes.of("2025-01-29T16:15:00Z"); + + ReindexingRuleProvider provider = + InlineReindexingRuleProvider + .builder() + .segmentGranularityRules(List.of( + new ReindexingSegmentGranularityRule("hour-rule", null, Period.days(30), Granularities.HOUR), + new ReindexingSegmentGranularityRule("day-rule", null, Period.days(90), Granularities.DAY) + )) + .dataSchemaRules(List.of( + new ReindexingDataSchemaRule("metrics-7d", null, Period.days(7), null, new AggregatorFactory[0], null, null, null) + )) + .build(); + + CascadingReindexingTemplate template = new CascadingReindexingTemplate( + "testDS", + null, + null, + provider, + null, + null, + null, + null, + Granularities.MONTH // MONTH is coarser than HOUR! + ); + + IllegalArgumentException exception = Assertions.assertThrows( + IllegalArgumentException.class, + () -> template.generateAlignedSearchIntervals(referenceTime) + ); + + Assertions.assertTrue( + exception.getMessage().contains("Invalid segment granularity timeline") + ); + Assertions.assertTrue( + exception.getMessage().contains("coarser granularity") + ); + } + + /** + * TEST: Validation failure - older rule has finer granularity than newer rule + *

+ * REFERENCE TIME: 2025-01-29T16:15:00Z + *

+ * INPUT RULES: + *

    + *
  • Segment Granularity Rules: P30D→DAY, P90D→HOUR
  • + *
  • Other Rules: None
  • + *
  • Default Segment Granularity: DAY
  • + *
+ *

+ * PROCESSING: + *

    + *
  1. Sort rules by period: P90D→HOUR (oldest), P30D→DAY (newest)
  2. + *
  3. Timeline would be: [-∞, HOUR_boundary) HOUR, [HOUR_boundary, DAY_boundary) DAY
  4. + *
  5. Validation: HOUR → DAY progression means granularity is getting COARSER toward present
  6. + *
  7. This violates the constraint: older data (P90D) has HOUR granularity, newer data (P30D) has DAY granularity
  8. + *
+ *

+ * EXPECTED: IllegalArgumentException with message about invalid granularity timeline + */ + @Test + public void test_generateAlignedSearchIntervals_failsWhenOlderRuleHasFinerGranularityThanNewerRule() + { + DateTime referenceTime = DateTimes.of("2025-01-29T16:15:00Z"); + + ReindexingRuleProvider provider = + InlineReindexingRuleProvider + .builder() + .segmentGranularityRules(List.of( + new ReindexingSegmentGranularityRule("day-rule", null, Period.days(30), Granularities.DAY), + new ReindexingSegmentGranularityRule("hour-rule", null, Period.days(90), Granularities.HOUR) + )) + .build(); + + CascadingReindexingTemplate template = new CascadingReindexingTemplate( + "testDS", + null, + null, + provider, + null, + null, + null, + null, + Granularities.DAY + ); + + IllegalArgumentException exception = Assertions.assertThrows( + IllegalArgumentException.class, + () -> template.generateAlignedSearchIntervals(referenceTime) + ); + + Assertions.assertTrue( + exception.getMessage().contains("Invalid segment granularity timeline") + ); + Assertions.assertTrue( + exception.getMessage().contains("coarser granularity") + ); + } + + private static class TestCascadingReindexingTemplate extends CascadingReindexingTemplate + { + // Capture intervals that were processed for assertions + private final List processedIntervals = new ArrayList<>(); + + public TestCascadingReindexingTemplate( + String dataSource, + Integer taskPriority, + Long inputSegmentSizeBytes, + ReindexingRuleProvider ruleProvider, + CompactionEngine engine, + Map taskContext, + Period skipOffsetFromLatest, + Period skipOffsetFromNow + ) + { + super(dataSource, taskPriority, inputSegmentSizeBytes, ruleProvider, + engine, taskContext, skipOffsetFromLatest, skipOffsetFromNow, Granularities.DAY); + } + + public List getProcessedIntervals() + { + return processedIntervals; + } + + @Override + protected CompactionJobTemplate createJobTemplateForInterval( + InlineSchemaDataSourceCompactionConfig config + ) + { + return new CompactionJobTemplate() { + @Override + public String getType() + { + return "test"; + } + + @Override + @Nullable + public Granularity getSegmentGranularity() + { + return null; + } + + @Override + public List createCompactionJobs( + DruidInputSource source, + CompactionJobParams params + ) + { + // Record the interval that was processed + processedIntervals.add(source.getInterval()); + + // Return a single mock job + return List.of(); + } + }; + } + } + + private SegmentTimeline createTestTimeline(DateTime start, DateTime end) + { + DataSegment segment = DataSegment.builder() + .dataSource("testDS") + .interval(new Interval(start, end)) + .version("v1") + .size(1000) + .build(); + return SegmentTimeline.forSegments(Collections.singletonList(segment)); + } + + private ReindexingRuleProvider createMockProvider(List periods) + { + // Create segment granularity rules for each period + List segmentGranularityRules = new ArrayList<>(); + for (int i = 0; i < periods.size(); i++) { + segmentGranularityRules.add(new ReindexingSegmentGranularityRule( + "segment-gran-rule-" + i, + null, + periods.get(i), + Granularities.HOUR + )); + } + + ReindexingRuleProvider mockProvider = EasyMock.createMock(ReindexingRuleProvider.class); + EasyMock.expect(mockProvider.isReady()).andReturn(true); + EasyMock.expect(mockProvider.getSegmentGranularityRules()).andReturn(segmentGranularityRules).anyTimes(); + // Return a fresh stream on each call to avoid "stream has already been operated upon or closed" errors + EasyMock.expect(mockProvider.streamAllRules()).andAnswer(() -> segmentGranularityRules.stream().map(r -> (ReindexingRule) r)).anyTimes(); + EasyMock.expect(mockProvider.getSegmentGranularityRule(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(segmentGranularityRules.get(0)).anyTimes(); + EasyMock.expect(mockProvider.getIOConfigRule(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(null).anyTimes(); + EasyMock.expect(mockProvider.getTuningConfigRule(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(null).anyTimes(); + EasyMock.expect(mockProvider.getDataSchemaRule(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(null).anyTimes(); + EasyMock.expect(mockProvider.getDeletionRules(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(Collections.emptyList()).anyTimes(); + EasyMock.replay(mockProvider); + return mockProvider; + } + + private CompactionJobParams createMockParams(DateTime referenceTime, SegmentTimeline timeline) + { + CompactionJobParams mockParams = EasyMock.createMock(CompactionJobParams.class); + EasyMock.expect(mockParams.getScheduleStartTime()).andReturn(referenceTime).anyTimes(); + EasyMock.expect(mockParams.getTimeline("testDS")).andReturn(timeline); + EasyMock.replay(mockParams); + return mockParams; + } + + private DruidInputSource createMockSource() + { + final Interval[] capturedInterval = new Interval[1]; + + DruidInputSource mockSource = EasyMock.createMock(DruidInputSource.class); + EasyMock.expect(mockSource.withInterval(EasyMock.anyObject(Interval.class))) + .andAnswer(() -> { + capturedInterval[0] = (Interval) EasyMock.getCurrentArguments()[0]; + return mockSource; + }) + .anyTimes(); + EasyMock.expect(mockSource.getInterval()) + .andAnswer(() -> capturedInterval[0]) + .anyTimes(); + EasyMock.replay(mockSource); + return mockSource; + } + + /** + * Helper method to create a ReindexingDataSchemaRule with minimal required fields for testing + *

+ * Helps quickly generate multiple rules to be used in testing formation of timelines and splits. + */ + private ReindexingDataSchemaRule createReindexingDataSchemaRule(String name, Period period) + { + return new ReindexingDataSchemaRule( + name, + null, + period, + null, + new AggregatorFactory[0], + null, + null, + null + ); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/ReindexingConfigBuilderTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/ReindexingConfigBuilderTest.java new file mode 100644 index 000000000000..c9bc0c69f11d --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/ReindexingConfigBuilderTest.java @@ -0,0 +1,327 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.query.filter.OrDimFilter; +import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.server.compaction.InlineReindexingRuleProvider; +import org.apache.druid.server.compaction.IntervalGranularityInfo; +import org.apache.druid.server.compaction.ReindexingDataSchemaRule; +import org.apache.druid.server.compaction.ReindexingDeletionRule; +import org.apache.druid.server.compaction.ReindexingIOConfigRule; +import org.apache.druid.server.compaction.ReindexingRuleProvider; +import org.apache.druid.server.compaction.ReindexingSegmentGranularityRule; +import org.apache.druid.server.compaction.ReindexingTuningConfigRule; +import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskDimensionsConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class ReindexingConfigBuilderTest +{ + private static final Interval TEST_INTERVAL = Intervals.of("2024-11-01/2024-11-02"); + private static final DateTime REFERENCE_TIME = DateTimes.of("2025-01-15"); + + @Test + public void test_applyTo_handlesSynteticSegmentGranularityInsertion() + { + ReindexingRuleProvider provider = InlineReindexingRuleProvider.builder() + .dataSchemaRules( + ImmutableList.of( + new ReindexingDataSchemaRule( + "schema-30d", + null, + Period.days(30), + new UserCompactionTaskDimensionsConfig(null), + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + Granularities.HOUR, + true, + ImmutableList.of() + ) + ) + ).build(); + + InlineSchemaDataSourceCompactionConfig.Builder builder = + InlineSchemaDataSourceCompactionConfig.builder() + .forDataSource("test_datasource"); + + // Create synthetic timeline with default granularity (no source rule since it's default) + ImmutableList syntheticTimeline = ImmutableList.of( + new IntervalGranularityInfo(TEST_INTERVAL, Granularities.DAY, null) + ); + + ReindexingConfigBuilder configBuilder = new ReindexingConfigBuilder( + provider, + TEST_INTERVAL, + REFERENCE_TIME, + syntheticTimeline + ); + + int count = configBuilder.applyTo(builder); + + Assertions.assertEquals(1, count); + + InlineSchemaDataSourceCompactionConfig config = builder.build(); + Assertions.assertNotNull(config.getGranularitySpec()); + Assertions.assertNotNull(config.getGranularitySpec().getSegmentGranularity()); + Assertions.assertEquals(Granularities.DAY, config.getGranularitySpec().getSegmentGranularity()); + Assertions.assertNotNull(config.getGranularitySpec().getQueryGranularity()); + Assertions.assertEquals(Granularities.HOUR, config.getGranularitySpec().getQueryGranularity()); + Assertions.assertNotNull(config.getGranularitySpec().isRollup()); + Assertions.assertTrue(config.getGranularitySpec().isRollup()); + + // Test applyToWithDetails() on a fresh builder + InlineSchemaDataSourceCompactionConfig.Builder builderForDetails = + InlineSchemaDataSourceCompactionConfig.builder() + .forDataSource("test_datasource"); + + ReindexingConfigBuilder.BuildResult buildResult = configBuilder.applyToWithDetails(builderForDetails); + + // Verify count matches + Assertions.assertEquals(count, buildResult.getRuleCount()); + + // Verify applied rules - should only contain the data schema rule, not the synthetic segment granularity + Assertions.assertNotNull(buildResult.getAppliedRules()); + Assertions.assertEquals(1, buildResult.getAppliedRules().size()); + Assertions.assertTrue(buildResult.getAppliedRules().get(0) instanceof ReindexingDataSchemaRule); + + // Verify config matches + InlineSchemaDataSourceCompactionConfig configFromDetails = builderForDetails.build(); + Assertions.assertEquals(config.getGranularitySpec(), configFromDetails.getGranularitySpec()); + } + + @Test + public void test_applyTo_allRulesPresent_appliesAllConfigsAndReturnsCorrectCount() + { + ReindexingRuleProvider provider = createFullyPopulatedProvider(); + InlineSchemaDataSourceCompactionConfig.Builder builder = + InlineSchemaDataSourceCompactionConfig.builder() + .forDataSource("test_datasource"); + + // Create the segment granularity rule used in the provider + ReindexingSegmentGranularityRule segmentGranularityRule = new ReindexingSegmentGranularityRule( + "gran-30d", + null, + Period.days(30), + Granularities.DAY + ); + + // Create synthetic timeline with granularity from the rule + ImmutableList syntheticTimeline = ImmutableList.of( + new IntervalGranularityInfo(TEST_INTERVAL, Granularities.DAY, segmentGranularityRule) + ); + + ReindexingConfigBuilder configBuilder = new ReindexingConfigBuilder( + provider, + TEST_INTERVAL, + REFERENCE_TIME, + syntheticTimeline + ); + + int count = configBuilder.applyTo(builder); + + Assertions.assertEquals(6, count); + + InlineSchemaDataSourceCompactionConfig config = builder.build(); + + Assertions.assertNotNull(config.getGranularitySpec().getSegmentGranularity()); + Assertions.assertEquals(Granularities.DAY, config.getGranularitySpec().getSegmentGranularity()); + + Assertions.assertNotNull(config.getGranularitySpec().getQueryGranularity()); + Assertions.assertEquals(Granularities.HOUR, config.getGranularitySpec().getQueryGranularity()); + Assertions.assertTrue(config.getGranularitySpec().isRollup()); + + Assertions.assertNotNull(config.getTuningConfig()); + Assertions.assertNotNull(config.getMetricsSpec()); + Assertions.assertEquals(1, config.getMetricsSpec().length); + Assertions.assertEquals("count", config.getMetricsSpec()[0].getName()); + + Assertions.assertNotNull(config.getDimensionsSpec()); + Assertions.assertNotNull(config.getIoConfig()); + + Assertions.assertNotNull(config.getProjections()); + Assertions.assertEquals(1, config.getProjections().size()); // only 1 as we match the 2nd dataSchemaRule + + Assertions.assertNotNull(config.getTransformSpec()); + DimFilter appliedFilter = config.getTransformSpec().getFilter(); + Assertions.assertTrue(appliedFilter instanceof NotDimFilter); + + NotDimFilter notFilter = (NotDimFilter) appliedFilter; + Assertions.assertTrue(notFilter.getField() instanceof OrDimFilter); + + OrDimFilter orFilter = (OrDimFilter) notFilter.getField(); + Assertions.assertEquals(2, orFilter.getFields().size()); // 2 filters combined + + // Now test applyToWithDetails() on a fresh builder + InlineSchemaDataSourceCompactionConfig.Builder builderForDetails = + InlineSchemaDataSourceCompactionConfig.builder() + .forDataSource("test_datasource"); + + ReindexingConfigBuilder.BuildResult buildResult = configBuilder.applyToWithDetails(builderForDetails); + + // Verify BuildResult count matches applyTo() count + Assertions.assertEquals(count, buildResult.getRuleCount()); + + // Verify applied rules list + Assertions.assertNotNull(buildResult.getAppliedRules()); + Assertions.assertEquals(6, buildResult.getAppliedRules().size()); + + // Verify rule types in order: tuning, io, dataSchema, 2 deletion rules, segment granularity + Assertions.assertTrue(buildResult.getAppliedRules().get(0) instanceof ReindexingTuningConfigRule); + Assertions.assertTrue(buildResult.getAppliedRules().get(1) instanceof ReindexingIOConfigRule); + Assertions.assertTrue(buildResult.getAppliedRules().get(2) instanceof ReindexingDataSchemaRule); + Assertions.assertTrue(buildResult.getAppliedRules().get(3) instanceof ReindexingDeletionRule); + Assertions.assertTrue(buildResult.getAppliedRules().get(4) instanceof ReindexingDeletionRule); + Assertions.assertTrue(buildResult.getAppliedRules().get(5) instanceof ReindexingSegmentGranularityRule); + + // Verify the config produced by applyToWithDetails() matches the original + InlineSchemaDataSourceCompactionConfig configFromDetails = builderForDetails.build(); + Assertions.assertEquals(config.getGranularitySpec(), configFromDetails.getGranularitySpec()); + Assertions.assertEquals(config.getTuningConfig(), configFromDetails.getTuningConfig()); + } + + @Test + public void test_applyTo_noRulesPresent_appliesNothingAndReturnsZero() + { + ReindexingRuleProvider provider = InlineReindexingRuleProvider.builder().build(); + InlineSchemaDataSourceCompactionConfig.Builder builder = + InlineSchemaDataSourceCompactionConfig.builder() + .forDataSource("test_datasource"); + + // Create synthetic timeline with default granularity (no source rule) + ImmutableList syntheticTimeline = ImmutableList.of( + new IntervalGranularityInfo(TEST_INTERVAL, Granularities.DAY, null) + ); + + ReindexingConfigBuilder configBuilder = new ReindexingConfigBuilder( + provider, + TEST_INTERVAL, + REFERENCE_TIME, + syntheticTimeline + ); + + int count = configBuilder.applyTo(builder); + + Assertions.assertEquals(0, count); + + InlineSchemaDataSourceCompactionConfig config = builder.build(); + + Assertions.assertNull(config.getTuningConfig()); + Assertions.assertNull(config.getMetricsSpec()); + Assertions.assertNull(config.getDimensionsSpec()); + Assertions.assertNull(config.getIoConfig()); + Assertions.assertNull(config.getProjections()); + Assertions.assertNull(config.getTransformSpec()); + } + + private ReindexingRuleProvider createFullyPopulatedProvider() + { + ReindexingSegmentGranularityRule segmentGranularityRule = new ReindexingSegmentGranularityRule( + "gran-30d", + null, + Period.days(30), + Granularities.DAY + ); + + ReindexingTuningConfigRule tuningConfigRule = new ReindexingTuningConfigRule( + "tuning-30d", + null, + Period.days(30), + new UserCompactionTaskQueryTuningConfig(null, null, null, null, null, null, + null, null, null, null, null, null, + null, null, null, null, null, null, null) + ); + + ReindexingDeletionRule filterRule1 = new ReindexingDeletionRule( + "filter-30d", + null, + Period.days(30), + new SelectorDimFilter("country", "US", null), + null + ); + + ReindexingDeletionRule filterRule2 = new ReindexingDeletionRule( + "filter-60d", + null, + Period.days(60), + new SelectorDimFilter("device", "mobile", null), + null + ); + + ReindexingIOConfigRule ioConfigRule = new ReindexingIOConfigRule( + "io-30d", + null, + Period.days(30), + new UserCompactionTaskIOConfig(null) + ); + + ReindexingDataSchemaRule dataSchemaRule1 = new ReindexingDataSchemaRule( + "schema-30d", + null, + Period.days(30), + new UserCompactionTaskDimensionsConfig(null), + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + Granularities.HOUR, + true, + ImmutableList.of( + new AggregateProjectionSpec("proj1", null, null, null, + new AggregatorFactory[]{new CountAggregatorFactory("count1")}), + new AggregateProjectionSpec("proj2", null, null, null, + new AggregatorFactory[]{new CountAggregatorFactory("count2")}) + ) + ); + + ReindexingDataSchemaRule dataSchemaRule2 = new ReindexingDataSchemaRule( + "schema-60d", + null, + Period.days(60), + new UserCompactionTaskDimensionsConfig(null), + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + Granularities.HOUR, + true, + ImmutableList.of( + new AggregateProjectionSpec("proj3", null, null, null, + new AggregatorFactory[]{new CountAggregatorFactory("count3")}) + ) + ); + + return InlineReindexingRuleProvider.builder() + .segmentGranularityRules(ImmutableList.of(segmentGranularityRule)) + .tuningConfigRules(ImmutableList.of(tuningConfigRule)) + .ioConfigRules(ImmutableList.of(ioConfigRule)) + .deletionRules(ImmutableList.of(filterRule1, filterRule2)) + .dataSchemaRules(ImmutableList.of(dataSchemaRule1, dataSchemaRule2)) + .build(); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/ReindexingDeletionRuleOptimizerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/ReindexingDeletionRuleOptimizerTest.java new file mode 100644 index 000000000000..47b69031218a --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/ReindexingDeletionRuleOptimizerTest.java @@ -0,0 +1,539 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.query.filter.OrDimFilter; +import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.TestDataSource; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.metadata.DefaultIndexingStateFingerprintMapper; +import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; +import org.apache.druid.segment.metadata.IndexingStateCache; +import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; +import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.server.compaction.CompactionCandidate; +import org.apache.druid.server.compaction.CompactionStatus; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; +import org.apache.druid.timeline.CompactionState; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.easymock.EasyMock; +import org.joda.time.DateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +public class ReindexingDeletionRuleOptimizerTest +{ + private static final DataSegment WIKI_SEGMENT + = DataSegment.builder(SegmentId.of(TestDataSource.WIKI, Intervals.of("2013-01-01/PT1H"), "v1", 0)) + .size(100_000_000L) + .build(); + + private HeapMemoryIndexingStateStorage indexingStateStorage; + private IndexingStateCache indexingStateCache; + private IndexingStateFingerprintMapper fingerprintMapper; + private ReindexingDeletionRuleOptimizer optimizer; + + @BeforeEach + public void setUp() + { + indexingStateStorage = new HeapMemoryIndexingStateStorage(); + indexingStateCache = new IndexingStateCache(); + fingerprintMapper = new DefaultIndexingStateFingerprintMapper( + indexingStateCache, + new DefaultObjectMapper() + ); + optimizer = new ReindexingDeletionRuleOptimizer(); + } + + @Test + public void testOptimize_SingleFilter_NotOrFilter_NoFingerprints_ReturnsUnchanged() + { + DimFilter filterA = new SelectorDimFilter("country", "US", null); + NotDimFilter expectedFilter = new NotDimFilter(filterA); + + CompactionCandidate candidate = createCandidateWithFingerprints("fp1"); + InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null); + CompactionJobParams params = createParams(); + + DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params); + + // No state for fp1, so config should be unchanged + Assertions.assertEquals(expectedFilter, result.getTransformSpec().getFilter()); + } + + @Test + public void testOptimize_SingleFilter_AlreadyApplied_RemovesTransformSpec() + { + DimFilter filterB = new SelectorDimFilter("country", "UK", null); + CompactionState state = createStateWithSingleFilter(filterB); + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, "fp1", state, DateTimes.nowUtc()); + syncCacheFromManager(); + + NotDimFilter expectedFilter = new NotDimFilter(filterB); + CompactionCandidate candidate = createCandidateWithFingerprints("fp1"); + InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null); + CompactionJobParams params = createParams(); + + DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params); + + // All filters optimized away, transform spec should be null + Assertions.assertNull(result.getTransformSpec()); + } + + @Test + public void testOptimize_AllFiltersAlreadyApplied_RemovesTransformSpec() + { + DimFilter filterA = new SelectorDimFilter("country", "US", null); + DimFilter filterB = new SelectorDimFilter("country", "UK", null); + DimFilter filterC = new SelectorDimFilter("country", "FR", null); + + CompactionState state = createStateWithFilters(filterA, filterB, filterC); + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, "fp1", state, DateTimes.nowUtc()); + syncCacheFromManager(); + + NotDimFilter expectedFilter = new NotDimFilter(new OrDimFilter(Arrays.asList(filterA, filterB, filterC))); + CompactionCandidate candidate = createCandidateWithFingerprints("fp1"); + InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null); + CompactionJobParams params = createParams(); + + DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params); + + // All filters already applied, transform spec should be removed + Assertions.assertNull(result.getTransformSpec()); + } + + @Test + public void testOptimize_NoFiltersApplied_ReturnsAllExpected() + { + DimFilter filterA = new SelectorDimFilter("country", "US", null); + DimFilter filterB = new SelectorDimFilter("country", "UK", null); + DimFilter filterC = new SelectorDimFilter("country", "FR", null); + + CompactionState state = createStateWithoutFilters(); + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, "fp1", state, DateTimes.nowUtc()); + syncCacheFromManager(); + + NotDimFilter expectedFilter = new NotDimFilter(new OrDimFilter(Arrays.asList(filterA, filterB, filterC))); + CompactionCandidate candidate = createCandidateWithFingerprints("fp1"); + InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null); + CompactionJobParams params = createParams(); + + DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params); + + // No filters were applied, so all should remain + NotDimFilter resultFilter = (NotDimFilter) result.getTransformSpec().getFilter(); + OrDimFilter innerOr = (OrDimFilter) resultFilter.getField(); + Assertions.assertEquals(3, innerOr.getFields().size()); + Assertions.assertTrue(innerOr.getFields().containsAll(Arrays.asList(filterA, filterB, filterC))); + } + + @Test + public void testOptimize_PartiallyApplied_ReturnsDelta() + { + DimFilter filterA = new SelectorDimFilter("country", "US", null); + DimFilter filterB = new SelectorDimFilter("country", "UK", null); + DimFilter filterC = new SelectorDimFilter("country", "FR", null); + DimFilter filterD = new SelectorDimFilter("country", "DE", null); + + CompactionState state = createStateWithFilters(filterA, filterB); + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, "fp1", state, DateTimes.nowUtc()); + syncCacheFromManager(); + + NotDimFilter expectedFilter = new NotDimFilter( + new OrDimFilter(Arrays.asList(filterA, filterB, filterC, filterD)) + ); + CompactionCandidate candidate = createCandidateWithFingerprints("fp1"); + InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null); + CompactionJobParams params = createParams(); + + DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params); + + // Only C and D should remain (A and B were already applied) + NotDimFilter resultFilter = (NotDimFilter) result.getTransformSpec().getFilter(); + OrDimFilter innerOr = (OrDimFilter) resultFilter.getField(); + Set resultSet = new HashSet<>(innerOr.getFields()); + Set expectedSet = new HashSet<>(Arrays.asList(filterC, filterD)); + + Assertions.assertEquals(expectedSet, resultSet); + } + + @Test + public void testOptimize_MultipleFingerprints_UnionOfMissing() + { + DimFilter filterA = new SelectorDimFilter("country", "US", null); + DimFilter filterB = new SelectorDimFilter("country", "UK", null); + DimFilter filterC = new SelectorDimFilter("country", "FR", null); + DimFilter filterD = new SelectorDimFilter("country", "DE", null); + + CompactionState state1 = createStateWithFilters(filterA, filterB); + CompactionState state2 = createStateWithFilters(filterA, filterC); + DateTime now = DateTimes.nowUtc(); + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, "fp1", state1, now); + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, "fp2", state2, now); + syncCacheFromManager(); + + NotDimFilter expectedFilter = new NotDimFilter( + new OrDimFilter(Arrays.asList(filterA, filterB, filterC, filterD)) + ); + CompactionCandidate candidate = createCandidateWithFingerprints("fp1", "fp2"); + InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null); + CompactionJobParams params = createParams(); + + DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params); + + // fp1 has A,B applied; fp2 has A,C applied + // Union of missing: B (missing from fp2), C (missing from fp1), D (missing from both) + NotDimFilter resultFilter = (NotDimFilter) result.getTransformSpec().getFilter(); + OrDimFilter innerOr = (OrDimFilter) resultFilter.getField(); + Set resultSet = new HashSet<>(innerOr.getFields()); + Set expectedSet = new HashSet<>(Arrays.asList(filterB, filterC, filterD)); + + Assertions.assertEquals(expectedSet, resultSet); + } + + @Test + public void testOptimize_MultipleFingerprints_NoDuplicates() + { + DimFilter filterA = new SelectorDimFilter("country", "US", null); + DimFilter filterB = new SelectorDimFilter("country", "UK", null); + DimFilter filterC = new SelectorDimFilter("country", "FR", null); + + CompactionState state1 = createStateWithFilters(filterA); + CompactionState state2 = createStateWithFilters(filterA); + DateTime now = DateTimes.nowUtc(); + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, "fp1", state1, now); + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, "fp2", state2, now); + syncCacheFromManager(); + + NotDimFilter expectedFilter = new NotDimFilter( + new OrDimFilter(Arrays.asList(filterA, filterB, filterC)) + ); + CompactionCandidate candidate = createCandidateWithFingerprints("fp1", "fp2"); + InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null); + CompactionJobParams params = createParams(); + + DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params); + + // Both fingerprints have A applied, so only B and C remain + NotDimFilter resultFilter = (NotDimFilter) result.getTransformSpec().getFilter(); + OrDimFilter innerOr = (OrDimFilter) resultFilter.getField(); + Set resultSet = new HashSet<>(innerOr.getFields()); + + Assertions.assertEquals(2, resultSet.size()); + Assertions.assertTrue(resultSet.containsAll(Arrays.asList(filterB, filterC))); + } + + @Test + public void testOptimize_MissingCompactionState_ReturnsAllFilters() + { + DimFilter filterA = new SelectorDimFilter("country", "US", null); + DimFilter filterB = new SelectorDimFilter("country", "UK", null); + DimFilter filterC = new SelectorDimFilter("country", "FR", null); + + // No state persisted for fp1 + NotDimFilter expectedFilter = new NotDimFilter( + new OrDimFilter(Arrays.asList(filterA, filterB, filterC)) + ); + CompactionCandidate candidate = createCandidateWithFingerprints("fp1"); + InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null); + CompactionJobParams params = createParams(); + + DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params); + + // No state available, all filters should remain + Assertions.assertEquals(expectedFilter, result.getTransformSpec().getFilter()); + } + + @Test + public void testOptimize_TransformSpecWithSingleFilter() + { + DimFilter filterA = new SelectorDimFilter("country", "US", null); + DimFilter filterB = new SelectorDimFilter("country", "UK", null); + DimFilter filterC = new SelectorDimFilter("country", "FR", null); + + CompactionState state = createStateWithSingleFilter(filterA); + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, "fp1", state, DateTimes.nowUtc()); + syncCacheFromManager(); + + NotDimFilter expectedFilter = new NotDimFilter( + new OrDimFilter(Arrays.asList(filterA, filterB, filterC)) + ); + CompactionCandidate candidate = createCandidateWithFingerprints("fp1"); + InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null); + CompactionJobParams params = createParams(); + + DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params); + + // A was already applied, only B and C should remain + NotDimFilter resultFilter = (NotDimFilter) result.getTransformSpec().getFilter(); + OrDimFilter innerOr = (OrDimFilter) resultFilter.getField(); + Assertions.assertEquals(2, innerOr.getFields().size()); + Assertions.assertTrue(innerOr.getFields().containsAll(Arrays.asList(filterB, filterC))); + } + + @Test + public void testOptimize_SegmentsWithNoFingerprints() + { + DimFilter filterA = new SelectorDimFilter("country", "US", null); + DimFilter filterB = new SelectorDimFilter("country", "UK", null); + DimFilter filterC = new SelectorDimFilter("country", "FR", null); + + CompactionCandidate candidate = createCandidateWithNullFingerprints(3); + + NotDimFilter expectedFilter = new NotDimFilter( + new OrDimFilter(Arrays.asList(filterA, filterB, filterC)) + ); + InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null); + CompactionJobParams params = createParams(); + + DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params); + + // No fingerprints, all filters should remain + Assertions.assertEquals(expectedFilter, result.getTransformSpec().getFilter()); + } + + + + + // Helper methods + + private InlineSchemaDataSourceCompactionConfig createConfigWithFilter( + @Nullable NotDimFilter filter, + @Nullable VirtualColumns virtualColumns + ) + { + CompactionTransformSpec transformSpec = filter == null && virtualColumns == null + ? null + : new CompactionTransformSpec(filter, virtualColumns); + + return InlineSchemaDataSourceCompactionConfig.builder() + .forDataSource(TestDataSource.WIKI) + .withTransformSpec(transformSpec) + .build(); + } + + private CompactionJobParams createParams() + { + CompactionJobParams mockParams = EasyMock.createMock(CompactionJobParams.class); + EasyMock.expect(mockParams.getFingerprintMapper()).andReturn(fingerprintMapper).anyTimes(); + EasyMock.replay(mockParams); + return mockParams; + } + + private CompactionCandidate createCandidateWithFingerprints(String... fingerprints) + { + List segments = Arrays.stream(fingerprints) + .map(fp -> DataSegment.builder(WIKI_SEGMENT).indexingStateFingerprint(fp).build()) + .collect(Collectors.toList()); + return CompactionCandidate.from(segments, null) + .withCurrentStatus(CompactionStatus.pending("segments need compaction")); + } + + private CompactionCandidate createCandidateWithNullFingerprints(int count) + { + List segments = new ArrayList<>(); + for (int i = 0; i < count; i++) { + segments.add(DataSegment.builder(WIKI_SEGMENT).indexingStateFingerprint(null).build()); + } + return CompactionCandidate.from(segments, null) + .withCurrentStatus(CompactionStatus.pending("segments need compaction")); + } + + private CompactionState createStateWithFilters(DimFilter... filters) + { + OrDimFilter orFilter = new OrDimFilter(Arrays.asList(filters)); + NotDimFilter notFilter = new NotDimFilter(orFilter); + CompactionTransformSpec transformSpec = new CompactionTransformSpec(notFilter, null); + + return new CompactionState( + null, + null, + null, + transformSpec, + IndexSpec.getDefault(), + null, + null + ); + } + + private CompactionState createStateWithSingleFilter(DimFilter filter) + { + NotDimFilter notFilter = new NotDimFilter(filter); + CompactionTransformSpec transformSpec = new CompactionTransformSpec(notFilter, null); + + return new CompactionState( + null, + null, + null, + transformSpec, + IndexSpec.getDefault(), + null, + null + ); + } + + private CompactionState createStateWithoutFilters() + { + return new CompactionState( + null, + null, + null, + null, + IndexSpec.getDefault(), + null, + null + ); + } + + @Test + public void testOptimize_FilterVirtualColumns_SomeColumnsReferenced() + { + // Create virtual columns vc1, vc2, vc3 + VirtualColumns virtualColumns = VirtualColumns.create( + ImmutableList.of( + new ExpressionVirtualColumn("vc1", "col1 + 1", ColumnType.LONG, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vc2", "col2 + 2", ColumnType.LONG, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vc3", "col3 + 3", ColumnType.LONG, TestExprMacroTable.INSTANCE) + ) + ); + + // Create a filter that only references vc1 and vc3 (vc2 is unreferenced) + DimFilter filter = new OrDimFilter( + Arrays.asList( + new SelectorDimFilter("vc1", "value1", null), + new SelectorDimFilter("vc3", "value3", null) + ) + ); + NotDimFilter notFilter = new NotDimFilter(filter); + + // Candidate has no filters applied, so all filters remain + CompactionCandidate candidate = createCandidateWithNullFingerprints(1); + InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(notFilter, virtualColumns); + CompactionJobParams params = createParams(); + + DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params); + + // Filter remains, but vc2 should be filtered out + VirtualColumns resultVCs = result.getTransformSpec().getVirtualColumns(); + Assertions.assertNotNull(resultVCs); + Assertions.assertEquals(2, resultVCs.getVirtualColumns().length); + + Set outputNames = new HashSet<>(); + for (org.apache.druid.segment.VirtualColumn vc : resultVCs.getVirtualColumns()) { + outputNames.add(vc.getOutputName()); + } + + Assertions.assertTrue(outputNames.contains("vc1")); + Assertions.assertFalse(outputNames.contains("vc2")); // vc2 should be filtered out + Assertions.assertTrue(outputNames.contains("vc3")); + } + + @Test + public void testOptimize_FilterVirtualColumns_NoColumnsReferenced() + { + // Create virtual columns + VirtualColumns virtualColumns = VirtualColumns.create( + ImmutableList.of( + new ExpressionVirtualColumn("vc1", "col1 + 1", ColumnType.LONG, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vc2", "col2 + 2", ColumnType.LONG, TestExprMacroTable.INSTANCE) + ) + ); + + // Create a filter that references a physical column, not virtual columns + DimFilter filter = new SelectorDimFilter("regularColumn", "value", null); + NotDimFilter notFilter = new NotDimFilter(filter); + + // Candidate has no filters applied + CompactionCandidate candidate = createCandidateWithNullFingerprints(1); + InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(notFilter, virtualColumns); + CompactionJobParams params = createParams(); + + DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params); + + Assertions.assertEquals(VirtualColumns.EMPTY, result.getTransformSpec().getVirtualColumns()); + } + + @Test + public void testOptimize_CandidateNeverCompacted_NoOptimization() + { + DimFilter filterA = new SelectorDimFilter("country", "US", null); + NotDimFilter expectedFilter = new NotDimFilter(filterA); + + // Candidate with NEVER_COMPACTED status + List segments = new ArrayList<>(); + segments.add(DataSegment.builder(WIKI_SEGMENT).indexingStateFingerprint(null).build()); + CompactionCandidate candidate = CompactionCandidate.from(segments, null) + .withCurrentStatus(CompactionStatus.pending(CompactionStatus.NEVER_COMPACTED_REASON)); + + InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null); + CompactionJobParams params = createParams(); + + DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params); + + // Should return config unchanged since candidate was never compacted + Assertions.assertSame(config, result); + } + + @Test + public void testOptimize_NoTransformSpec_NoOptimization() + { + // Config without transform spec + InlineSchemaDataSourceCompactionConfig config = InlineSchemaDataSourceCompactionConfig.builder() + .forDataSource(TestDataSource.WIKI) + .build(); + + CompactionCandidate candidate = createCandidateWithFingerprints("fp1"); + CompactionJobParams params = createParams(); + + DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params); + + // Should return config unchanged + Assertions.assertSame(config, result); + } + + /** + * Helper to sync the cache with states stored in the manager (for tests that persist states). + */ + private void syncCacheFromManager() + { + indexingStateCache.resetIndexingStatesForPublishedSegments(indexingStateStorage.getAllStoredStates()); + } +} diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 2310ff84072d..e65a1994f4aa 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -73,6 +73,7 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.indexing.CombinedDataSchema; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.security.Action; @@ -265,7 +266,7 @@ public List createMsqControllerTasks( Query query; Interval interval = intervalDataSchema.getKey(); DataSchema dataSchema = intervalDataSchema.getValue(); - Map inputColToVirtualCol = getVirtualColumns(dataSchema, interval); + Map inputColToVirtualCol = getVirtualColumns(dataSchema, interval, compactionTask.getTransformSpec()); if (isGroupBy(dataSchema)) { query = buildGroupByQuery(compactionTask, interval, dataSchema, inputColToVirtualCol); @@ -565,7 +566,7 @@ private static boolean isQueryGranularityEmptyOrNone(DataSchema dataSchema) * grouping on them without unnesting. * */ - private Map getVirtualColumns(DataSchema dataSchema, Interval interval) + private Map getVirtualColumns(DataSchema dataSchema, Interval interval, CompactionTransformSpec compactionTransformSpec) { Map inputColToVirtualCol = new HashMap<>(); if (!isQueryGranularityEmptyOrNone(dataSchema)) { @@ -626,6 +627,13 @@ private Map getVirtualColumns(DataSchema dataSchema, Inte ); } } + + if (compactionTransformSpec != null && compactionTransformSpec.getVirtualColumns() != null) { + for (VirtualColumn vc : compactionTransformSpec.getVirtualColumns().getVirtualColumns()) { + inputColToVirtualCol.put(vc.getOutputName(), vc); + } + } + return inputColToVirtualCol; } @@ -645,7 +653,7 @@ private Query buildGroupByQuery( List postAggregators = inputColToVirtualCol.entrySet() .stream() - .filter(entry -> !entry.getKey().equals(ColumnHolder.TIME_COLUMN_NAME)) + .filter(entry -> entry.getValue().getOutputName().startsWith(ARRAY_VIRTUAL_COLUMN_PREFIX)) .map( entry -> new ExpressionPostAggregator( diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index 46cb30deb202..c532e8334744 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -1018,7 +1018,8 @@ public void testReplaceOnFoo1WithWhereExtern(String contextName, Map conte ), GranularityType.DAY, Intervals.ETERNITY, - new CompactionTransformSpec(new NotDimFilter(new NullFilter("dim1", null))) + new CompactionTransformSpec(new NotDimFilter(new NullFilter("dim1", null)), null) ) ) .verifyResults(); @@ -2206,7 +2207,7 @@ public void testReplaceTombstonesOverPartiallyOverlappingSegments(String context ), GranularityType.QUARTER, Intervals.of("2000-01-01T00:00:00.000Z/2002-01-01T00:00:00.000Z"), - new CompactionTransformSpec(new NotDimFilter(new NullFilter("dim1", null))) + new CompactionTransformSpec(new NotDimFilter(new NullFilter("dim1", null)), null) )) .verifyResults(); } @@ -2287,7 +2288,7 @@ public void testEmptyReplaceAllOverExistingSegment(String contextName, Map metricsSpec = List.of(new CountAggregatorFactory("count")); - CompactionTransformSpec transformSpec = new CompactionTransformSpec(null); + CompactionTransformSpec transformSpec = new CompactionTransformSpec(null, null); IndexSpec indexSpec = IndexSpec.getDefault(); UniformGranularitySpec granularitySpec = new UniformGranularitySpec( Granularities.DAY, @@ -75,7 +75,7 @@ public void testBuilderWithProjections() DynamicPartitionsSpec partitionsSpec = new DynamicPartitionsSpec(null, null); DimensionsSpec dimensionsSpec = DimensionsSpec.builder().build(); List metricsSpec = List.of(new CountAggregatorFactory("count")); - CompactionTransformSpec transformSpec = new CompactionTransformSpec(null); + CompactionTransformSpec transformSpec = new CompactionTransformSpec(null, null); IndexSpec indexSpec = IndexSpec.getDefault(); UniformGranularitySpec granularitySpec = new UniformGranularitySpec( Granularities.DAY, @@ -111,7 +111,7 @@ public void testToBuilder() DynamicPartitionsSpec partitionsSpec = new DynamicPartitionsSpec(null, null); DimensionsSpec dimensionsSpec = DimensionsSpec.builder().build(); List metricsSpec = List.of(new CountAggregatorFactory("count")); - CompactionTransformSpec transformSpec = new CompactionTransformSpec(null); + CompactionTransformSpec transformSpec = new CompactionTransformSpec(null, null); IndexSpec indexSpec = IndexSpec.getDefault(); UniformGranularitySpec granularitySpec = new UniformGranularitySpec( Granularities.DAY, @@ -156,7 +156,7 @@ public void testToBuilderWithModifications() DynamicPartitionsSpec partitionsSpec = new DynamicPartitionsSpec(null, null); DimensionsSpec dimensionsSpec = DimensionsSpec.builder().build(); List metricsSpec = List.of(new CountAggregatorFactory("count")); - CompactionTransformSpec transformSpec = new CompactionTransformSpec(null); + CompactionTransformSpec transformSpec = new CompactionTransformSpec(null, null); IndexSpec indexSpec = IndexSpec.getDefault(); UniformGranularitySpec granularitySpec = new UniformGranularitySpec( Granularities.DAY, @@ -197,7 +197,7 @@ public void testBuilderIndependence() DynamicPartitionsSpec partitionsSpec2 = new DynamicPartitionsSpec(200, null); DimensionsSpec dimensionsSpec = DimensionsSpec.builder().build(); List metricsSpec = List.of(new CountAggregatorFactory("count")); - CompactionTransformSpec transformSpec = new CompactionTransformSpec(null); + CompactionTransformSpec transformSpec = new CompactionTransformSpec(null, null); IndexSpec indexSpec = IndexSpec.getDefault(); UniformGranularitySpec granularitySpec = new UniformGranularitySpec( Granularities.DAY, diff --git a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java index 23e88564c8f3..37c175b57f27 100644 --- a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -32,11 +32,16 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; @@ -66,6 +71,7 @@ public void setUp() { InjectableValues.Std injectableValues = new InjectableValues.Std(); injectableValues.addValue(PruneSpecsHolder.class, PruneSpecsHolder.DEFAULT); + injectableValues.addValue(ExprMacroTable.class, TestExprMacroTable.INSTANCE); MAPPER.setInjectableValues(injectableValues); } @@ -82,7 +88,19 @@ public void testSerializationWithLatestFormat() throws Exception new HashedPartitionsSpec(100000, null, ImmutableList.of("dim1")), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "bar", "foo"))), ImmutableList.of(new CountAggregatorFactory("count")), - new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null)), + new CompactionTransformSpec( + new SelectorDimFilter("dim1", "foo", null), + VirtualColumns.create( + ImmutableList.of( + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() + ) + ) + ) + ), MAPPER.convertValue(ImmutableMap.of(), IndexSpec.class), MAPPER.convertValue(ImmutableMap.of(), GranularitySpec.class), null @@ -147,7 +165,19 @@ public void testV1Serialization() throws Exception DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "bar", "foo")) ), ImmutableList.of(new CountAggregatorFactory("count")), - new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null)), + new CompactionTransformSpec( + new SelectorDimFilter("dim1", "foo", null), + VirtualColumns.create( + ImmutableList.of( + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() + ) + ) + ) + ), MAPPER.convertValue(ImmutableMap.of(), IndexSpec.class), MAPPER.convertValue(ImmutableMap.of(), GranularitySpec.class), null @@ -347,7 +377,19 @@ public void testWithLastCompactionState() new DynamicPartitionsSpec(null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))), ImmutableList.of(new CountAggregatorFactory("count")), - new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null)), + new CompactionTransformSpec( + new SelectorDimFilter("dim1", "foo", null), + VirtualColumns.create( + ImmutableList.of( + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() + ) + ) + ) + ), MAPPER.convertValue(Map.of("test", "map"), IndexSpec.class), MAPPER.convertValue(Map.of("test2", "map2"), GranularitySpec.class), null @@ -377,7 +419,17 @@ public void testAnnotateWithLastCompactionState() DimensionsSpec dimensionsSpec = new DimensionsSpec(DimensionsSpec.getDefaultSchemas(List.of("bar", "foo"))); List metricsSpec = ImmutableList.of(new CountAggregatorFactory("count")); CompactionTransformSpec transformSpec = new CompactionTransformSpec( - new SelectorDimFilter("dim1", "foo", null) + new SelectorDimFilter("dim1", "foo", null), + VirtualColumns.create( + ImmutableList.of( + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() + ) + ) + ) ); IndexSpec indexSpec = MAPPER.convertValue(Map.of("test", "map"), IndexSpec.class).getEffectiveSpec(); GranularitySpec granularitySpec = MAPPER.convertValue(Map.of("test2", "map"), GranularitySpec.class); diff --git a/server/src/main/java/org/apache/druid/server/compaction/AbstractReindexingRule.java b/server/src/main/java/org/apache/druid/server/compaction/AbstractReindexingRule.java new file mode 100644 index 000000000000..3131df31cbbf --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/compaction/AbstractReindexingRule.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.logger.Logger; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Base implementation for reindexing rules that apply based on data age thresholds. + *

+ * Provides period-based applicability logic: a rule with {@link AbstractReindexingRule#olderThan} of P7D applies to + * data older than 7 days as compared to the time of evaluation. Subclasses define specific reindexing configuration + * (granularity, filters, tuning, etc.) and whether multiple rules can combine (additive vs non-additive). + *

+ * The {@link #appliesTo(Interval, DateTime)} method determines if an interval is fully, + * partially, or not covered by this rule's threshold, enabling cascading reindexing + * strategies where different rules apply to different age tiers of data. + */ +public abstract class AbstractReindexingRule implements ReindexingRule +{ + private static final Logger LOG = new Logger(AbstractReindexingRule.class); + + private final String id; + private final String description; + private final Period olderThan; + + public AbstractReindexingRule( + @Nonnull String id, + @Nullable String description, + @Nonnull Period olderThan + ) + { + this.id = Objects.requireNonNull(id, "id cannot be null"); + this.description = description; + this.olderThan = Objects.requireNonNull(olderThan, "olderThan period cannot be null"); + + validatePeriodIsNonNegative(olderThan); + } + + /** + * Validates that a period represents a non-negative duration (>= 0). + *

+ * Zero periods (P0D) are allowed - they indicate rules that should apply immediately to all data. + * Negative periods are rejected as they would be nonsensical. + *

+ * For periods with precise units (days, hours, minutes, seconds), validates by converting + * to a standard duration. For periods with variable-length units (months, years), validates + * that no components are negative, since these cannot be converted to a precise duration. + * + * @param period the period to validate + * @throws IllegalArgumentException if the period is negative + */ + private static void validatePeriodIsNonNegative(Period period) + { + if (hasMonthsOrYears(period)) { + if (isPeriodNegative(period)) { + throw new IllegalArgumentException("period must not be negative. Supplied period: " + period); + } + } else { + if (period.toStandardDuration().getMillis() < 0) { + throw new IllegalArgumentException("period must not be negative. Supplied period: " + period); + } + } + } + + /** + * Checks if a period with variable-length components (months/years) has any negative components. + *

+ * This is purposely an unscientific check that simply ensures no negative values are present in any component of the period. + * It should be "good enough" for almost all reasonable use cases. + * + * @param period the period to check + * @return true if any component is negative + */ + private static boolean isPeriodNegative(Period period) + { + return period.getYears() < 0 + || period.getMonths() < 0 + || period.getWeeks() < 0 + || period.getDays() < 0 + || period.getHours() < 0 + || period.getMinutes() < 0 + || period.getSeconds() < 0 + || period.getMillis() < 0; + } + + @JsonProperty + @Override + public String getId() + { + return id; + } + + @JsonProperty + @Override + public String getDescription() + { + return description; + } + + @JsonProperty + @Override + public Period getOlderThan() + { + return olderThan; + } + + @Override + public AppliesToMode appliesTo(Interval interval, @Nullable DateTime referenceTime) + { + DateTime now = (referenceTime != null) ? referenceTime : DateTimes.nowUtc(); + DateTime intervalEnd = interval.getEnd(); + DateTime intervalStart = interval.getStart(); + + DateTime threshold = now.minus(olderThan); + + if (intervalEnd.isBefore(threshold) || intervalEnd.isEqual(threshold)) { + LOG.debug("Reindexing rule [%s] applies FULLY to interval [%s]. Threshold: [%s]", id, interval, threshold); + return AppliesToMode.FULL; + } else if (intervalStart.isAfter(threshold)) { + LOG.debug("Reindexing rule [%s] does NOT apply to interval [%s]. Threshold: [%s]", id, interval, threshold); + return AppliesToMode.NONE; + } else { + LOG.debug("Reindexing rule [%s] applies PARTIALLY to interval [%s]. Threshold: [%s]", id, interval, threshold); + return AppliesToMode.PARTIAL; + } + } + + /** + * Checks if a period contains months or years components which have variable lenghts and require special handling + */ + private static boolean hasMonthsOrYears(Period period) + { + return period.getYears() != 0 || period.getMonths() != 0; + } + +} diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java index 99e1eef21465..10076a719241 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java @@ -60,6 +60,7 @@ public class CompactionStatus private static final Logger log = new Logger(CompactionStatus.class); private static final CompactionStatus COMPLETE = new CompactionStatus(State.COMPLETE, null, null, null); + public static final String NEVER_COMPACTED_REASON = "not compacted yet"; public enum State { @@ -86,7 +87,7 @@ public enum State Evaluator::rollupIsUpToDate, Evaluator::dimensionsSpecIsUpToDate, Evaluator::metricsSpecIsUpToDate, - Evaluator::transformSpecFilterIsUpToDate, + Evaluator::transformSpecIsUpToDate, Evaluator::projectionsAreUpToDate ); @@ -522,7 +523,7 @@ private CompactionStatus segmentsHaveBeenCompactedAtLeastOnce() if (uncompactedSegments.isEmpty()) { return COMPLETE; } else { - return CompactionStatus.pending("not compacted yet"); + return CompactionStatus.pending(NEVER_COMPACTED_REASON); } } @@ -566,9 +567,9 @@ private CompactionStatus metricsSpecIsUpToDate() return evaluateForAllCompactionStates(this::metricsSpecIsUpToDate); } - private CompactionStatus transformSpecFilterIsUpToDate() + private CompactionStatus transformSpecIsUpToDate() { - return evaluateForAllCompactionStates(this::transformSpecFilterIsUpToDate); + return evaluateForAllCompactionStates(this::transformSpecIsUpToDate); } private CompactionStatus partitionsSpecIsUpToDate(CompactionState lastCompactionState) @@ -750,17 +751,22 @@ private CompactionStatus metricsSpecIsUpToDate(CompactionState lastCompactionSta } } - private CompactionStatus transformSpecFilterIsUpToDate(CompactionState lastCompactionState) + private CompactionStatus transformSpecIsUpToDate(CompactionState lastCompactionState) { - if (compactionConfig.getTransformSpec() == null) { + final CompactionTransformSpec configuredSpec = compactionConfig.getTransformSpec(); + if (configuredSpec == null + || (configuredSpec.getFilter() == null && configuredSpec.getVirtualColumns().isEmpty())) { return COMPLETE; } - CompactionTransformSpec existingTransformSpec = lastCompactionState.getTransformSpec(); + final CompactionTransformSpec existingSpec = Configs.valueOrDefault( + lastCompactionState.getTransformSpec(), + new CompactionTransformSpec(null, null) + ); return CompactionStatus.completeIfNullOrEqual( - "transformSpec filter", - compactionConfig.getTransformSpec().getFilter(), - existingTransformSpec == null ? null : existingTransformSpec.getFilter(), + "transformSpec", + configuredSpec, + existingSpec, String::valueOf ); } diff --git a/server/src/main/java/org/apache/druid/server/compaction/ComposingReindexingRuleProvider.java b/server/src/main/java/org/apache/druid/server/compaction/ComposingReindexingRuleProvider.java new file mode 100644 index 000000000000..e1152ff48266 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/compaction/ComposingReindexingRuleProvider.java @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +/** + * A meta-provider that composes multiple {@link ReindexingRuleProvider}s with first-wins semantics. + *

+ * This provider delegates rule queries to a list of child providers in order. For each rule type, + * it returns the result from the first provider that has non-empty rules of that type. + *

+ * First-Wins Strategy: + * Provider order determines precedence. If provider A returns rules of a given type, those rules + * are used and subsequent providers are not consulted for that type. This applies to all rule + * types, regardless of whether they are additive or non-additive. + *

+ * Readiness: + * The composing provider is considered ready only when ALL child providers are ready. + * This ensures consistent behavior during startup. + *

+ * Example Usage: + *

{@code
+ * {
+ *   "type": "composing",
+ *   "providers": [
+ *     {
+ *       "type": "inline",
+ *       "segmentGranularityRules": [
+ *         {
+ *           "id": "recent-data-granularity",
+ *           "olderThan": "P7D",
+ *           "segmentGranularity": "HOUR"
+ *         }
+ *       ]
+ *     }
+ *   ]
+ * }
+ * }
+ * In this example: + *
    + *
  • Composing rule with a single provider to simply show the inline definition.
  • + *
  • Once multiple provider types exist, this will allow operators to chain them as needed.
  • + *
+ */ +public class ComposingReindexingRuleProvider implements ReindexingRuleProvider +{ + public static final String TYPE = "composing"; + + private final List providers; + + @JsonCreator + public ComposingReindexingRuleProvider( + @JsonProperty("providers") List providers + ) + { + this.providers = Objects.requireNonNull(providers, "providers cannot be null"); + + for (ReindexingRuleProvider provider : providers) { + Objects.requireNonNull(provider, "providers list contains null element"); + } + } + + @JsonProperty("providers") + public List getProviders() + { + return providers; + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + public boolean isReady() + { + // All providers must be ready + return providers.stream().allMatch(ReindexingRuleProvider::isReady); + } + + @Override + public List getDeletionRules() + { + return providers.stream() + .map(ReindexingRuleProvider::getDeletionRules) + .filter(rules -> !rules.isEmpty()) + .findFirst() + .orElse(Collections.emptyList()); + } + + @Override + public List getDeletionRules(Interval interval, DateTime referenceTime) + { + return providers.stream() + .map(p -> p.getDeletionRules(interval, referenceTime)) + .filter(rules -> !rules.isEmpty()) + .findFirst() + .orElse(Collections.emptyList()); + } + + @Override + @Nullable + public ReindexingDataSchemaRule getDataSchemaRule(Interval interval, DateTime referenceTime) + { + return providers.stream() + .map(p -> p.getDataSchemaRule(interval, referenceTime)) + .filter(Objects::nonNull) + .findFirst() + .orElse(null); + } + + @Override + public List getDataSchemaRules() + { + return providers.stream() + .map(ReindexingRuleProvider::getDataSchemaRules) + .filter(rules -> !rules.isEmpty()) + .findFirst() + .orElse(Collections.emptyList()); + } + + @Override + public List getIOConfigRules() + { + return providers.stream() + .map(ReindexingRuleProvider::getIOConfigRules) + .filter(rules -> !rules.isEmpty()) + .findFirst() + .orElse(Collections.emptyList()); + } + + @Override + @Nullable + public ReindexingIOConfigRule getIOConfigRule(Interval interval, DateTime referenceTime) + { + return providers.stream() + .map(p -> p.getIOConfigRule(interval, referenceTime)) + .filter(Objects::nonNull) + .findFirst() + .orElse(null); + } + + @Override + @Nullable + public ReindexingSegmentGranularityRule getSegmentGranularityRule(Interval interval, DateTime referenceTime) + { + return providers.stream() + .map(p -> p.getSegmentGranularityRule(interval, referenceTime)) + .filter(Objects::nonNull) + .findFirst() + .orElse(null); + } + + @Override + public List getSegmentGranularityRules() + { + return providers.stream() + .map(ReindexingRuleProvider::getSegmentGranularityRules) + .filter(rules -> !rules.isEmpty()) + .findFirst() + .orElse(Collections.emptyList()); + } + + @Override + public List getTuningConfigRules() + { + return providers.stream() + .map(ReindexingRuleProvider::getTuningConfigRules) + .filter(rules -> !rules.isEmpty()) + .findFirst() + .orElse(Collections.emptyList()); + } + + @Override + @Nullable + public ReindexingTuningConfigRule getTuningConfigRule(Interval interval, DateTime referenceTime) + { + return providers.stream() + .map(p -> p.getTuningConfigRule(interval, referenceTime)) + .filter(Objects::nonNull) + .findFirst() + .orElse(null); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ComposingReindexingRuleProvider that = (ComposingReindexingRuleProvider) o; + return Objects.equals(providers, that.providers); + } + + @Override + public int hashCode() + { + return Objects.hash(providers); + } + + @Override + public String toString() + { + return "ComposingReindexingRuleProvider{" + + "providers=" + providers + + ", ready=" + isReady() + + '}'; + } +} diff --git a/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java index 1994e87a6388..ed80ec967b6f 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java +++ b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java @@ -509,7 +509,7 @@ skipInterval, new Interval(remainingStart, remainingEnd) } } - if (!remainingStart.equals(remainingEnd)) { + if (remainingStart.isBefore(remainingEnd)) { filteredIntervals.add(new Interval(remainingStart, remainingEnd)); } diff --git a/server/src/main/java/org/apache/druid/server/compaction/InlineReindexingRuleProvider.java b/server/src/main/java/org/apache/druid/server/compaction/InlineReindexingRuleProvider.java new file mode 100644 index 000000000000..d713b1ec3a9f --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/compaction/InlineReindexingRuleProvider.java @@ -0,0 +1,334 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.common.config.Configs; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; + +/** + * Rule provider that returns a static list of rules defined inline in the configuration. + *

+ * This is the simplest provider implementation, suitable for testing and use cases where the number of rules is + * relatively small and can be defined directly in the supervisor spec. + *

+ * When filtering rules by interval, this provider only returns rules where {@link ReindexingRule#appliesTo(Interval, DateTime)} + * returns {@link ReindexingRule.AppliesToMode#FULL}. Rules with partial or no overlap are excluded. + *

+ * For non-additive rule types, when multiple rules fully match an interval, only the rule with the oldest threshold + * (largest period) is returned. For example, if both a P30D and P90D granularity rule match an interval, the P90D + * rule is selected because it has the oldest threshold (now - 90 days is older than now - 30 days). + *

+ * Example usage: + *

{@code
+ * {
+ *   "type": "inline",
+ *   "reindexingDeletionRules": [
+ *     {
+ *       "id": "remove-bots-90d",
+ *       "olderThan": "P90D",
+ *       "deleteWhere": {
+ *         "type": "not",
+ *         "field": {
+ *           "type": "equals",
+ *           "column": "is_bot",
+ *           "matchValueType": "STRING"
+ *           "matchValue": "true"
+ *         }
+ *       },
+ *       "description": "Remove bot traffic from segments older than 90 days"
+ *     },
+ *     {
+ *       "id": "remove-low-priority-180d",
+ *       "olderThan": "P180D",
+ *       "deleteWhere": {
+ *         "type": "not",
+ *         "field": {
+ *           {
+ *             "type": "inType",
+ *             "column": "priority",
+ *             "matchValueType": "STRING",
+ *             "sortedValues": ["low", "spam"]
+ *           }
+ *         }
+ *       },
+ *       "description": "Remove low-priority data from segments older than 180 days"
+ *     }
+ *   ]
+ * }
+ * }
+ */ +public class InlineReindexingRuleProvider implements ReindexingRuleProvider +{ + public static final String TYPE = "inline"; + + private final List deletionRules; + private final List ioConfigRules; + private final List segmentGranularityRules; + private final List tuningConfigRules; + private final List dataSchemaRules; + + + @JsonCreator + public InlineReindexingRuleProvider( + @JsonProperty("deletionRules") @Nullable List deletionRules, + @JsonProperty("ioConfigRules") @Nullable List ioConfigRules, + @JsonProperty("segmentGranularityRules") @Nullable List segmentGranularityRules, + @JsonProperty("tuningConfigRules") @Nullable List tuningConfigRules, + @JsonProperty("dataSchemaRules") @Nullable List dataSchemaRules + ) + { + this.deletionRules = Configs.valueOrDefault(deletionRules, Collections.emptyList()); + this.ioConfigRules = Configs.valueOrDefault(ioConfigRules, Collections.emptyList()); + this.segmentGranularityRules = Configs.valueOrDefault(segmentGranularityRules, Collections.emptyList()); + this.tuningConfigRules = Configs.valueOrDefault(tuningConfigRules, Collections.emptyList()); + this.dataSchemaRules = Configs.valueOrDefault(dataSchemaRules, Collections.emptyList()); + } + + public static Builder builder() + { + return new Builder(); + } + + @Override + @JsonProperty("type") + public String getType() + { + return TYPE; + } + + @Override + @JsonProperty("deletionRules") + public List getDeletionRules() + { + return deletionRules; + } + + @Override + @Nullable + public ReindexingDataSchemaRule getDataSchemaRule(Interval interval, DateTime referenceTime) + { + return getApplicableRule(dataSchemaRules, interval, referenceTime); + } + + @Override + @JsonProperty("dataSchemaRules") + public List getDataSchemaRules() + { + return dataSchemaRules; + } + + @Override + @JsonProperty("ioConfigRules") + public List getIOConfigRules() + { + return ioConfigRules; + } + + @Override + @JsonProperty("segmentGranularityRules") + public List getSegmentGranularityRules() + { + return segmentGranularityRules; + } + + @Override + @JsonProperty("tuningConfigRules") + public List getTuningConfigRules() + { + return tuningConfigRules; + } + + @Override + public List getDeletionRules(Interval interval, DateTime referenceTime) + { + return getApplicableRules(deletionRules, interval, referenceTime); + } + + @Override + @Nullable + public ReindexingIOConfigRule getIOConfigRule(Interval interval, DateTime referenceTime) + { + return getApplicableRule(ioConfigRules, interval, referenceTime); + } + + @Override + @Nullable + public ReindexingSegmentGranularityRule getSegmentGranularityRule( + Interval interval, + DateTime referenceTime + ) + { + return getApplicableRule(segmentGranularityRules, interval, referenceTime); + } + + @Override + @Nullable + public ReindexingTuningConfigRule getTuningConfigRule(Interval interval, DateTime referenceTime) + { + return getApplicableRule(tuningConfigRules, interval, referenceTime); + } + + /** + * Returns the list of rules that apply to the given interval. + *

+ * This provider implementation only returns rules that fully apply to the given interval. + *

+ */ + private List getApplicableRules(List rules, Interval interval, DateTime referenceTime) + { + List applicableRules = new ArrayList<>(); + for (T rule : rules) { + if (rule.appliesTo(interval, referenceTime) == ReindexingRule.AppliesToMode.FULL) { + applicableRules.add(rule); + } + } + return applicableRules; + } + + /** + * Returns the single most applicable rule for the given interval. + *

+ * "most applicable" means if multiple rules match, the one returned is the one with the oldest + * threshold (i.e., the largest period into the past from "now"). + */ + @Nullable + private T getApplicableRule(List rules, Interval interval, DateTime referenceTime) + { + List applicableRules = new ArrayList<>(); + for (T rule : rules) { + if (rule.appliesTo(interval, referenceTime) == ReindexingRule.AppliesToMode.FULL) { + applicableRules.add(rule); + } + } + + if (applicableRules.isEmpty()) { + return null; + } + + return Collections.min( + applicableRules, + Comparator.comparingLong(r -> { + DateTime threshold = referenceTime.minus(r.getOlderThan()); + return threshold.getMillis(); + }) + ); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + InlineReindexingRuleProvider that = (InlineReindexingRuleProvider) o; + return Objects.equals(deletionRules, that.deletionRules) + && Objects.equals(ioConfigRules, that.ioConfigRules) + && Objects.equals(segmentGranularityRules, that.segmentGranularityRules) + && Objects.equals(tuningConfigRules, that.tuningConfigRules) + && Objects.equals(dataSchemaRules, that.dataSchemaRules); + } + + @Override + public int hashCode() + { + return Objects.hash( + deletionRules, + ioConfigRules, + segmentGranularityRules, + tuningConfigRules, + dataSchemaRules + ); + } + + @Override + public String toString() + { + return "InlineReindexingRuleProvider{" + + "deletionRules=" + deletionRules + + ", ioConfigRules=" + ioConfigRules + + ", segmentGranularityRules=" + segmentGranularityRules + + ", tuningConfigRules=" + tuningConfigRules + + ", dataSchemaRules=" + dataSchemaRules + + '}'; + } + + public static class Builder + { + private List deletionRules; + private List ioConfigRules; + private List segmentGranularityRules; + private List tuningConfigRules; + private List dataSchemaRules; + + public Builder deletionRules(List deletionRules) + { + this.deletionRules = deletionRules; + return this; + } + + public Builder dataSchemaRules(List dataSchemaRules) + { + this.dataSchemaRules = dataSchemaRules; + return this; + } + + public Builder ioConfigRules(List ioConfigRules) + { + this.ioConfigRules = ioConfigRules; + return this; + } + + public Builder segmentGranularityRules(List segmentGranularityRules) + { + this.segmentGranularityRules = segmentGranularityRules; + return this; + } + + public Builder tuningConfigRules(List tuningConfigRules) + { + this.tuningConfigRules = tuningConfigRules; + return this; + } + + public InlineReindexingRuleProvider build() + { + return new InlineReindexingRuleProvider( + deletionRules, + ioConfigRules, + segmentGranularityRules, + tuningConfigRules, + dataSchemaRules + ); + } + } +} diff --git a/server/src/main/java/org/apache/druid/server/compaction/IntervalGranularityInfo.java b/server/src/main/java/org/apache/druid/server/compaction/IntervalGranularityInfo.java new file mode 100644 index 000000000000..691010a4769e --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/compaction/IntervalGranularityInfo.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import org.apache.druid.java.util.common.granularity.Granularity; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Associates a time interval with its segment granularity and optional source rule. + * Used to pass synthetic timeline information from timeline generation to config building. + */ +public class IntervalGranularityInfo +{ + private final Interval interval; + private final Granularity granularity; + private final ReindexingSegmentGranularityRule sourceRule; + + public IntervalGranularityInfo( + Interval interval, + Granularity granularity, + @Nullable ReindexingSegmentGranularityRule sourceRule + ) + { + this.interval = interval; + this.granularity = granularity; + this.sourceRule = sourceRule; + } + + public Interval getInterval() + { + return interval; + } + + public Granularity getGranularity() + { + return granularity; + } + + @Nullable + public ReindexingSegmentGranularityRule getSourceRule() + { + return sourceRule; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + IntervalGranularityInfo that = (IntervalGranularityInfo) o; + return Objects.equals(interval, that.interval) + && Objects.equals(granularity, that.granularity) + && Objects.equals(sourceRule, that.sourceRule); + } + + @Override + public int hashCode() + { + return Objects.hash(interval, granularity, sourceRule); + } + + @Override + public String toString() + { + return "IntervalGranularityInfo{" + + "interval=" + interval + + ", granularity=" + granularity + + ", sourceRule=" + (sourceRule != null ? sourceRule.getId() : "null") + + '}'; + } +} diff --git a/server/src/main/java/org/apache/druid/server/compaction/ReindexingDataSchemaRule.java b/server/src/main/java/org/apache/druid/server/compaction/ReindexingDataSchemaRule.java new file mode 100644 index 000000000000..6ed678245ccf --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/compaction/ReindexingDataSchemaRule.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.server.coordinator.UserCompactionTaskDimensionsConfig; +import org.joda.time.Period; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; + +/** + * A {@link ReindexingRule} that specifies a data schema for reindexing tasks to configure. + *

+ * This rule allows users to specify dimensionsspec metricsspec, query granularity, rollup, and projections for reindexing tasks to apply + *

+ * This is a non-additive rule. Multiple data schema rules cannot be applied to the same interval being reindexed. + *

+ * Example inline usage: + *

{@code
+ * {
+ *   "id": "change-query-granularity-30d",
+ *   "olderThan": "P30D",
+ *   "queryGranularity": "HOUR",
+ *   "rollup": true
+ * }
+ */ +public class ReindexingDataSchemaRule extends AbstractReindexingRule +{ + private final UserCompactionTaskDimensionsConfig dimensionsSpec; + private final AggregatorFactory[] metricsSpec; + private final Granularity queryGranularity; + private final Boolean rollup; + private final List projections; + + public ReindexingDataSchemaRule( + @JsonProperty("id") @Nonnull String id, + @JsonProperty("description") @Nullable String description, + @JsonProperty("olderThan") @Nonnull Period olderThan, + @JsonProperty("dimensionsSpec") @Nullable UserCompactionTaskDimensionsConfig dimensionsSpec, + @JsonProperty("metricsSpec") @Nullable AggregatorFactory[] metricsSpec, + @JsonProperty("queryGranularity") @Nullable Granularity queryGranularity, + @JsonProperty("rollup") @Nullable Boolean rollup, + @JsonProperty("projections") @Nullable List projections + ) + { + super(id, description, olderThan); + InvalidInput.conditionalException( + (dimensionsSpec != null || metricsSpec != null || queryGranularity != null || rollup != null || projections != null), + "At least oe of 'dimensionsSpec', 'metricsSpec', 'queryGranularity', 'rollup' or 'projections' must be non-null" + ); + this.dimensionsSpec = dimensionsSpec; + this.metricsSpec = metricsSpec; + this.queryGranularity = queryGranularity; + this.rollup = rollup; + this.projections = projections; + } + + @JsonProperty + public UserCompactionTaskDimensionsConfig getDimensionsSpec() + { + return dimensionsSpec; + } + + @JsonProperty + public List getProjections() + { + return projections; + } + + @JsonProperty + public AggregatorFactory[] getMetricsSpec() + { + return metricsSpec; + } + + @JsonProperty + public Granularity getQueryGranularity() + { + return queryGranularity; + } + + @JsonProperty + public Boolean getRollup() + { + return rollup; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ReindexingDataSchemaRule that = (ReindexingDataSchemaRule) o; + return Objects.equals(getId(), that.getId()) + && Objects.equals(getDescription(), that.getDescription()) + && Objects.equals(getOlderThan(), that.getOlderThan()) + && Objects.equals(dimensionsSpec, that.dimensionsSpec) + && Objects.deepEquals(metricsSpec, that.metricsSpec) + && Objects.equals(queryGranularity, that.queryGranularity) + && Objects.equals(rollup, that.rollup) + && Objects.equals(projections, that.projections); + } + + @Override + public int hashCode() + { + int result = Objects.hash( + getId(), + getDescription(), + getOlderThan(), + dimensionsSpec, + queryGranularity, + rollup, + projections + ); + result = 31 * result + Arrays.hashCode(metricsSpec); + return result; + } + + @Override + public String toString() + { + return "ReindexingDataSchemaRule{" + + "id='" + getId() + '\'' + + ", description='" + getDescription() + '\'' + + ", olderThan=" + getOlderThan() + + ", dimensionsSpec=" + dimensionsSpec + + ", metricsSpec=" + Arrays.toString(metricsSpec) + + ", queryGranularity=" + queryGranularity + + ", rollup=" + rollup + + ", projections=" + projections + + '}'; + } +} diff --git a/server/src/main/java/org/apache/druid/server/compaction/ReindexingDeletionRule.java b/server/src/main/java/org/apache/druid/server/compaction/ReindexingDeletionRule.java new file mode 100644 index 000000000000..a6b40aa40d62 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/compaction/ReindexingDeletionRule.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.segment.VirtualColumns; +import org.joda.time.Period; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * A {@link ReindexingRule} that specifies patterns to match for removing rows during reindexing. + *

+ * The {@link #deleteWhere} clause defines rows to REMOVE from reindexed segments. For example, a `deleteWhere` {@link DimFilter} + * {@code selector(isRobot=true)} with {@link AbstractReindexingRule#olderThan} P90D will "remove rows where isRobot=true + * from data older than 90 days". The reindexing framework automatically wraps these delete where clauses in NOT logic during + * processing. + *

+ * This is an additive rule. Multiple rules can apply to the same segment. When multiple rules apply, they are combined + * as NOT(A OR B OR C) where A, B, and C come from distinct {@link ReindexingDeletionRule}s. + *

+ * Example usage: + *

{@code
+ * {
+ *   "id": "remove-robots-90d",
+ *   "olderThan": "P90D",
+ *   "deleteWhere": {
+ *     "type": "selector",
+ *     "dimension": "isRobot",
+ *     "value": "true"
+ *   },
+ *   "description": "Remove robot traffic from data older than 90 days"
+ * }
+ * }
+ *

+ * Virtual column support for filtering on nested fields (MSQ engine only): + *

+ * It is important to note that when using virtual columns in the filter, the virtual columns must be defined + * with unique names. Users will have to take care to ensure a rule always has the same unique virtual column names + * to not impact the fingerprinting of segments reindexed with the rule. + *

+ * Example inline useage with virtual column: + *

{@code
+ * {
+ *   "id": "remove-using-nested-field-filter",
+ *   "olderThan": "P90D",
+ *   "deleteWhere": {
+ *     "type": "selector",
+ *     "dimension": "extractedField",
+ *     "value": "unwantedValue"
+ *   },
+ *   "virtualColumns": [
+ *     {
+ *       "type": "expression",
+ *       "name": "extractedField",
+ *       "expression": "json_value(metadata, '$.category')",
+ *       "outputType": "STRING"
+ *     }
+ *   ],
+ *   "description": "Remove rows where metadata.category = 'unwantedValue' from segments older than 90 days"
+ * }
+ * }
+ */ +public class ReindexingDeletionRule extends AbstractReindexingRule +{ + private final DimFilter deleteWhere; + private final VirtualColumns virtualColumns; + + @JsonCreator + public ReindexingDeletionRule( + @JsonProperty("id") @Nonnull String id, + @JsonProperty("description") @Nullable String description, + @JsonProperty("olderThan") @Nonnull Period olderThan, + @JsonProperty("deleteWhere") @Nonnull DimFilter deleteWhere, + @JsonProperty("virtualColumns") @Nullable VirtualColumns virtualColumns + ) + { + super(id, description, olderThan); + InvalidInput.conditionalException(deleteWhere != null, "'deleteWhere' cannot be null"); + this.deleteWhere = deleteWhere; + this.virtualColumns = virtualColumns; + } + + @JsonProperty + public DimFilter getDeleteWhere() + { + return deleteWhere; + } + + @JsonProperty + @Nullable + public VirtualColumns getVirtualColumns() + { + return virtualColumns; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ReindexingDeletionRule that = (ReindexingDeletionRule) o; + return Objects.equals(getId(), that.getId()) + && Objects.equals(getDescription(), that.getDescription()) + && Objects.equals(getOlderThan(), that.getOlderThan()) + && Objects.equals(deleteWhere, that.deleteWhere) + && Objects.equals(virtualColumns, that.virtualColumns); + } + + @Override + public int hashCode() + { + return Objects.hash( + getId(), + getDescription(), + getOlderThan(), + deleteWhere, + virtualColumns + ); + } + + @Override + public String toString() + { + return "ReindexingDeletionRule{" + + "id='" + getId() + '\'' + + ", description='" + getDescription() + '\'' + + ", olderThan=" + getOlderThan() + + ", deleteWhere=" + deleteWhere + + ", virtualColumns=" + virtualColumns + + '}'; + } +} diff --git a/server/src/main/java/org/apache/druid/server/compaction/ReindexingIOConfigRule.java b/server/src/main/java/org/apache/druid/server/compaction/ReindexingIOConfigRule.java new file mode 100644 index 000000000000..1c34c269133a --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/compaction/ReindexingIOConfigRule.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; +import org.joda.time.Period; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * A {@link ReindexingRule} that specifies a {@link UserCompactionTaskIOConfig} for tasks to configure. + *

+ * This is a non-additive rule. Multiple IO config rules cannot be applied to the same interval safely, + * as a compaction job can only use one IO configuration. + *

+ * Example inline usage: + *

{@code
+ * {
+ *   "id": "dropExistingFalse-false",
+ *   "olderThan": "P90D",
+ *   "ioConfig": {
+ *     "dropExisting": false
+ *   },
+ * }
+ * }
+ */ +public class ReindexingIOConfigRule extends AbstractReindexingRule +{ + private final UserCompactionTaskIOConfig ioConfig; + + @JsonCreator + public ReindexingIOConfigRule( + @JsonProperty("id") @Nonnull String id, + @JsonProperty("description") @Nullable String description, + @JsonProperty("olderThan") @Nonnull Period olderThan, + @JsonProperty("ioConfig") @Nonnull UserCompactionTaskIOConfig ioConfig + ) + { + super(id, description, olderThan); + InvalidInput.conditionalException(ioConfig != null, "'ioConfig' cannot be null"); + this.ioConfig = ioConfig; + } + + @JsonProperty + public UserCompactionTaskIOConfig getIoConfig() + { + return ioConfig; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ReindexingIOConfigRule that = (ReindexingIOConfigRule) o; + return Objects.equals(getId(), that.getId()) + && Objects.equals(getDescription(), that.getDescription()) + && Objects.equals(getOlderThan(), that.getOlderThan()) + && Objects.equals(ioConfig, that.ioConfig); + } + + @Override + public int hashCode() + { + return Objects.hash( + getId(), + getDescription(), + getOlderThan(), + ioConfig + ); + } + + @Override + public String toString() + { + return "ReindexingIOConfigRule{" + + "id='" + getId() + '\'' + + ", description='" + getDescription() + '\'' + + ", olderThan=" + getOlderThan() + + ", ioConfig=" + ioConfig + + '}'; + } +} diff --git a/server/src/main/java/org/apache/druid/server/compaction/ReindexingRule.java b/server/src/main/java/org/apache/druid/server/compaction/ReindexingRule.java new file mode 100644 index 000000000000..e2ed8ae42236 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/compaction/ReindexingRule.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; + +import javax.annotation.Nullable; + +/** + * Defines a reindexing configuration that applies to data based on age thresholds. + *

+ * Rules encapsulate specific aspects of reindexing (granularity, filters, tuning, etc.) + * and specify when they should apply via a {@link Period} which defines the age threshold for applicability. + *

+ * Rules conditionally apply to data "older than" the rules threshold relative to the time of rule evaluation. + */ +public interface ReindexingRule +{ + /** + * Indicates how a rule applies to a given time interval based on the rule's period threshold. + *

    + *
  • PARTIAL: The rule applies to part of the interval.
  • + *
  • FULL: The rule applies to the entire interval.
  • + *
  • NONE: The rule does not apply to the interval at all.
  • + *
+ */ + enum AppliesToMode + { + PARTIAL, + FULL, + NONE + } + + /** + * Provides an identifier for the rule, which can be used for referencing, logging, or management purposes. + *

+ * Note that there is no inherent contract for uniqueness across rule sets provided by this interface. + */ + String getId(); + + /** + * Provides a human-readable description of the rule, which can be used for logging, debugging, or user interfaces to explain the purpose and behavior of the rule. + */ + String getDescription(); + + /** + * Defines the age threshold for rule applicability. The rule applies to data older than a reference time minus this period. + *

+ * For example, if the period is "P30D" (30 days) and the reference time is "2024-01-31T00:00:00Z", the rule applies to data older than "2024-01-01T00:00:00Z". + * @return The period representing the age threshold for rule applicability. + */ + Period getOlderThan(); + + /** + * Check if this rule applies to the given interval. + *

+ *

    + *
  • If the rule applies to the entire interval, return {@link AppliesToMode#FULL}.
  • + *
  • If the rule applies to only part of the interval, return {@link AppliesToMode#PARTIAL}.
  • + *
  • If the rule does not apply to the interval at all, return {@link AppliesToMode#NONE}.
  • + *
+ * @param interval The interval to check applicability against. + * @param referenceTime The reference time to use for period calculations. null results in using current UTC instant at runtime. + * @return The applicability mode of the rule for the given interval and reference time. + */ + AppliesToMode appliesTo(Interval interval, @Nullable DateTime referenceTime); + +} diff --git a/server/src/main/java/org/apache/druid/server/compaction/ReindexingRuleProvider.java b/server/src/main/java/org/apache/druid/server/compaction/ReindexingRuleProvider.java new file mode 100644 index 000000000000..be617169ec5b --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/compaction/ReindexingRuleProvider.java @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.stream.Stream; + +/** + * Provides compaction rules for different aspects of reindexing configuration. + *

+ * This abstraction allows rules to be sourced from different locations: inline definitions, + * database storage, external services, or dynamically generated based on metrics. Each method + * returns rules for a specific reindexing aspect (granularity, filters, tuning, etc.), either + * for all rules or filtered by interval applicability. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = InlineReindexingRuleProvider.TYPE, value = InlineReindexingRuleProvider.class), + @JsonSubTypes.Type(name = ComposingReindexingRuleProvider.TYPE, value = ComposingReindexingRuleProvider.class) +}) +public interface ReindexingRuleProvider +{ + /** + * Returns the type identifier for this provider implementation. + *

+ * This value is used in JSON serialization to identify which provider implementation + * to use when deserializing. + * + * @return the type identifier (e.g., "inline", "external") + */ + String getType(); + + /** + * Returns true if this provider is ready to supply rules. + *

+ * Providers that depend on external state (HTTP services, databases) should return false + * until they have successfully initialized and loaded their rules. Reindexing supervisors + * should check this before generating tasks to avoid creating tasks with incomplete rule sets. + *

+ * The default implementation returns true, which is appropriate for providers that have + * their rules available immediately (such as inline providers with static configuration). + * + * @return true if the provider is ready to supply rules, false otherwise + */ + default boolean isReady() + { + return true; + } + + /** + * Returns all reindexing deletion rules that apply to the given interval. + *

+ * Handling partial overlaps is the responsibility of the provider implementation and should be clearly documented. + *

+ * @param interval The interval to check applicability against. + * @param referenceTime The reference time to use for period calculations while determining rule applicability for an interval. + * e.g., a rule with period P7D applies to data older than 7 days from the reference time. + * @return The list of {@link ReindexingDeletionRule} rules that apply to the given interval. + */ + List getDeletionRules(Interval interval, DateTime referenceTime); + + /** + * Returns ALL reindexing deletion rules. + */ + List getDeletionRules(); + + /** + * Returns the matched reindexing data schema rule that applies to the given interval. + *

+ * Handling cases of multiple applicable rules and/orpartial overlaps is the responsibility of the provider + * implementation and should be clearly documented. + *

+ * + * @param interval The interval to check applicability against. + * @param referenceTime The reference time to use for period calculations while determining rule applicability for an interval. + * e.g., a rule with period P7D applies to data older than 7 days from the reference time. + * @return {@link ReindexingDataSchemaRule} rule that applies to the given interval. + */ + @Nullable + ReindexingDataSchemaRule getDataSchemaRule(Interval interval, DateTime referenceTime); + + /** + * Returns ALL reindexing data schema rules. + */ + List getDataSchemaRules(); + + /** + * Returns the matched reindexing IO config rule that applies to the given interval. + *

+ * Handling cases of multiple applicable rules and/or partial overlaps is the responsibility of the provider + * implementation and should be clearly documented. + *

+ * + * @param interval The interval to check applicability against. + * @param referenceTime The reference time to use for period calculations while determining rule applicability for an interval. + * e.g., a rule with period P7D applies to data older than 7 days from the reference time. + * @return {@link ReindexingIOConfigRule} that applies to the given interval. + */ + @Nullable + ReindexingIOConfigRule getIOConfigRule(Interval interval, DateTime referenceTime); + + /** + * Returns ALL reindexing IO config rules. + */ + List getIOConfigRules(); + + /** + * Returns the matched reindexing segment granularity rule that applies to the given interval. + *

+ * Handling cases of multiple applicable rules and/or partial overlaps is the responsibility of the provider + * implementation and should be clearly documented. + *

+ * + * @param interval The interval to check applicability against. + * @param referenceTime The reference time to use for period calculations while determining rule applicability for an interval. + * e.g., a rule with period P7D applies to data older than 7 days from the reference time. + * @return {@link ReindexingSegmentGranularityRule} rule that applies to the given interval. + */ + @Nullable + ReindexingSegmentGranularityRule getSegmentGranularityRule(Interval interval, DateTime referenceTime); + + /** + * Returns ALL reindexing segment granularity rules. + */ + List getSegmentGranularityRules(); + + /** + * Returns the matched reindexing tuning config rule that applies to the given interval. + *

+ * Handling cases of multiple applicable rules and/or partial overlaps is the responsibility of the provider + * implementation and should be clearly documented. + *

+ * + * @param interval The interval to check applicability against. + * @param referenceTime The reference time to use for period calculations while determining rule applicability for an interval. + * e.g., a rule with period P7D applies to data older than 7 days from the reference time. + */ + @Nullable + ReindexingTuningConfigRule getTuningConfigRule(Interval interval, DateTime referenceTime); + + /** + * Returns ALL reindexing tuning config rules. + */ + List getTuningConfigRules(); + + /** + * Returns a stream of all reindexing rules across all types. + *

+ * This provides a flexible way to filter, map, and process rules without needing + * specific methods for every possible combination. For example, to get all non-segment-granularity + * rules, you can filter: {@code streamAllRules().filter(rule -> !(rule instanceof ReindexingSegmentGranularityRule))} + * + * @return a stream of all rules from all rule types + */ + default Stream streamAllRules() + { + return Stream.of( + getIOConfigRules().stream(), + getTuningConfigRules().stream(), + getDeletionRules().stream(), + getSegmentGranularityRules().stream(), + getDataSchemaRules().stream() + ).flatMap(s -> s); + } +} diff --git a/server/src/main/java/org/apache/druid/server/compaction/ReindexingSegmentGranularityRule.java b/server/src/main/java/org/apache/druid/server/compaction/ReindexingSegmentGranularityRule.java new file mode 100644 index 000000000000..47bcc2b59116 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/compaction/ReindexingSegmentGranularityRule.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.joda.time.Period; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.List; +import java.util.Objects; + +/** + * A {@link ReindexingRule} that specifies a segment granularity for reindexing tasks to configure. + *

+ * This rule controls how time-series data is bucketed into segments during reindexing. For example, changing from + * 15-minute segments to hourly segments reduces segment count. There is a strict allow list of supported granularities + * to prevent misconfiguration. + *

+ * This is a non-additive rule. Multiple segment granularity rules cannot be applied to the same segment. + *

+ * Example inline usage: + *

{@code
+ * {
+ *     "id": "daily-30d",
+ *     "olderThan": "P30D",
+ *     "segmentGranularity": "DAY"
+ *     "description": "Compact to daily segments for data older than 30 days"
+ * }
+ * }
+ */ +public class ReindexingSegmentGranularityRule extends AbstractReindexingRule +{ + private static final List SUPPORTED_SEGMENT_GRANULARITIES = List.of( + Granularities.MINUTE, + Granularities.FIFTEEN_MINUTE, + Granularities.HOUR, + Granularities.DAY, + Granularities.MONTH, + Granularities.QUARTER, + Granularities.YEAR + ); + + private final Granularity segmentGranularity; + + @JsonCreator + public ReindexingSegmentGranularityRule( + @JsonProperty("id") @Nonnull String id, + @JsonProperty("description") @Nullable String description, + @JsonProperty("olderThan") @Nonnull Period olderThan, + @JsonProperty("segmentGranularity") @Nonnull Granularity segmentGranularity + ) + { + super(id, description, olderThan); + InvalidInput.conditionalException( + SUPPORTED_SEGMENT_GRANULARITIES.contains(segmentGranularity), + "Unsupported segment granularity [%s]. Supported values are: MINUTE, FIFTEEN_MINUTE, HOUR, DAY, MONTH, QUARTER, YEAR", + segmentGranularity + ); + this.segmentGranularity = segmentGranularity; + } + + @JsonProperty + public Granularity getSegmentGranularity() + { + return segmentGranularity; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ReindexingSegmentGranularityRule that = (ReindexingSegmentGranularityRule) o; + return Objects.equals(getId(), that.getId()) + && Objects.equals(getDescription(), that.getDescription()) + && Objects.equals(getOlderThan(), that.getOlderThan()) + && Objects.equals(segmentGranularity, that.segmentGranularity); + } + + @Override + public int hashCode() + { + return Objects.hash( + getId(), + getDescription(), + getOlderThan(), + segmentGranularity + ); + } + + @Override + public String toString() + { + return "ReindexingSegmentGranularityRule{" + + "id='" + getId() + '\'' + + ", description='" + getDescription() + '\'' + + ", olderThan=" + getOlderThan() + + ", segmentGranularity=" + segmentGranularity + + '}'; + } + +} diff --git a/server/src/main/java/org/apache/druid/server/compaction/ReindexingTuningConfigRule.java b/server/src/main/java/org/apache/druid/server/compaction/ReindexingTuningConfigRule.java new file mode 100644 index 000000000000..e3bc98f3b0d8 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/compaction/ReindexingTuningConfigRule.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.segment.indexing.TuningConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; +import org.joda.time.Period; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * A {@link ReindexingRule} that specifies a {@link TuningConfig} for tasks to configure. + *

+ * This rule controls things like partitioning strategy. For example, applying range partitioning over specific + * dimensions to older data can optimize query performance for common access patterns. + *

+ * This is a non-additive rule. Multiple tuning config rules cannot be applied to the same interval, as a compaction + * job can only use one tuning configuration. + *

+ * Example inline usage: + *

{@code
+ * {
+ *   "id": "range-partition-30d",
+ *   "olderThan": "P30D",
+ *   "tuningConfig": {
+ *     "partitionsSpec": {
+ *       "type": "range",
+ *       "targetRowsPerSegment": 5000000,
+ *       "partitionDimensions": ["country", "city"]
+ *     }
+ *   },
+ *   "description": "Use range partitioning for data older than 30 days"
+ * }
+ * }
+ */ +public class ReindexingTuningConfigRule extends AbstractReindexingRule +{ + private final UserCompactionTaskQueryTuningConfig tuningConfig; + + @JsonCreator + public ReindexingTuningConfigRule( + @JsonProperty("id") @Nonnull String id, + @JsonProperty("description") @Nullable String description, + @JsonProperty("olderThan") @Nonnull Period olderThan, + @JsonProperty("tuningConfig") @Nonnull UserCompactionTaskQueryTuningConfig tuningConfig + ) + { + super(id, description, olderThan); + InvalidInput.conditionalException(tuningConfig != null, "'tuningConfig' cannot be null"); + this.tuningConfig = tuningConfig; + } + + @JsonProperty + public UserCompactionTaskQueryTuningConfig getTuningConfig() + { + return tuningConfig; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ReindexingTuningConfigRule that = (ReindexingTuningConfigRule) o; + return Objects.equals(getId(), that.getId()) + && Objects.equals(getDescription(), that.getDescription()) + && Objects.equals(getOlderThan(), that.getOlderThan()) + && Objects.equals(tuningConfig, that.tuningConfig); + } + + @Override + public int hashCode() + { + return Objects.hash( + getId(), + getDescription(), + getOlderThan(), + tuningConfig + ); + } + + @Override + public String toString() + { + return "ReindexingTuningConfigRule{" + + "id='" + getId() + '\'' + + ", description='" + getDescription() + '\'' + + ", olderThan=" + getOlderThan() + + ", tuningConfig=" + tuningConfig + + '}'; + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/InlineSchemaDataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/InlineSchemaDataSourceCompactionConfig.java index b88c4a0bcaf4..58a7206b8ab8 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/InlineSchemaDataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/InlineSchemaDataSourceCompactionConfig.java @@ -276,6 +276,29 @@ public int hashCode() return result; } + /** + * Creates a builder initialized with all fields from this config. + * Useful for creating modified copies of an existing config. + */ + public Builder toBuilder() + { + return new Builder() + .forDataSource(this.dataSource) + .withTaskPriority(this.taskPriority) + .withInputSegmentSizeBytes(this.inputSegmentSizeBytes) + .withMaxRowsPerSegment(this.maxRowsPerSegment) + .withSkipOffsetFromLatest(this.skipOffsetFromLatest) + .withTuningConfig(this.tuningConfig) + .withGranularitySpec(this.granularitySpec) + .withDimensionsSpec(this.dimensionsSpec) + .withMetricsSpec(this.metricsSpec) + .withTransformSpec(this.transformSpec) + .withProjections(this.projections) + .withIoConfig(this.ioConfig) + .withEngine(this.engine) + .withTaskContext(this.taskContext); + } + public static class Builder { private String dataSource; @@ -360,6 +383,34 @@ public Builder withGranularitySpec( return this; } + public Builder withSegmentGranularity(Granularity segmentGranularity) + { + if (this.granularitySpec == null) { + this.granularitySpec = new UserCompactionTaskGranularityConfig(segmentGranularity, null, null); + } else { + this.granularitySpec = new UserCompactionTaskGranularityConfig( + segmentGranularity, + this.granularitySpec.getQueryGranularity(), + this.granularitySpec.isRollup() + ); + } + return this; + } + + public Builder withQueryGranularityAndRollup(Granularity queryGranularity, Boolean rollup) + { + if (this.granularitySpec == null) { + this.granularitySpec = new UserCompactionTaskGranularityConfig(null, queryGranularity, rollup); + } else { + this.granularitySpec = new UserCompactionTaskGranularityConfig( + this.granularitySpec.getSegmentGranularity(), + queryGranularity, + rollup + ); + } + return this; + } + public Builder withDimensionsSpec( UserCompactionTaskDimensionsConfig dimensionsSpec ) diff --git a/server/src/test/java/org/apache/druid/server/compaction/AbstractReindexingRuleTest.java b/server/src/test/java/org/apache/druid/server/compaction/AbstractReindexingRuleTest.java new file mode 100644 index 000000000000..e6b55c406bed --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/compaction/AbstractReindexingRuleTest.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import org.apache.druid.query.filter.SelectorDimFilter; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class AbstractReindexingRuleTest +{ + @Test + public void test_constructor_positiveMonthsNegativeDays_throwsException() + { + Period period = Period.months(1).withDays(-40); + + Assertions.assertThrows( + IllegalArgumentException.class, + () -> new ReindexingDeletionRule( + "test-rule", + null, + period, + new SelectorDimFilter("dim", "val", null), + null + ) + ); + } + + @Test + public void test_constructor_positiveYearsNegativeMonths_throwsException() + { + Period period = new Period(1, -13, 0, 0, 0, 0, 0, 0); + + Assertions.assertThrows( + IllegalArgumentException.class, + () -> new ReindexingDeletionRule( + "test-rule", + null, + period, + new SelectorDimFilter("dim", "val", null), + null + ) + ); + } +} diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java index 2b274e805d4c..370dcaf444d5 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java @@ -35,16 +35,22 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.TestDataSource; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.metadata.DefaultIndexingStateFingerprintMapper; import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; +import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskDimensionsConfig; @@ -474,6 +480,107 @@ public void testStatusWhenProjectionsMismatch() Assert.assertFalse(status.isComplete()); } + @Test + public void testStatusWhenTransformSpecVirtualColumnsMatch() + { + ExpressionVirtualColumn vc = new ExpressionVirtualColumn( + "extractedField", "concat(metadata, '_category')", ColumnType.STRING, ExprMacroTable.nil() + ); + CompactionTransformSpec transformSpec = new CompactionTransformSpec( + new SelectorDimFilter("extractedField", "foo", null), + VirtualColumns.create(vc) + ); + CompactionState lastCompactionState = new CompactionState( + null, + null, + null, + transformSpec, + IndexSpec.getDefault(), + null, + null + ); + DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withTransformSpec(transformSpec) + .build(); + + DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); + CompactionStatus status = CompactionStatus.compute( + CompactionCandidate.from(List.of(segment), null), compactionConfig, fingerprintMapper + ); + Assert.assertTrue(status.isComplete()); + } + + @Test + public void testStatusWhenTransformSpecVirtualColumnsMismatch() + { + SelectorDimFilter filter = new SelectorDimFilter("extractedField", "foo", null); + ExpressionVirtualColumn oldVc = new ExpressionVirtualColumn( + "extractedField", "concat(metadata, '_old')", ColumnType.STRING, ExprMacroTable.nil() + ); + ExpressionVirtualColumn newVc = new ExpressionVirtualColumn( + "extractedField", "concat(metadata, '_new')", ColumnType.STRING, ExprMacroTable.nil() + ); + + CompactionState lastCompactionState = new CompactionState( + null, + null, + null, + new CompactionTransformSpec(filter, VirtualColumns.create(oldVc)), + IndexSpec.getDefault(), + null, + null + ); + DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withTransformSpec(new CompactionTransformSpec(filter, VirtualColumns.create(newVc))) + .build(); + + DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); + CompactionStatus status = CompactionStatus.compute( + CompactionCandidate.from(List.of(segment), null), compactionConfig, fingerprintMapper + ); + Assert.assertFalse(status.isComplete()); + Assert.assertTrue(status.getReason().startsWith("'transformSpec' mismatch")); + } + + @Test + public void test_evaluate_needsCompactionWhenMismatchedFingerprintStateHasDifferentVirtualColumns() + { + SelectorDimFilter filter = new SelectorDimFilter("extractedField", "foo", null); + ExpressionVirtualColumn vc = new ExpressionVirtualColumn( + "extractedField", "concat(metadata, '_category')", ColumnType.STRING, ExprMacroTable.nil() + ); + + DataSourceCompactionConfig oldConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withTransformSpec(new CompactionTransformSpec(filter, null)) + .build(); + CompactionState oldState = oldConfig.toCompactionState(); + String oldFingerprint = fingerprintMapper.generateFingerprint(TestDataSource.WIKI, oldState); + + DataSourceCompactionConfig newConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withTransformSpec(new CompactionTransformSpec(filter, VirtualColumns.create(vc))) + .build(); + + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, oldFingerprint, oldState, DateTimes.nowUtc()); + syncCacheFromManager(); + + List segments = List.of( + DataSegment.builder(WIKI_SEGMENT).indexingStateFingerprint(oldFingerprint).build() + ); + CompactionStatus status = CompactionStatus.compute( + CompactionCandidate.from(segments, null), newConfig, fingerprintMapper + ); + Assert.assertFalse(status.isComplete()); + Assert.assertTrue(status.getReason().startsWith("'transformSpec' mismatch")); + } + @Test public void testStatusWhenAutoSchemaMatch() { diff --git a/server/src/test/java/org/apache/druid/server/compaction/ComposingReindexingRuleProviderTest.java b/server/src/test/java/org/apache/druid/server/compaction/ComposingReindexingRuleProviderTest.java new file mode 100644 index 000000000000..03ce574273e1 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/compaction/ComposingReindexingRuleProviderTest.java @@ -0,0 +1,503 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.server.coordinator.UserCompactionTaskDimensionsConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.function.BiFunction; +import java.util.function.Function; + +public class ComposingReindexingRuleProviderTest +{ + private static final DateTime REFERENCE_TIME = DateTimes.of("2025-12-19T12:00:00Z"); + + @Test + public void test_constructor_nullProviders_throwsNullPointerException() + { + Assertions.assertThrows( + NullPointerException.class, + () -> new ComposingReindexingRuleProvider(null) + ); + } + + @Test + public void test_constructor_nullProviderInList_throwsNullPointerException() + { + List providers = new ArrayList<>(); + providers.add(InlineReindexingRuleProvider.builder().build()); + providers.add(null); // Null provider + + NullPointerException exception = Assertions.assertThrows( + NullPointerException.class, + () -> new ComposingReindexingRuleProvider(providers) + ); + + Assertions.assertTrue(exception.getMessage().contains("providers list contains null element")); + } + + @Test + public void test_constructor_emptyProviderList_succeeds() + { + ComposingReindexingRuleProvider composing = new ComposingReindexingRuleProvider( + Collections.emptyList() + ); + + Assertions.assertEquals("composing", composing.getType()); + Assertions.assertTrue(composing.isReady()); + Assertions.assertTrue(composing.getDeletionRules().isEmpty()); + } + + + @Test + public void test_isReady_allProvidersReady_returnsTrue() + { + ReindexingRuleProvider provider1 = InlineReindexingRuleProvider.builder().build(); + ReindexingRuleProvider provider2 = InlineReindexingRuleProvider.builder().build(); + + ComposingReindexingRuleProvider composing = new ComposingReindexingRuleProvider( + ImmutableList.of(provider1, provider2) + ); + + Assertions.assertTrue(composing.isReady()); + } + + @Test + public void test_isReady_someProvidersNotReady_returnsFalse() + { + ReindexingRuleProvider readyProvider = InlineReindexingRuleProvider.builder().build(); + ReindexingRuleProvider notReadyProvider = createNotReadyProvider(); + + ComposingReindexingRuleProvider composing = new ComposingReindexingRuleProvider( + ImmutableList.of(readyProvider, notReadyProvider) + ); + + Assertions.assertFalse(composing.isReady()); + } + + @Test + public void test_isReady_emptyProviderList_returnsTrue() + { + ComposingReindexingRuleProvider composing = new ComposingReindexingRuleProvider( + Collections.emptyList() + ); + + Assertions.assertTrue(composing.isReady()); + } + + @Test + public void test_getDeletionRules_compositingBehavior() + { + testComposingBehaviorForRuleType( + rules -> InlineReindexingRuleProvider.builder().deletionRules(rules).build(), + ComposingReindexingRuleProvider::getDeletionRules, + createFilterRule("rule1", Period.days(7)), + createFilterRule("rule2", Period.days(30)), + ReindexingDeletionRule::getId + ); + } + + @Test + public void test_getDeletionRulesWithInterval_compositingBehavior() + { + testComposingBehaviorForAdditiveRuleTypeWithInterval( + rules -> InlineReindexingRuleProvider.builder().deletionRules(rules).build(), + (provider, it) -> provider.getDeletionRules(it.interval, it.time), + createFilterRule("rule1", Period.days(7)), + createFilterRule("rule2", Period.days(30)), + ReindexingDeletionRule::getId + ); + } + + @Test + public void test_getDataSchemaRules_compositingBehavior() + { + testComposingBehaviorForRuleType( + rules -> InlineReindexingRuleProvider.builder().dataSchemaRules(rules).build(), + ComposingReindexingRuleProvider::getDataSchemaRules, + createDataSchemaRule("rule1", Period.days(7)), + createDataSchemaRule("rule2", Period.days(30)), + ReindexingDataSchemaRule::getId + ); + } + + @Test + public void test_getDataSchemaRulesWithInterval_compositingBehavior() + { + testComposingBehaviorForNonAdditiveRuleTypeWithInterval( + rules -> InlineReindexingRuleProvider.builder().dataSchemaRules(rules).build(), + (provider, it) -> provider.getDataSchemaRule(it.interval, it.time), + createDataSchemaRule("rule1", Period.days(7)), + createDataSchemaRule("rule2", Period.days(30)), + ReindexingDataSchemaRule::getId + ); + } + + @Test + public void test_getSegmentGranularityRules_compositingBehavior() + { + testComposingBehaviorForRuleType( + rules -> InlineReindexingRuleProvider.builder().segmentGranularityRules(rules).build(), + ComposingReindexingRuleProvider::getSegmentGranularityRules, + createSegmentGranularityRule("rule1", Period.days(7)), + createSegmentGranularityRule("rule2", Period.days(30)), + ReindexingSegmentGranularityRule::getId + ); + } + + @Test + public void test_getIOConfigRules_compositingBehavior() + { + testComposingBehaviorForRuleType( + rules -> InlineReindexingRuleProvider.builder().ioConfigRules(rules).build(), + ComposingReindexingRuleProvider::getIOConfigRules, + createIOConfigRule("rule1", Period.days(7)), + createIOConfigRule("rule2", Period.days(30)), + ReindexingIOConfigRule::getId + ); + } + + @Test + public void test_getIOConfigRuleWithInterval_compositingBehavior() + { + testComposingBehaviorForNonAdditiveRuleTypeWithInterval( + rules -> InlineReindexingRuleProvider.builder().ioConfigRules(rules).build(), + (provider, it) -> provider.getIOConfigRule(it.interval, it.time), + createIOConfigRule("rule1", Period.days(7)), + createIOConfigRule("rule2", Period.days(30)), + ReindexingIOConfigRule::getId + ); + } + + @Test + public void test_getTuningConfigRules_compositingBehavior() + { + testComposingBehaviorForRuleType( + rules -> InlineReindexingRuleProvider.builder().tuningConfigRules(rules).build(), + ComposingReindexingRuleProvider::getTuningConfigRules, + createTuningConfigRule("rule1", Period.days(7)), + createTuningConfigRule("rule2", Period.days(30)), + ReindexingTuningConfigRule::getId + ); + } + + @Test + public void test_getTuningConfigRuleWithInterval_compositingBehavior() + { + testComposingBehaviorForNonAdditiveRuleTypeWithInterval( + rules -> InlineReindexingRuleProvider.builder().tuningConfigRules(rules).build(), + (provider, it) -> provider.getTuningConfigRule(it.interval, it.time), + createTuningConfigRule("rule1", Period.days(7)), + createTuningConfigRule("rule2", Period.days(30)), + ReindexingTuningConfigRule::getId + ); + } + + @Test + public void test_getSegmentGranularityRuleWithInterval_compositingBehavior() + { + testComposingBehaviorForNonAdditiveRuleTypeWithInterval( + rules -> InlineReindexingRuleProvider.builder().segmentGranularityRules(rules).build(), + (provider, it) -> provider.getSegmentGranularityRule(it.interval, it.time), + createSegmentGranularityRule("rule1", Period.days(7)), + createSegmentGranularityRule("rule2", Period.days(30)), + ReindexingSegmentGranularityRule::getId + ); + } + + @Test + public void test_equals_sameProviders_returnsTrue() + { + ReindexingRuleProvider provider1 = InlineReindexingRuleProvider.builder().build(); + ReindexingRuleProvider provider2 = InlineReindexingRuleProvider.builder() + .deletionRules(ImmutableList.of(createFilterRule("rule1", Period.days(30)))) + .build(); + + ComposingReindexingRuleProvider composing1 = new ComposingReindexingRuleProvider( + ImmutableList.of(provider1, provider2) + ); + ComposingReindexingRuleProvider composing2 = new ComposingReindexingRuleProvider( + ImmutableList.of(provider1, provider2) + ); + + Assertions.assertEquals(composing1, composing2); + Assertions.assertEquals(composing1.hashCode(), composing2.hashCode()); + } + + @Test + public void test_equals_differentProviders_returnsFalse() + { + ReindexingRuleProvider provider1 = InlineReindexingRuleProvider.builder().build(); + ReindexingRuleProvider provider2 = InlineReindexingRuleProvider.builder() + .deletionRules(ImmutableList.of(createFilterRule("rule1", Period.days(30)))) + .build(); + + ComposingReindexingRuleProvider composing1 = new ComposingReindexingRuleProvider( + ImmutableList.of(provider1) + ); + ComposingReindexingRuleProvider composing2 = new ComposingReindexingRuleProvider( + ImmutableList.of(provider1, provider2) + ); + + Assertions.assertNotEquals(composing1, composing2); + } + + + + /** + * Helper class to pass interval + time together + */ + private static class IntervalAndTime + { + final Interval interval; + final DateTime time; + + IntervalAndTime(Interval interval, DateTime time) + { + this.interval = interval; + this.time = time; + } + } + + /** + * Tests composing behavior for getXxxRules() - all three scenarios: + * 1. First provider has rules → returns first provider's rules + * 2. First provider empty → falls through to second provider + * 3. Both providers empty → returns empty list + */ + private void testComposingBehaviorForRuleType( + Function, ReindexingRuleProvider> providerFactory, + Function> ruleGetter, + T rule1, + T rule2, + Function idExtractor + ) + { + ReindexingRuleProvider provider1 = providerFactory.apply(ImmutableList.of(rule1)); + ReindexingRuleProvider provider2 = providerFactory.apply(ImmutableList.of(rule2)); + + ComposingReindexingRuleProvider composing = new ComposingReindexingRuleProvider( + ImmutableList.of(provider1, provider2) + ); + + List result = ruleGetter.apply(composing); + Assertions.assertEquals(1, result.size()); + Assertions.assertEquals("rule1", idExtractor.apply(result.get(0))); + + ReindexingRuleProvider emptyProvider = InlineReindexingRuleProvider.builder().build(); + composing = new ComposingReindexingRuleProvider( + ImmutableList.of(emptyProvider, provider2) + ); + + result = ruleGetter.apply(composing); + Assertions.assertEquals(1, result.size()); + Assertions.assertEquals("rule2", idExtractor.apply(result.get(0))); + + ReindexingRuleProvider emptyProvider2 = InlineReindexingRuleProvider.builder().build(); + composing = new ComposingReindexingRuleProvider( + ImmutableList.of(emptyProvider, emptyProvider2) + ); + + result = ruleGetter.apply(composing); + Assertions.assertTrue(result.isEmpty()); + } + + private void testComposingBehaviorForNonAdditiveRuleTypeWithInterval( + Function, ReindexingRuleProvider> providerFactory, + BiFunction ruleGetter, + T rule1, + T rule2, + Function idExtractor + ) + { + Interval interval = Intervals.of("2025-11-01/2025-11-15"); + + ReindexingRuleProvider provider1 = providerFactory.apply(ImmutableList.of(rule1)); + ReindexingRuleProvider provider2 = providerFactory.apply(ImmutableList.of(rule2)); + + ComposingReindexingRuleProvider composing = new ComposingReindexingRuleProvider( + ImmutableList.of(provider1, provider2) + ); + + T result = ruleGetter.apply(composing, new IntervalAndTime(interval, REFERENCE_TIME)); + Assertions.assertNotNull(result); + Assertions.assertEquals("rule1", idExtractor.apply(result)); + + ReindexingRuleProvider emptyProvider = InlineReindexingRuleProvider.builder().build(); + composing = new ComposingReindexingRuleProvider( + ImmutableList.of(emptyProvider, provider2) + ); + + result = ruleGetter.apply(composing, new IntervalAndTime(interval, REFERENCE_TIME)); + Assertions.assertNotNull(result); + Assertions.assertEquals("rule2", idExtractor.apply(result)); + + ReindexingRuleProvider emptyProvider2 = InlineReindexingRuleProvider.builder().build(); + composing = new ComposingReindexingRuleProvider( + ImmutableList.of(emptyProvider, emptyProvider2) + ); + + result = ruleGetter.apply(composing, new IntervalAndTime(interval, REFERENCE_TIME)); + Assertions.assertNull(result); + } + + /** + * Tests composing behavior for getXxxRules(interval, time) - all three scenarios: + * 1. First provider has rules → returns first provider's rules + * 2. First provider empty → falls through to second provider + * 3. Both providers empty → returns empty list + */ + private void testComposingBehaviorForAdditiveRuleTypeWithInterval( + Function, ReindexingRuleProvider> providerFactory, + BiFunction> ruleGetter, + T rule1, + T rule2, + Function idExtractor + ) + { + Interval interval = Intervals.of("2025-11-01/2025-11-15"); + + ReindexingRuleProvider provider1 = providerFactory.apply(ImmutableList.of(rule1)); + ReindexingRuleProvider provider2 = providerFactory.apply(ImmutableList.of(rule2)); + + ComposingReindexingRuleProvider composing = new ComposingReindexingRuleProvider( + ImmutableList.of(provider1, provider2) + ); + + List result = ruleGetter.apply(composing, new IntervalAndTime(interval, REFERENCE_TIME)); + Assertions.assertEquals(1, result.size()); + Assertions.assertEquals("rule1", idExtractor.apply(result.get(0))); + + ReindexingRuleProvider emptyProvider = InlineReindexingRuleProvider.builder().build(); + composing = new ComposingReindexingRuleProvider( + ImmutableList.of(emptyProvider, provider2) + ); + + result = ruleGetter.apply(composing, new IntervalAndTime(interval, REFERENCE_TIME)); + Assertions.assertEquals(1, result.size()); + Assertions.assertEquals("rule2", idExtractor.apply(result.get(0))); + + ReindexingRuleProvider emptyProvider2 = InlineReindexingRuleProvider.builder().build(); + composing = new ComposingReindexingRuleProvider( + ImmutableList.of(emptyProvider, emptyProvider2) + ); + + result = ruleGetter.apply(composing, new IntervalAndTime(interval, REFERENCE_TIME)); + Assertions.assertTrue(result.isEmpty()); + } + + /** + * Creates a test provider that is not ready + */ + private ReindexingRuleProvider createNotReadyProvider() + { + return new InlineReindexingRuleProvider(null, null, null, null, null) + { + @Override + public boolean isReady() + { + return false; + } + }; + } + + private ReindexingDeletionRule createFilterRule(String id, Period period) + { + return new ReindexingDeletionRule( + id, + "Test rule", + period, + new SelectorDimFilter("test", "value", null), + null + ); + } + + private ReindexingSegmentGranularityRule createSegmentGranularityRule(String id, Period period) + { + return new ReindexingSegmentGranularityRule( + id, + "Test granularity rule", + period, + Granularities.DAY + ); + } + + private ReindexingIOConfigRule createIOConfigRule(String id, Period period) + { + return new ReindexingIOConfigRule( + id, + "Test IO config rule", + period, + new UserCompactionTaskIOConfig(null) + ); + } + + private ReindexingTuningConfigRule createTuningConfigRule(String id, Period period) + { + return new ReindexingTuningConfigRule( + id, + "Test tuning config rule", + period, + new UserCompactionTaskQueryTuningConfig(null, null, null, null, null, null, null, null, + null, null, null, null, null, null, null, null, null, null, null + ) + ); + } + + private ReindexingDataSchemaRule createDataSchemaRule(String id, Period period) + { + return new ReindexingDataSchemaRule( + id, + "Test data schema rule", + period, + new UserCompactionTaskDimensionsConfig(null), + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + Granularities.DAY, + true, + ImmutableList.of( + new AggregateProjectionSpec( + "test_projection", + null, + null, + null, + new AggregatorFactory[]{new CountAggregatorFactory("count")} + ) + ) + ); + } + +} diff --git a/server/src/test/java/org/apache/druid/server/compaction/InlineReindexingRuleProviderTest.java b/server/src/test/java/org/apache/druid/server/compaction/InlineReindexingRuleProviderTest.java new file mode 100644 index 000000000000..efd37311d33c --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/compaction/InlineReindexingRuleProviderTest.java @@ -0,0 +1,279 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.server.coordinator.UserCompactionTaskDimensionsConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.function.BiFunction; + +public class InlineReindexingRuleProviderTest +{ + private static final DateTime REFERENCE_TIME = DateTimes.of("2025-12-19T12:00:00Z"); + + private static final Interval INTERVAL_100_DAYS_OLD = Intervals.of( + "2025-09-01T00:00:00Z/2025-09-02T00:00:00Z" + ); // Ends 109 days before reference time + + private static final Interval INTERVAL_50_DAYS_OLD = Intervals.of( + "2025-10-20T00:00:00Z/2025-10-21T00:00:00Z" + ); // Ends 59 days before reference time + + private static final Interval INTERVAL_20_DAYS_OLD = Intervals.of( + "2025-11-20T00:00:00Z/2025-11-21T00:00:00Z" + ); // Ends 28 days before reference time + + @Test + public void test_constructor_nullListsDefaultToEmpty() + { + InlineReindexingRuleProvider provider = new InlineReindexingRuleProvider( + null, + null, + null, + null, + null + ); + + Assertions.assertNotNull(provider.getDeletionRules()); + Assertions.assertTrue(provider.getDeletionRules().isEmpty()); + Assertions.assertNotNull(provider.getIOConfigRules()); + Assertions.assertTrue(provider.getIOConfigRules().isEmpty()); + Assertions.assertNotNull(provider.getSegmentGranularityRules()); + Assertions.assertTrue(provider.getSegmentGranularityRules().isEmpty()); + Assertions.assertNotNull(provider.getTuningConfigRules()); + Assertions.assertTrue(provider.getTuningConfigRules().isEmpty()); + Assertions.assertTrue(provider.getDataSchemaRules().isEmpty()); + } + + @Test + public void test_reindexingRules_validateAdditivity() + { + ReindexingDeletionRule rule30d = createFilterRule("filter-30d", Period.days(30)); + ReindexingDeletionRule rule60d = createFilterRule("filter-60d", Period.days(60)); + ReindexingDeletionRule rule90d = createFilterRule("filter-90d", Period.days(90)); + + InlineReindexingRuleProvider provider = InlineReindexingRuleProvider.builder() + .deletionRules(ImmutableList.of(rule30d, rule60d, rule90d)) + .build(); + + List noMatch = provider.getDeletionRules(INTERVAL_20_DAYS_OLD, REFERENCE_TIME); + Assertions.assertTrue(noMatch.isEmpty(), "No rules should match interval that's too recent"); + + List oneMatch = provider.getDeletionRules(INTERVAL_50_DAYS_OLD, REFERENCE_TIME); + Assertions.assertEquals(1, oneMatch.size(), "Only rule30d should match"); + Assertions.assertEquals("filter-30d", oneMatch.get(0).getId()); + + List multiMatch = provider.getDeletionRules(INTERVAL_100_DAYS_OLD, REFERENCE_TIME); + Assertions.assertEquals(3, multiMatch.size(), "All 3 additive rules should be returned"); + Assertions.assertTrue(multiMatch.stream().anyMatch(r -> r.getId().equals("filter-30d"))); + Assertions.assertTrue(multiMatch.stream().anyMatch(r -> r.getId().equals("filter-60d"))); + Assertions.assertTrue(multiMatch.stream().anyMatch(r -> r.getId().equals("filter-90d"))); + } + + @Test + public void test_allNonAdditiveRules_validateNonAdditivity() + { + // Test IOConfig rules + testNonAdditivity( + "ioConfig", + this::createIOConfigRule, + InlineReindexingRuleProvider.Builder::ioConfigRules, + InlineReindexingRuleProvider::getIOConfigRule + ); + + // Test segment granularity rules + testNonAdditivity( + "segmentGranularity", + this::createSegmentGranularityRule, + InlineReindexingRuleProvider.Builder::segmentGranularityRules, + InlineReindexingRuleProvider::getSegmentGranularityRule + ); + + // Test tuning config rules + testNonAdditivity( + "tuningConfig", + this::createTuningConfigRule, + InlineReindexingRuleProvider.Builder::tuningConfigRules, + InlineReindexingRuleProvider::getTuningConfigRule + ); + + testNonAdditivity( + "dataSchema", + this::createDataSchemaRule, + InlineReindexingRuleProvider.Builder::dataSchemaRules, + InlineReindexingRuleProvider::getDataSchemaRule + ); + } + + @Test + public void test_allRuleTypesWireCorrectly_withInterval() + { + ReindexingDeletionRule filterRule = createFilterRule("filter", Period.days(30)); + ReindexingIOConfigRule ioConfigRule = createIOConfigRule("ioconfig", Period.days(30)); + ReindexingSegmentGranularityRule segmentGranularityRule = createSegmentGranularityRule("segmentGranularity", Period.days(30)); + ReindexingTuningConfigRule tuningConfigRule = createTuningConfigRule("tuning", Period.days(30)); + ReindexingDataSchemaRule dataSchemaRule = createDataSchemaRule("dataSchema", Period.days(30)); + + InlineReindexingRuleProvider provider = InlineReindexingRuleProvider.builder() + .deletionRules(ImmutableList.of(filterRule)) + .ioConfigRules(ImmutableList.of(ioConfigRule)) + .segmentGranularityRules(ImmutableList.of(segmentGranularityRule)) + .tuningConfigRules(ImmutableList.of(tuningConfigRule)) + .dataSchemaRules(ImmutableList.of(dataSchemaRule)) + .build(); + + Assertions.assertEquals(1, provider.getDeletionRules(INTERVAL_100_DAYS_OLD, REFERENCE_TIME).size()); + Assertions.assertEquals("filter", provider.getDeletionRules(INTERVAL_100_DAYS_OLD, REFERENCE_TIME).get(0).getId()); + + Assertions.assertEquals("ioconfig", provider.getIOConfigRule(INTERVAL_100_DAYS_OLD, REFERENCE_TIME).getId()); + + Assertions.assertEquals("segmentGranularity", provider.getSegmentGranularityRule(INTERVAL_100_DAYS_OLD, REFERENCE_TIME).getId()); + + Assertions.assertEquals("tuning", provider.getTuningConfigRule(INTERVAL_100_DAYS_OLD, REFERENCE_TIME).getId()); + + Assertions.assertEquals("dataSchema", provider.getDataSchemaRule(INTERVAL_100_DAYS_OLD, REFERENCE_TIME).getId()); + } + + /** + * Generic test helper for validating non-additive rule behavior. + *

+ * Tests that when multiple rules match an interval, only the rule with the oldest threshold + * (largest period) is returned. + * + * @param ruleTypeName descriptive name for error messages + * @param ruleFactory function to create a rule instance + * @param builderSetter function to set rules on the builder + * @param ruleGetter function to retrieve the applicable rule from the provider + */ + private void testNonAdditivity( + String ruleTypeName, + BiFunction ruleFactory, + BiFunction, InlineReindexingRuleProvider.Builder> builderSetter, + TriFunction ruleGetter + ) + { + T rule30d = ruleFactory.apply(ruleTypeName + "-30d", Period.days(30)); + T rule60d = ruleFactory.apply(ruleTypeName + "-60d", Period.days(60)); + T rule90d = ruleFactory.apply(ruleTypeName + "-90d", Period.days(90)); + + InlineReindexingRuleProvider.Builder builder = InlineReindexingRuleProvider.builder(); + builder = builderSetter.apply(builder, ImmutableList.of(rule30d, rule60d, rule90d)); + InlineReindexingRuleProvider provider = builder.build(); + + Assertions.assertNull( + ruleGetter.apply(provider, INTERVAL_20_DAYS_OLD, REFERENCE_TIME), + ruleTypeName + ": No rule should match interval that's too recent" + ); + + T oneMatch = ruleGetter.apply(provider, INTERVAL_50_DAYS_OLD, REFERENCE_TIME); + Assertions.assertEquals( + ruleTypeName + "-30d", + oneMatch.getId(), + ruleTypeName + ": Only 30d rule should match" + ); + + T multiMatch = ruleGetter.apply(provider, INTERVAL_100_DAYS_OLD, REFERENCE_TIME); + Assertions.assertEquals( + ruleTypeName + "-90d", + multiMatch.getId(), + ruleTypeName + ": Should return rule with oldest threshold (P90D)" + ); + } + + @FunctionalInterface + private interface TriFunction + { + R apply(T t, U u, V v); + } + + @Test + public void test_getType_returnsInline() + { + InlineReindexingRuleProvider provider = InlineReindexingRuleProvider.builder().build(); + Assertions.assertEquals("inline", provider.getType()); + } + + private ReindexingDeletionRule createFilterRule(String id, Period period) + { + return new ReindexingDeletionRule(id, null, period, new SelectorDimFilter("dim", "val", null), null); + } + + private ReindexingIOConfigRule createIOConfigRule(String id, Period period) + { + return new ReindexingIOConfigRule(id, null, period, new UserCompactionTaskIOConfig(null)); + } + + private ReindexingSegmentGranularityRule createSegmentGranularityRule(String id, Period period) + { + return new ReindexingSegmentGranularityRule( + id, + null, + period, + Granularities.DAY + ); + } + + private ReindexingTuningConfigRule createTuningConfigRule(String id, Period period) + { + return new ReindexingTuningConfigRule( + id, + null, + period, + new UserCompactionTaskQueryTuningConfig(null, null, null, null, null, null, null, null, + null, null, null, null, null, null, null, null, null, null, null + ) + ); + } + + private ReindexingDataSchemaRule createDataSchemaRule(String id, Period period) + { + return new ReindexingDataSchemaRule( + id, + null, + period, + new UserCompactionTaskDimensionsConfig(null), + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + Granularities.DAY, + true, + ImmutableList.of(new AggregateProjectionSpec( + "test_projection", + null, + null, + null, + new AggregatorFactory[]{new CountAggregatorFactory("count")} + )) + ); + } +} diff --git a/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java index 4e86abcd2468..ee2a442d2921 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java @@ -1431,7 +1431,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil partitionsSpec, null, null, - new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null)), + new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null), null), indexSpec, null, null @@ -1445,7 +1445,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil partitionsSpec, null, null, - new CompactionTransformSpec(new SelectorDimFilter("dim1", "bar", null)), + new CompactionTransformSpec(new SelectorDimFilter("dim1", "bar", null), null), indexSpec, null, null @@ -1459,7 +1459,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil partitionsSpec, null, null, - new CompactionTransformSpec(null), + new CompactionTransformSpec(null, null), indexSpec, null, null @@ -1474,7 +1474,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil // Auto compaction config sets filter=SelectorDimFilter("dim1", "bar", null) CompactionSegmentIterator iterator = createIterator( configBuilder().withTransformSpec( - new CompactionTransformSpec(new SelectorDimFilter("dim1", "bar", null)) + new CompactionTransformSpec(new SelectorDimFilter("dim1", "bar", null), null) ).build(), timeline ); @@ -1518,7 +1518,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil // Auto compaction config sets filter=null iterator = createIterator( configBuilder().withTransformSpec( - new CompactionTransformSpec(null) + new CompactionTransformSpec(null, null) ).build(), timeline ); diff --git a/server/src/test/java/org/apache/druid/server/compaction/ReindexingDataSchemaRuleTest.java b/server/src/test/java/org/apache/druid/server/compaction/ReindexingDataSchemaRuleTest.java new file mode 100644 index 000000000000..7afef2dce4d7 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/compaction/ReindexingDataSchemaRuleTest.java @@ -0,0 +1,370 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import org.apache.druid.data.input.impl.LongDimensionSchema; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.server.coordinator.UserCompactionTaskDimensionsConfig; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.List; + +public class ReindexingDataSchemaRuleTest +{ + private static final DateTime REFERENCE_TIME = DateTimes.of("2025-12-19T12:00:00Z"); + private static final Period PERIOD_14_DAYS = Period.days(14); + + private final UserCompactionTaskDimensionsConfig dimensionsSpec = new UserCompactionTaskDimensionsConfig(List.of(new LongDimensionSchema("dim1"))); + private final AggregatorFactory[] metricsSpec = new AggregatorFactory[]{ + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("sum_metric", "metric") + }; + private final Granularity queryGranularity = Granularities.MINUTE; + private final Boolean rollup = true; + + private final ReindexingDataSchemaRule rule = new ReindexingDataSchemaRule( + "test-schema-rule", + "Custom data schema", + PERIOD_14_DAYS, + dimensionsSpec, + metricsSpec, + queryGranularity, + rollup, + Collections.emptyList() + ); + + @Test + public void test_appliesTo_intervalFullyBeforeThreshold_returnsFull() + { + // Threshold is 2025-12-05T12:00:00Z (14 days before reference time) + // Interval ends at 2025-12-03, which is fully before threshold + Interval interval = Intervals.of("2025-12-02T00:00:00Z/2025-12-03T00:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.FULL, result); + } + + @Test + public void test_appliesTo_intervalEndsAtThreshold_returnsFull() + { + // Threshold is 2025-12-05T12:00:00Z (14 days before reference time) + // Interval ends exactly at threshold - should be FULL (boundary case) + Interval interval = Intervals.of("2025-12-04T12:00:00Z/2025-12-05T12:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.FULL, result); + } + + @Test + public void test_appliesTo_intervalSpansThreshold_returnsPartial() + { + // Threshold is 2025-12-05T12:00:00Z (14 days before reference time) + // Interval starts before threshold and ends after - PARTIAL + Interval interval = Intervals.of("2025-12-04T00:00:00Z/2025-12-06T00:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.PARTIAL, result); + } + + @Test + public void test_appliesTo_intervalStartsAfterThreshold_returnsNone() + { + // Threshold is 2025-12-05T12:00:00Z (14 days before reference time) + // Interval starts after threshold - NONE + Interval interval = Intervals.of("2025-12-15T00:00:00Z/2025-12-16T00:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.NONE, result); + } + + @Test + public void test_getDimensionsSpec_returnsConfiguredValue() + { + UserCompactionTaskDimensionsConfig dimensions = rule.getDimensionsSpec(); + + Assertions.assertNotNull(dimensions); + Assertions.assertEquals(dimensionsSpec, dimensions); + } + + @Test + public void test_getMetricsSpec_returnsConfiguredValue() + { + AggregatorFactory[] metrics = rule.getMetricsSpec(); + + Assertions.assertNotNull(metrics); + Assertions.assertEquals(2, metrics.length); + Assertions.assertEquals("count", metrics[0].getName()); + Assertions.assertEquals("sum_metric", metrics[1].getName()); + } + + @Test + public void test_getQueryGranularity_returnsConfiguredValue() + { + Granularity granularity = rule.getQueryGranularity(); + + Assertions.assertNotNull(granularity); + Assertions.assertEquals(queryGranularity, granularity); + } + + @Test + public void test_getRollup_returnsConfiguredValue() + { + Boolean rollupValue = rule.getRollup(); + + Assertions.assertNotNull(rollupValue); + Assertions.assertEquals(true, rollupValue); + } + + @Test + public void test_getProjections_returnsConfiguredValue() + { + Assertions.assertNotNull(rule.getProjections()); + Assertions.assertEquals(Collections.emptyList(), rule.getProjections()); + } + + @Test + public void test_getId_returnsConfiguredId() + { + Assertions.assertEquals("test-schema-rule", rule.getId()); + } + + @Test + public void test_getDescription_returnsConfiguredDescription() + { + Assertions.assertEquals("Custom data schema", rule.getDescription()); + } + + @Test + public void test_getOlderThan_returnsConfiguredPeriod() + { + Assertions.assertEquals(PERIOD_14_DAYS, rule.getOlderThan()); + } + + @Test + public void test_constructor_nullId_throwsNullPointerException() + { + Assertions.assertThrows( + NullPointerException.class, + () -> new ReindexingDataSchemaRule( + null, + "description", + PERIOD_14_DAYS, + dimensionsSpec, + metricsSpec, + queryGranularity, + rollup, + Collections.emptyList() + ) + ); + } + + @Test + public void test_constructor_nullPeriod_throwsNullPointerException() + { + Assertions.assertThrows( + NullPointerException.class, + () -> new ReindexingDataSchemaRule( + "test-id", + "description", + null, + dimensionsSpec, + metricsSpec, + queryGranularity, + rollup, + Collections.emptyList() + ) + ); + } + + @Test + public void test_constructor_zeroPeriod_succeeds() + { + // P0D is valid - indicates rules that apply immediately to all data + Period zeroPeriod = Period.days(0); + ReindexingDataSchemaRule rule = new ReindexingDataSchemaRule( + "test-id", + "description", + zeroPeriod, + dimensionsSpec, + metricsSpec, + queryGranularity, + rollup, + Collections.emptyList() + ); + Assertions.assertEquals(zeroPeriod, rule.getOlderThan()); + } + + @Test + public void test_constructor_negativePeriod_throwsIllegalArgumentException() + { + Period negativePeriod = Period.days(-14); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> new ReindexingDataSchemaRule( + "test-id", + "description", + negativePeriod, + dimensionsSpec, + metricsSpec, + queryGranularity, + rollup, + Collections.emptyList() + ) + ); + } + + @Test + public void test_constructor_nullDimensionsSpec_succeeds() + { + // Null dimensions spec is allowed + ReindexingDataSchemaRule rule = new ReindexingDataSchemaRule( + "test-id", + "description", + PERIOD_14_DAYS, + null, + metricsSpec, + queryGranularity, + rollup, + Collections.emptyList() + ); + Assertions.assertNull(rule.getDimensionsSpec()); + } + + @Test + public void test_constructor_nullMetricsSpec_succeeds() + { + // Null metrics spec is allowed + ReindexingDataSchemaRule rule = new ReindexingDataSchemaRule( + "test-id", + "description", + PERIOD_14_DAYS, + dimensionsSpec, + null, + queryGranularity, + rollup, + Collections.emptyList() + ); + Assertions.assertNull(rule.getMetricsSpec()); + } + + @Test + public void test_constructor_nullQueryGranularity_succeeds() + { + // Null query granularity is allowed + ReindexingDataSchemaRule rule = new ReindexingDataSchemaRule( + "test-id", + "description", + PERIOD_14_DAYS, + dimensionsSpec, + metricsSpec, + null, + rollup, + Collections.emptyList() + ); + Assertions.assertNull(rule.getQueryGranularity()); + } + + @Test + public void test_constructor_nullRollup_succeeds() + { + // Null rollup is allowed + ReindexingDataSchemaRule rule = new ReindexingDataSchemaRule( + "test-id", + "description", + PERIOD_14_DAYS, + dimensionsSpec, + metricsSpec, + queryGranularity, + null, + Collections.emptyList() + ); + Assertions.assertNull(rule.getRollup()); + } + + @Test + public void test_constructor_nullProjections_succeeds() + { + // Null projections is allowed + ReindexingDataSchemaRule rule = new ReindexingDataSchemaRule( + "test-id", + "description", + PERIOD_14_DAYS, + dimensionsSpec, + metricsSpec, + queryGranularity, + rollup, + null + ); + Assertions.assertNull(rule.getProjections()); + } + + @Test + public void test_constructor_emptyMetricsSpec_succeeds() + { + // Empty metrics array is allowed + AggregatorFactory[] emptyMetrics = new AggregatorFactory[0]; + ReindexingDataSchemaRule rule = new ReindexingDataSchemaRule( + "test-id", + "description", + PERIOD_14_DAYS, + dimensionsSpec, + emptyMetrics, + queryGranularity, + rollup, + Collections.emptyList() + ); + Assertions.assertNotNull(rule.getMetricsSpec()); + Assertions.assertEquals(0, rule.getMetricsSpec().length); + } + + @Test + public void test_constructor_allNull_throwsDruidException() + { + Assertions.assertThrows( + DruidException.class, + () -> new ReindexingDataSchemaRule( + "test-id", + "description", + PERIOD_14_DAYS, + null, + null, + null, + null, + null + ) + ); + } +} diff --git a/server/src/test/java/org/apache/druid/server/compaction/ReindexingDeletionRuleTest.java b/server/src/test/java/org/apache/druid/server/compaction/ReindexingDeletionRuleTest.java new file mode 100644 index 000000000000..a99f3157527a --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/compaction/ReindexingDeletionRuleTest.java @@ -0,0 +1,370 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.EqualityFilter; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + + +public class ReindexingDeletionRuleTest +{ + private static final DateTime REFERENCE_TIME = DateTimes.of("2025-12-19T12:00:00Z"); + private static final Period PERIOD_30_DAYS = Period.days(30); + + private final DimFilter testFilter = new EqualityFilter("isRobot", ColumnType.STRING, "true", null); + private final VirtualColumns virtualColumns = VirtualColumns.create( + ImmutableList.of( + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() + ) + ) + ); + + + private final ReindexingDeletionRule rule = new ReindexingDeletionRule( + "test-filter-rule", + "Remove robot traffic", + PERIOD_30_DAYS, + testFilter, + virtualColumns + ); + + @Test + public void test_appliesTo_intervalFullyBeforeThreshold_returnsFull() + { + // Threshold is 2025-11-19T12:00:00Z (30 days before reference time) + // Interval ends at 2025-11-15, which is fully before threshold + Interval interval = Intervals.of("2025-11-14T00:00:00Z/2025-11-15T00:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.FULL, result); + } + + @Test + public void test_appliesTo_intervalEndsAtThreshold_returnsFull() + { + // Threshold is 2025-11-19T12:00:00Z (30 days before reference time) + // Interval ends exactly at threshold - should be FULL (boundary case) + Interval interval = Intervals.of("2025-11-18T12:00:00Z/2025-11-19T12:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.FULL, result); + } + + @Test + public void test_appliesTo_intervalSpansThreshold_returnsPartial() + { + // Threshold is 2025-11-19T12:00:00Z (30 days before reference time) + // Interval starts before threshold and ends after - PARTIAL + Interval interval = Intervals.of("2025-11-18T00:00:00Z/2025-11-20T00:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.PARTIAL, result); + } + + @Test + public void test_appliesTo_intervalStartsAfterThreshold_returnsNone() + { + // Threshold is 2025-11-19T12:00:00Z (30 days before reference time) + // Interval starts after threshold - NONE + Interval interval = Intervals.of("2025-12-15T00:00:00Z/2025-12-16T00:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.NONE, result); + } + + @Test + public void test_getVirtualColumns_returnsConfiguredVirtualColumns() + { + VirtualColumns vCols = rule.getVirtualColumns(); + + Assertions.assertNotNull(vCols); + Assertions.assertEquals(virtualColumns, vCols); + } + + @Test + public void test_getDeleteWhere_returnsConfiguredFilter() + { + DimFilter filter = rule.getDeleteWhere(); + + Assertions.assertNotNull(filter); + Assertions.assertEquals(testFilter, filter); + } + + @Test + public void test_getId_returnsConfiguredId() + { + Assertions.assertEquals("test-filter-rule", rule.getId()); + } + + @Test + public void test_getDescription_returnsConfiguredDescription() + { + Assertions.assertEquals("Remove robot traffic", rule.getDescription()); + } + + @Test + public void test_getOlderThan_returnsConfiguredPeriod() + { + Assertions.assertEquals(PERIOD_30_DAYS, rule.getOlderThan()); + } + + @Test + public void test_constructor_nullId_throwsNullPointerException() + { + Assertions.assertThrows( + NullPointerException.class, + () -> new ReindexingDeletionRule(null, "description", PERIOD_30_DAYS, testFilter, null) + ); + } + + @Test + public void test_constructor_nullPeriod_throwsNullPointerException() + { + Assertions.assertThrows( + NullPointerException.class, + () -> new ReindexingDeletionRule("test-id", "description", null, testFilter, null) + ); + } + + @Test + public void test_constructor_zeroPeriod_succeeds() + { + // P0D is valid - indicates rules that apply immediately to all data + Period zeroPeriod = Period.days(0); + ReindexingDeletionRule rule = new ReindexingDeletionRule( + "test-id", + "description", + zeroPeriod, + testFilter, + null + ); + Assertions.assertEquals(zeroPeriod, rule.getOlderThan()); + } + + @Test + public void test_constructor_negativePeriod_throwsIllegalArgumentException() + { + Period negativePeriod = Period.days(-30); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> new ReindexingDeletionRule("test-id", "description", negativePeriod, testFilter, null) + ); + } + + @Test + public void test_constructor_nullDeleteWhere_throwsDruidException() + { + Assertions.assertThrows( + DruidException.class, + () -> new ReindexingDeletionRule("test-id", "description", PERIOD_30_DAYS, null, null) + ); + } + + // ========== Tests for variable-length periods (months/years) ========== + + @Test + public void test_constructor_periodWithMonths_succeeds() + { + // P6M should work - months are valid even though they're variable length + Period period = Period.months(6); + ReindexingDeletionRule rule = new ReindexingDeletionRule( + "test-id", + "6 month rule", + period, + testFilter, + null + ); + + Assertions.assertEquals(period, rule.getOlderThan()); + } + + @Test + public void test_constructor_periodWithYears_succeeds() + { + // P1Y should work - years are valid even though they're variable length + Period period = Period.years(1); + ReindexingDeletionRule rule = new ReindexingDeletionRule( + "test-id", + "1 year rule", + period, + testFilter, + null + ); + + Assertions.assertEquals(period, rule.getOlderThan()); + } + + @Test + public void test_constructor_periodWithMixedMonthsAndDays_succeeds() + { + // P6M15D should work - mixed months and days + Period period = Period.months(6).plusDays(15); + ReindexingDeletionRule rule = new ReindexingDeletionRule( + "test-id", + "6 months 15 days rule", + period, + testFilter, + null + ); + + Assertions.assertEquals(period, rule.getOlderThan()); + } + + @Test + public void test_constructor_periodWithYearsMonthsDays_succeeds() + { + // P1Y3M10D should work - complex period with years, months, and days + Period period = Period.years(1).plusMonths(3).plusDays(10); + ReindexingDeletionRule rule = new ReindexingDeletionRule( + "test-id", + "1 year 3 months 10 days rule", + period, + testFilter, + null + ); + + Assertions.assertEquals(period, rule.getOlderThan()); + } + + @Test + public void test_constructor_zeroMonthsPeriod_succeeds() + { + // P0M is valid - equivalent to P0D, indicates rules that apply immediately to all data + Period zeroPeriod = Period.months(0); + ReindexingDeletionRule rule = new ReindexingDeletionRule( + "test-id", + "description", + zeroPeriod, + testFilter, + null + ); + Assertions.assertEquals(zeroPeriod, rule.getOlderThan()); + } + + @Test + public void test_constructor_negativeMonthsPeriod_throwsIllegalArgumentException() + { + // P-6M should fail - negative months + Period negativePeriod = Period.months(-6); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> new ReindexingDeletionRule("test-id", "description", negativePeriod, testFilter, null) + ); + } + + @Test + public void test_appliesTo_periodWithMonths_calculatesThresholdCorrectly() + { + // Test that month-based periods correctly calculate threshold using calendar arithmetic + // Reference time: 2025-12-19T12:00:00Z + // Period: P6M (6 months) + // Expected threshold: 2025-06-19T12:00:00Z (6 months before reference) + + Period sixMonths = Period.months(6); + ReindexingDeletionRule monthRule = new ReindexingDeletionRule( + "test-month-rule", + "6 months rule", + sixMonths, + testFilter, + null + ); + + // Interval ending before 6-month threshold - should be FULL + Interval beforeThreshold = Intervals.of("2025-06-01T00:00:00Z/2025-06-15T00:00:00Z"); + Assertions.assertEquals( + ReindexingRule.AppliesToMode.FULL, + monthRule.appliesTo(beforeThreshold, REFERENCE_TIME) + ); + + // Interval spanning the 6-month threshold - should be PARTIAL + Interval spanningThreshold = Intervals.of("2025-06-15T00:00:00Z/2025-07-15T00:00:00Z"); + Assertions.assertEquals( + ReindexingRule.AppliesToMode.PARTIAL, + monthRule.appliesTo(spanningThreshold, REFERENCE_TIME) + ); + + // Interval starting after 6-month threshold - should be NONE + Interval afterThreshold = Intervals.of("2025-07-01T00:00:00Z/2025-07-15T00:00:00Z"); + Assertions.assertEquals( + ReindexingRule.AppliesToMode.NONE, + monthRule.appliesTo(afterThreshold, REFERENCE_TIME) + ); + } + + @Test + public void test_appliesTo_periodWithYears_calculatesThresholdCorrectly() + { + // Test that year-based periods correctly calculate threshold using calendar arithmetic + // Reference time: 2025-12-19T12:00:00Z + // Period: P1Y (1 year) + // Expected threshold: 2024-12-19T12:00:00Z (1 year before reference) + + Period oneYear = Period.years(1); + ReindexingDeletionRule yearRule = new ReindexingDeletionRule( + "test-year-rule", + "1 year rule", + oneYear, + testFilter, + null + ); + + // Interval ending before 1-year threshold - should be FULL + Interval beforeThreshold = Intervals.of("2024-11-01T00:00:00Z/2024-12-01T00:00:00Z"); + Assertions.assertEquals( + ReindexingRule.AppliesToMode.FULL, + yearRule.appliesTo(beforeThreshold, REFERENCE_TIME) + ); + + // Interval spanning the 1-year threshold - should be PARTIAL + Interval spanningThreshold = Intervals.of("2024-12-01T00:00:00Z/2025-01-01T00:00:00Z"); + Assertions.assertEquals( + ReindexingRule.AppliesToMode.PARTIAL, + yearRule.appliesTo(spanningThreshold, REFERENCE_TIME) + ); + + // Interval starting after 1-year threshold - should be NONE + Interval afterThreshold = Intervals.of("2025-01-01T00:00:00Z/2025-02-01T00:00:00Z"); + Assertions.assertEquals( + ReindexingRule.AppliesToMode.NONE, + yearRule.appliesTo(afterThreshold, REFERENCE_TIME) + ); + } +} diff --git a/server/src/test/java/org/apache/druid/server/compaction/ReindexingIOConfigRuleTest.java b/server/src/test/java/org/apache/druid/server/compaction/ReindexingIOConfigRuleTest.java new file mode 100644 index 000000000000..92ecd4d5f0b5 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/compaction/ReindexingIOConfigRuleTest.java @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class ReindexingIOConfigRuleTest +{ + private static final DateTime REFERENCE_TIME = DateTimes.of("2025-12-19T12:00:00Z"); + private static final Period PERIOD_60_DAYS = Period.days(60); + + private final ReindexingIOConfigRule rule = new ReindexingIOConfigRule( + "test-ioconfig-rule", + "Custom IO config", + PERIOD_60_DAYS, + new UserCompactionTaskIOConfig(null) + ); + + @Test + public void test_appliesTo_intervalFullyBeforeThreshold_returnsFull() + { + // Threshold is 2025-10-20T12:00:00Z (60 days before reference time) + // Interval ends at 2025-10-15, which is fully before threshold + Interval interval = Intervals.of("2025-10-14T00:00:00Z/2025-10-15T00:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.FULL, result); + } + + @Test + public void test_appliesTo_intervalEndsAtThreshold_returnsFull() + { + // Threshold is 2025-10-20T12:00:00Z (60 days before reference time) + // Interval ends exactly at threshold - should be FULL (boundary case) + Interval interval = Intervals.of("2025-10-19T12:00:00Z/2025-10-20T12:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.FULL, result); + } + + @Test + public void test_appliesTo_intervalSpansThreshold_returnsPartial() + { + // Threshold is 2025-10-20T12:00:00Z (60 days before reference time) + // Interval starts before threshold and ends after - PARTIAL + Interval interval = Intervals.of("2025-10-19T00:00:00Z/2025-10-21T00:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.PARTIAL, result); + } + + @Test + public void test_appliesTo_intervalStartsAfterThreshold_returnsNone() + { + // Threshold is 2025-10-20T12:00:00Z (60 days before reference time) + // Interval starts after threshold - NONE + Interval interval = Intervals.of("2025-12-15T00:00:00Z/2025-12-16T00:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.NONE, result); + } + + @Test + public void test_getIoConfig_returnsConfiguredValue() + { + UserCompactionTaskIOConfig config = rule.getIoConfig(); + + Assertions.assertNotNull(config); + } + + @Test + public void test_getId_returnsConfiguredId() + { + Assertions.assertEquals("test-ioconfig-rule", rule.getId()); + } + + @Test + public void test_getDescription_returnsConfiguredDescription() + { + Assertions.assertEquals("Custom IO config", rule.getDescription()); + } + + @Test + public void test_getOlderThan_returnsConfiguredPeriod() + { + Assertions.assertEquals(PERIOD_60_DAYS, rule.getOlderThan()); + } + + @Test + public void test_constructor_nullId_throwsNullPointerException() + { + UserCompactionTaskIOConfig config = new UserCompactionTaskIOConfig(null); + Assertions.assertThrows( + NullPointerException.class, + () -> new ReindexingIOConfigRule(null, "description", PERIOD_60_DAYS, config) + ); + } + + @Test + public void test_constructor_nullPeriod_throwsNullPointerException() + { + UserCompactionTaskIOConfig config = new UserCompactionTaskIOConfig(null); + Assertions.assertThrows( + NullPointerException.class, + () -> new ReindexingIOConfigRule("test-id", "description", null, config) + ); + } + + @Test + public void test_constructor_zeroPeriod_succeeds() + { + // P0D is valid - indicates rules that apply immediately to all data + UserCompactionTaskIOConfig config = new UserCompactionTaskIOConfig(null); + Period zeroPeriod = Period.days(0); + ReindexingIOConfigRule rule = new ReindexingIOConfigRule( + "test-id", + "description", + zeroPeriod, + config + ); + Assertions.assertEquals(zeroPeriod, rule.getOlderThan()); + } + + @Test + public void test_constructor_negativePeriod_throwsIllegalArgumentException() + { + UserCompactionTaskIOConfig config = new UserCompactionTaskIOConfig(null); + Period negativePeriod = Period.days(-60); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> new ReindexingIOConfigRule("test-id", "description", negativePeriod, config) + ); + } + + @Test + public void test_constructor_nullIOConfig_throwsDruidException() + { + Assertions.assertThrows( + DruidException.class, + () -> new ReindexingIOConfigRule("test-id", "description", PERIOD_60_DAYS, null) + ); + } +} diff --git a/server/src/test/java/org/apache/druid/server/compaction/ReindexingSegmentGranularityRuleTest.java b/server/src/test/java/org/apache/druid/server/compaction/ReindexingSegmentGranularityRuleTest.java new file mode 100644 index 000000000000..9fa076d69ae0 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/compaction/ReindexingSegmentGranularityRuleTest.java @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.granularity.PeriodGranularity; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class ReindexingSegmentGranularityRuleTest +{ + private static final DateTime REFERENCE_TIME = DateTimes.of("2025-12-19T12:00:00Z"); + private static final Period PERIOD_7_DAYS = Period.days(7); + + private final ReindexingSegmentGranularityRule rule = new ReindexingSegmentGranularityRule( + "test-rule", + "Test segment granularity rule", + PERIOD_7_DAYS, + Granularities.HOUR + ); + + @Test + public void test_appliesTo_intervalFullyBeforeThreshold_returnsFull() + { + // Threshold is 2025-12-12T12:00:00Z (7 days before reference time) + // Interval ends at 2025-12-10, which is fully before threshold + Interval interval = Intervals.of("2025-12-09T00:00:00Z/2025-12-10T00:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.FULL, result); + } + + @Test + public void test_appliesTo_intervalEndsAtThreshold_returnsFull() + { + // Threshold is 2025-12-12T12:00:00Z (7 days before reference time) + // Interval ends exactly at threshold - should be FULL (boundary case) + Interval interval = Intervals.of("2025-12-11T12:00:00Z/2025-12-12T12:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.FULL, result); + } + + @Test + public void test_appliesTo_intervalSpansThreshold_returnsPartial() + { + // Threshold is 2025-12-12T12:00:00Z (7 days before reference time) + // Interval starts before threshold and ends after - PARTIAL + Interval interval = Intervals.of("2025-12-11T00:00:00Z/2025-12-13T00:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.PARTIAL, result); + } + + @Test + public void test_appliesTo_intervalStartsAfterThreshold_returnsNone() + { + // Threshold is 2025-12-12T12:00:00Z (7 days before reference time) + // Interval starts after threshold - NONE + Interval interval = Intervals.of("2025-12-18T00:00:00Z/2025-12-19T00:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.NONE, result); + } + + @Test + public void test_getGranularity_returnsConfiguredValue() + { + Granularity granularity = rule.getSegmentGranularity(); + + Assertions.assertNotNull(granularity); + Assertions.assertEquals(Granularities.HOUR, granularity); + } + + @Test + public void test_getId_returnsConfiguredId() + { + Assertions.assertEquals("test-rule", rule.getId()); + } + + @Test + public void test_getDescription_returnsConfiguredDescription() + { + Assertions.assertEquals("Test segment granularity rule", rule.getDescription()); + } + + @Test + public void test_getOlderThan_returnsConfiguredPeriod() + { + Assertions.assertEquals(PERIOD_7_DAYS, rule.getOlderThan()); + } + + @Test + public void test_constructor_nullId_throwsNullPointerException() + { + Assertions.assertThrows( + NullPointerException.class, + () -> new ReindexingSegmentGranularityRule(null, "description", PERIOD_7_DAYS, Granularities.HOUR) + ); + } + + @Test + public void test_constructor_nullPeriod_throwsNullPointerException() + { + Assertions.assertThrows( + NullPointerException.class, + () -> new ReindexingSegmentGranularityRule("test-id", "description", null, Granularities.HOUR) + ); + } + + @Test + public void test_constructor_zeroPeriod_succeeds() + { + // P0D is valid - indicates rules that apply immediately to all data + Period zeroPeriod = Period.days(0); + ReindexingSegmentGranularityRule rule = new ReindexingSegmentGranularityRule( + "test-id", + "description", + zeroPeriod, + Granularities.HOUR + ); + Assertions.assertEquals(zeroPeriod, rule.getOlderThan()); + } + + @Test + public void test_constructor_negativePeriod_throwsIllegalArgumentException() + { + Period negativePeriod = Period.days(-7); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> new ReindexingSegmentGranularityRule("test-id", "description", negativePeriod, Granularities.HOUR) + ); + } + + @Test + public void test_constructor_nullGranularity_throwsNullPointerException() + { + Assertions.assertThrows( + NullPointerException.class, + () -> new ReindexingSegmentGranularityRule("test-id", "description", PERIOD_7_DAYS, null) + ); + } + + @Test + public void test_constructor_supportedGranularities_allSucceed() + { + Granularity[] supportedGranularities = { + Granularities.MINUTE, + Granularities.FIFTEEN_MINUTE, + Granularities.HOUR, + Granularities.DAY, + Granularities.MONTH, + Granularities.QUARTER, + Granularities.YEAR + }; + + for (Granularity granularity : supportedGranularities) { + ReindexingSegmentGranularityRule rule = new ReindexingSegmentGranularityRule( + "test-id", + "description", + PERIOD_7_DAYS, + granularity + ); + Assertions.assertEquals(granularity, rule.getSegmentGranularity()); + } + } + + @Test + public void test_constructor_unsupportedGranularities_allThrowDruidException() + { + Granularity[] unsupportedGranularities = { + Granularities.THIRTY_MINUTE, + Granularities.SIX_HOUR, + Granularities.EIGHT_HOUR, + Granularities.WEEK, + new PeriodGranularity(Period.days(3), null, DateTimeZone.UTC), // Custom period + new PeriodGranularity(Period.days(1), null, DateTimes.inferTzFromString("America/Los_Angeles")) // With timezone + }; + + for (Granularity granularity : unsupportedGranularities) { + DruidException exception = Assertions.assertThrows( + DruidException.class, + () -> new ReindexingSegmentGranularityRule("test-id", "description", PERIOD_7_DAYS, granularity) + ); + Assertions.assertTrue( + exception.getMessage().contains("Unsupported segment granularity"), + "Expected exception message to contain 'Unsupported segment granularity' but got: " + exception.getMessage() + ); + } + } +} diff --git a/server/src/test/java/org/apache/druid/server/compaction/ReindexingTuningConfigRuleTest.java b/server/src/test/java/org/apache/druid/server/compaction/ReindexingTuningConfigRuleTest.java new file mode 100644 index 000000000000..b3625c1e834b --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/compaction/ReindexingTuningConfigRuleTest.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import org.apache.druid.error.DruidException; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class ReindexingTuningConfigRuleTest +{ + private static final DateTime REFERENCE_TIME = DateTimes.of("2025-12-19T12:00:00Z"); + private static final Period PERIOD_21_DAYS = Period.days(21); + + private final ReindexingTuningConfigRule rule = new ReindexingTuningConfigRule( + "test-tuning-rule", + "Custom tuning config", + PERIOD_21_DAYS, + createTestTuningConfig() + + ); + + @Test + public void test_appliesTo_intervalFullyBeforeThreshold_returnsFull() + { + // Threshold is 2025-11-28T12:00:00Z (21 days before reference time) + // Interval ends at 2025-11-25, which is fully before threshold + Interval interval = Intervals.of("2025-11-24T00:00:00Z/2025-11-25T00:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.FULL, result); + } + + @Test + public void test_appliesTo_intervalEndsAtThreshold_returnsFull() + { + // Threshold is 2025-11-28T12:00:00Z (21 days before reference time) + // Interval ends exactly at threshold - should be FULL (boundary case) + Interval interval = Intervals.of("2025-11-27T12:00:00Z/2025-11-28T12:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.FULL, result); + } + + @Test + public void test_appliesTo_intervalSpansThreshold_returnsPartial() + { + // Threshold is 2025-11-28T12:00:00Z (21 days before reference time) + // Interval starts before threshold and ends after - PARTIAL + Interval interval = Intervals.of("2025-11-27T00:00:00Z/2025-11-29T00:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.PARTIAL, result); + } + + @Test + public void test_appliesTo_intervalStartsAfterThreshold_returnsNone() + { + // Threshold is 2025-11-28T12:00:00Z (21 days before reference time) + // Interval starts after threshold - NONE + Interval interval = Intervals.of("2025-12-15T00:00:00Z/2025-12-16T00:00:00Z"); + + ReindexingRule.AppliesToMode result = rule.appliesTo(interval, REFERENCE_TIME); + + Assertions.assertEquals(ReindexingRule.AppliesToMode.NONE, result); + } + + @Test + public void test_getTuningConfig_returnsConfiguredValue() + { + UserCompactionTaskQueryTuningConfig config = rule.getTuningConfig(); + + Assertions.assertNotNull(config); + Assertions.assertNotNull(config.getPartitionsSpec()); + } + + @Test + public void test_getId_returnsConfiguredId() + { + Assertions.assertEquals("test-tuning-rule", rule.getId()); + } + + @Test + public void test_getDescription_returnsConfiguredDescription() + { + Assertions.assertEquals("Custom tuning config", rule.getDescription()); + } + + @Test + public void test_getOlderThan_returnsConfiguredPeriod() + { + Assertions.assertEquals(PERIOD_21_DAYS, rule.getOlderThan()); + } + + @Test + public void test_constructor_nullId_throwsNullPointerException() + { + Assertions.assertThrows( + NullPointerException.class, + () -> new ReindexingTuningConfigRule(null, "description", PERIOD_21_DAYS, createTestTuningConfig()) + ); + } + + @Test + public void test_constructor_nullPeriod_throwsNullPointerException() + { + Assertions.assertThrows( + NullPointerException.class, + () -> new ReindexingTuningConfigRule("test-id", "description", null, createTestTuningConfig()) + ); + } + + @Test + public void test_constructor_zeroPeriod_succeeds() + { + // P0D is valid - indicates rules that apply immediately to all data + Period zeroPeriod = Period.days(0); + ReindexingTuningConfigRule rule = new ReindexingTuningConfigRule( + "test-id", + "description", + zeroPeriod, + createTestTuningConfig() + ); + Assertions.assertEquals(zeroPeriod, rule.getOlderThan()); + } + + @Test + public void test_constructor_negativePeriod_throwsIllegalArgumentException() + { + Period negativePeriod = Period.days(-21); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> new ReindexingTuningConfigRule("test-id", "description", negativePeriod, createTestTuningConfig()) + ); + } + + @Test + public void test_constructor_nullTuningConfig_throwsDruidException() + { + Assertions.assertThrows( + DruidException.class, + () -> new ReindexingTuningConfigRule("test-id", "description", PERIOD_21_DAYS, null) + ); + } + + private UserCompactionTaskQueryTuningConfig createTestTuningConfig() + { + return new UserCompactionTaskQueryTuningConfig( + null, + null, + null, + null, + null, + new DynamicPartitionsSpec(5000000, null), + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordination/LoadableDataSegmentTest.java b/server/src/test/java/org/apache/druid/server/coordination/LoadableDataSegmentTest.java index 62dda2cd694b..21e5626cae72 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/LoadableDataSegmentTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/LoadableDataSegmentTest.java @@ -19,16 +19,23 @@ package org.apache.druid.server.coordination; +import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -44,7 +51,15 @@ public class LoadableDataSegmentTest { - private static final ObjectMapper MAPPER = new DefaultObjectMapper(); + private static final ObjectMapper MAPPER; + + static { + MAPPER = new DefaultObjectMapper(); + MAPPER.setInjectableValues( + new InjectableValues.Std().addValue(ExprMacroTable.class, TestExprMacroTable.INSTANCE) + ); + } + private static final int TEST_VERSION = 0x9; @Test @@ -59,7 +74,19 @@ public void testSerde_LoadableDataSegment() throws Exception new HashedPartitionsSpec(100000, null, List.of("dim1")), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(List.of("dim1", "bar", "foo"))), List.of(new CountAggregatorFactory("count")), - new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null)), + new CompactionTransformSpec( + new SelectorDimFilter("dim1", "foo", null), + VirtualColumns.create( + ImmutableList.of( + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() + ) + ) + ) + ), MAPPER.convertValue(Map.of(), IndexSpec.class), MAPPER.convertValue(Map.of(), GranularitySpec.class), null diff --git a/server/src/test/java/org/apache/druid/server/coordinator/InlineSchemaDataSourceCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/InlineSchemaDataSourceCompactionConfigTest.java index 85ce0bfef017..4c58d551509d 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/InlineSchemaDataSourceCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/InlineSchemaDataSourceCompactionConfigTest.java @@ -19,6 +19,7 @@ package org.apache.druid.server.coordinator; +import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -29,14 +30,19 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.CompressionFactory.LongEncodingStrategy; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.Duration; @@ -48,7 +54,15 @@ public class InlineSchemaDataSourceCompactionConfigTest extends InitializedNullHandlingTest { - private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER; + + static { + OBJECT_MAPPER = new DefaultObjectMapper(); + OBJECT_MAPPER.setInjectableValues( + new InjectableValues.Std().addValue(ExprMacroTable.class, TestExprMacroTable.INSTANCE) + ); + } + @Test public void testSerdeBasic() throws IOException @@ -436,7 +450,21 @@ public void testSerdeTransformSpec() throws IOException .forDataSource("dataSource") .withInputSegmentSizeBytes(500L) .withSkipOffsetFromLatest(new Period(3600)) - .withTransformSpec(new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null))) + .withTransformSpec( + new CompactionTransformSpec( + new SelectorDimFilter("dim1", "foo", null), + VirtualColumns.create( + ImmutableList.of( + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() + ) + ) + ) + ) + ) .withTaskContext(ImmutableMap.of("key", "val")) .build(); final String json = OBJECT_MAPPER.writeValueAsString(config); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 68cef0430d2f..4ea98e43ad57 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 @@ -1289,7 +1289,8 @@ public void testCompactWithTransformSpec() .withTuningConfig(getTuningConfig(3)) .withTransformSpec( new CompactionTransformSpec( - new SelectorDimFilter("dim1", "foo", null) + new SelectorDimFilter("dim1", "foo", null), + null ) ) .withEngine(engine) diff --git a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java index c86a64bb077b..84d8021700aa 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java @@ -32,10 +32,15 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -58,6 +63,7 @@ public class DataSegmentPlusTest public void setUp() { InjectableValues.Std injectableValues = new InjectableValues.Std(); + injectableValues.addValue(ExprMacroTable.class, TestExprMacroTable.INSTANCE); injectableValues.addValue(DataSegment.PruneSpecsHolder.class, DataSegment.PruneSpecsHolder.DEFAULT); MAPPER.setInjectableValues(injectableValues); } @@ -95,7 +101,19 @@ public void testSerde() throws JsonProcessingException DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "bar", "foo")) ), ImmutableList.of(new CountAggregatorFactory("cnt")), - new CompactionTransformSpec(new SelectorDimFilter("dim1", "foo", null)), + new CompactionTransformSpec( + new SelectorDimFilter("dim1", "foo", null), + VirtualColumns.create( + ImmutableList.of( + new ExpressionVirtualColumn( + "isRobotFiltered", + "concat(isRobot, '_filtered')", + ColumnType.STRING, + ExprMacroTable.nil() + ) + ) + ) + ), MAPPER.convertValue(ImmutableMap.of(), IndexSpec.class), MAPPER.convertValue(ImmutableMap.of(), GranularitySpec.class), null