From d729fdff375348ac617744e57bbd2d2cb9504bae Mon Sep 17 00:00:00 2001
From: Abhishek Balaji Radhakrishnan
Date: Tue, 26 Sep 2023 22:07:09 -0700
Subject: [PATCH] Add separate test classes. Do some code cleanup.
---
.../PeriodBroadcastDistributionRule.java | 4 +-
.../rules/PeriodDropBeforeRule.java | 11 +-
.../coordinator/rules/PeriodDropRule.java | 3 +-
.../coordinator/rules/PeriodLoadRule.java | 3 +-
.../druid/server/coordinator/rules/Rule.java | 6 +-
.../druid/server/coordinator/rules/Rules.java | 23 +-
.../rules/RulesEligibleIntervalTest.java | 195 ++++++++++
.../server/coordinator/rules/RulesTest.java | 149 ++++++++
.../druid/server/http/RulesResourceTest.java | 354 ++----------------
9 files changed, 408 insertions(+), 340 deletions(-)
create mode 100644 server/src/test/java/org/apache/druid/server/coordinator/rules/RulesEligibleIntervalTest.java
create mode 100644 server/src/test/java/org/apache/druid/server/coordinator/rules/RulesTest.java
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodBroadcastDistributionRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodBroadcastDistributionRule.java
index 5de02add3316..17bbf7be789b 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodBroadcastDistributionRule.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodBroadcastDistributionRule.java
@@ -21,6 +21,8 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@@ -68,7 +70,7 @@ public boolean appliesTo(Interval interval, DateTime referenceTimestamp)
@Override
public Interval getEligibleInterval(DateTime referenceTimestamp)
{
- return includeFuture ? new Interval(referenceTimestamp.minus(period), referenceTimestamp.plus(period))
+ return includeFuture ? new Interval(referenceTimestamp.minus(period), DateTimes.utc(JodaUtils.MAX_INSTANT))
: new Interval(referenceTimestamp.minus(period), referenceTimestamp);
}
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodDropBeforeRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodDropBeforeRule.java
index 9b9b2c0a72aa..7198e5af7006 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodDropBeforeRule.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodDropBeforeRule.java
@@ -22,7 +22,6 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@@ -69,8 +68,14 @@ public boolean appliesTo(Interval theInterval, DateTime referenceTimestamp)
@Override
public Interval getEligibleInterval(DateTime referenceTimestamp)
{
-// return new Interval(DateTimes.utc(JodaUtils.MIN_INSTANT), referenceTimestamp.minus(period));
- return new Interval(DateTimes.utc(Long.MIN_VALUE), referenceTimestamp.minus(period));
+ final DateTime end = referenceTimestamp.minus(period);
+ if (end.isBefore(DateTimes.MIN)) {
+ // We use Long.MIN_VALUE as the start here (instead of DateTimes.MIN) when end is < DateTimes.MIN because the
+ // resulting interval will be invalid where start > end. This is true for referenceTimestamp = DateTimes.MIN.
+ return new Interval(DateTimes.utc(Long.MIN_VALUE), end);
+ } else {
+ return new Interval(DateTimes.MIN, end);
+ }
}
@Override
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodDropRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodDropRule.java
index 47744dffb8ba..e92e55a3c9a8 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodDropRule.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodDropRule.java
@@ -22,7 +22,6 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@@ -87,7 +86,7 @@ public boolean appliesTo(Interval theInterval, DateTime referenceTimestamp)
@Override
public Interval getEligibleInterval(DateTime referenceTimestamp)
{
- return includeFuture ? new Interval(referenceTimestamp.minus(period), DateTimes.utc(JodaUtils.MAX_INSTANT))
+ return includeFuture ? new Interval(referenceTimestamp.minus(period), DateTimes.MAX)
: new Interval(referenceTimestamp.minus(period), referenceTimestamp);
}
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodLoadRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodLoadRule.java
index 0e6f32f1cadb..b799a664eb45 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodLoadRule.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodLoadRule.java
@@ -22,7 +22,6 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@@ -88,7 +87,7 @@ public boolean appliesTo(Interval interval, DateTime referenceTimestamp)
@Override
public Interval getEligibleInterval(DateTime referenceTimestamp)
{
- return includeFuture ? new Interval(referenceTimestamp.minus(period), DateTimes.utc(JodaUtils.MAX_INSTANT))
+ return includeFuture ? new Interval(referenceTimestamp.minus(period), DateTimes.MAX)
: new Interval(referenceTimestamp.minus(period), referenceTimestamp);
}
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java
index c34b086484c8..72c02ceb5416 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java
@@ -52,9 +52,9 @@ public interface Rule
void run(DataSegment segment, SegmentActionHandler segmentHandler);
/**
- * Return an eligible interval from the reference timestamp. Implementations
- * must return a valid interval based on the rule type.
- * @param referenceTimestamp base timestamp
+ * Returns the interval eligible for this rule. The interval must be computed based on the rule type
+ * optionally using {@code referenceTimestamp}. {@code referenceTimestamp} must be a timestamp
+ * between [{@link org.apache.druid.java.util.common.DateTimes.MIN}, {@link org.apache.druid.java.util.common.DateTimes.MAX}).
*/
Interval getEligibleInterval(DateTime referenceTimestamp);
}
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/Rules.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/Rules.java
index 540bd1842ff7..bf068de088c2 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/rules/Rules.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/Rules.java
@@ -50,10 +50,22 @@ private Rules()
}
/**
- * Validate rules. This method throws an exception if a rule contain an interval that
- * fully covers another subsequent rules' interval in the list. Rules that will be evaluated at some point
- * are considered to be legitimate.
- * @param rules Datasource rules.
+ * Validates the given list of retention rules for a datasource (or cluster default).
+ * A rule is considered valid only if it will be evaluated at some point, i.e. its eligible interval
+ * is not fully covered by the eligible interval of any preceding rule in the list.
+ *
+ * Consider two rules r1 and r2. Assume r1 and r2's eligible intervals at the time when the rules are evaluated are
+ * i1 and i2 respectively. r1 and r2 are invalid if:
+ *
+ * - i1 is eternity. i.e., eternity fully covers i2 and any other interval that follows it. Or
+ * - i1 fully contains i2 and
+ * - r1's eligible interval at i1's start and end fully contain r2's eligible interval at i1's start and end
+ * respectively. This boundary check is used to identify rules that will fire at some point. i.e., period based rules
+ * will return distinct eligible intervals at the boundaries, whereas broadcast and interval based rules will return
+ * fixed intervals regardless of the boundary.
+ *
+ *
+ * @throws org.apache.druid.error.DruidException with error code "invalidInput" if any of the given rules is not valid.
*/
public static void validateRules(final List rules)
{
@@ -70,9 +82,6 @@ public static void validateRules(final List rules)
final Rule nextRule = rules.get(j);
final Interval nextInterval = nextRule.getEligibleInterval(now);
if (currInterval.contains(nextInterval)) {
- // If the current rule has eternity, it covers everything following it.
- // Or if the current rule still covers the next rule at the current interval boundaries, then the
- // next rule will never fire at any time, so throw an exception.
if (Intervals.ETERNITY.equals(currInterval) ||
(currRule.getEligibleInterval(currInterval.getStart())
.contains(nextRule.getEligibleInterval(currInterval.getStart()))
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/RulesEligibleIntervalTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/RulesEligibleIntervalTest.java
new file mode 100644
index 000000000000..83fafbde0f34
--- /dev/null
+++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/RulesEligibleIntervalTest.java
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server.coordinator.rules;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.JodaUtils;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+import org.joda.time.Period;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class RulesEligibleIntervalTest
+{
+ private final DateTime referenceTime;
+
+ @Parameterized.Parameters
+ public static Object[] getReferenceTimestamps()
+ {
+ final DateTime now = DateTimes.nowUtc();
+ return new Object[]{
+ now,
+ now.minusYears(5),
+ now.plusYears(10),
+ DateTimes.utc(0),
+ DateTimes.utc(JodaUtils.MIN_INSTANT + 1),
+ DateTimes.utc(JodaUtils.MIN_INSTANT),
+ DateTimes.utc(JodaUtils.MAX_INSTANT - 1),
+ DateTimes.utc(JodaUtils.MAX_INSTANT)
+ };
+ }
+
+ public RulesEligibleIntervalTest(final DateTime referenceTimestamp)
+ {
+ this.referenceTime = referenceTimestamp;
+ }
+
+ @Test
+ public void testPeriodLoadEligibleInterval()
+ {
+ final Period period = new Period("P50Y");
+ final PeriodLoadRule loadPT1H = new PeriodLoadRule(
+ period,
+ true,
+ null,
+ null
+ );
+ Assert.assertEquals(
+ new Interval(this.referenceTime.minus(period), DateTimes.MAX),
+ loadPT1H.getEligibleInterval(this.referenceTime)
+ );
+ }
+
+ @Test
+ public void testPeriodLoadExcludingFutureEligibleInterval()
+ {
+ final Period period = new Period("PT1H");
+ final PeriodLoadRule rule = new PeriodLoadRule(
+ period,
+ false,
+ null,
+ null
+ );
+ Assert.assertEquals(new Interval(period, this.referenceTime), rule.getEligibleInterval(this.referenceTime));
+ }
+
+ @Test
+ public void testIntervalLoadEligibleInterval()
+ {
+ final Interval interval = Intervals.of("2000/3000");
+ final IntervalLoadRule rule = new IntervalLoadRule(
+ interval,
+ null,
+ null
+ );
+ Assert.assertEquals(interval, rule.getEligibleInterval(this.referenceTime));
+ }
+
+ @Test
+ public void testForeverLoadEligibleInterval()
+ {
+ final ForeverLoadRule rule = new ForeverLoadRule(ImmutableMap.of(), false);
+ Assert.assertEquals(Intervals.ETERNITY, rule.getEligibleInterval(this.referenceTime));
+ }
+
+ @Test
+ public void testPeriodDropEligibleInterval()
+ {
+ final Period period = new Period("P5000Y");
+ final PeriodDropRule rule = new PeriodDropRule(
+ period,
+ true
+ );
+ Assert.assertEquals(
+ new Interval(this.referenceTime.minus(period), DateTimes.utc(JodaUtils.MAX_INSTANT)),
+ rule.getEligibleInterval(this.referenceTime)
+ );
+ }
+
+ @Test
+ public void testPeriodDropExcludingFutureEligibleInterval()
+ {
+ final Period period = new Period("P50Y");
+ final PeriodDropRule rule = new PeriodDropRule(
+ period,
+ false
+ );
+ Assert.assertEquals(
+ new Interval(this.referenceTime.minus(period), this.referenceTime),
+ rule.getEligibleInterval(this.referenceTime)
+ );
+ }
+
+ @Test
+ public void testPeriodDropBeforeEligibleInterval()
+ {
+ final Period period = new Period("P50Y");
+ final PeriodDropBeforeRule rule = new PeriodDropBeforeRule(period);
+
+ if (this.referenceTime.minus(period).isBefore(DateTimes.MIN)) {
+ Assert.assertEquals(
+ new Interval(DateTimes.utc(Long.MIN_VALUE), this.referenceTime.minus(period)),
+ rule.getEligibleInterval(this.referenceTime)
+ );
+ } else {
+ Assert.assertEquals(
+ new Interval(DateTimes.MIN, this.referenceTime.minus(period)),
+ rule.getEligibleInterval(this.referenceTime)
+ );
+ }
+ }
+
+ @Test
+ public void testForeverDropEligibleInterval()
+ {
+ final ForeverDropRule rule = new ForeverDropRule();
+ Assert.assertEquals(Intervals.ETERNITY, rule.getEligibleInterval(this.referenceTime));
+ }
+
+ @Test
+ public void testPeriodBroadcastEligibleInterval()
+ {
+ final Period period = new Period("P15Y");
+ final PeriodBroadcastDistributionRule rule = new PeriodBroadcastDistributionRule(period, true);
+ Assert.assertEquals(
+ new Interval(referenceTime.minus(period), DateTimes.MAX),
+ rule.getEligibleInterval(referenceTime)
+ );
+ }
+
+ @Test
+ public void testPeriodBroadcastExcludingFutureEligibleInterval()
+ {
+ final Period period = new Period("P15Y");
+ final PeriodBroadcastDistributionRule rule = new PeriodBroadcastDistributionRule(period, false);
+ Assert.assertEquals(new Interval(period, this.referenceTime), rule.getEligibleInterval(this.referenceTime));
+ }
+
+ @Test
+ public void testForeverBroadcastEligibleInterval()
+ {
+ final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule();
+ Assert.assertEquals(Intervals.ETERNITY, rule.getEligibleInterval(this.referenceTime));
+ }
+
+ @Test
+ public void testIntervalBroadcastEligibleInterval()
+ {
+ final Interval interval = Intervals.of("1993/2070");
+ final IntervalBroadcastDistributionRule rule = new IntervalBroadcastDistributionRule(interval);
+ Assert.assertEquals(interval, rule.getEligibleInterval(this.referenceTime));
+ }
+}
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/RulesTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/RulesTest.java
new file mode 100644
index 000000000000..b6bde979a302
--- /dev/null
+++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/RulesTest.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server.coordinator.rules;
+
+import org.apache.druid.error.DruidExceptionMatcher;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.StringUtils;
+import org.joda.time.Period;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+@RunWith(Parameterized.class)
+public class RulesTest
+{
+ // Load rules
+ private static final PeriodLoadRule LOAD_PT1H = new PeriodLoadRule(
+ new Period("PT1H"), true, null, null
+ );
+ private static final PeriodLoadRule LOAD_PT1H_EXLUDE_FUTURE = new PeriodLoadRule(
+ new Period("PT1H"), false, null, null
+ );
+ private static final PeriodLoadRule LOAD_P3M = new PeriodLoadRule(
+ new Period("P3M"), true, null, null
+ );
+ private static final IntervalLoadRule LOAD_2020_2023 = new IntervalLoadRule(
+ Intervals.of("2020/2023"), null, null
+ );
+ private static final IntervalLoadRule LOAD_2021_2022 = new IntervalLoadRule(
+ Intervals.of("2021/2022"), null, null
+ );
+ private static final IntervalLoadRule LOAD_1980_2050 = new IntervalLoadRule(
+ Intervals.of("1980/2050"), null, null
+ );
+ private static final ForeverLoadRule LOAD_FOREVER = new ForeverLoadRule(null, null);
+
+ // Drop rules
+ private static final PeriodDropBeforeRule DROP_BEFORE_P3M = new PeriodDropBeforeRule(new Period("P3M"));
+ private static final PeriodDropBeforeRule DROP_BEFORE_P6M = new PeriodDropBeforeRule(new Period("P6M"));
+ private static final PeriodDropRule DROP_P1M = new PeriodDropRule(new Period("P1M"), true);
+ private static final PeriodDropRule DROP_P2M = new PeriodDropRule(new Period("P2M"), true);
+ private static final IntervalDropRule DROP_2000_2020 = new IntervalDropRule(Intervals.of("2000/2020"));
+ private static final IntervalDropRule DROP_2010_2020 = new IntervalDropRule(Intervals.of("2010/2020"));
+ private static final ForeverDropRule DROP_FOREVER = new ForeverDropRule();
+
+ // Broadcast rules
+ private static final PeriodBroadcastDistributionRule BROADCAST_PT1H = new PeriodBroadcastDistributionRule(
+ new Period("PT1H"), true
+ );
+ private static final PeriodBroadcastDistributionRule BROADCAST_PT1H_EXCLUDE_FUTURE = new PeriodBroadcastDistributionRule(
+ new Period("PT1H"), false
+ );
+ private static final PeriodBroadcastDistributionRule BROADCAST_PT2H = new PeriodBroadcastDistributionRule(
+ new Period("PT2H"), true
+ );
+ private static final IntervalBroadcastDistributionRule BROADCAST_2000_2050 = new IntervalBroadcastDistributionRule(
+ Intervals.of("2000/2050")
+ );
+ private static final IntervalBroadcastDistributionRule BROADCAST_2010_2020 = new IntervalBroadcastDistributionRule(
+ Intervals.of("2010/2020")
+ );
+ private static final ForeverBroadcastDistributionRule BROADCAST_FOREVER = new ForeverBroadcastDistributionRule();
+
+ private final List rules;
+ private final boolean isInvalid;
+ private final Rule invalidRule1;
+ private final Rule invalidRule2;
+
+ public RulesTest(final List rules, final boolean isInvalid, final Rule invalidRule1, final Rule invalidRule2)
+ {
+ this.rules = rules;
+ this.isInvalid = isInvalid;
+ this.invalidRule1 = invalidRule1;
+ this.invalidRule2 = invalidRule2;
+ }
+
+ @Parameterized.Parameters
+ public static Object[] inputsAndExpectations()
+ {
+ return new Object[][] {
+ // Invalid rules
+ {getRules(LOAD_PT1H, LOAD_2021_2022, LOAD_FOREVER, LOAD_P3M), true, LOAD_FOREVER, LOAD_P3M},
+ {getRules(LOAD_PT1H, LOAD_P3M, LOAD_PT1H_EXLUDE_FUTURE, LOAD_1980_2050), true, LOAD_PT1H, LOAD_PT1H_EXLUDE_FUTURE},
+ {getRules(LOAD_PT1H, LOAD_P3M, LOAD_2020_2023, LOAD_P3M), true, LOAD_P3M, LOAD_P3M},
+ {getRules(LOAD_2020_2023, LOAD_2021_2022, LOAD_P3M, LOAD_FOREVER), true, LOAD_2020_2023, LOAD_2021_2022},
+ {getRules(LOAD_P3M, LOAD_2021_2022, LOAD_PT1H, LOAD_2020_2023), true, LOAD_P3M, LOAD_PT1H},
+ {getRules(LOAD_P3M, LOAD_2021_2022, LOAD_FOREVER, LOAD_2020_2023), true, LOAD_FOREVER, LOAD_2020_2023},
+ {getRules(DROP_BEFORE_P3M, DROP_P1M, DROP_BEFORE_P6M, DROP_FOREVER), true, DROP_BEFORE_P3M, DROP_BEFORE_P6M},
+ {getRules(DROP_P2M, DROP_P1M, DROP_FOREVER), true, DROP_P2M, DROP_P1M},
+ {getRules(DROP_2000_2020, DROP_P1M, DROP_P2M, DROP_2010_2020), true, DROP_2000_2020, DROP_2010_2020},
+ {getRules(DROP_P1M, DROP_FOREVER, DROP_P2M), true, DROP_FOREVER, DROP_P2M},
+ {getRules(BROADCAST_2000_2050, BROADCAST_PT1H, BROADCAST_2010_2020, BROADCAST_FOREVER), true, BROADCAST_2000_2050, BROADCAST_2010_2020},
+ {getRules(BROADCAST_PT2H, BROADCAST_2000_2050, BROADCAST_PT1H, BROADCAST_FOREVER), true, BROADCAST_PT2H, BROADCAST_PT1H},
+ {getRules(BROADCAST_PT1H, BROADCAST_PT1H_EXCLUDE_FUTURE, BROADCAST_FOREVER), true, BROADCAST_PT1H, BROADCAST_PT1H_EXCLUDE_FUTURE},
+ {getRules(BROADCAST_PT1H, BROADCAST_PT2H, BROADCAST_2010_2020, BROADCAST_FOREVER, BROADCAST_2000_2050), true, BROADCAST_FOREVER, BROADCAST_2000_2050},
+ {getRules(LOAD_PT1H, LOAD_1980_2050, LOAD_P3M, LOAD_FOREVER, DROP_FOREVER), true, LOAD_FOREVER, DROP_FOREVER},
+
+ // Valid rules
+ {null, false, null, null},
+ {getRules(), false, null, null},
+ {getRules(LOAD_FOREVER), false, null, null},
+ {getRules(LOAD_PT1H_EXLUDE_FUTURE, LOAD_PT1H, LOAD_P3M, LOAD_FOREVER), false, null, null},
+ {getRules(DROP_2010_2020, DROP_2000_2020, DROP_P1M, DROP_P2M, DROP_BEFORE_P3M, DROP_FOREVER), false, null, null},
+ {getRules(BROADCAST_PT1H, BROADCAST_PT2H, BROADCAST_2010_2020, BROADCAST_2000_2050, BROADCAST_FOREVER), false, null, null},
+ {getRules(DROP_BEFORE_P6M, DROP_P1M, DROP_BEFORE_P3M, LOAD_2020_2023, DROP_FOREVER), false, null, null},
+ {getRules(DROP_2000_2020, LOAD_PT1H, BROADCAST_2000_2050, LOAD_1980_2050, LOAD_P3M, LOAD_FOREVER), false, null, null},
+ };
+ }
+
+ private static ArrayList getRules(Rule... rules)
+ {
+ return new ArrayList<>(Arrays.asList(rules));
+ }
+
+ @Test
+ public void testValidateRules()
+ {
+ if (this.isInvalid) {
+ DruidExceptionMatcher.invalidInput().expectMessageContains(
+ StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].",
+ invalidRule1, invalidRule2
+ )
+ ).assertThrowsAndMatches(() -> Rules.validateRules(rules));
+ } else {
+ Rules.validateRules(rules);
+ }
+ }
+}
diff --git a/server/src/test/java/org/apache/druid/server/http/RulesResourceTest.java b/server/src/test/java/org/apache/druid/server/http/RulesResourceTest.java
index 7ec3b5acc351..16e394ccc3c3 100644
--- a/server/src/test/java/org/apache/druid/server/http/RulesResourceTest.java
+++ b/server/src/test/java/org/apache/druid/server/http/RulesResourceTest.java
@@ -28,12 +28,8 @@
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.metadata.MetadataRuleManager;
-import org.apache.druid.server.coordinator.rules.ForeverBroadcastDistributionRule;
-import org.apache.druid.server.coordinator.rules.ForeverDropRule;
import org.apache.druid.server.coordinator.rules.ForeverLoadRule;
-import org.apache.druid.server.coordinator.rules.IntervalBroadcastDistributionRule;
import org.apache.druid.server.coordinator.rules.IntervalDropRule;
-import org.apache.druid.server.coordinator.rules.IntervalLoadRule;
import org.apache.druid.server.coordinator.rules.PeriodBroadcastDistributionRule;
import org.apache.druid.server.coordinator.rules.PeriodDropBeforeRule;
import org.apache.druid.server.coordinator.rules.PeriodDropRule;
@@ -57,6 +53,17 @@ public class RulesResourceTest
private MetadataRuleManager databaseRuleManager;
private AuditManager auditManager;
+ private static final PeriodLoadRule LOAD_P3M = new PeriodLoadRule(
+ new Period("P3M"), true, null, null
+ );
+ private final ForeverLoadRule LOAD_FOREVER = new ForeverLoadRule(null, null);
+ private static final PeriodDropRule DROP_P2M = new PeriodDropRule(new Period("P2M"), true);
+ private static final PeriodDropBeforeRule DROP_BEFORE_P6M = new PeriodDropBeforeRule(new Period("P6M"));
+ private static final IntervalDropRule DROP_2010_2020 = new IntervalDropRule(Intervals.of("2010/2020"));
+ private static final PeriodBroadcastDistributionRule BROADCAST_PT2H = new PeriodBroadcastDistributionRule(
+ new Period("PT2H"), true
+ );
+
@Before
public void setUp()
{
@@ -184,344 +191,47 @@ public void testSetDatasourceRulesWithEffectivelyNoRule()
}
@Test
- public void testSetDatasourceRulesWithInvalidLoadRules()
- {
- EasyMock.replay(auditManager);
-
- final RulesResource rulesResource = new RulesResource(databaseRuleManager, auditManager);
- final HttpServletRequest req = EasyMock.createMock(HttpServletRequest.class);
-
- final PeriodLoadRule loadPT1H = new PeriodLoadRule(new Period("PT1H"), true, null, null);
- final PeriodLoadRule loadPT1HNoFuture = new PeriodLoadRule(new Period("PT1H"), false, null, null);
- final PeriodLoadRule loadP3M = new PeriodLoadRule(new Period("P3M"), true, null, null);
- final PeriodLoadRule loadP6M = new PeriodLoadRule(new Period("P6M"), true, null, null);
- final IntervalLoadRule loadInterval2020To2023 = new IntervalLoadRule(
- Intervals.of("2020/2023"),
- null,
- null
- );
- final IntervalLoadRule loadInterval2021To2022 = new IntervalLoadRule(
- Intervals.of("2021/2022"),
- null,
- null
- );
- final ForeverLoadRule loadForever = new ForeverLoadRule(null, null);
-
- final List rules = new ArrayList<>();
- rules.add(loadP6M);
- rules.add(loadP3M);
- rules.add(loadForever);
-
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", loadP6M, loadP3M)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
-
- rules.clear();
- rules.add(loadP3M);
- rules.add(loadForever);
- rules.add(loadP6M);
-
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", loadForever, loadP6M)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
-
- rules.clear();
- rules.add(loadForever);
- rules.add(loadPT1H);
- rules.add(loadInterval2021To2022);
- rules.add(loadP6M);
-
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", loadForever, loadPT1H)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
-
- rules.clear();
- rules.add(loadPT1H);
- rules.add(loadPT1HNoFuture);
- rules.add(loadP6M);
-
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", loadPT1H, loadPT1HNoFuture)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
-
- rules.clear();
- rules.add(loadPT1H);
- rules.add(loadPT1H);
- rules.add(loadP6M);
-
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", loadPT1H, loadPT1H)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
-
- rules.clear();
- rules.add(loadInterval2020To2023);
- rules.add(loadInterval2021To2022);
- rules.add(loadP6M);
- rules.add(loadForever);
-
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", loadInterval2020To2023, loadInterval2021To2022)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
-
- rules.clear();
- rules.add(loadP6M);
- rules.add(loadInterval2021To2022);
- rules.add(loadP3M);
- rules.add(loadInterval2020To2023);
-
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", loadP6M, loadP3M)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
-
-
- rules.clear();
- rules.add(loadP6M);
- rules.add(loadInterval2020To2023);
- rules.add(loadForever);
- rules.add(loadInterval2021To2022);
-
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", loadInterval2020To2023, loadInterval2021To2022)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
- }
-
- @Test
- public void testDatasourceRulesWithInvalidDropRules()
- {
- EasyMock.replay(auditManager);
-
- final RulesResource rulesResource = new RulesResource(databaseRuleManager, auditManager);
- final HttpServletRequest req = EasyMock.createMock(HttpServletRequest.class);
-
- final PeriodDropBeforeRule dropBeforeByP3M = new PeriodDropBeforeRule(new Period("P3M"));
- final PeriodDropBeforeRule dropBeforeByP6M = new PeriodDropBeforeRule(new Period("P6M"));
- final PeriodDropRule dropByP1M = new PeriodDropRule(new Period("P1M"), true);
- final IntervalDropRule dropInterval2000To2020 = new IntervalDropRule(Intervals.of("2000/2020"));
- final IntervalDropRule dropInterval2010To2020 = new IntervalDropRule(Intervals.of("2010/2020"));
- final PeriodDropRule dropByP1MNoFuture = new PeriodDropRule(new Period("P1M"), false);
- final PeriodDropRule dropByP2M = new PeriodDropRule(new Period("P2M"), true);
- final ForeverDropRule dropForever = new ForeverDropRule();
-
- final List rules = new ArrayList<>();
- rules.add(dropBeforeByP3M);
- rules.add(dropBeforeByP6M);
- rules.add(dropByP1M);
- rules.add(dropForever);
-
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", dropBeforeByP3M, dropBeforeByP6M)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
-
- rules.clear();
- rules.add(dropByP2M);
- rules.add(dropByP1M);
- rules.add(dropForever);
-
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", dropByP2M, dropByP1M)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
-
- rules.clear();
- rules.add(dropInterval2000To2020);
- rules.add(dropByP1M);
- rules.add(dropByP2M);
- rules.add(dropInterval2010To2020);
-
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", dropInterval2000To2020, dropInterval2010To2020)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
-
- rules.clear();
- rules.add(dropByP1M);
- rules.add(dropByP1MNoFuture);
- rules.add(dropForever);
-
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", dropByP1M, dropByP1MNoFuture)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
-
-
- rules.clear();
- rules.add(dropForever);
- rules.add(dropByP1M);
- rules.add(dropByP2M);
-
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", dropForever, dropByP1M)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
- }
-
- @Test
- public void testDatasourceRulesWithInvalidBroadcastRules()
+ public void testSetDatasourceRulesWithValidRules()
{
- EasyMock.replay(auditManager);
-
+ EasyMock.expect(databaseRuleManager.overrideRule(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()))
+ .andReturn(true).anyTimes();
+ EasyMock.replay(databaseRuleManager);
final RulesResource rulesResource = new RulesResource(databaseRuleManager, auditManager);
- final HttpServletRequest req = EasyMock.createMock(HttpServletRequest.class);
-
- final ForeverBroadcastDistributionRule broadcastForever = new ForeverBroadcastDistributionRule();
- final PeriodBroadcastDistributionRule broadcastPT1H = new PeriodBroadcastDistributionRule(new Period("PT1H"), true);
- final PeriodBroadcastDistributionRule broadcastPT1HNoFuture = new PeriodBroadcastDistributionRule(new Period("PT1H"), false);
- final PeriodBroadcastDistributionRule broadcastPT2H = new PeriodBroadcastDistributionRule(new Period("PT2H"), true);
- final IntervalBroadcastDistributionRule broadcastInterval2000To2050 = new IntervalBroadcastDistributionRule(
- Intervals.of("2000/2050")
- );
- final IntervalBroadcastDistributionRule broadcastInterval2010To2020 = new IntervalBroadcastDistributionRule(
- Intervals.of("2010/2020")
- );
final List rules = new ArrayList<>();
- rules.add(broadcastInterval2000To2050);
- rules.add(broadcastInterval2010To2020);
- rules.add(broadcastForever);
-
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", broadcastInterval2000To2050, broadcastInterval2010To2020)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
-
- rules.clear();
- rules.add(broadcastPT2H);
- rules.add(broadcastPT1H);
- rules.add(broadcastForever);
-
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", broadcastPT2H, broadcastPT1H)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
-
- rules.clear();
- rules.add(broadcastPT1H);
- rules.add(broadcastPT1H);
- rules.add(broadcastForever);
-
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", broadcastPT1H, broadcastPT1H)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
-
- // Same interval with and without future.
- rules.clear();
- rules.add(broadcastPT1H);
- rules.add(broadcastPT1HNoFuture);
- rules.add(broadcastForever);
+ rules.add(BROADCAST_PT2H);
+ rules.add(DROP_P2M);
+ rules.add(DROP_2010_2020);
+ rules.add(LOAD_P3M);
+ rules.add(DROP_BEFORE_P6M);
+ rules.add(LOAD_FOREVER);
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", broadcastPT1H, broadcastPT1HNoFuture)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
-
- rules.clear();
- rules.add(broadcastPT1H);
- rules.add(broadcastPT2H);
- rules.add(broadcastInterval2010To2020);
- rules.add(broadcastForever);
- rules.add(broadcastInterval2000To2050);
-
- DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", broadcastForever, broadcastInterval2000To2050)
- ).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
+ final Response resp = rulesResource.setDatasourceRules("dataSource1", rules, null, null, EasyMock.createMock(HttpServletRequest.class));
+ Assert.assertEquals(200, resp.getStatus());
+ EasyMock.verify(databaseRuleManager);
}
@Test
- public void testSetDatasourceRulesWithDifferentInvalidRules()
+ public void testSetDatasourceRulesWithInvalidRules()
{
EasyMock.replay(auditManager);
final RulesResource rulesResource = new RulesResource(databaseRuleManager, auditManager);
final HttpServletRequest req = EasyMock.createMock(HttpServletRequest.class);
- final IntervalLoadRule loadInterval1980To2050 = new IntervalLoadRule(
- Intervals.of("1980/2050"),
- null,
- null
- );
- final IntervalLoadRule loadInterval2020To2025 = new IntervalLoadRule(
- Intervals.of("2020/2025"),
- null,
- null
- );
- final PeriodLoadRule loadPT1H = new PeriodLoadRule(new Period("PT1H"), null, null, null);
- final PeriodLoadRule loadP3M = new PeriodLoadRule(new Period("P3M"), null, null, null);
- final PeriodLoadRule loadP6M = new PeriodLoadRule(new Period("P6M"), null, null, null);
- final ForeverLoadRule loadForever = new ForeverLoadRule(null, null);
- final ForeverDropRule dropForever = new ForeverDropRule();
- final PeriodBroadcastDistributionRule broadcastPT15m = new PeriodBroadcastDistributionRule(new Period("PT15m"), true);
-
final List rules = new ArrayList<>();
- rules.add(loadInterval2020To2025);
- rules.add(broadcastPT15m);
- rules.add(loadInterval1980To2050);
- rules.add(loadPT1H);
- rules.add(loadP3M);
- rules.add(loadP6M);
- rules.add(loadForever);
- rules.add(dropForever);
+ rules.add(DROP_P2M);
+ rules.add(LOAD_P3M);
+ rules.add(DROP_BEFORE_P6M);
+ rules.add(BROADCAST_PT2H);
+ rules.add(DROP_2010_2020);
+ rules.add(LOAD_FOREVER);
DruidExceptionMatcher.invalidInput().expectMessageContains(
- StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", loadForever, dropForever)
+ StringUtils.format("Rule[%s] has an interval that fully contains the interval for rule[%s].", DROP_P2M, BROADCAST_PT2H)
).assertThrowsAndMatches(() -> rulesResource.setDatasourceRules("dataSource1", rules, null, null, req));
}
- @Test
- public void testSetDatasourceRulesWithValidRules()
- {
- EasyMock.expect(databaseRuleManager.overrideRule(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()))
- .andReturn(true).anyTimes();
- EasyMock.replay(databaseRuleManager);
- final RulesResource rulesResource = new RulesResource(databaseRuleManager, auditManager);
-
- final IntervalLoadRule loadLargeInteval = new IntervalLoadRule(
- Intervals.of("1980-07-29T01:00:00Z/2050-12-30T01:00:00Z"),
- null,
- null
- );
- final IntervalLoadRule loadSmallInterval = new IntervalLoadRule(
- Intervals.of("2020-09-29T01:00:00Z/2025-10-30T01:00:00Z"),
- null,
- null
- );
- final PeriodLoadRule loadPT1H = new PeriodLoadRule(new Period("PT1H"), true, null, null);
- final PeriodLoadRule loadPT1HNoFuture = new PeriodLoadRule(new Period("PT1H"), false, null, null);
- final PeriodLoadRule loadP3M = new PeriodLoadRule(new Period("P3M"), true, null, null);
- final PeriodLoadRule loadP6M = new PeriodLoadRule(new Period("P6M"), true, null, null);
- final PeriodDropRule dropByP1M = new PeriodDropRule(new Period("P1M"), true);
- final PeriodDropRule dropByP1MNoFuture = new PeriodDropRule(new Period("P1M"), false);
- final ForeverLoadRule loadForever = new ForeverLoadRule(null, null);
- final ForeverDropRule dropForever = new ForeverDropRule();
- final PeriodBroadcastDistributionRule broadcastPT15m = new PeriodBroadcastDistributionRule(new Period("PT15m"), true);
- final PeriodBroadcastDistributionRule broadcastPT15mNoFuture = new PeriodBroadcastDistributionRule(new Period("PT15m"), false);
-
- final List rules = new ArrayList<>();
- rules.add(loadSmallInterval);
- rules.add(loadLargeInteval);
- rules.add(broadcastPT15m);
- rules.add(loadPT1HNoFuture);
- rules.add(loadPT1H);
- rules.add(dropByP1MNoFuture);
- rules.add(dropByP1M);
- rules.add(loadP3M);
- rules.add(loadP6M);
- rules.add(loadForever);
-
- final Response resp = rulesResource.setDatasourceRules("dataSource1", rules, null, null, EasyMock.createMock(HttpServletRequest.class));
- Assert.assertEquals(200, resp.getStatus());
- EasyMock.verify(databaseRuleManager);
-
- rules.clear();
- rules.add(broadcastPT15mNoFuture);
- rules.add(broadcastPT15m);
- rules.add(loadPT1HNoFuture);
- rules.add(loadPT1H);
- rules.add(dropByP1MNoFuture);
- rules.add(dropByP1M);
- rules.add(loadP3M);
- rules.add(loadP6M);
- rules.add(loadSmallInterval);
- rules.add(loadLargeInteval);
- rules.add(dropForever);
-
- final Response resp2 = rulesResource.setDatasourceRules("dataSource1", rules, null, null, EasyMock.createMock(HttpServletRequest.class));
- Assert.assertEquals(200, resp2.getStatus());
- EasyMock.verify(databaseRuleManager);
- }
-
@Test
public void testGetAllDatasourcesRuleHistoryWithCount()
{