mirror of https://github.com/apache/druid.git
make reference timestamp explicit for time based rules
This commit is contained in:
parent
2a188996cd
commit
19c3fd3363
|
@ -23,6 +23,7 @@ import com.metamx.druid.client.DataSegment;
|
|||
import com.metamx.druid.db.DatabaseRuleManager;
|
||||
import com.metamx.druid.master.rules.Rule;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
|
@ -51,13 +52,14 @@ public class DruidMasterRuleRunner implements DruidMasterHelper
|
|||
}
|
||||
|
||||
// Run through all matched rules for available segments
|
||||
DateTime now = new DateTime();
|
||||
DatabaseRuleManager databaseRuleManager = params.getDatabaseRuleManager();
|
||||
for (DataSegment segment : params.getAvailableSegments()) {
|
||||
List<Rule> rules = databaseRuleManager.getRulesWithDefault(segment.getDataSource());
|
||||
|
||||
boolean foundMatchingRule = false;
|
||||
for (Rule rule : rules) {
|
||||
if (rule.appliesTo(segment)) {
|
||||
if (rule.appliesTo(segment, now)) {
|
||||
stats.accumulate(rule.run(master, params, segment));
|
||||
foundMatchingRule = true;
|
||||
break;
|
||||
|
|
|
@ -22,6 +22,7 @@ package com.metamx.druid.master.rules;
|
|||
import com.metamx.druid.client.DataSegment;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
/**
|
||||
|
@ -52,7 +53,7 @@ public class IntervalDropRule extends DropRule
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean appliesTo(DataSegment segment)
|
||||
public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp)
|
||||
{
|
||||
return interval.contains(segment.getInterval());
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.metamx.common.logger.Logger;
|
|||
import com.metamx.druid.client.DataSegment;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
/**
|
||||
|
@ -81,7 +82,7 @@ public class IntervalLoadRule extends LoadRule
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean appliesTo(DataSegment segment)
|
||||
public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp)
|
||||
{
|
||||
return interval.contains(segment.getInterval());
|
||||
}
|
||||
|
|
|
@ -54,9 +54,9 @@ public class PeriodDropRule extends DropRule
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean appliesTo(DataSegment segment)
|
||||
public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp)
|
||||
{
|
||||
final Interval currInterval = new Interval(new DateTime().minus(period), period);
|
||||
final Interval currInterval = new Interval(referenceTimestamp.minus(period), period);
|
||||
return currInterval.contains(segment.getInterval());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -81,9 +81,9 @@ public class PeriodLoadRule extends LoadRule
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean appliesTo(DataSegment segment)
|
||||
public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp)
|
||||
{
|
||||
final Interval currInterval = new Interval(period, new DateTime());
|
||||
final Interval currInterval = new Interval(period, referenceTimestamp);
|
||||
return currInterval.overlaps(segment.getInterval());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import com.metamx.druid.master.DruidMasterRuntimeParams;
|
|||
import com.metamx.druid.master.MasterStats;
|
||||
import org.codehaus.jackson.annotate.JsonSubTypes;
|
||||
import org.codehaus.jackson.annotate.JsonTypeInfo;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -40,7 +41,7 @@ public interface Rule
|
|||
{
|
||||
public String getType();
|
||||
|
||||
public boolean appliesTo(DataSegment segment);
|
||||
public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp);
|
||||
|
||||
public MasterStats run(DruidMaster master, DruidMasterRuntimeParams params, DataSegment segment);
|
||||
}
|
||||
|
|
|
@ -51,13 +51,15 @@ public class PeriodDropRuleTest
|
|||
now.minusDays(2),
|
||||
now.minusDays(1)
|
||||
)
|
||||
).build()
|
||||
).build(),
|
||||
now
|
||||
)
|
||||
);
|
||||
Assert.assertTrue(
|
||||
rule.appliesTo(
|
||||
builder.interval(new Interval(now.minusYears(100), now.minusDays(1)))
|
||||
.build()
|
||||
.build(),
|
||||
now
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -73,19 +75,22 @@ public class PeriodDropRuleTest
|
|||
Assert.assertTrue(
|
||||
rule.appliesTo(
|
||||
builder.interval(new Interval(now.minusWeeks(1), now.minusDays(1)))
|
||||
.build()
|
||||
.build(),
|
||||
now
|
||||
)
|
||||
);
|
||||
Assert.assertFalse(
|
||||
rule.appliesTo(
|
||||
builder.interval(new Interval(now.minusYears(1), now.minusDays(1)))
|
||||
.build()
|
||||
.build(),
|
||||
now
|
||||
)
|
||||
);
|
||||
Assert.assertFalse(
|
||||
rule.appliesTo(
|
||||
builder.interval(new Interval(now.minusMonths(2), now.minusDays(1)))
|
||||
.build()
|
||||
.build(),
|
||||
now
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -39,38 +39,41 @@ public class PeriodLoadRuleTest
|
|||
@Test
|
||||
public void testAppliesToAll()
|
||||
{
|
||||
DateTime now = new DateTime("2013-01-01");
|
||||
PeriodLoadRule rule = new PeriodLoadRule(
|
||||
new Period("P5000Y"),
|
||||
0,
|
||||
""
|
||||
);
|
||||
|
||||
Assert.assertTrue(rule.appliesTo(builder.interval(new Interval("2012-01-01/2012-12-31")).build()));
|
||||
Assert.assertTrue(rule.appliesTo(builder.interval(new Interval("1000-01-01/2012-12-31")).build()));
|
||||
Assert.assertTrue(rule.appliesTo(builder.interval(new Interval("0500-01-01/2100-12-31")).build()));
|
||||
Assert.assertTrue(rule.appliesTo(builder.interval(new Interval("2012-01-01/2012-12-31")).build(), now));
|
||||
Assert.assertTrue(rule.appliesTo(builder.interval(new Interval("1000-01-01/2012-12-31")).build(), now));
|
||||
Assert.assertTrue(rule.appliesTo(builder.interval(new Interval("0500-01-01/2100-12-31")).build(), now));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAppliesToPeriod()
|
||||
{
|
||||
DateTime now = new DateTime();
|
||||
DateTime now = new DateTime("2012-12-31T01:00:00");
|
||||
PeriodLoadRule rule = new PeriodLoadRule(
|
||||
new Period("P1M"),
|
||||
0,
|
||||
""
|
||||
);
|
||||
|
||||
Assert.assertTrue(rule.appliesTo(builder.interval(new Interval(now.minusWeeks(1), now)).build()));
|
||||
Assert.assertTrue(rule.appliesTo(builder.interval(new Interval(now.minusWeeks(1), now)).build(), now));
|
||||
Assert.assertTrue(
|
||||
rule.appliesTo(
|
||||
builder.interval(new Interval(now.minusDays(1), now.plusDays(1)))
|
||||
.build()
|
||||
.build(),
|
||||
now
|
||||
)
|
||||
);
|
||||
Assert.assertFalse(
|
||||
rule.appliesTo(
|
||||
builder.interval(new Interval(now.plusDays(1), now.plusDays(2)))
|
||||
.build()
|
||||
.build(),
|
||||
now
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue