update joda-time and fix min/max instant

This commit is contained in:
Xavier Léauté 2014-12-02 16:55:32 -08:00
parent c3bcf34c64
commit a79389a9e5
11 changed files with 25 additions and 15 deletions

View File

@ -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();

View File

@ -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)))
);
}

View File

@ -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

View File

@ -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>

View File

@ -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

View File

@ -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();

View File

@ -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());

View File

@ -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());

View File

@ -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);

View File

@ -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()

View File

@ -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);