mirror of https://github.com/apache/druid.git
update joda-time and fix min/max instant
This commit is contained in:
parent
c3bcf34c64
commit
a79389a9e5
|
@ -36,6 +36,10 @@ import java.util.TreeSet;
|
|||
*/
|
||||
public class JodaUtils
|
||||
{
|
||||
// joda limits years to [-292275054,292278993] that should be reasonable
|
||||
public static final long MAX_INSTANT = new DateTime("292278993").getMillis();
|
||||
public static final long MIN_INSTANT = new DateTime("-292275054").getMillis();
|
||||
|
||||
public static ArrayList<Interval> condenseIntervals(Iterable<Interval> intervals)
|
||||
{
|
||||
ArrayList<Interval> retVal = Lists.newArrayList();
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.google.common.collect.Iterables;
|
|||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.Pair;
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
import io.druid.timeline.partition.ImmutablePartitionHolder;
|
||||
import io.druid.timeline.partition.IntegerPartitionChunk;
|
||||
import io.druid.timeline.partition.PartitionChunk;
|
||||
|
@ -1294,7 +1295,7 @@ public class VersionedIntervalTimelineTest
|
|||
createExpected("2011-01-05/2011-01-10", "2", 2),
|
||||
createExpected("2011-01-10/2011-01-15", "3", 3)
|
||||
),
|
||||
timeline.lookup(new Interval(new DateTime(0), new DateTime(Long.MAX_VALUE)))
|
||||
timeline.lookup(new Interval(new DateTime(0), new DateTime(JodaUtils.MAX_INSTANT)))
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -36,6 +36,7 @@ import com.metamx.common.Pair;
|
|||
import com.metamx.common.guava.Comparators;
|
||||
import com.metamx.common.guava.FunctionalIterable;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
import io.druid.indexing.common.TaskLock;
|
||||
import io.druid.indexing.common.task.Task;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -462,11 +463,11 @@ public class TaskLockbox
|
|||
final NavigableSet<Interval> dsLockbox = dsRunning.navigableKeySet();
|
||||
final Iterable<Interval> searchIntervals = Iterables.concat(
|
||||
// Single interval that starts at or before ours
|
||||
Collections.singletonList(dsLockbox.floor(new Interval(interval.getStart(), new DateTime(Long.MAX_VALUE)))),
|
||||
Collections.singletonList(dsLockbox.floor(new Interval(interval.getStart(), new DateTime(JodaUtils.MAX_INSTANT)))),
|
||||
|
||||
// All intervals that start somewhere between our start instant (exclusive) and end instant (exclusive)
|
||||
dsLockbox.subSet(
|
||||
new Interval(interval.getStart(), new DateTime(Long.MAX_VALUE)),
|
||||
new Interval(interval.getStart(), new DateTime(JodaUtils.MAX_INSTANT)),
|
||||
false,
|
||||
new Interval(interval.getEnd(), interval.getEnd()),
|
||||
false
|
||||
|
|
2
pom.xml
2
pom.xml
|
@ -348,7 +348,7 @@
|
|||
<dependency>
|
||||
<groupId>joda-time</groupId>
|
||||
<artifactId>joda-time</artifactId>
|
||||
<version>2.1</version>
|
||||
<version>2.6</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.code.findbugs</groupId>
|
||||
|
|
|
@ -20,19 +20,20 @@
|
|||
package io.druid.granularity;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
|
||||
public final class AllGranularity extends BaseQueryGranularity
|
||||
{
|
||||
@Override
|
||||
public long next(long offset)
|
||||
{
|
||||
return Long.MAX_VALUE;
|
||||
return JodaUtils.MAX_INSTANT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long truncate(long offset)
|
||||
{
|
||||
return Long.MIN_VALUE;
|
||||
return JodaUtils.MIN_INSTANT;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.base.Charsets;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
import io.druid.query.BaseQuery;
|
||||
import io.druid.query.DataSource;
|
||||
import io.druid.query.Query;
|
||||
|
@ -155,8 +156,8 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
return Lists.newArrayList();
|
||||
}
|
||||
|
||||
DateTime min = new DateTime(Long.MAX_VALUE);
|
||||
DateTime max = new DateTime(Long.MIN_VALUE);
|
||||
DateTime min = new DateTime(JodaUtils.MAX_INSTANT);
|
||||
DateTime max = new DateTime(JodaUtils.MIN_INSTANT);
|
||||
for (Result<TimeBoundaryResultValue> result : results) {
|
||||
TimeBoundaryResultValue val = result.getValue();
|
||||
|
||||
|
|
|
@ -1363,8 +1363,8 @@ public class IndexMaker
|
|||
cols.writeToChannel(writer);
|
||||
dims.writeToChannel(writer);
|
||||
|
||||
DateTime minTime = new DateTime(Long.MAX_VALUE);
|
||||
DateTime maxTime = new DateTime(0l);
|
||||
DateTime minTime = new DateTime(JodaUtils.MAX_INSTANT);
|
||||
DateTime maxTime = new DateTime(JodaUtils.MIN_INSTANT);
|
||||
|
||||
for (IndexableAdapter index : adapters) {
|
||||
minTime = JodaUtils.minDateTime(minTime, index.getDataInterval().getStart());
|
||||
|
|
|
@ -457,8 +457,8 @@ public class IndexMerger
|
|||
GenericIndexed.fromIterable(mergedDimensions, GenericIndexed.stringStrategy).writeToChannel(channel);
|
||||
GenericIndexed.fromIterable(mergedMetrics, GenericIndexed.stringStrategy).writeToChannel(channel);
|
||||
|
||||
DateTime minTime = new DateTime(Long.MAX_VALUE);
|
||||
DateTime maxTime = new DateTime(0l);
|
||||
DateTime minTime = new DateTime(JodaUtils.MAX_INSTANT);
|
||||
DateTime maxTime = new DateTime(JodaUtils.MIN_INSTANT);
|
||||
|
||||
for (IndexableAdapter index : indexes) {
|
||||
minTime = JodaUtils.minDateTime(minTime, index.getDataInterval().getStart());
|
||||
|
|
|
@ -28,6 +28,7 @@ import com.google.common.collect.PeekingIterator;
|
|||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.Granularity;
|
||||
import com.metamx.common.guava.Comparators;
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
@ -90,7 +91,7 @@ public class ArbitraryGranularitySpec implements GranularitySpec
|
|||
public Optional<Interval> bucketInterval(DateTime dt)
|
||||
{
|
||||
// First interval with start time ≤ dt
|
||||
final Interval interval = intervals.floor(new Interval(dt, new DateTime(Long.MAX_VALUE)));
|
||||
final Interval interval = intervals.floor(new Interval(dt, new DateTime(JodaUtils.MAX_INSTANT)));
|
||||
|
||||
if (interval != null && interval.contains(dt)) {
|
||||
return Optional.of(interval);
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package io.druid.segment.realtime.plumber;
|
||||
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Period;
|
||||
|
||||
|
@ -31,7 +32,7 @@ public class MessageTimeRejectionPolicyFactory implements RejectionPolicyFactory
|
|||
|
||||
return new RejectionPolicy()
|
||||
{
|
||||
private volatile long maxTimestamp = Long.MIN_VALUE;
|
||||
private volatile long maxTimestamp = JodaUtils.MIN_INSTANT;
|
||||
|
||||
@Override
|
||||
public DateTime getCurrMaxTime()
|
||||
|
|
|
@ -193,7 +193,7 @@ public class RealtimePlumberSchoolTest
|
|||
public void testPersist() throws Exception
|
||||
{
|
||||
final MutableBoolean committed = new MutableBoolean(false);
|
||||
plumber.getSinks().put(0L, new Sink(new Interval(0, TimeUnit.HOURS.toMillis(1)),schema, tuningConfig, DateTime.now().toString()));
|
||||
plumber.getSinks().put(0L, new Sink(new Interval(0, TimeUnit.HOURS.toMillis(1)),schema, tuningConfig, new DateTime("2014-12-01T12:34:56.789").toString()));
|
||||
plumber.startJob();
|
||||
final InputRow row = EasyMock.createNiceMock(InputRow.class);
|
||||
EasyMock.expect(row.getTimestampFromEpoch()).andReturn(0L);
|
||||
|
|
Loading…
Reference in New Issue