Add forbidden-apis plugin; prohibit using system time zone (#4611)

* Forbidden APIs WIP

* Remove some tests

* Restore io.druid.math.expr.Function

* Integration tests fix

* Add comments

* Fix in SimpleWorkerProvisioningStrategy

* Formatting

* Replace String.format() with StringUtils.format() in RemoteTaskRunnerTest

* Address comments

* Fix GroupByMultiSegmentTest
This commit is contained in:
Roman Leventov 2017-08-21 15:02:42 -05:00 committed by Charles Allen
parent fa87eaa6e8
commit cbd1902db8
448 changed files with 4231 additions and 4104 deletions

View File

@ -19,6 +19,7 @@
package io.druid.data.input; package io.druid.data.input;
import io.druid.java.util.common.DateTimes;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import java.util.List; import java.util.List;
@ -60,7 +61,7 @@ public class MapBasedInputRow extends MapBasedRow implements InputRow
public String toString() public String toString()
{ {
return "MapBasedInputRow{" + return "MapBasedInputRow{" +
"timestamp=" + new DateTime(getTimestampFromEpoch()) + "timestamp=" + DateTimes.utc(getTimestampFromEpoch()) +
", event=" + getEvent() + ", event=" + getEvent() +
", dimensions=" + dimensions + ", dimensions=" + dimensions +
'}'; '}';

View File

@ -22,6 +22,7 @@ package io.druid.data.input;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.parsers.ParseException; import io.druid.java.util.common.parsers.ParseException;
import org.joda.time.DateTime; import org.joda.time.DateTime;
@ -54,7 +55,7 @@ public class MapBasedRow implements Row
Map<String, Object> event Map<String, Object> event
) )
{ {
this(new DateTime(timestamp), event); this(DateTimes.utc(timestamp), event);
} }
@Override @Override

View File

@ -91,7 +91,7 @@ public class DataSegmentUtils
return new SegmentIdentifierParts( return new SegmentIdentifierParts(
dataSource, dataSource,
new Interval(start.getMillis(), end.getMillis()), new Interval(start, end),
version, version,
trail trail
); );

View File

@ -29,6 +29,7 @@ import com.fasterxml.jackson.databind.SerializationFeature;
import com.fasterxml.jackson.databind.deser.std.StdDeserializer; import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
import com.fasterxml.jackson.databind.module.SimpleModule; import com.fasterxml.jackson.databind.module.SimpleModule;
import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; import com.fasterxml.jackson.databind.ser.std.ToStringSerializer;
import io.druid.java.util.common.Intervals;
import org.joda.time.Interval; import org.joda.time.Interval;
import java.io.IOException; import java.io.IOException;
@ -61,7 +62,7 @@ public class TestObjectMapper extends ObjectMapper
JsonParser jsonParser, DeserializationContext deserializationContext JsonParser jsonParser, DeserializationContext deserializationContext
) throws IOException, JsonProcessingException ) throws IOException, JsonProcessingException
{ {
return new Interval(jsonParser.getText()); return Intervals.of(jsonParser.getText());
} }
} }
); );

View File

@ -19,19 +19,18 @@
package io.druid.data.input; package io.druid.data.input;
import org.joda.time.DateTime; import com.google.common.collect.ImmutableMap;
import io.druid.java.util.common.DateTimes;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import com.google.common.collect.ImmutableMap;
public class MapBasedRowTest public class MapBasedRowTest
{ {
@Test @Test
public void testGetLongMetricFromString() public void testGetLongMetricFromString()
{ {
MapBasedRow row = new MapBasedRow( MapBasedRow row = new MapBasedRow(
new DateTime(), DateTimes.nowUtc(),
ImmutableMap.<String, Object>builder() ImmutableMap.<String, Object>builder()
.put("k0", "-1.2") .put("k0", "-1.2")
.put("k1", "1.23") .put("k1", "1.23")

View File

@ -27,8 +27,8 @@ import com.google.common.collect.Lists;
import io.druid.TestObjectMapper; import io.druid.TestObjectMapper;
import io.druid.data.input.ByteBufferInputRowParser; import io.druid.data.input.ByteBufferInputRowParser;
import io.druid.data.input.InputRow; import io.druid.data.input.InputRow;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import org.joda.time.DateTime;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -63,7 +63,7 @@ public class InputRowParserSerdeTest
Assert.assertEquals(ImmutableList.of("foo", "bar"), parsed.getDimensions()); Assert.assertEquals(ImmutableList.of("foo", "bar"), parsed.getDimensions());
Assert.assertEquals(ImmutableList.of("x"), parsed.getDimension("foo")); Assert.assertEquals(ImmutableList.of("x"), parsed.getDimension("foo"));
Assert.assertEquals(ImmutableList.of("y"), parsed.getDimension("bar")); Assert.assertEquals(ImmutableList.of("y"), parsed.getDimension("bar"));
Assert.assertEquals(new DateTime("2000").getMillis(), parsed.getTimestampFromEpoch()); Assert.assertEquals(DateTimes.of("2000").getMillis(), parsed.getTimestampFromEpoch());
} }
@Test @Test
@ -79,7 +79,7 @@ public class InputRowParserSerdeTest
Assert.assertEquals(ImmutableList.of("foo", "bar"), parsed.getDimensions()); Assert.assertEquals(ImmutableList.of("foo", "bar"), parsed.getDimensions());
Assert.assertEquals(ImmutableList.of("x"), parsed.getDimension("foo")); Assert.assertEquals(ImmutableList.of("x"), parsed.getDimension("foo"));
Assert.assertEquals(ImmutableList.of("y"), parsed.getDimension("bar")); Assert.assertEquals(ImmutableList.of("y"), parsed.getDimension("bar"));
Assert.assertEquals(new DateTime("3000").getMillis(), parsed.getTimestampFromEpoch()); Assert.assertEquals(DateTimes.of("3000").getMillis(), parsed.getTimestampFromEpoch());
} }
} }
@ -219,7 +219,7 @@ public class InputRowParserSerdeTest
Assert.assertEquals(ImmutableList.of("asdf"), parsed.getDimension("hey0barx")); Assert.assertEquals(ImmutableList.of("asdf"), parsed.getDimension("hey0barx"));
Assert.assertEquals(ImmutableList.of("456"), parsed.getDimension("metA")); Assert.assertEquals(ImmutableList.of("456"), parsed.getDimension("metA"));
Assert.assertEquals(ImmutableList.of("5"), parsed.getDimension("newmet")); Assert.assertEquals(ImmutableList.of("5"), parsed.getDimension("newmet"));
Assert.assertEquals(new DateTime("2999").getMillis(), parsed.getTimestampFromEpoch()); Assert.assertEquals(DateTimes.of("2999").getMillis(), parsed.getTimestampFromEpoch());
String testSpec = "{\"enabled\": true,\"useFieldDiscovery\": true, \"fields\": [\"parseThisRootField\"]}"; String testSpec = "{\"enabled\": true,\"useFieldDiscovery\": true, \"fields\": [\"parseThisRootField\"]}";
final JSONPathSpec parsedSpec = jsonMapper.readValue(testSpec, JSONPathSpec.class); final JSONPathSpec parsedSpec = jsonMapper.readValue(testSpec, JSONPathSpec.class);

View File

@ -24,10 +24,10 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import io.druid.data.input.Firehose; import io.druid.data.input.Firehose;
import io.druid.data.input.Row; import io.druid.data.input.Row;
import io.druid.java.util.common.DateTimes;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.commons.io.filefilter.TrueFileFilter; import org.apache.commons.io.filefilter.TrueFileFilter;
import org.hamcrest.CoreMatchers; import org.hamcrest.CoreMatchers;
import org.joda.time.DateTime;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
import org.junit.BeforeClass; import org.junit.BeforeClass;
@ -124,7 +124,7 @@ public class PrefetchableTextFilesFirehoseFactoryTest
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {
for (int j = 0; j < 100; j++) { for (int j = 0; j < 100; j++) {
final Row row = rows.get(i * 100 + j); final Row row = rows.get(i * 100 + j);
Assert.assertEquals(new DateTime(20171220 + i), row.getTimestamp()); Assert.assertEquals(DateTimes.utc(20171220 + i), row.getTimestamp());
Assert.assertEquals(String.valueOf(i), row.getDimension("a").get(0)); Assert.assertEquals(String.valueOf(i), row.getDimension("a").get(0));
Assert.assertEquals(String.valueOf(j), row.getDimension("b").get(0)); Assert.assertEquals(String.valueOf(j), row.getDimension("b").get(0));
} }

View File

@ -20,6 +20,7 @@
package io.druid.data.input.impl; package io.druid.data.input.impl;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import io.druid.java.util.common.DateTimes;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.format.ISODateTimeFormat; import org.joda.time.format.ISODateTimeFormat;
import org.junit.Assert; import org.junit.Assert;
@ -32,7 +33,7 @@ public class TimestampSpecTest
{ {
TimestampSpec spec = new TimestampSpec("TIMEstamp", "yyyy-MM-dd", null); TimestampSpec spec = new TimestampSpec("TIMEstamp", "yyyy-MM-dd", null);
Assert.assertEquals( Assert.assertEquals(
new DateTime("2014-03-01"), DateTimes.of("2014-03-01"),
spec.extractTimestamp(ImmutableMap.<String, Object>of("TIMEstamp", "2014-03-01")) spec.extractTimestamp(ImmutableMap.<String, Object>of("TIMEstamp", "2014-03-01"))
); );
} }
@ -40,9 +41,9 @@ public class TimestampSpecTest
@Test @Test
public void testExtractTimestampWithMissingTimestampColumn() throws Exception public void testExtractTimestampWithMissingTimestampColumn() throws Exception
{ {
TimestampSpec spec = new TimestampSpec(null, null, new DateTime(0)); TimestampSpec spec = new TimestampSpec(null, null, DateTimes.EPOCH);
Assert.assertEquals( Assert.assertEquals(
new DateTime("1970-01-01"), DateTimes.of("1970-01-01"),
spec.extractTimestamp(ImmutableMap.<String, Object>of("dim", "foo")) spec.extractTimestamp(ImmutableMap.<String, Object>of("dim", "foo"))
); );
} }

View File

@ -28,11 +28,12 @@ import com.google.common.collect.Range;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import io.druid.TestObjectMapper; import io.druid.TestObjectMapper;
import io.druid.data.input.InputRow; import io.druid.data.input.InputRow;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.Intervals;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import io.druid.timeline.partition.PartitionChunk; import io.druid.timeline.partition.PartitionChunk;
import io.druid.timeline.partition.ShardSpec; import io.druid.timeline.partition.ShardSpec;
import io.druid.timeline.partition.ShardSpecLookup; import io.druid.timeline.partition.ShardSpecLookup;
import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -90,7 +91,7 @@ public class DataSegmentTest
public void testV1Serialization() throws Exception public void testV1Serialization() throws Exception
{ {
final Interval interval = new Interval("2011-10-01/2011-10-02"); final Interval interval = Intervals.of("2011-10-01/2011-10-02");
final ImmutableMap<String, Object> loadSpec = ImmutableMap.<String, Object>of("something", "or_other"); final ImmutableMap<String, Object> loadSpec = ImmutableMap.<String, Object>of("something", "or_other");
DataSegment segment = new DataSegment( DataSegment segment = new DataSegment(
@ -150,8 +151,8 @@ public class DataSegmentTest
{ {
final DataSegment segment = DataSegment.builder() final DataSegment segment = DataSegment.builder()
.dataSource("foo") .dataSource("foo")
.interval(new Interval("2012-01-01/2012-01-02")) .interval(Intervals.of("2012-01-01/2012-01-02"))
.version(new DateTime("2012-01-01T11:22:33.444Z").toString()) .version(DateTimes.of("2012-01-01T11:22:33.444Z").toString())
.shardSpec(NoneShardSpec.instance()) .shardSpec(NoneShardSpec.instance())
.build(); .build();
@ -166,8 +167,8 @@ public class DataSegmentTest
{ {
final DataSegment segment = DataSegment.builder() final DataSegment segment = DataSegment.builder()
.dataSource("foo") .dataSource("foo")
.interval(new Interval("2012-01-01/2012-01-02")) .interval(Intervals.of("2012-01-01/2012-01-02"))
.version(new DateTime("2012-01-01T11:22:33.444Z").toString()) .version(DateTimes.of("2012-01-01T11:22:33.444Z").toString())
.shardSpec(getShardSpec(0)) .shardSpec(getShardSpec(0))
.build(); .build();
@ -182,8 +183,8 @@ public class DataSegmentTest
{ {
final DataSegment segment = DataSegment.builder() final DataSegment segment = DataSegment.builder()
.dataSource("foo") .dataSource("foo")
.interval(new Interval("2012-01-01/2012-01-02")) .interval(Intervals.of("2012-01-01/2012-01-02"))
.version(new DateTime("2012-01-01T11:22:33.444Z").toString()) .version(DateTimes.of("2012-01-01T11:22:33.444Z").toString())
.shardSpec(getShardSpec(7)) .shardSpec(getShardSpec(7))
.build(); .build();
@ -198,8 +199,8 @@ public class DataSegmentTest
{ {
final DataSegment segment = DataSegment.builder() final DataSegment segment = DataSegment.builder()
.dataSource("foo") .dataSource("foo")
.interval(new Interval("2012-01-01/2012-01-02")) .interval(Intervals.of("2012-01-01/2012-01-02"))
.version(new DateTime("2012-01-01T11:22:33.444Z").toString()) .version(DateTimes.of("2012-01-01T11:22:33.444Z").toString())
.build(); .build();
final DataSegment segment2 = mapper.readValue(mapper.writeValueAsString(segment), DataSegment.class); final DataSegment segment2 = mapper.readValue(mapper.writeValueAsString(segment), DataSegment.class);
@ -240,7 +241,7 @@ public class DataSegmentTest
{ {
return DataSegment.builder() return DataSegment.builder()
.dataSource(dataSource) .dataSource(dataSource)
.interval(new Interval(interval)) .interval(Intervals.of(interval))
.version(version) .version(version)
.size(1) .size(1)
.build(); .build();

View File

@ -19,8 +19,8 @@
package io.druid.timeline; package io.druid.timeline;
import io.druid.java.util.common.Intervals;
import io.druid.timeline.DataSegmentUtils.SegmentIdentifierParts; import io.druid.timeline.DataSegmentUtils.SegmentIdentifierParts;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -32,19 +32,19 @@ public class DataSegmentUtilsTest
public void testBasic() public void testBasic()
{ {
String datasource = "datasource"; String datasource = "datasource";
SegmentIdentifierParts desc = new SegmentIdentifierParts(datasource, new Interval("2015-01-02/2015-01-03"), "ver", "0_0"); SegmentIdentifierParts desc = new SegmentIdentifierParts(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver", "0_0");
Assert.assertEquals("datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_0", desc.toString()); Assert.assertEquals("datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_0", desc.toString());
Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString())); Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
desc = desc.withInterval(new Interval("2014-10-20T00:00:00Z/P1D")); desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals("datasource_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_0", desc.toString()); Assert.assertEquals("datasource_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_0", desc.toString());
Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString())); Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
desc = new SegmentIdentifierParts(datasource, new Interval("2015-01-02/2015-01-03"), "ver", null); desc = new SegmentIdentifierParts(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver", null);
Assert.assertEquals("datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver", desc.toString()); Assert.assertEquals("datasource_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver", desc.toString());
Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString())); Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
desc = desc.withInterval(new Interval("2014-10-20T00:00:00Z/P1D")); desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals("datasource_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver", desc.toString()); Assert.assertEquals("datasource_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver", desc.toString());
Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString())); Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
} }
@ -53,19 +53,19 @@ public class DataSegmentUtilsTest
public void testDataSourceWithUnderscore1() public void testDataSourceWithUnderscore1()
{ {
String datasource = "datasource_1"; String datasource = "datasource_1";
SegmentIdentifierParts desc = new SegmentIdentifierParts(datasource, new Interval("2015-01-02/2015-01-03"), "ver", "0_0"); SegmentIdentifierParts desc = new SegmentIdentifierParts(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver", "0_0");
Assert.assertEquals("datasource_1_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_0", desc.toString()); Assert.assertEquals("datasource_1_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_0", desc.toString());
Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString())); Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
desc = desc.withInterval(new Interval("2014-10-20T00:00:00Z/P1D")); desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals("datasource_1_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_0", desc.toString()); Assert.assertEquals("datasource_1_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_0", desc.toString());
Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString())); Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
desc = new SegmentIdentifierParts(datasource, new Interval("2015-01-02/2015-01-03"), "ver", null); desc = new SegmentIdentifierParts(datasource, Intervals.of("2015-01-02/2015-01-03"), "ver", null);
Assert.assertEquals("datasource_1_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver", desc.toString()); Assert.assertEquals("datasource_1_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver", desc.toString());
Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString())); Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
desc = desc.withInterval(new Interval("2014-10-20T00:00:00Z/P1D")); desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals("datasource_1_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver", desc.toString()); Assert.assertEquals("datasource_1_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver", desc.toString());
Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString())); Assert.assertEquals(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
} }
@ -74,28 +74,28 @@ public class DataSegmentUtilsTest
public void testDataSourceWithUnderscore2() public void testDataSourceWithUnderscore2()
{ {
String dataSource = "datasource_2015-01-01T00:00:00.000Z"; String dataSource = "datasource_2015-01-01T00:00:00.000Z";
SegmentIdentifierParts desc = new SegmentIdentifierParts(dataSource, new Interval("2015-01-02/2015-01-03"), "ver", "0_0"); SegmentIdentifierParts desc = new SegmentIdentifierParts(dataSource, Intervals.of("2015-01-02/2015-01-03"), "ver", "0_0");
Assert.assertEquals( Assert.assertEquals(
"datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_0", "datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver_0_0",
desc.toString() desc.toString()
); );
Assert.assertEquals(desc, DataSegmentUtils.valueOf(dataSource, desc.toString())); Assert.assertEquals(desc, DataSegmentUtils.valueOf(dataSource, desc.toString()));
desc = desc.withInterval(new Interval("2014-10-20T00:00:00Z/P1D")); desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals( Assert.assertEquals(
"datasource_2015-01-01T00:00:00.000Z_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_0", "datasource_2015-01-01T00:00:00.000Z_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver_0_0",
desc.toString() desc.toString()
); );
Assert.assertEquals(desc, DataSegmentUtils.valueOf(dataSource, desc.toString())); Assert.assertEquals(desc, DataSegmentUtils.valueOf(dataSource, desc.toString()));
desc = new SegmentIdentifierParts(dataSource, new Interval("2015-01-02/2015-01-03"), "ver", null); desc = new SegmentIdentifierParts(dataSource, Intervals.of("2015-01-02/2015-01-03"), "ver", null);
Assert.assertEquals( Assert.assertEquals(
"datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver", "datasource_2015-01-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver",
desc.toString() desc.toString()
); );
Assert.assertEquals(desc, DataSegmentUtils.valueOf(dataSource, desc.toString())); Assert.assertEquals(desc, DataSegmentUtils.valueOf(dataSource, desc.toString()));
desc = desc.withInterval(new Interval("2014-10-20T00:00:00Z/P1D")); desc = desc.withInterval(Intervals.of("2014-10-20T00:00:00Z/P1D"));
Assert.assertEquals( Assert.assertEquals(
"datasource_2015-01-01T00:00:00.000Z_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver", "datasource_2015-01-01T00:00:00.000Z_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver",
desc.toString() desc.toString()

View File

@ -24,6 +24,7 @@ import com.google.common.collect.Iterables;
import io.druid.benchmark.datagen.BenchmarkColumnSchema; import io.druid.benchmark.datagen.BenchmarkColumnSchema;
import io.druid.benchmark.datagen.BenchmarkSchemaInfo; import io.druid.benchmark.datagen.BenchmarkSchemaInfo;
import io.druid.benchmark.datagen.SegmentGenerator; import io.druid.benchmark.datagen.SegmentGenerator;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Sequences;
@ -41,7 +42,6 @@ import io.druid.segment.VirtualColumns;
import io.druid.segment.column.ValueType; import io.druid.segment.column.ValueType;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.LinearShardSpec; import io.druid.timeline.partition.LinearShardSpec;
import org.joda.time.Interval;
import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Fork;
@ -89,7 +89,7 @@ public class ExpressionBenchmark
BenchmarkColumnSchema.makeNormal("y", ValueType.FLOAT, false, 1, 0d, 0d, 10000d, false) BenchmarkColumnSchema.makeNormal("y", ValueType.FLOAT, false, 1, 0d, 0d, 10000d, false)
), ),
ImmutableList.of(), ImmutableList.of(),
new Interval("2000/P1D"), Intervals.of("2000/P1D"),
false false
); );

View File

@ -39,6 +39,7 @@ import org.openjdk.jmh.runner.options.OptionsBuilder;
import java.text.SimpleDateFormat; import java.text.SimpleDateFormat;
import java.util.Date; import java.util.Date;
import java.util.Locale;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@State(Scope.Benchmark) @State(Scope.Benchmark)
@ -60,7 +61,7 @@ public class TimeParseBenchmark
@Setup @Setup
public void setup() public void setup()
{ {
SimpleDateFormat format = new SimpleDateFormat(DATA_FORMAT); SimpleDateFormat format = new SimpleDateFormat(DATA_FORMAT, Locale.ENGLISH);
long start = System.currentTimeMillis(); long start = System.currentTimeMillis();
int rowsPerBatch = numRows / numBatches; int rowsPerBatch = numRows / numBatches;
int numRowInBatch = 0; int numRowInBatch = 0;

View File

@ -20,6 +20,7 @@
package io.druid.benchmark.datagen; package io.druid.benchmark.datagen;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import io.druid.java.util.common.Intervals;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.query.aggregation.DoubleMinAggregatorFactory; import io.druid.query.aggregation.DoubleMinAggregatorFactory;
@ -84,7 +85,7 @@ public class BenchmarkSchemas
basicSchemaIngestAggs.add(new DoubleMinAggregatorFactory("minFloatZipf", "metFloatZipf")); basicSchemaIngestAggs.add(new DoubleMinAggregatorFactory("minFloatZipf", "metFloatZipf"));
basicSchemaIngestAggs.add(new HyperUniquesAggregatorFactory("hyper", "dimHyperUnique")); basicSchemaIngestAggs.add(new HyperUniquesAggregatorFactory("hyper", "dimHyperUnique"));
Interval basicSchemaDataInterval = new Interval(0, 1000000); Interval basicSchemaDataInterval = Intervals.utc(0, 1000000);
BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo( BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo(
basicSchemaColumns, basicSchemaColumns,
@ -104,7 +105,7 @@ public class BenchmarkSchemas
List<AggregatorFactory> basicSchemaIngestAggs = new ArrayList<>(); List<AggregatorFactory> basicSchemaIngestAggs = new ArrayList<>();
basicSchemaIngestAggs.add(new CountAggregatorFactory("rows")); basicSchemaIngestAggs.add(new CountAggregatorFactory("rows"));
Interval basicSchemaDataInterval = new Interval(0, 1000000); Interval basicSchemaDataInterval = Intervals.utc(0, 1000000);
BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo( BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo(
basicSchemaColumns, basicSchemaColumns,
@ -125,7 +126,7 @@ public class BenchmarkSchemas
basicSchemaIngestAggs.add(new LongSumAggregatorFactory("dimSequential", "dimSequential")); basicSchemaIngestAggs.add(new LongSumAggregatorFactory("dimSequential", "dimSequential"));
basicSchemaIngestAggs.add(new CountAggregatorFactory("rows")); basicSchemaIngestAggs.add(new CountAggregatorFactory("rows"));
Interval basicSchemaDataInterval = new Interval(0, 1000000); Interval basicSchemaDataInterval = Intervals.utc(0, 1000000);
BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo( BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo(
basicSchemaColumns, basicSchemaColumns,
@ -146,7 +147,7 @@ public class BenchmarkSchemas
basicSchemaIngestAggs.add(new DoubleSumAggregatorFactory("dimSequential", "dimSequential")); basicSchemaIngestAggs.add(new DoubleSumAggregatorFactory("dimSequential", "dimSequential"));
basicSchemaIngestAggs.add(new CountAggregatorFactory("rows")); basicSchemaIngestAggs.add(new CountAggregatorFactory("rows"));
Interval basicSchemaDataInterval = new Interval(0, 1000000); Interval basicSchemaDataInterval = Intervals.utc(0, 1000000);
BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo( BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo(
basicSchemaColumns, basicSchemaColumns,

View File

@ -25,7 +25,7 @@ import com.google.common.io.Files;
import io.druid.benchmark.datagen.BenchmarkSchemaInfo; import io.druid.benchmark.datagen.BenchmarkSchemaInfo;
import io.druid.benchmark.datagen.BenchmarkSchemas; import io.druid.benchmark.datagen.BenchmarkSchemas;
import io.druid.benchmark.datagen.SegmentGenerator; import io.druid.benchmark.datagen.SegmentGenerator;
import io.druid.common.utils.JodaUtils; import io.druid.java.util.common.Intervals;
import io.druid.data.input.Row; import io.druid.data.input.Row;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequence;
@ -48,7 +48,6 @@ import io.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.LinearShardSpec; import io.druid.timeline.partition.LinearShardSpec;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.joda.time.Interval;
import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Fork;
@ -124,7 +123,7 @@ public class SqlBenchmark
groupByQuery = GroupByQuery groupByQuery = GroupByQuery
.builder() .builder()
.setDataSource("foo") .setDataSource("foo")
.setInterval(new Interval(JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT)) .setInterval(Intervals.ETERNITY)
.setDimensions( .setDimensions(
Arrays.<DimensionSpec>asList( Arrays.<DimensionSpec>asList(
new DefaultDimensionSpec("dimZipf", "d0"), new DefaultDimensionSpec("dimZipf", "d0"),

View File

@ -30,6 +30,7 @@ import io.druid.concurrent.Execs;
import io.druid.data.input.InputRow; import io.druid.data.input.InputRow;
import io.druid.hll.HyperLogLogHash; import io.druid.hll.HyperLogLogHash;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Sequences;
@ -72,7 +73,6 @@ import io.druid.segment.column.ColumnConfig;
import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.serde.ComplexMetrics; import io.druid.segment.serde.ComplexMetrics;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.joda.time.Interval;
import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Fork;
@ -214,7 +214,7 @@ public class TimeseriesBenchmark
basicQueries.put("timeFilterAlphanumeric", timeFilterQuery); basicQueries.put("timeFilterAlphanumeric", timeFilterQuery);
} }
{ {
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(new Interval(200000, 300000))); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.utc(200000, 300000)));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
LongSumAggregatorFactory lsaf = new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"); LongSumAggregatorFactory lsaf = new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential");
queryAggs.add(lsaf); queryAggs.add(lsaf);

View File

@ -19,6 +19,7 @@
package io.druid.server.coordinator; package io.druid.server.coordinator;
import io.druid.java.util.common.DateTimes;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -39,7 +40,7 @@ import java.util.concurrent.TimeUnit;
@State(Scope.Benchmark) @State(Scope.Benchmark)
public class CostBalancerStrategyBenchmark public class CostBalancerStrategyBenchmark
{ {
private final static DateTime t0 = new DateTime("2016-01-01T01:00:00Z"); private final static DateTime t0 = DateTimes.of("2016-01-01T01:00:00Z");
private List<DataSegment> segments; private List<DataSegment> segments;
private DataSegment segment; private DataSegment segment;

View File

@ -22,8 +22,8 @@ package io.druid.benchmark;
import io.druid.benchmark.datagen.BenchmarkColumnSchema; import io.druid.benchmark.datagen.BenchmarkColumnSchema;
import io.druid.benchmark.datagen.BenchmarkDataGenerator; import io.druid.benchmark.datagen.BenchmarkDataGenerator;
import io.druid.data.input.InputRow; import io.druid.data.input.InputRow;
import io.druid.java.util.common.Intervals;
import io.druid.segment.column.ValueType; import io.druid.segment.column.ValueType;
import org.joda.time.Interval;
import org.junit.Test; import org.junit.Test;
import java.util.ArrayList; import java.util.ArrayList;
@ -378,13 +378,13 @@ public class BenchmarkDataGeneratorTest
) )
); );
BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator(schemas, 9999, new Interval(50000, 600000), 100); BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator(schemas, 9999, Intervals.utc(50000, 600000), 100);
for (int i = 0; i < 100; i++) { for (int i = 0; i < 100; i++) {
InputRow row = dataGenerator.nextRow(); InputRow row = dataGenerator.nextRow();
//System.out.println("S-ROW: " + row); //System.out.println("S-ROW: " + row);
} }
BenchmarkDataGenerator dataGenerator2 = new BenchmarkDataGenerator(schemas, 9999, new Interval(50000, 50001), 100); BenchmarkDataGenerator dataGenerator2 = new BenchmarkDataGenerator(schemas, 9999, Intervals.utc(50000, 50001), 100);
for (int i = 0; i < 100; i++) { for (int i = 0; i < 100; i++) {
InputRow row = dataGenerator2.nextRow(); InputRow row = dataGenerator2.nextRow();
//System.out.println("S2-ROW: " + row); //System.out.println("S2-ROW: " + row);

View File

@ -0,0 +1,46 @@
@defaultMessage Uses default time zone
org.joda.time.DateTime#<init>()
org.joda.time.DateTime#<init>(long)
org.joda.time.DateTime#<init>(java.lang.Object)
org.joda.time.DateTime#<init>(int, int, int, int, int)
org.joda.time.DateTime#<init>(int, int, int, int, int, int)
org.joda.time.DateTime#<init>(int, int, int, int, int, int, int)
org.joda.time.DateTime#now()
org.joda.time.Instant#toDateTime()
org.joda.time.Instant#toMutableDateTime()
org.joda.time.Instant#toMutableDateTimeISO()
org.joda.time.base.AbstractInstant#toDateTimeISO()
org.joda.time.base.AbstractInstant#toDateTime()
org.joda.time.base.AbstractInstant#toMutableDateTime()
org.joda.time.base.AbstractInstant#toMutableDateTimeISO()
org.joda.time.LocalDateTime#<init>()
org.joda.time.LocalDateTime#<init>(long)
org.joda.time.LocalDateTime#<init>(java.lang.Object)
org.joda.time.LocalDateTime#now()
org.joda.time.LocalDateTime#fromDateFields(java.util.Date)
org.joda.time.LocalDateTime#toDate()
org.joda.time.LocalDateTime#toDateTime()
org.joda.time.LocalDate#<init>()
org.joda.time.LocalDate#<init>(long)
org.joda.time.LocalDate#<init>(java.lang.Object)
org.joda.time.LocalDate#fromDateFields(java.util.Date)
org.joda.time.LocalDate#now()
org.joda.time.LocalDate#toDate()
org.joda.time.LocalDate#toDateTime(org.joda.time.LocalTime)
org.joda.time.LocalDate#toDateTimeAtCurrentTime()
org.joda.time.LocalDate#toDateTimeAtStartOfDay()
org.joda.time.LocalDate#toInterval()
org.joda.time.LocalTime#<init>()
org.joda.time.LocalTime#<init>(long)
org.joda.time.LocalTime#<init>(java.lang.Object)
org.joda.time.LocalTime#fromDateFields(java.util.Date)
org.joda.time.LocalTime#now()
org.joda.time.LocalTime#toDateTimeToday()
org.joda.time.Interval#<init>(long, long)
org.joda.time.Interval#<init>(java.lang.Object)
org.joda.time.Interval#parse(java.lang.String)
org.joda.time.Interval#parseWithOffset(java.lang.String)
@defaultMessage Doesn't handle edge cases where the start of day isn't midnight.
org.joda.time.LocalDate#toDateTimeAtMidnight()
org.joda.time.DateMidnight

View File

@ -22,6 +22,7 @@ package io.druid.audit;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import io.druid.java.util.common.DateTimes;
import org.joda.time.DateTime; import org.joda.time.DateTime;
/** /**
@ -50,7 +51,7 @@ public class AuditEntry
this.key = key; this.key = key;
this.type = type; this.type = type;
this.auditInfo = authorInfo; this.auditInfo = authorInfo;
this.auditTime = auditTime == null ? DateTime.now() : auditTime; this.auditTime = auditTime == null ? DateTimes.nowUtc() : auditTime;
this.payload = payload; this.payload = payload;
} }
@ -151,7 +152,7 @@ public class AuditEntry
this.key = null; this.key = null;
this.auditInfo = null; this.auditInfo = null;
this.payload = null; this.payload = null;
this.auditTime = DateTime.now(); this.auditTime = DateTimes.nowUtc();
} }
public Builder key(String key) public Builder key(String key)

View File

@ -30,7 +30,9 @@ import org.joda.time.format.ISODateTimeFormat;
import java.util.List; import java.util.List;
/** /**
* Do NOT remove "unused" members in this class. They are used by generated Antlr
*/ */
@SuppressWarnings("unused")
interface Function interface Function
{ {
String name(); String name();
@ -1024,7 +1026,7 @@ interface Function
} }
final String arg = args.get(0).eval(bindings).asString(); final String arg = args.get(0).eval(bindings).asString();
return ExprEval.of(Strings.nullToEmpty(arg).toLowerCase()); return ExprEval.of(StringUtils.toLowerCase(Strings.nullToEmpty(arg)));
} }
} }
@ -1044,7 +1046,7 @@ interface Function
} }
final String arg = args.get(0).eval(bindings).asString(); final String arg = args.get(0).eval(bindings).asString();
return ExprEval.of(Strings.nullToEmpty(arg).toUpperCase()); return ExprEval.of(StringUtils.toUpperCase(Strings.nullToEmpty(arg)));
} }
} }
} }

View File

@ -25,7 +25,7 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.collect.Ordering; import com.google.common.collect.Ordering;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import io.druid.common.utils.JodaUtils; import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.UOE; import io.druid.java.util.common.UOE;
import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.guava.Comparators;
import io.druid.timeline.partition.ImmutablePartitionHolder; import io.druid.timeline.partition.ImmutablePartitionHolder;
@ -288,7 +288,7 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
} }
Interval lower = completePartitionsTimeline.floorKey( Interval lower = completePartitionsTimeline.floorKey(
new Interval(interval.getStartMillis(), JodaUtils.MAX_INSTANT) new Interval(interval.getStart(), DateTimes.MAX)
); );
if (lower == null || !lower.overlaps(interval)) { if (lower == null || !lower.overlaps(interval)) {

View File

@ -19,6 +19,8 @@
package io.druid.common.utils; package io.druid.common.utils;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.JodaUtils;
import org.joda.time.Duration; import org.joda.time.Duration;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.joda.time.Period; import org.joda.time.Period;
@ -37,18 +39,18 @@ public class JodaUtilsTest
public void testUmbrellaIntervalsSimple() throws Exception public void testUmbrellaIntervalsSimple() throws Exception
{ {
List<Interval> intervals = Arrays.asList( List<Interval> intervals = Arrays.asList(
new Interval("2011-03-03/2011-03-04"), Intervals.of("2011-03-03/2011-03-04"),
new Interval("2011-01-01/2011-01-02"), Intervals.of("2011-01-01/2011-01-02"),
new Interval("2011-02-01/2011-02-05"), Intervals.of("2011-02-01/2011-02-05"),
new Interval("2011-02-03/2011-02-08"), Intervals.of("2011-02-03/2011-02-08"),
new Interval("2011-01-01/2011-01-03"), Intervals.of("2011-01-01/2011-01-03"),
new Interval("2011-03-01/2011-03-02"), Intervals.of("2011-03-01/2011-03-02"),
new Interval("2011-03-05/2011-03-06"), Intervals.of("2011-03-05/2011-03-06"),
new Interval("2011-02-01/2011-02-02") Intervals.of("2011-02-01/2011-02-02")
); );
Assert.assertEquals( Assert.assertEquals(
new Interval("2011-01-01/2011-03-06"), Intervals.of("2011-01-01/2011-03-06"),
JodaUtils.umbrellaInterval(intervals) JodaUtils.umbrellaInterval(intervals)
); );
} }
@ -71,23 +73,23 @@ public class JodaUtilsTest
public void testCondenseIntervalsSimple() throws Exception public void testCondenseIntervalsSimple() throws Exception
{ {
List<Interval> intervals = Arrays.asList( List<Interval> intervals = Arrays.asList(
new Interval("2011-01-01/2011-01-02"), Intervals.of("2011-01-01/2011-01-02"),
new Interval("2011-01-02/2011-01-03"), Intervals.of("2011-01-02/2011-01-03"),
new Interval("2011-02-01/2011-02-05"), Intervals.of("2011-02-01/2011-02-05"),
new Interval("2011-02-01/2011-02-02"), Intervals.of("2011-02-01/2011-02-02"),
new Interval("2011-02-03/2011-02-08"), Intervals.of("2011-02-03/2011-02-08"),
new Interval("2011-03-01/2011-03-02"), Intervals.of("2011-03-01/2011-03-02"),
new Interval("2011-03-03/2011-03-04"), Intervals.of("2011-03-03/2011-03-04"),
new Interval("2011-03-05/2011-03-06") Intervals.of("2011-03-05/2011-03-06")
); );
Assert.assertEquals( Assert.assertEquals(
Arrays.asList( Arrays.asList(
new Interval("2011-01-01/2011-01-03"), Intervals.of("2011-01-01/2011-01-03"),
new Interval("2011-02-01/2011-02-08"), Intervals.of("2011-02-01/2011-02-08"),
new Interval("2011-03-01/2011-03-02"), Intervals.of("2011-03-01/2011-03-02"),
new Interval("2011-03-03/2011-03-04"), Intervals.of("2011-03-03/2011-03-04"),
new Interval("2011-03-05/2011-03-06") Intervals.of("2011-03-05/2011-03-06")
), ),
JodaUtils.condenseIntervals(intervals) JodaUtils.condenseIntervals(intervals)
); );
@ -97,31 +99,31 @@ public class JodaUtilsTest
public void testCondenseIntervalsMixedUp() throws Exception public void testCondenseIntervalsMixedUp() throws Exception
{ {
List<Interval> intervals = Arrays.asList( List<Interval> intervals = Arrays.asList(
new Interval("2011-01-01/2011-01-02"), Intervals.of("2011-01-01/2011-01-02"),
new Interval("2011-01-02/2011-01-03"), Intervals.of("2011-01-02/2011-01-03"),
new Interval("2011-02-01/2011-02-05"), Intervals.of("2011-02-01/2011-02-05"),
new Interval("2011-02-01/2011-02-02"), Intervals.of("2011-02-01/2011-02-02"),
new Interval("2011-02-03/2011-02-08"), Intervals.of("2011-02-03/2011-02-08"),
new Interval("2011-03-01/2011-03-02"), Intervals.of("2011-03-01/2011-03-02"),
new Interval("2011-03-03/2011-03-04"), Intervals.of("2011-03-03/2011-03-04"),
new Interval("2011-03-05/2011-03-06"), Intervals.of("2011-03-05/2011-03-06"),
new Interval("2011-04-01/2011-04-05"), Intervals.of("2011-04-01/2011-04-05"),
new Interval("2011-04-02/2011-04-03"), Intervals.of("2011-04-02/2011-04-03"),
new Interval("2011-05-01/2011-05-05"), Intervals.of("2011-05-01/2011-05-05"),
new Interval("2011-05-02/2011-05-07") Intervals.of("2011-05-02/2011-05-07")
); );
for (int i = 0; i < 20; ++i) { for (int i = 0; i < 20; ++i) {
Collections.shuffle(intervals); Collections.shuffle(intervals);
Assert.assertEquals( Assert.assertEquals(
Arrays.asList( Arrays.asList(
new Interval("2011-01-01/2011-01-03"), Intervals.of("2011-01-01/2011-01-03"),
new Interval("2011-02-01/2011-02-08"), Intervals.of("2011-02-01/2011-02-08"),
new Interval("2011-03-01/2011-03-02"), Intervals.of("2011-03-01/2011-03-02"),
new Interval("2011-03-03/2011-03-04"), Intervals.of("2011-03-03/2011-03-04"),
new Interval("2011-03-05/2011-03-06"), Intervals.of("2011-03-05/2011-03-06"),
new Interval("2011-04-01/2011-04-05"), Intervals.of("2011-04-01/2011-04-05"),
new Interval("2011-05-01/2011-05-07") Intervals.of("2011-05-01/2011-05-07")
), ),
JodaUtils.condenseIntervals(intervals) JodaUtils.condenseIntervals(intervals)
); );
@ -131,15 +133,13 @@ public class JodaUtilsTest
@Test @Test
public void testMinMaxInterval() public void testMinMaxInterval()
{ {
final Interval interval = new Interval(JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT); Assert.assertEquals(Long.MAX_VALUE, Intervals.ETERNITY.toDuration().getMillis());
Assert.assertEquals(Long.MAX_VALUE, interval.toDuration().getMillis());
} }
@Test @Test
public void testMinMaxDuration() public void testMinMaxDuration()
{ {
final Interval interval = new Interval(JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT); final Duration duration = Intervals.ETERNITY.toDuration();
final Duration duration = interval.toDuration();
Assert.assertEquals(Long.MAX_VALUE, duration.getMillis()); Assert.assertEquals(Long.MAX_VALUE, duration.getMillis());
Assert.assertEquals("PT9223372036854775.807S", duration.toString()); Assert.assertEquals("PT9223372036854775.807S", duration.toString());
} }
@ -148,8 +148,7 @@ public class JodaUtilsTest
@Test(expected = ArithmeticException.class) @Test(expected = ArithmeticException.class)
public void testMinMaxPeriod() public void testMinMaxPeriod()
{ {
final Interval interval = new Interval(JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT); final Period period = Intervals.ETERNITY.toDuration().toPeriod();
final Period period = interval.toDuration().toPeriod();
Assert.assertEquals(Long.MAX_VALUE, period.getMinutes()); Assert.assertEquals(Long.MAX_VALUE, period.getMinutes());
} }

View File

@ -76,7 +76,7 @@ public class ExecsTest
{ {
for (int i = 0; i < nTasks; i++) { for (int i = 0; i < nTasks; i++) {
final int taskID = i; final int taskID = i;
System.out.println("Produced task" + taskID); log.info("Produced task %d", taskID);
blockingExecutor.submit( blockingExecutor.submit(
new Runnable() new Runnable()
{ {

View File

@ -57,7 +57,7 @@ public class LifecycleLockTest
finishLatch.countDown(); finishLatch.countDown();
} }
catch (InterruptedException e) { catch (InterruptedException e) {
e.printStackTrace(); throw new RuntimeException(e);
} }
} }
}.start(); }.start();
@ -99,7 +99,7 @@ public class LifecycleLockTest
finishLatch.countDown(); finishLatch.countDown();
} }
catch (InterruptedException e) { catch (InterruptedException e) {
e.printStackTrace(); throw new RuntimeException(e);
} }
} }
}.start(); }.start();

View File

@ -25,8 +25,8 @@ import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.Ordering; import com.google.common.collect.Ordering;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import io.druid.java.util.common.DateTimes;
import io.druid.common.utils.JodaUtils; import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.Pair; import io.druid.java.util.common.Pair;
import io.druid.timeline.partition.ImmutablePartitionHolder; import io.druid.timeline.partition.ImmutablePartitionHolder;
import io.druid.timeline.partition.IntegerPartitionChunk; import io.druid.timeline.partition.IntegerPartitionChunk;
@ -86,7 +86,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-04-02/2011-04-06", "2", 1), createExpected("2011-04-02/2011-04-06", "2", 1),
createExpected("2011-04-06/2011-04-09", "3", 4) createExpected("2011-04-06/2011-04-09", "3", 4)
), ),
timeline.lookup(new Interval("2011-04-01/2011-04-09")) timeline.lookup(Intervals.of("2011-04-01/2011-04-09"))
); );
} }
@ -95,7 +95,7 @@ public class VersionedIntervalTimelineTest
{ {
Assert.assertEquals( Assert.assertEquals(
makeSingle(1), makeSingle(1),
timeline.remove(new Interval("2011-04-01/2011-04-09"), "2", makeSingle(1)) timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "2", makeSingle(1))
); );
assertValues( assertValues(
Arrays.asList( Arrays.asList(
@ -104,7 +104,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-04-03/2011-04-06", "1", 3), createExpected("2011-04-03/2011-04-06", "1", 3),
createExpected("2011-04-06/2011-04-09", "3", 4) createExpected("2011-04-06/2011-04-09", "3", 4)
), ),
timeline.lookup(new Interval("2011-04-01/2011-04-09")) timeline.lookup(Intervals.of("2011-04-01/2011-04-09"))
); );
} }
@ -113,11 +113,11 @@ public class VersionedIntervalTimelineTest
{ {
Assert.assertEquals( Assert.assertEquals(
makeSingle(1), makeSingle(1),
timeline.remove(new Interval("2011-04-01/2011-04-09"), "2", makeSingle(1)) timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "2", makeSingle(1))
); );
Assert.assertEquals( Assert.assertEquals(
makeSingle(2), makeSingle(2),
timeline.remove(new Interval("2011-04-01/2011-04-03"), "1", makeSingle(2)) timeline.remove(Intervals.of("2011-04-01/2011-04-03"), "1", makeSingle(2))
); );
assertValues( assertValues(
Arrays.asList( Arrays.asList(
@ -125,7 +125,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-04-03/2011-04-06", "1", 3), createExpected("2011-04-03/2011-04-06", "1", 3),
createExpected("2011-04-06/2011-04-09", "3", 4) createExpected("2011-04-06/2011-04-09", "3", 4)
), ),
timeline.lookup(new Interval("2011-04-01/2011-04-09")) timeline.lookup(Intervals.of("2011-04-01/2011-04-09"))
); );
} }
@ -134,7 +134,7 @@ public class VersionedIntervalTimelineTest
{ {
Assert.assertEquals( Assert.assertEquals(
makeSingle(1), makeSingle(1),
timeline.remove(new Interval("2011-04-01/2011-04-09"), "2", makeSingle(1)) timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "2", makeSingle(1))
); );
assertValues( assertValues(
Arrays.asList( Arrays.asList(
@ -142,7 +142,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-04-02/2011-04-03", "1", 2), createExpected("2011-04-02/2011-04-03", "1", 2),
createExpected("2011-04-03/2011-04-05", "1", 3) createExpected("2011-04-03/2011-04-05", "1", 3)
), ),
timeline.lookup(new Interval("2011-04-01/2011-04-05")) timeline.lookup(Intervals.of("2011-04-01/2011-04-05"))
); );
assertValues( assertValues(
@ -150,7 +150,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-04-02T18/2011-04-03", "1", 2), createExpected("2011-04-02T18/2011-04-03", "1", 2),
createExpected("2011-04-03/2011-04-04T01", "1", 3) createExpected("2011-04-03/2011-04-04T01", "1", 3)
), ),
timeline.lookup(new Interval("2011-04-02T18/2011-04-04T01")) timeline.lookup(Intervals.of("2011-04-02T18/2011-04-04T01"))
); );
} }
@ -161,21 +161,21 @@ public class VersionedIntervalTimelineTest
Collections.singletonList( Collections.singletonList(
createExpected("2011-05-01/2011-05-09", "4", 9) createExpected("2011-05-01/2011-05-09", "4", 9)
), ),
timeline.lookup(new Interval("2011-05-01/2011-05-09")) timeline.lookup(Intervals.of("2011-05-01/2011-05-09"))
); );
} }
@Test @Test
public void testMay2() throws Exception public void testMay2() throws Exception
{ {
Assert.assertNotNull(timeline.remove(new Interval("2011-05-01/2011-05-10"), "4", makeSingle(1))); Assert.assertNotNull(timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "4", makeSingle(1)));
assertValues( assertValues(
Arrays.asList( Arrays.asList(
createExpected("2011-05-01/2011-05-03", "2", 7), createExpected("2011-05-01/2011-05-03", "2", 7),
createExpected("2011-05-03/2011-05-04", "3", 8), createExpected("2011-05-03/2011-05-04", "3", 8),
createExpected("2011-05-04/2011-05-05", "2", 7) createExpected("2011-05-04/2011-05-05", "2", 7)
), ),
timeline.lookup(new Interval("2011-05-01/2011-05-09")) timeline.lookup(Intervals.of("2011-05-01/2011-05-09"))
); );
} }
@ -184,25 +184,25 @@ public class VersionedIntervalTimelineTest
{ {
Assert.assertEquals( Assert.assertEquals(
makeSingle(9), makeSingle(9),
timeline.remove(new Interval("2011-05-01/2011-05-10"), "4", makeSingle(9)) timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "4", makeSingle(9))
); );
Assert.assertEquals( Assert.assertEquals(
makeSingle(7), makeSingle(7),
timeline.remove(new Interval("2011-05-01/2011-05-05"), "2", makeSingle(7)) timeline.remove(Intervals.of("2011-05-01/2011-05-05"), "2", makeSingle(7))
); );
assertValues( assertValues(
Arrays.asList( Arrays.asList(
createExpected("2011-05-01/2011-05-02", "1", 6), createExpected("2011-05-01/2011-05-02", "1", 6),
createExpected("2011-05-03/2011-05-04", "3", 8) createExpected("2011-05-03/2011-05-04", "3", 8)
), ),
timeline.lookup(new Interval("2011-05-01/2011-05-09")) timeline.lookup(Intervals.of("2011-05-01/2011-05-09"))
); );
} }
@Test @Test
public void testInsertInWrongOrder() throws Exception public void testInsertInWrongOrder() throws Exception
{ {
DateTime overallStart = new DateTime().minus(Hours.TWO); DateTime overallStart = DateTimes.nowUtc().minus(Hours.TWO);
Assert.assertTrue( Assert.assertTrue(
"These timestamps have to be at the end AND include now for this test to work.", "These timestamps have to be at the end AND include now for this test to work.",
@ -241,32 +241,32 @@ public class VersionedIntervalTimelineTest
{ {
Assert.assertEquals( Assert.assertEquals(
new ImmutablePartitionHolder<Integer>(new PartitionHolder<Integer>(makeSingle(1))), new ImmutablePartitionHolder<Integer>(new PartitionHolder<Integer>(makeSingle(1))),
timeline.findEntry(new Interval("2011-10-01/2011-10-02"), "1") timeline.findEntry(Intervals.of("2011-10-01/2011-10-02"), "1")
); );
Assert.assertEquals( Assert.assertEquals(
new ImmutablePartitionHolder<Integer>(new PartitionHolder<Integer>(makeSingle(1))), new ImmutablePartitionHolder<Integer>(new PartitionHolder<Integer>(makeSingle(1))),
timeline.findEntry(new Interval("2011-10-01/2011-10-01T10"), "1") timeline.findEntry(Intervals.of("2011-10-01/2011-10-01T10"), "1")
); );
Assert.assertEquals( Assert.assertEquals(
new ImmutablePartitionHolder<Integer>(new PartitionHolder<Integer>(makeSingle(1))), new ImmutablePartitionHolder<Integer>(new PartitionHolder<Integer>(makeSingle(1))),
timeline.findEntry(new Interval("2011-10-01T02/2011-10-02"), "1") timeline.findEntry(Intervals.of("2011-10-01T02/2011-10-02"), "1")
); );
Assert.assertEquals( Assert.assertEquals(
new ImmutablePartitionHolder<Integer>(new PartitionHolder<Integer>(makeSingle(1))), new ImmutablePartitionHolder<Integer>(new PartitionHolder<Integer>(makeSingle(1))),
timeline.findEntry(new Interval("2011-10-01T04/2011-10-01T17"), "1") timeline.findEntry(Intervals.of("2011-10-01T04/2011-10-01T17"), "1")
); );
Assert.assertEquals( Assert.assertEquals(
null, null,
timeline.findEntry(new Interval("2011-10-01T04/2011-10-01T17"), "2") timeline.findEntry(Intervals.of("2011-10-01T04/2011-10-01T17"), "2")
); );
Assert.assertEquals( Assert.assertEquals(
null, null,
timeline.findEntry(new Interval("2011-10-01T04/2011-10-02T17"), "1") timeline.findEntry(Intervals.of("2011-10-01T04/2011-10-02T17"), "1")
); );
} }
@ -280,7 +280,7 @@ public class VersionedIntervalTimelineTest
Assert.assertEquals( Assert.assertEquals(
new ImmutablePartitionHolder<Integer>(new PartitionHolder<Integer>(makeSingle(1))), new ImmutablePartitionHolder<Integer>(new PartitionHolder<Integer>(makeSingle(1))),
timeline.findEntry(new Interval("2011-01-02T02/2011-01-04"), "1") timeline.findEntry(Intervals.of("2011-01-02T02/2011-01-04"), "1")
); );
} }
@ -301,7 +301,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-10-04/2011-10-05", "4", 4), createExpected("2011-10-04/2011-10-05", "4", 4),
createExpected("2011-10-05/2011-10-06", "5", 5) createExpected("2011-10-05/2011-10-06", "5", 5)
), ),
timeline.lookup(new Interval("2011-10-01/2011-10-06")) timeline.lookup(Intervals.of("2011-10-01/2011-10-06"))
); );
} }
@ -313,14 +313,14 @@ public class VersionedIntervalTimelineTest
add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(null, 10, 0, 60)); add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(null, 10, 0, 60));
assertValues( assertValues(
ImmutableList.of(createExpected("2011-10-05/2011-10-06", "5", 5)), ImmutableList.of(createExpected("2011-10-05/2011-10-06", "5", 5)),
timeline.lookup(new Interval("2011-10-05/2011-10-07")) timeline.lookup(Intervals.of("2011-10-05/2011-10-07"))
); );
Assert.assertTrue("Expected no overshadowed entries", timeline.findOvershadowed().isEmpty()); Assert.assertTrue("Expected no overshadowed entries", timeline.findOvershadowed().isEmpty());
add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(10, 20, 1, 61)); add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(10, 20, 1, 61));
assertValues( assertValues(
ImmutableList.of(createExpected("2011-10-05/2011-10-06", "5", 5)), ImmutableList.of(createExpected("2011-10-05/2011-10-06", "5", 5)),
timeline.lookup(new Interval("2011-10-05/2011-10-07")) timeline.lookup(Intervals.of("2011-10-05/2011-10-07"))
); );
Assert.assertTrue("Expected no overshadowed entries", timeline.findOvershadowed().isEmpty()); Assert.assertTrue("Expected no overshadowed entries", timeline.findOvershadowed().isEmpty());
@ -337,7 +337,7 @@ public class VersionedIntervalTimelineTest
) )
) )
), ),
timeline.lookup(new Interval("2011-10-05/2011-10-07")) timeline.lookup(Intervals.of("2011-10-05/2011-10-07"))
); );
Assert.assertTrue("Expected no overshadowed entries", timeline.findOvershadowed().isEmpty()); Assert.assertTrue("Expected no overshadowed entries", timeline.findOvershadowed().isEmpty());
} }
@ -368,10 +368,10 @@ public class VersionedIntervalTimelineTest
testIncompletePartitionDoesNotOvershadow(); testIncompletePartitionDoesNotOvershadow();
final IntegerPartitionChunk<Integer> chunk = IntegerPartitionChunk.make(null, 10, 0, 60); final IntegerPartitionChunk<Integer> chunk = IntegerPartitionChunk.make(null, 10, 0, 60);
Assert.assertEquals(chunk, timeline.remove(new Interval("2011-10-05/2011-10-07"), "6", chunk)); Assert.assertEquals(chunk, timeline.remove(Intervals.of("2011-10-05/2011-10-07"), "6", chunk));
assertValues( assertValues(
ImmutableList.of(createExpected("2011-10-05/2011-10-06", "5", 5)), ImmutableList.of(createExpected("2011-10-05/2011-10-06", "5", 5)),
timeline.lookup(new Interval("2011-10-05/2011-10-07")) timeline.lookup(Intervals.of("2011-10-05/2011-10-07"))
); );
Assert.assertTrue("Expected no overshadowed entries", timeline.findOvershadowed().isEmpty()); Assert.assertTrue("Expected no overshadowed entries", timeline.findOvershadowed().isEmpty());
} }
@ -384,18 +384,18 @@ public class VersionedIntervalTimelineTest
Collections.singletonList( Collections.singletonList(
createExpected("2011-05-01/2011-05-09", "5", 10) createExpected("2011-05-01/2011-05-09", "5", 10)
), ),
timeline.lookup(new Interval("2011-05-01/2011-05-09")) timeline.lookup(Intervals.of("2011-05-01/2011-05-09"))
); );
Assert.assertEquals( Assert.assertEquals(
makeSingle(10), makeSingle(10),
timeline.remove(new Interval("2011-05-01/2011-05-10"), "5", makeSingle(10)) timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "5", makeSingle(10))
); );
assertValues( assertValues(
Collections.singletonList( Collections.singletonList(
createExpected("2011-05-01/2011-05-09", "4", 9) createExpected("2011-05-01/2011-05-09", "4", 9)
), ),
timeline.lookup(new Interval("2011-05-01/2011-05-09")) timeline.lookup(Intervals.of("2011-05-01/2011-05-09"))
); );
add("2011-05-01/2011-05-10", "5", 10); add("2011-05-01/2011-05-10", "5", 10);
@ -403,18 +403,18 @@ public class VersionedIntervalTimelineTest
Collections.singletonList( Collections.singletonList(
createExpected("2011-05-01/2011-05-09", "5", 10) createExpected("2011-05-01/2011-05-09", "5", 10)
), ),
timeline.lookup(new Interval("2011-05-01/2011-05-09")) timeline.lookup(Intervals.of("2011-05-01/2011-05-09"))
); );
Assert.assertEquals( Assert.assertEquals(
makeSingle(9), makeSingle(9),
timeline.remove(new Interval("2011-05-01/2011-05-10"), "4", makeSingle(9)) timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "4", makeSingle(9))
); );
assertValues( assertValues(
Collections.singletonList( Collections.singletonList(
createExpected("2011-05-01/2011-05-09", "5", 10) createExpected("2011-05-01/2011-05-09", "5", 10)
), ),
timeline.lookup(new Interval("2011-05-01/2011-05-09")) timeline.lookup(Intervals.of("2011-05-01/2011-05-09"))
); );
} }
@ -454,7 +454,7 @@ public class VersionedIntervalTimelineTest
Collections.singletonList( Collections.singletonList(
createExpected("2011-01-01/2011-01-10", "2", 2) createExpected("2011-01-01/2011-01-10", "2", 2)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-10")) timeline.lookup(Intervals.of("2011-01-01/2011-01-10"))
); );
} }
@ -475,7 +475,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-05/2011-01-15", "2", 3), createExpected("2011-01-05/2011-01-15", "2", 3),
createExpected("2011-01-15/2011-01-20", "1", 2) createExpected("2011-01-15/2011-01-20", "1", 2)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -496,7 +496,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-05/2011-01-15", "2", 3), createExpected("2011-01-05/2011-01-15", "2", 3),
createExpected("2011-01-15/2011-01-20", "1", 2) createExpected("2011-01-15/2011-01-20", "1", 2)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -515,7 +515,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-01/2011-01-05", "1", 1), createExpected("2011-01-01/2011-01-05", "1", 1),
createExpected("2011-01-05/2011-01-15", "2", 3) createExpected("2011-01-05/2011-01-15", "2", 3)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-15")) timeline.lookup(Intervals.of("2011-01-01/2011-01-15"))
); );
} }
@ -534,7 +534,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-01/2011-01-05", "1", 1), createExpected("2011-01-01/2011-01-05", "1", 1),
createExpected("2011-01-05/2011-01-15", "2", 3) createExpected("2011-01-05/2011-01-15", "2", 3)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-15")) timeline.lookup(Intervals.of("2011-01-01/2011-01-15"))
); );
} }
@ -553,7 +553,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-01/2011-01-10", "2", 1), createExpected("2011-01-01/2011-01-10", "2", 1),
createExpected("2011-01-10/2011-01-15", "1", 3) createExpected("2011-01-10/2011-01-15", "1", 3)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-15")) timeline.lookup(Intervals.of("2011-01-01/2011-01-15"))
); );
} }
@ -572,7 +572,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-01/2011-01-10", "2", 1), createExpected("2011-01-01/2011-01-10", "2", 1),
createExpected("2011-01-10/2011-01-15", "1", 3) createExpected("2011-01-10/2011-01-15", "1", 3)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-15")) timeline.lookup(Intervals.of("2011-01-01/2011-01-15"))
); );
} }
@ -592,7 +592,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-05/2011-01-15", "2", 3), createExpected("2011-01-05/2011-01-15", "2", 3),
createExpected("2011-01-15/2011-01-20", "1", 2) createExpected("2011-01-15/2011-01-20", "1", 2)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -612,7 +612,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-05/2011-01-15", "2", 3), createExpected("2011-01-05/2011-01-15", "2", 3),
createExpected("2011-01-15/2011-01-20", "1", 2) createExpected("2011-01-15/2011-01-20", "1", 2)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -632,7 +632,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-01/2011-01-15", "2", 3), createExpected("2011-01-01/2011-01-15", "2", 3),
createExpected("2011-01-15/2011-01-20", "1", 2) createExpected("2011-01-15/2011-01-20", "1", 2)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -652,7 +652,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-01/2011-01-15", "2", 3), createExpected("2011-01-01/2011-01-15", "2", 3),
createExpected("2011-01-15/2011-01-20", "1", 2) createExpected("2011-01-15/2011-01-20", "1", 2)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -674,7 +674,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-03/2011-01-06", "3", 3), createExpected("2011-01-03/2011-01-06", "3", 3),
createExpected("2011-01-06/2011-01-20", "2", 2) createExpected("2011-01-06/2011-01-20", "2", 2)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -696,7 +696,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-03/2011-01-06", "3", 3), createExpected("2011-01-03/2011-01-06", "3", 3),
createExpected("2011-01-06/2011-01-20", "2", 2) createExpected("2011-01-06/2011-01-20", "2", 2)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -718,7 +718,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-03/2011-01-06", "3", 3), createExpected("2011-01-03/2011-01-06", "3", 3),
createExpected("2011-01-06/2011-01-20", "2", 2) createExpected("2011-01-06/2011-01-20", "2", 2)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -741,7 +741,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-06/2011-01-10", "2", 2), createExpected("2011-01-06/2011-01-10", "2", 2),
createExpected("2011-01-10/2011-01-20", "1", 1) createExpected("2011-01-10/2011-01-20", "1", 1)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -764,7 +764,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-06/2011-01-10", "2", 2), createExpected("2011-01-06/2011-01-10", "2", 2),
createExpected("2011-01-10/2011-01-20", "1", 1) createExpected("2011-01-10/2011-01-20", "1", 1)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -787,7 +787,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-06/2011-01-10", "2", 2), createExpected("2011-01-06/2011-01-10", "2", 2),
createExpected("2011-01-10/2011-01-20", "1", 1) createExpected("2011-01-10/2011-01-20", "1", 1)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -811,7 +811,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-13/2011-01-15", "1", 2), createExpected("2011-01-13/2011-01-15", "1", 2),
createExpected("2011-01-15/2011-01-20", "1", 3) createExpected("2011-01-15/2011-01-20", "1", 3)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -835,7 +835,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-13/2011-01-15", "1", 2), createExpected("2011-01-13/2011-01-15", "1", 2),
createExpected("2011-01-15/2011-01-20", "1", 3) createExpected("2011-01-15/2011-01-20", "1", 3)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -855,7 +855,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-01/2011-01-10", "2", 2), createExpected("2011-01-01/2011-01-10", "2", 2),
createExpected("2011-01-10/2011-01-20", "2", 3) createExpected("2011-01-10/2011-01-20", "2", 3)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -875,7 +875,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-01/2011-01-10", "2", 2), createExpected("2011-01-01/2011-01-10", "2", 2),
createExpected("2011-01-10/2011-01-20", "2", 3) createExpected("2011-01-10/2011-01-20", "2", 3)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -897,7 +897,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-05/2011-01-15", "2", 2), createExpected("2011-01-05/2011-01-15", "2", 2),
createExpected("2011-01-15/2011-01-25", "3", 3) createExpected("2011-01-15/2011-01-25", "3", 3)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-25")) timeline.lookup(Intervals.of("2011-01-01/2011-01-25"))
); );
} }
@ -919,7 +919,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-05/2011-01-15", "2", 2), createExpected("2011-01-05/2011-01-15", "2", 2),
createExpected("2011-01-15/2011-01-25", "3", 3) createExpected("2011-01-15/2011-01-25", "3", 3)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-25")) timeline.lookup(Intervals.of("2011-01-01/2011-01-25"))
); );
} }
@ -941,7 +941,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-05/2011-01-13", "1", 1), createExpected("2011-01-05/2011-01-13", "1", 1),
createExpected("2011-01-13/2011-01-20", "2", 2) createExpected("2011-01-13/2011-01-20", "2", 2)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -961,7 +961,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-01/2011-01-15", "2", 2), createExpected("2011-01-01/2011-01-15", "2", 2),
createExpected("2011-01-15/2011-01-20", "1", 1) createExpected("2011-01-15/2011-01-20", "1", 1)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -983,7 +983,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-13/2011-01-17", "2", 3), createExpected("2011-01-13/2011-01-17", "2", 3),
createExpected("2011-01-17/2011-01-20", "1", 1) createExpected("2011-01-17/2011-01-20", "1", 1)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -1005,7 +1005,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-13/2011-01-17", "2", 3), createExpected("2011-01-13/2011-01-17", "2", 3),
createExpected("2011-01-17/2011-01-20", "1", 1) createExpected("2011-01-17/2011-01-20", "1", 1)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -1026,7 +1026,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-05/2011-01-15", "1", 1), createExpected("2011-01-05/2011-01-15", "1", 1),
createExpected("2011-01-15/2011-01-20", "2", 3) createExpected("2011-01-15/2011-01-20", "2", 3)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -1047,7 +1047,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-05/2011-01-15", "1", 1), createExpected("2011-01-05/2011-01-15", "1", 1),
createExpected("2011-01-15/2011-01-20", "2", 3) createExpected("2011-01-15/2011-01-20", "2", 3)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -1072,7 +1072,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-04/2011-01-05", "3", 3), createExpected("2011-01-04/2011-01-05", "3", 3),
createExpected("2011-01-05/2011-01-06", "4", 4) createExpected("2011-01-05/2011-01-06", "4", 4)
), ),
timeline.lookup(new Interval("0000-01-01/3000-01-01")) timeline.lookup(Intervals.of("0000-01-01/3000-01-01"))
); );
} }
@ -1093,7 +1093,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-01T12/2011-01-02", "3", 3), createExpected("2011-01-01T12/2011-01-02", "3", 3),
createExpected("2011-01-02/3011-01-03", "1", 1) createExpected("2011-01-02/3011-01-03", "1", 1)
), ),
timeline.lookup(new Interval("2011-01-01/3011-01-03")) timeline.lookup(Intervals.of("2011-01-01/3011-01-03"))
); );
} }
@ -1175,13 +1175,13 @@ public class VersionedIntervalTimelineTest
add("2011-01-01/2011-01-20", "1", 1); add("2011-01-01/2011-01-20", "1", 1);
add("2011-01-10/2011-01-15", "2", 2); add("2011-01-10/2011-01-15", "2", 2);
timeline.remove(new Interval("2011-01-10/2011-01-15"), "2", makeSingle(2)); timeline.remove(Intervals.of("2011-01-10/2011-01-15"), "2", makeSingle(2));
assertValues( assertValues(
Collections.singletonList( Collections.singletonList(
createExpected("2011-01-01/2011-01-20", "1", 1) createExpected("2011-01-01/2011-01-20", "1", 1)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -1194,7 +1194,7 @@ public class VersionedIntervalTimelineTest
add("2011-01-10/2011-01-20", "2", 2); add("2011-01-10/2011-01-20", "2", 2);
add("2011-01-20/2011-01-30", "3", 4); add("2011-01-20/2011-01-30", "3", 4);
timeline.remove(new Interval("2011-01-10/2011-01-20"), "2", makeSingle(2)); timeline.remove(Intervals.of("2011-01-10/2011-01-20"), "2", makeSingle(2));
assertValues( assertValues(
Arrays.asList( Arrays.asList(
@ -1202,7 +1202,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-20/2011-01-30", "3", 4) createExpected("2011-01-20/2011-01-30", "3", 4)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-30")) timeline.lookup(Intervals.of("2011-01-01/2011-01-30"))
); );
} }
@ -1215,15 +1215,15 @@ public class VersionedIntervalTimelineTest
add("2011-01-02/2011-01-03", "2", 2); add("2011-01-02/2011-01-03", "2", 2);
add("2011-01-10/2011-01-14", "2", 3); add("2011-01-10/2011-01-14", "2", 3);
timeline.remove(new Interval("2011-01-02/2011-01-03"), "2", makeSingle(2)); timeline.remove(Intervals.of("2011-01-02/2011-01-03"), "2", makeSingle(2));
timeline.remove(new Interval("2011-01-10/2011-01-14"), "2", makeSingle(3)); timeline.remove(Intervals.of("2011-01-10/2011-01-14"), "2", makeSingle(3));
assertValues( assertValues(
Collections.singletonList( Collections.singletonList(
createExpected("2011-01-01/2011-01-20", "1", 1) createExpected("2011-01-01/2011-01-20", "1", 1)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -1236,7 +1236,7 @@ public class VersionedIntervalTimelineTest
add("2011-01-10/2011-01-15", "2", 2); add("2011-01-10/2011-01-15", "2", 2);
add("2011-01-15/2011-01-20", "2", 3); add("2011-01-15/2011-01-20", "2", 3);
timeline.remove(new Interval("2011-01-15/2011-01-20"), "2", makeSingle(3)); timeline.remove(Intervals.of("2011-01-15/2011-01-20"), "2", makeSingle(3));
assertValues( assertValues(
Arrays.asList( Arrays.asList(
@ -1244,7 +1244,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-10/2011-01-15", "2", 2), createExpected("2011-01-10/2011-01-15", "2", 2),
createExpected("2011-01-15/2011-01-20", "1", 1) createExpected("2011-01-15/2011-01-20", "1", 1)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -1255,14 +1255,14 @@ public class VersionedIntervalTimelineTest
add("2011-01-01/2011-01-20", "1", 1); add("2011-01-01/2011-01-20", "1", 1);
add("2011-01-10/2011-01-15", "2", 2); add("2011-01-10/2011-01-15", "2", 2);
timeline.remove(new Interval("2011-01-10/2011-01-15"), "2", makeSingle(2)); timeline.remove(Intervals.of("2011-01-10/2011-01-15"), "2", makeSingle(2));
add("2011-01-01/2011-01-20", "1", 1); add("2011-01-01/2011-01-20", "1", 1);
assertValues( assertValues(
Collections.singletonList( Collections.singletonList(
createExpected("2011-01-01/2011-01-20", "1", 1) createExpected("2011-01-01/2011-01-20", "1", 1)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
); );
} }
@ -1271,11 +1271,11 @@ public class VersionedIntervalTimelineTest
{ {
Assert.assertNull( Assert.assertNull(
"Don't have it, should be null", "Don't have it, should be null",
timeline.remove(new Interval("1970-01-01/2025-04-20"), "1", makeSingle(1)) timeline.remove(Intervals.of("1970-01-01/2025-04-20"), "1", makeSingle(1))
); );
Assert.assertNull( Assert.assertNull(
"Don't have it, should be null", "Don't have it, should be null",
timeline.remove(new Interval("2011-04-01/2011-04-09"), "version does not exist", makeSingle(1)) timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "version does not exist", makeSingle(1))
); );
} }
@ -1289,7 +1289,7 @@ public class VersionedIntervalTimelineTest
add("2011-01-10/2011-01-15", "3", 3); add("2011-01-10/2011-01-15", "3", 3);
add("2011-01-15/2011-01-20", "4", 4); add("2011-01-15/2011-01-20", "4", 4);
timeline.remove(new Interval("2011-01-15/2011-01-20"), "4", makeSingle(4)); timeline.remove(Intervals.of("2011-01-15/2011-01-20"), "4", makeSingle(4));
assertValues( assertValues(
Arrays.asList( Arrays.asList(
@ -1297,7 +1297,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-05/2011-01-10", "2", 2), createExpected("2011-01-05/2011-01-10", "2", 2),
createExpected("2011-01-10/2011-01-15", "3", 3) createExpected("2011-01-10/2011-01-15", "3", 3)
), ),
timeline.lookup(new Interval(new DateTime(0), new DateTime(JodaUtils.MAX_INSTANT))) timeline.lookup(new Interval(DateTimes.EPOCH, DateTimes.MAX))
); );
} }
@ -1504,7 +1504,7 @@ public class VersionedIntervalTimelineTest
add("2011-04-01/2011-04-09", "1", 1); add("2011-04-01/2011-04-09", "1", 1);
Assert.assertTrue(timeline.lookup(Interval.parse("1970/1980")).isEmpty()); Assert.assertTrue(timeline.lookup(Intervals.of("1970/1980")).isEmpty());
} }
// https://github.com/druid-io/druid/issues/3010 // https://github.com/druid-io/druid/issues/3010
@ -1526,7 +1526,7 @@ public class VersionedIntervalTimelineTest
) )
) )
), ),
timeline.lookup(new Interval("2011-04-01/2011-04-02")) timeline.lookup(Intervals.of("2011-04-01/2011-04-02"))
); );
add("2011-04-01/2011-04-02", "3", IntegerPartitionChunk.make(null, 1, 0, 110)); add("2011-04-01/2011-04-02", "3", IntegerPartitionChunk.make(null, 1, 0, 110));
@ -1540,7 +1540,7 @@ public class VersionedIntervalTimelineTest
) )
) )
), ),
timeline.lookup(new Interval("2011-04-01/2011-04-02")) timeline.lookup(Intervals.of("2011-04-01/2011-04-02"))
); );
assertValues( assertValues(
Sets.newHashSet( Sets.newHashSet(
@ -1564,7 +1564,7 @@ public class VersionedIntervalTimelineTest
) )
) )
), ),
timeline.lookup(new Interval("2011-04-01/2011-04-02")) timeline.lookup(Intervals.of("2011-04-01/2011-04-02"))
); );
} }
@ -1579,58 +1579,58 @@ public class VersionedIntervalTimelineTest
add("2011-04-15/2011-04-17", "1", new SingleElementPartitionChunk<Integer>(1)); add("2011-04-15/2011-04-17", "1", new SingleElementPartitionChunk<Integer>(1));
add("2011-04-17/2011-04-19", "1", new SingleElementPartitionChunk<Integer>(1)); add("2011-04-17/2011-04-19", "1", new SingleElementPartitionChunk<Integer>(1));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-03"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-03"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-05"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-05"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-06"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-06"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-07"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-07"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-08"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-08"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-09"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-09"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-10"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-10"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-30"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-30"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-06"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-07"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-08"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-09"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-06"), "1")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "1"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-07"), "1")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "1"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-08"), "1")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "1"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-09"), "1")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "1"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-06"), "2")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "2"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-07"), "2")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "2"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-08"), "2")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "2"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-09"), "2")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "2"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-07"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-07"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-08"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-08"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-09"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-09"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-10"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-10"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-30"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-30"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-07/2011-04-08"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-08"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-07/2011-04-09"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-09"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-07/2011-04-10"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-10"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-07/2011-04-30"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-30"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-08/2011-04-09"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-08/2011-04-09"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-08/2011-04-10"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-08/2011-04-10"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-08/2011-04-30"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-08/2011-04-30"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-09/2011-04-10"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-10"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-09/2011-04-15"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-15"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-09/2011-04-17"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-17"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-09/2011-04-19"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-19"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-09/2011-04-30"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-30"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-15/2011-04-16"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-16"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-15/2011-04-17"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-17"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-15/2011-04-18"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-18"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-15/2011-04-19"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-19"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-15/2011-04-20"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-20"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-15/2011-04-30"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-30"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-19/2011-04-20"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-19/2011-04-20"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-21/2011-04-22"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-21/2011-04-22"), "0"));
} }
@Test @Test
@ -1645,77 +1645,77 @@ public class VersionedIntervalTimelineTest
add("2011-04-17/2011-04-21", "11", new SingleElementPartitionChunk<Integer>(1)); add("2011-04-17/2011-04-21", "11", new SingleElementPartitionChunk<Integer>(1));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-03"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-03"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-05"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-05"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-06"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-06"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-07"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-07"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-08"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-08"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-09"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-09"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-10"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-10"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-11"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-11"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-30"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-30"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-06"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-07"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-08"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-09"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-10"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-11"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-11"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-06"), "12")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "12"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-07"), "12")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "12"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-08"), "12")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "12"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-09"), "12")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "12"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-10"), "12")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "12"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-11"), "12")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-11"), "12"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-06"), "13")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "13"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-07"), "13")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "13"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-08"), "13")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "13"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-09"), "13")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "13"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-10"), "13")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "13"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-11"), "13")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-11"), "13"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-12"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-12"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-15"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-15"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-16"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-16"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-17"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-17"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-18"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-18"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-19"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-19"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-20"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-20"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-21"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-21"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-22"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-22"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-07"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-07"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-08"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-08"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-09"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-09"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-10"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-10"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-11"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-11"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-12"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-12"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-15"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-15"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-16"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-16"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-17"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-17"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-18"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-18"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-19"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-19"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-20"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-20"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-21"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-21"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-22"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-22"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-12/2011-04-15"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-15"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-12/2011-04-16"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-16"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-12/2011-04-17"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-17"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-12/2011-04-18"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-18"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-12/2011-04-19"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-19"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-12/2011-04-20"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-20"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-12/2011-04-21"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-21"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-12/2011-04-22"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-22"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-15/2011-04-21"), "0")); Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-21"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-21/2011-04-22"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-21/2011-04-22"), "0"));
} }
private Pair<Interval, Pair<String, PartitionHolder<Integer>>> createExpected( private Pair<Interval, Pair<String, PartitionHolder<Integer>>> createExpected(
@ -1738,7 +1738,7 @@ public class VersionedIntervalTimelineTest
) )
{ {
return Pair.of( return Pair.of(
new Interval(intervalString), Intervals.of(intervalString),
Pair.of(version, new PartitionHolder<Integer>(values)) Pair.of(version, new PartitionHolder<Integer>(values))
); );
} }
@ -1750,17 +1750,17 @@ public class VersionedIntervalTimelineTest
private void add(String interval, String version, Integer value) private void add(String interval, String version, Integer value)
{ {
add(new Interval(interval), version, value); add(Intervals.of(interval), version, value);
} }
private void add(Interval interval, String version, Integer value) private void add(Interval interval, String version, Integer value)
{ {
add(new Interval(interval), version, makeSingle(value)); add(interval, version, makeSingle(value));
} }
private void add(String interval, String version, PartitionChunk<Integer> value) private void add(String interval, String version, PartitionChunk<Integer> value)
{ {
add(new Interval(interval), version, value); add(Intervals.of(interval), version, value);
} }
private void add(Interval interval, String version, PartitionChunk<Integer> value) private void add(Interval interval, String version, PartitionChunk<Integer> value)

View File

@ -22,6 +22,7 @@ package io.druid.emitter.ambari.metrics;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
import junitparams.JUnitParamsRunner; import junitparams.JUnitParamsRunner;
import junitparams.Parameters; import junitparams.Parameters;
import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric; import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
@ -44,7 +45,7 @@ public class WhiteListBasedDruidToTimelineEventConverterTest
new DefaultObjectMapper() new DefaultObjectMapper()
); );
private ServiceMetricEvent event; private ServiceMetricEvent event;
private final DateTime createdTime = new DateTime(); private final DateTime createdTime = DateTimes.nowUtc();
private final String hostname = "testHost:8080"; private final String hostname = "testHost:8080";
private final String serviceName = "historical"; private final String serviceName = "historical";
private final String defaultNamespace = prefix + "." + serviceName; private final String defaultNamespace = prefix + "." + serviceName;

View File

@ -21,11 +21,11 @@ package io.druid.storage.azure;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.microsoft.azure.storage.StorageException; import com.microsoft.azure.storage.StorageException;
import io.druid.java.util.common.Intervals;
import io.druid.segment.loading.SegmentLoadingException; import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import org.easymock.EasyMockSupport; import org.easymock.EasyMockSupport;
import org.joda.time.Interval;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -44,7 +44,7 @@ public class AzureDataSegmentKillerTest extends EasyMockSupport
private static final DataSegment dataSegment = new DataSegment( private static final DataSegment dataSegment = new DataSegment(
"test", "test",
new Interval("2015-04-12/2015-04-13"), Intervals.of("2015-04-12/2015-04-13"),
"1", "1",
ImmutableMap.<String, Object>of("containerName", containerName, "blobPath", blobPath), ImmutableMap.<String, Object>of("containerName", containerName, "blobPath", blobPath),
null, null,

View File

@ -22,11 +22,11 @@ package io.druid.storage.azure;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.microsoft.azure.storage.StorageException; import com.microsoft.azure.storage.StorageException;
import io.druid.java.util.common.FileUtils; import io.druid.java.util.common.FileUtils;
import io.druid.java.util.common.Intervals;
import io.druid.segment.loading.SegmentLoadingException; import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import org.easymock.EasyMockSupport; import org.easymock.EasyMockSupport;
import org.joda.time.Interval;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -50,7 +50,7 @@ public class AzureDataSegmentPullerTest extends EasyMockSupport
private static final String blobPath = "/path/to/storage/index.zip"; private static final String blobPath = "/path/to/storage/index.zip";
private static final DataSegment dataSegment = new DataSegment( private static final DataSegment dataSegment = new DataSegment(
"test", "test",
new Interval("2015-04-12/2015-04-13"), Intervals.of("2015-04-12/2015-04-13"),
"1", "1",
ImmutableMap.<String, Object>of("containerName", containerName, "blobPath", blobPath), ImmutableMap.<String, Object>of("containerName", containerName, "blobPath", blobPath),
null, null,

View File

@ -26,12 +26,12 @@ import com.google.common.collect.Maps;
import com.google.common.io.Files; import com.google.common.io.Files;
import com.microsoft.azure.storage.StorageException; import com.microsoft.azure.storage.StorageException;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.MapUtils; import io.druid.java.util.common.MapUtils;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import org.easymock.EasyMockSupport; import org.easymock.EasyMockSupport;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Rule; import org.junit.Rule;
@ -55,7 +55,7 @@ public class AzureDataSegmentPusherTest extends EasyMockSupport
private static final String blobPath = "test/2015-04-12T00:00:00.000Z_2015-04-13T00:00:00.000Z/1/0/index.zip"; private static final String blobPath = "test/2015-04-12T00:00:00.000Z_2015-04-13T00:00:00.000Z/1/0/index.zip";
private static final DataSegment dataSegment = new DataSegment( private static final DataSegment dataSegment = new DataSegment(
"test", "test",
new Interval("2015-04-12/2015-04-13"), Intervals.of("2015-04-12/2015-04-13"),
"1", "1",
ImmutableMap.<String, Object>of("containerName", containerName, "blobPath", blobPath), ImmutableMap.<String, Object>of("containerName", containerName, "blobPath", blobPath),
null, null,
@ -94,7 +94,7 @@ public class AzureDataSegmentPusherTest extends EasyMockSupport
DataSegment segmentToPush = new DataSegment( DataSegment segmentToPush = new DataSegment(
"foo", "foo",
new Interval("2015/2016"), Intervals.of("2015/2016"),
"0", "0",
Maps.<String, Object>newHashMap(), Maps.<String, Object>newHashMap(),
Lists.<String>newArrayList(), Lists.<String>newArrayList(),

View File

@ -23,13 +23,13 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.io.Files; import com.google.common.io.Files;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.jclouds.io.Payload; import org.jclouds.io.Payload;
import org.jclouds.openstack.swift.v1.features.ObjectApi; import org.jclouds.openstack.swift.v1.features.ObjectApi;
import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi; import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Rule; import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
@ -74,7 +74,7 @@ public class CloudFilesDataSegmentPusherTest
DataSegment segmentToPush = new DataSegment( DataSegment segmentToPush = new DataSegment(
"foo", "foo",
new Interval("2015/2016"), Intervals.of("2015/2016"),
"0", "0",
Maps.<String, Object>newHashMap(), Maps.<String, Object>newHashMap(),
Lists.<String>newArrayList(), Lists.<String>newArrayList(),

View File

@ -22,6 +22,7 @@ package io.druid.query.aggregation.distinctcount;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.MapBasedInputRow;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Sequences;
import io.druid.query.Druids; import io.druid.query.Druids;
@ -61,7 +62,7 @@ public class DistinctCountTimeseriesQueryTest
String visitor_id = "visitor_id"; String visitor_id = "visitor_id";
String client_type = "client_type"; String client_type = "client_type";
DateTime time = new DateTime("2016-03-04T00:00:00.000Z"); DateTime time = DateTimes.of("2016-03-04T00:00:00.000Z");
long timestamp = time.getMillis(); long timestamp = time.getMillis();
index.add( index.add(
new MapBasedInputRow( new MapBasedInputRow(

View File

@ -22,9 +22,9 @@ package io.druid.query.aggregation.distinctcount;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import io.druid.collections.StupidPool; import io.druid.collections.StupidPool;
import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.MapBasedInputRow;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Sequences;
import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryRunnerTestHelper;
@ -79,7 +79,7 @@ public class DistinctCountTopNQueryTest
String visitor_id = "visitor_id"; String visitor_id = "visitor_id";
String client_type = "client_type"; String client_type = "client_type";
DateTime time = new DateTime("2016-03-04T00:00:00.000Z"); DateTime time = DateTimes.of("2016-03-04T00:00:00.000Z");
long timestamp = time.getMillis(); long timestamp = time.getMillis();
index.add( index.add(
new MapBasedInputRow( new MapBasedInputRow(

View File

@ -20,11 +20,11 @@
package io.druid.storage.google; package io.druid.storage.google;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import io.druid.java.util.common.Intervals;
import io.druid.segment.loading.SegmentLoadingException; import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import org.easymock.EasyMockSupport; import org.easymock.EasyMockSupport;
import org.joda.time.Interval;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -39,7 +39,7 @@ public class GoogleDataSegmentKillerTest extends EasyMockSupport
private static final DataSegment dataSegment = new DataSegment( private static final DataSegment dataSegment = new DataSegment(
"test", "test",
new Interval("2015-04-12/2015-04-13"), Intervals.of("2015-04-12/2015-04-13"),
"1", "1",
ImmutableMap.<String, Object>of("bucket", bucket, "path", indexPath), ImmutableMap.<String, Object>of("bucket", bucket, "path", indexPath),
null, null,

View File

@ -21,11 +21,11 @@ package io.druid.storage.google;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import io.druid.java.util.common.FileUtils; import io.druid.java.util.common.FileUtils;
import io.druid.java.util.common.Intervals;
import io.druid.segment.loading.SegmentLoadingException; import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import org.easymock.EasyMockSupport; import org.easymock.EasyMockSupport;
import org.joda.time.Interval;
import org.junit.Test; import org.junit.Test;
import java.io.File; import java.io.File;
@ -42,7 +42,7 @@ public class GoogleDataSegmentPullerTest extends EasyMockSupport
private static final String path = "/path/to/storage/index.zip"; private static final String path = "/path/to/storage/index.zip";
private static final DataSegment dataSegment = new DataSegment( private static final DataSegment dataSegment = new DataSegment(
"test", "test",
new Interval("2015-04-12/2015-04-13"), Intervals.of("2015-04-12/2015-04-13"),
"1", "1",
ImmutableMap.<String, Object>of("bucket", bucket, "path", path), ImmutableMap.<String, Object>of("bucket", bucket, "path", path),
null, null,

View File

@ -25,11 +25,11 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.io.Files; import com.google.common.io.Files;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.easymock.EasyMockSupport; import org.easymock.EasyMockSupport;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Rule; import org.junit.Rule;
@ -50,7 +50,7 @@ public class GoogleDataSegmentPusherTest extends EasyMockSupport
private static final String path = "prefix/test/2015-04-12T00:00:00.000Z_2015-04-13T00:00:00.000Z/1/0/index.zip"; private static final String path = "prefix/test/2015-04-12T00:00:00.000Z_2015-04-13T00:00:00.000Z/1/0/index.zip";
private static final DataSegment dataSegment = new DataSegment( private static final DataSegment dataSegment = new DataSegment(
"test", "test",
new Interval("2015-04-12/2015-04-13"), Intervals.of("2015-04-12/2015-04-13"),
"1", "1",
ImmutableMap.<String, Object>of("bucket", bucket, "path", path), ImmutableMap.<String, Object>of("bucket", bucket, "path", path),
null, null,
@ -87,7 +87,7 @@ public class GoogleDataSegmentPusherTest extends EasyMockSupport
DataSegment segmentToPush = new DataSegment( DataSegment segmentToPush = new DataSegment(
"foo", "foo",
new Interval("2015/2016"), Intervals.of("2015/2016"),
"0", "0",
Maps.<String, Object>newHashMap(), Maps.<String, Object>newHashMap(),
Lists.<String>newArrayList(), Lists.<String>newArrayList(),

View File

@ -22,6 +22,7 @@ package io.druid.emitter.graphite;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
import junitparams.JUnitParamsRunner; import junitparams.JUnitParamsRunner;
import junitparams.Parameters; import junitparams.Parameters;
import org.easymock.EasyMock; import org.easymock.EasyMock;
@ -44,7 +45,7 @@ public class WhiteListBasedConverterTest
new DefaultObjectMapper() new DefaultObjectMapper()
); );
private ServiceMetricEvent event; private ServiceMetricEvent event;
private DateTime createdTime = new DateTime(); private DateTime createdTime = DateTimes.nowUtc();
private String hostname = "testHost.yahoo.com:8080"; private String hostname = "testHost.yahoo.com:8080";
private String serviceName = "historical"; private String serviceName = "historical";
private String defaultNamespace = prefix + "." + serviceName + "." + GraphiteEmitter.sanitize(hostname); private String defaultNamespace = prefix + "." + serviceName + "." + GraphiteEmitter.sanitize(hostname);

View File

@ -18,9 +18,10 @@
*/ */
package io.druid.data.input.orc; package io.druid.data.input.orc;
import io.druid.java.util.common.StringUtils;
import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.MapBasedInputRow;
import io.druid.indexer.HadoopDruidIndexerConfig; import io.druid.indexer.HadoopDruidIndexerConfig;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.StringUtils;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
@ -41,7 +42,6 @@ import org.apache.orc.CompressionKind;
import org.apache.orc.OrcFile; import org.apache.orc.OrcFile;
import org.apache.orc.TypeDescription; import org.apache.orc.TypeDescription;
import org.apache.orc.Writer; import org.apache.orc.Writer;
import org.joda.time.DateTime;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Rule; import org.junit.Rule;
@ -101,7 +101,7 @@ public class DruidOrcInputFormatTest
MapBasedInputRow row = (MapBasedInputRow) parser.parse(data); MapBasedInputRow row = (MapBasedInputRow) parser.parse(data);
Assert.assertTrue(row.getEvent().keySet().size() == 4); Assert.assertTrue(row.getEvent().keySet().size() == 4);
Assert.assertEquals(new DateTime(timestamp), row.getTimestamp()); Assert.assertEquals(DateTimes.of(timestamp), row.getTimestamp());
Assert.assertEquals(parser.getParseSpec().getDimensionsSpec().getDimensionNames(), row.getDimensions()); Assert.assertEquals(parser.getParseSpec().getDimensionsSpec().getDimensionNames(), row.getDimensions());
Assert.assertEquals(col1, row.getEvent().get("col1")); Assert.assertEquals(col1, row.getEvent().get("col1"));
Assert.assertEquals(Arrays.asList(col2), row.getDimension("col2")); Assert.assertEquals(Arrays.asList(col2), row.getDimension("col2"));

View File

@ -36,6 +36,7 @@ import io.druid.data.input.impl.TimestampSpec;
import io.druid.guice.GuiceInjectors; import io.druid.guice.GuiceInjectors;
import io.druid.initialization.Initialization; import io.druid.initialization.Initialization;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
import org.apache.hadoop.hive.ql.io.orc.OrcStruct; import org.apache.hadoop.hive.ql.io.orc.OrcStruct;
import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
import org.apache.hadoop.hive.serde2.objectinspector.SettableStructObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.SettableStructObjectInspector;
@ -43,7 +44,6 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
import org.apache.hadoop.io.FloatWritable; import org.apache.hadoop.io.FloatWritable;
import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.joda.time.DateTime;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -167,7 +167,7 @@ public class OrcHadoopInputRowParserTest
oi.setStructFieldData(struct, oi.getStructFieldRef("col6"), null); oi.setStructFieldData(struct, oi.getStructFieldRef("col6"), null);
final InputRow row = parser.parse(struct); final InputRow row = parser.parse(struct);
Assert.assertEquals("timestamp", new DateTime("2000-01-01"), row.getTimestamp()); Assert.assertEquals("timestamp", DateTimes.of("2000-01-01"), row.getTimestamp());
Assert.assertEquals("col1", "foo", row.getRaw("col1")); Assert.assertEquals("col1", "foo", row.getRaw("col1"));
Assert.assertEquals("col2", ImmutableList.of("foo", "bar"), row.getRaw("col2")); Assert.assertEquals("col2", ImmutableList.of("foo", "bar"), row.getRaw("col2"));
Assert.assertEquals("col3", 1.0f, row.getRaw("col3")); Assert.assertEquals("col3", 1.0f, row.getRaw("col3"));

View File

@ -25,7 +25,6 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.io.Files; import com.google.common.io.Files;
import io.druid.java.util.common.StringUtils;
import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.InputRowParser; import io.druid.data.input.impl.InputRowParser;
import io.druid.data.input.impl.TimeAndDimsParseSpec; import io.druid.data.input.impl.TimeAndDimsParseSpec;
@ -38,6 +37,8 @@ import io.druid.indexer.HadoopyShardSpec;
import io.druid.indexer.IndexGeneratorJob; import io.druid.indexer.IndexGeneratorJob;
import io.druid.indexer.JobHelper; import io.druid.indexer.JobHelper;
import io.druid.indexer.Jobby; import io.druid.indexer.Jobby;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory;
@ -117,7 +118,7 @@ public class OrcIndexGeneratorJobTest
"2014102212,i.example.com,963", "2014102212,i.example.com,963",
"2014102212,j.example.com,333" "2014102212,j.example.com,333"
); );
private final Interval interval = new Interval("2014-10-22T00:00:00Z/P1D"); private final Interval interval = Intervals.of("2014-10-22T00:00:00Z/P1D");
private File dataRoot; private File dataRoot;
private File outputRoot; private File outputRoot;
private Integer[][][] shardInfoForEachSegment = new Integer[][][]{{ private Integer[][][] shardInfoForEachSegment = new Integer[][][]{{

View File

@ -29,6 +29,7 @@ import io.druid.data.input.impl.DimensionSchema;
import io.druid.data.input.impl.InputRowParser; import io.druid.data.input.impl.InputRowParser;
import io.druid.data.input.impl.ParseSpec; import io.druid.data.input.impl.ParseSpec;
import io.druid.data.input.impl.TimestampSpec; import io.druid.data.input.impl.TimestampSpec;
import io.druid.java.util.common.DateTimes;
import org.apache.avro.LogicalType; import org.apache.avro.LogicalType;
import org.apache.avro.LogicalTypes; import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema; import org.apache.avro.Schema;
@ -92,7 +93,7 @@ public class ParquetHadoopInputRowParser implements InputRowParser<GenericRecord
if (logicalType instanceof LogicalTypes.Date) { if (logicalType instanceof LogicalTypes.Date) {
int daysSinceEpoch = (Integer) genericRecordAsMap.get(timestampSpec.getTimestampColumn()); int daysSinceEpoch = (Integer) genericRecordAsMap.get(timestampSpec.getTimestampColumn());
dateTime = new DateTime(TimeUnit.DAYS.toMillis(daysSinceEpoch)); dateTime = DateTimes.utc(TimeUnit.DAYS.toMillis(daysSinceEpoch));
} else { } else {
// Fall back to a binary format that will be parsed using joda-time // Fall back to a binary format that will be parsed using joda-time
dateTime = timestampSpec.extractTimestamp(genericRecordAsMap); dateTime = timestampSpec.extractTimestamp(genericRecordAsMap);

View File

@ -38,6 +38,7 @@ import java.util.Date;
import java.util.List; import java.util.List;
import java.util.Locale; import java.util.Locale;
import java.util.Random; import java.util.Random;
import java.util.TimeZone;
/** /**
* *
@ -178,7 +179,7 @@ public class RabbitMQProducerMain
Date stop = sdf.parse(cmd.getOptionValue("stop", sdf.format(new Date()))); Date stop = sdf.parse(cmd.getOptionValue("stop", sdf.format(new Date())));
Random r = new Random(); Random r = new Random();
Calendar timer = Calendar.getInstance(Locale.ENGLISH); Calendar timer = Calendar.getInstance(TimeZone.getTimeZone("UTC"), Locale.ENGLISH);
timer.setTime(sdf.parse(cmd.getOptionValue("start", "2010-01-01T00:00:00"))); timer.setTime(sdf.parse(cmd.getOptionValue("start", "2010-01-01T00:00:00")));
String msg_template = "{\"utcdt\": \"%s\", \"wp\": %d, \"gender\": \"%s\", \"age\": %d}"; String msg_template = "{\"utcdt\": \"%s\", \"wp\": %d, \"gender\": \"%s\", \"age\": %d}";

View File

@ -20,7 +20,7 @@ package io.druid.query.scan;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.google.inject.Inject; import com.google.inject.Inject;
import io.druid.common.utils.JodaUtils; import io.druid.java.util.common.JodaUtils;
import io.druid.java.util.common.ISE; import io.druid.java.util.common.ISE;
import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Sequences;

View File

@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.io.CharSource; import com.google.common.io.CharSource;
import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.MoreExecutors;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.MergeSequence; import io.druid.java.util.common.guava.MergeSequence;
import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequence;
@ -43,7 +44,6 @@ import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.StringUtils;
import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
@ -145,7 +145,7 @@ public class MultiSegmentScanQueryTest
private static IncrementalIndex newIndex(String minTimeStamp, int maxRowCount) private static IncrementalIndex newIndex(String minTimeStamp, int maxRowCount)
{ {
final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
.withMinTimestamp(new DateTime(minTimeStamp).getMillis()) .withMinTimestamp(DateTimes.of(minTimeStamp).getMillis())
.withQueryGranularity(Granularities.HOUR) .withQueryGranularity(Granularities.HOUR)
.withMetrics(TestIndex.METRIC_AGGS) .withMetrics(TestIndex.METRIC_AGGS)
.build(); .build();

View File

@ -25,7 +25,9 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.collect.ObjectArrays; import com.google.common.collect.ObjectArrays;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.ISE; import io.druid.java.util.common.ISE;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Sequences;
import io.druid.query.DefaultGenericQueryMetricsFactory; import io.druid.query.DefaultGenericQueryMetricsFactory;
import io.druid.query.QueryPlus; import io.druid.query.QueryPlus;
@ -39,8 +41,6 @@ import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.lookup.LookupExtractionFn; import io.druid.query.lookup.LookupExtractionFn;
import io.druid.query.spec.LegacySegmentSpec; import io.druid.query.spec.LegacySegmentSpec;
import io.druid.query.spec.QuerySegmentSpec; import io.druid.query.spec.QuerySegmentSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
@ -94,7 +94,7 @@ public class ScanQueryRunnerTest
}; };
public static final QuerySegmentSpec I_0112_0114 = new LegacySegmentSpec( public static final QuerySegmentSpec I_0112_0114 = new LegacySegmentSpec(
new Interval("2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z") Intervals.of("2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z")
); );
public static final String[] V_0112_0114 = ObjectArrays.concat(V_0112, V_0113, String.class); public static final String[] V_0112_0114 = ObjectArrays.concat(V_0112, V_0113, String.class);
@ -506,7 +506,7 @@ public class ScanQueryRunnerTest
event.put( event.put(
specs[0], specs[0],
specs.length == 1 || specs[1].equals("STRING") ? values[i] : specs.length == 1 || specs[1].equals("STRING") ? values[i] :
specs[1].equals("TIME") ? new DateTime(values[i]) : specs[1].equals("TIME") ? DateTimes.of(values[i]) :
specs[1].equals("FLOAT") ? Float.valueOf(values[i]) : specs[1].equals("FLOAT") ? Float.valueOf(values[i]) :
specs[1].equals("DOUBLE") ? Double.valueOf(values[i]) : specs[1].equals("DOUBLE") ? Double.valueOf(values[i]) :
specs[1].equals("LONG") ? Long.valueOf(values[i]) : specs[1].equals("LONG") ? Long.valueOf(values[i]) :

View File

@ -21,10 +21,10 @@ package io.druid.query.scan;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.TableDataSource; import io.druid.query.TableDataSource;
import io.druid.query.spec.LegacySegmentSpec; import io.druid.query.spec.LegacySegmentSpec;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -58,7 +58,7 @@ public class ScanQuerySpecTest
ScanQuery query = new ScanQuery( ScanQuery query = new ScanQuery(
new TableDataSource(QueryRunnerTestHelper.dataSource), new TableDataSource(QueryRunnerTestHelper.dataSource),
new LegacySegmentSpec(new Interval("2011-01-12/2011-01-14")), new LegacySegmentSpec(Intervals.of("2011-01-12/2011-01-14")),
null, null,
0, 0,
3, 3,

View File

@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableList;
import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.emitter.statsd.DimensionConverter; import io.druid.emitter.statsd.DimensionConverter;
import io.druid.emitter.statsd.StatsDMetric; import io.druid.emitter.statsd.StatsDMetric;
import org.joda.time.DateTime; import io.druid.java.util.common.DateTimes;
import org.junit.Test; import org.junit.Test;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -46,7 +46,7 @@ public class DimensionConverterTest
.setDimension("remoteAddress", "194.0.90.2") .setDimension("remoteAddress", "194.0.90.2")
.setDimension("id", "ID") .setDimension("id", "ID")
.setDimension("context", "{context}") .setDimension("context", "{context}")
.build(new DateTime(), "query/time", 10) .build(DateTimes.nowUtc(), "query/time", 10)
.build("broker", "brokerHost1"); .build("broker", "brokerHost1");
ImmutableList.Builder<String> actual = new ImmutableList.Builder<>(); ImmutableList.Builder<String> actual = new ImmutableList.Builder<>();

View File

@ -22,14 +22,13 @@ import com.metamx.emitter.service.ServiceMetricEvent;
import com.timgroup.statsd.StatsDClient; import com.timgroup.statsd.StatsDClient;
import io.druid.emitter.statsd.StatsDEmitter; import io.druid.emitter.statsd.StatsDEmitter;
import io.druid.emitter.statsd.StatsDEmitterConfig; import io.druid.emitter.statsd.StatsDEmitterConfig;
import io.druid.java.util.common.DateTimes;
import org.junit.Test;
import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.createMock;
import static org.easymock.EasyMock.replay; import static org.easymock.EasyMock.replay;
import static org.easymock.EasyMock.verify; import static org.easymock.EasyMock.verify;
import org.joda.time.DateTime;
import org.junit.Test;
/** /**
*/ */
public class StatsDEmitterTest public class StatsDEmitterTest
@ -47,7 +46,7 @@ public class StatsDEmitterTest
replay(client); replay(client);
emitter.emit(new ServiceMetricEvent.Builder() emitter.emit(new ServiceMetricEvent.Builder()
.setDimension("dataSource", "data-source") .setDimension("dataSource", "data-source")
.build(new DateTime(), "query/cache/total/hitRate", 0.54) .build(DateTimes.nowUtc(), "query/cache/total/hitRate", 0.54)
.build("broker", "brokerHost1") .build("broker", "brokerHost1")
); );
verify(client); verify(client);
@ -75,7 +74,7 @@ public class StatsDEmitterTest
.setDimension("remoteAddress", "194.0.90.2") .setDimension("remoteAddress", "194.0.90.2")
.setDimension("id", "ID") .setDimension("id", "ID")
.setDimension("context", "{context}") .setDimension("context", "{context}")
.build(new DateTime(), "query/time", 10) .build(DateTimes.nowUtc(), "query/time", 10)
.build("broker", "brokerHost1") .build("broker", "brokerHost1")
); );
verify(client); verify(client);
@ -103,7 +102,7 @@ public class StatsDEmitterTest
.setDimension("remoteAddress", "194.0.90.2") .setDimension("remoteAddress", "194.0.90.2")
.setDimension("id", "ID") .setDimension("id", "ID")
.setDimension("context", "{context}") .setDimension("context", "{context}")
.build(new DateTime(), "query/time", 10) .build(DateTimes.nowUtc(), "query/time", 10)
.build("broker", "brokerHost1") .build("broker", "brokerHost1")
); );
verify(client); verify(client);

View File

@ -21,8 +21,9 @@ package io.druid.query.aggregation;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.primitives.Longs; import com.google.common.primitives.Longs;
import io.druid.java.util.common.StringUtils;
import io.druid.data.input.impl.TimestampSpec; import io.druid.data.input.impl.TimestampSpec;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.StringUtils;
import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ColumnSelectorFactory;
import org.joda.time.DateTime; import org.joda.time.DateTime;
@ -116,7 +117,7 @@ public class TimestampAggregatorFactory extends AggregatorFactory
@Override @Override
public Object finalizeComputation(Object object) public Object finalizeComputation(Object object)
{ {
return new DateTime((long) object); return DateTimes.utc((long) object);
} }
@Override @Override

View File

@ -30,6 +30,7 @@ import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.StringInputRowParser;
import io.druid.data.input.impl.TimestampSpec; import io.druid.data.input.impl.TimestampSpec;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Sequences;
import io.druid.query.Druids; import io.druid.query.Druids;
import io.druid.query.QueryPlus; import io.druid.query.QueryPlus;
@ -46,7 +47,6 @@ import io.druid.query.select.SelectQueryRunnerFactory;
import io.druid.query.select.SelectResultValue; import io.druid.query.select.SelectResultValue;
import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.IncrementalIndexSchema;
import org.joda.time.DateTime;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
@ -85,7 +85,7 @@ public class MapVirtualColumnTest
); );
final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
.withMinTimestamp(new DateTime("2011-01-12T00:00:00.000Z").getMillis()) .withMinTimestamp(DateTimes.of("2011-01-12T00:00:00.000Z").getMillis())
.build(); .build();
final IncrementalIndex index = new IncrementalIndex.Builder() final IncrementalIndex index = new IncrementalIndex.Builder()
.setIndexSchema(schema) .setIndexSchema(schema)

View File

@ -39,6 +39,7 @@ import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.DatumWriter; import org.apache.avro.io.DatumWriter;
import org.apache.avro.io.EncoderFactory; import org.apache.avro.io.EncoderFactory;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.chrono.ISOChronology;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.schemarepo.InMemoryRepository; import org.schemarepo.InMemoryRepository;
@ -75,7 +76,7 @@ public class AvroStreamInputRowParserTest
public static final float SOME_FLOAT_VALUE = 0.23555f; public static final float SOME_FLOAT_VALUE = 0.23555f;
public static final int SOME_INT_VALUE = 1; public static final int SOME_INT_VALUE = 1;
public static final long SOME_LONG_VALUE = 679865987569912369L; public static final long SOME_LONG_VALUE = 679865987569912369L;
public static final DateTime DATE_TIME = new DateTime(2015, 10, 25, 19, 30); public static final DateTime DATE_TIME = new DateTime(2015, 10, 25, 19, 30, ISOChronology.getInstanceUTC());
public static final List<String> DIMENSIONS = Arrays.asList(EVENT_TYPE, ID, SOME_OTHER_ID, IS_VALID); public static final List<String> DIMENSIONS = Arrays.asList(EVENT_TYPE, ID, SOME_OTHER_ID, IS_VALID);
public static final TimeAndDimsParseSpec PARSE_SPEC = new TimeAndDimsParseSpec( public static final TimeAndDimsParseSpec PARSE_SPEC = new TimeAndDimsParseSpec(
new TimestampSpec("timestamp", "millis", null), new TimestampSpec("timestamp", "millis", null),

View File

@ -26,6 +26,7 @@ import com.google.common.collect.Lists;
import com.google.common.io.Files; import com.google.common.io.Files;
import io.druid.data.input.MapBasedRow; import io.druid.data.input.MapBasedRow;
import io.druid.data.input.Row; import io.druid.data.input.Row;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Sequences;
@ -221,7 +222,7 @@ public class SketchAggregationWithSimpleDataTest
Sequences.toList(seq, Lists.newArrayList()) Sequences.toList(seq, Lists.newArrayList())
); );
Assert.assertEquals(new DateTime("2014-10-20T00:00:00.000Z"), result.getTimestamp()); Assert.assertEquals(DateTimes.of("2014-10-20T00:00:00.000Z"), result.getTimestamp());
Assert.assertEquals(50.0, result.getValue().getDoubleMetric("sketch_count"), 0.01); Assert.assertEquals(50.0, result.getValue().getDoubleMetric("sketch_count"), 0.01);
Assert.assertEquals(50.0, result.getValue().getDoubleMetric("sketchEstimatePostAgg"), 0.01); Assert.assertEquals(50.0, result.getValue().getDoubleMetric("sketchEstimatePostAgg"), 0.01);
@ -249,7 +250,7 @@ public class SketchAggregationWithSimpleDataTest
Sequences.toList(seq, Lists.newArrayList()) Sequences.toList(seq, Lists.newArrayList())
); );
Assert.assertEquals(new DateTime("2014-10-20T00:00:00.000Z"), result.getTimestamp()); Assert.assertEquals(DateTimes.of("2014-10-20T00:00:00.000Z"), result.getTimestamp());
DimensionAndMetricValueExtractor value = Iterables.getOnlyElement(result.getValue().getValue()); DimensionAndMetricValueExtractor value = Iterables.getOnlyElement(result.getValue().getValue());
Assert.assertEquals(38.0, value.getDoubleMetric("sketch_count"), 0.01); Assert.assertEquals(38.0, value.getDoubleMetric("sketch_count"), 0.01);
@ -277,7 +278,7 @@ public class SketchAggregationWithSimpleDataTest
); );
Result<SelectResultValue> result = (Result<SelectResultValue>) Iterables.getOnlyElement(Sequences.toList(seq, Lists.newArrayList())); Result<SelectResultValue> result = (Result<SelectResultValue>) Iterables.getOnlyElement(Sequences.toList(seq, Lists.newArrayList()));
Assert.assertEquals(new DateTime("2014-10-20T00:00:00.000Z"), result.getTimestamp()); Assert.assertEquals(DateTimes.of("2014-10-20T00:00:00.000Z"), result.getTimestamp());
Assert.assertEquals(100, result.getValue().getEvents().size()); Assert.assertEquals(100, result.getValue().getEvents().size());
Assert.assertEquals("AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI=", result.getValue().getEvents().get(0).getEvent().get("pty_country")); Assert.assertEquals("AgMDAAAazJMCAAAAAACAPzz9j7pWTMdROWGf15uY1nI=", result.getValue().getEvents().get(0).getEvent().get("pty_country"));
} }

View File

@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.IOE; import io.druid.java.util.common.IOE;
import io.druid.java.util.common.Intervals;
import io.druid.storage.hdfs.HdfsDataSegmentFinder; import io.druid.storage.hdfs.HdfsDataSegmentFinder;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NumberedShardSpec; import io.druid.timeline.partition.NumberedShardSpec;
@ -36,7 +37,6 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.joda.time.Interval;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
@ -56,80 +56,62 @@ public class HdfsDataSegmentFinderTest
private static final ObjectMapper mapper = new DefaultObjectMapper(); private static final ObjectMapper mapper = new DefaultObjectMapper();
private static final String DESCRIPTOR_JSON = "descriptor.json"; private static final String DESCRIPTOR_JSON = "descriptor.json";
private static final String INDEX_ZIP = "index.zip"; private static final String INDEX_ZIP = "index.zip";
private static final DataSegment SEGMENT_1 = DataSegment.builder() private static final DataSegment SEGMENT_1 = DataSegment
.dataSource("wikipedia") .builder()
.interval( .dataSource("wikipedia")
new Interval( .interval(Intervals.of("2013-08-31T00:00:00.000Z/2013-09-01T00:00:00.000Z"))
"2013-08-31T00:00:00.000Z/2013-09-01T00:00:00.000Z" .version("2015-10-21T22:07:57.074Z")
) .loadSpec(
) ImmutableMap.<String, Object>of(
.version("2015-10-21T22:07:57.074Z") "type",
.loadSpec( "hdfs",
ImmutableMap.<String, Object>of( "path",
"type", "hdfs://abc.com:1234/somewhere/index.zip"
"hdfs", )
"path", )
"hdfs://abc.com:1234/somewhere/index.zip" .dimensions(ImmutableList.of("language", "page"))
) .metrics(ImmutableList.of("count"))
) .build();
.dimensions(ImmutableList.of("language", "page"))
.metrics(ImmutableList.of("count"))
.build();
private static final DataSegment SEGMENT_2 = DataSegment.builder(SEGMENT_1) private static final DataSegment SEGMENT_2 = DataSegment
.interval( .builder(SEGMENT_1)
new Interval( .interval(Intervals.of("2013-09-01T00:00:00.000Z/2013-09-02T00:00:00.000Z"))
"2013-09-01T00:00:00.000Z/2013-09-02T00:00:00.000Z" .build();
)
)
.build();
private static final DataSegment SEGMENT_3 = DataSegment.builder(SEGMENT_1) private static final DataSegment SEGMENT_3 = DataSegment
.interval( .builder(SEGMENT_1)
new Interval( .interval(Intervals.of("2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"))
"2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z" .version("2015-10-22T22:07:57.074Z")
) .build();
)
.version("2015-10-22T22:07:57.074Z")
.build();
private static final DataSegment SEGMENT_4_0 = DataSegment.builder(SEGMENT_1) private static final DataSegment SEGMENT_4_0 = DataSegment
.interval( .builder(SEGMENT_1)
new Interval( .interval(Intervals.of("2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"))
"2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z" .shardSpec(new NumberedShardSpec(0, 2))
) .build();
)
.shardSpec(new NumberedShardSpec(0, 2))
.build();
private static final DataSegment SEGMENT_4_1 = DataSegment.builder(SEGMENT_1) private static final DataSegment SEGMENT_4_1 = DataSegment
.interval( .builder(SEGMENT_1)
new Interval( .interval(Intervals.of("2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"))
"2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z" .shardSpec(new NumberedShardSpec(1, 2))
) .build();
)
.shardSpec(new NumberedShardSpec(1, 2))
.build();
private static final DataSegment SEGMENT_5 = DataSegment.builder() private static final DataSegment SEGMENT_5 = DataSegment
.dataSource("wikipedia") .builder()
.interval( .dataSource("wikipedia")
new Interval( .interval(Intervals.of("2013-09-03T00:00:00.000Z/2013-09-04T00:00:00.000Z"))
"2013-09-03T00:00:00.000Z/2013-09-04T00:00:00.000Z" .version("2015-10-21T22:07:57.074Z")
) .loadSpec(
) ImmutableMap.<String, Object>of(
.version("2015-10-21T22:07:57.074Z") "type",
.loadSpec( "hdfs",
ImmutableMap.<String, Object>of( "path",
"type", "hdfs://abc.com:1234/somewhere/1_index.zip"
"hdfs", )
"path", )
"hdfs://abc.com:1234/somewhere/1_index.zip" .dimensions(ImmutableList.of("language", "page"))
) .metrics(ImmutableList.of("count"))
) .build();
.dimensions(ImmutableList.of("language", "page"))
.metrics(ImmutableList.of("count"))
.build();
private static MiniDFSCluster miniCluster; private static MiniDFSCluster miniCluster;
private static File hdfsTmpDir; private static File hdfsTmpDir;

View File

@ -21,6 +21,7 @@ package io.druid.storage.hdfs;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
@ -28,7 +29,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -183,7 +183,7 @@ public class HdfsDataSegmentKillerTest
{ {
return new DataSegment( return new DataSegment(
"dataSource", "dataSource",
Interval.parse("2000/3000"), Intervals.of("2000/3000"),
"ver", "ver",
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(
"type", "hdfs", "type", "hdfs",

View File

@ -20,7 +20,6 @@
package io.druid.storage.hdfs; package io.druid.storage.hdfs;
import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.InjectableValues;
@ -42,6 +41,7 @@ import io.druid.indexer.HadoopIngestionSpec;
import io.druid.indexer.JobHelper; import io.druid.indexer.JobHelper;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.jackson.GranularityModule; import io.druid.jackson.GranularityModule;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import io.druid.segment.loading.LocalDataSegmentPusher; import io.druid.segment.loading.LocalDataSegmentPusher;
import io.druid.segment.loading.LocalDataSegmentPusherConfig; import io.druid.segment.loading.LocalDataSegmentPusherConfig;
@ -57,6 +57,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.TaskType; import org.apache.hadoop.mapreduce.TaskType;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.joda.time.chrono.ISOChronology;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Rule; import org.junit.Rule;
@ -147,7 +148,7 @@ public class HdfsDataSegmentPusherTest
DataSegment segmentToPush = new DataSegment( DataSegment segmentToPush = new DataSegment(
"foo", "foo",
new Interval("2015/2016"), Intervals.of("2015/2016"),
"0", "0",
Maps.<String, Object>newHashMap(), Maps.<String, Object>newHashMap(),
Lists.<String>newArrayList(), Lists.<String>newArrayList(),
@ -230,7 +231,7 @@ public class HdfsDataSegmentPusherTest
for (int i = 0; i < numberOfSegments; i++) { for (int i = 0; i < numberOfSegments; i++) {
segments[i] = new DataSegment( segments[i] = new DataSegment(
"foo", "foo",
new Interval("2015/2016"), Intervals.of("2015/2016"),
"0", "0",
Maps.<String, Object>newHashMap(), Maps.<String, Object>newHashMap(),
Lists.<String>newArrayList(), Lists.<String>newArrayList(),
@ -337,11 +338,10 @@ public class HdfsDataSegmentPusherTest
Interval.class, new StdDeserializer<Interval>(Interval.class) Interval.class, new StdDeserializer<Interval>(Interval.class)
{ {
@Override @Override
public Interval deserialize( public Interval deserialize(JsonParser jsonParser, DeserializationContext deserializationContext)
JsonParser jsonParser, DeserializationContext deserializationContext throws IOException
) throws IOException, JsonProcessingException
{ {
return new Interval(jsonParser.getText()); return Intervals.of(jsonParser.getText());
} }
} }
); );
@ -353,7 +353,7 @@ public class HdfsDataSegmentPusherTest
public void shouldNotHaveColonsInHdfsStorageDir() throws Exception public void shouldNotHaveColonsInHdfsStorageDir() throws Exception
{ {
Interval interval = new Interval("2011-10-01/2011-10-02"); Interval interval = Intervals.of("2011-10-01/2011-10-02");
ImmutableMap<String, Object> loadSpec = ImmutableMap.<String, Object>of("something", "or_other"); ImmutableMap<String, Object> loadSpec = ImmutableMap.<String, Object>of("something", "or_other");
DataSegment segment = new DataSegment( DataSegment segment = new DataSegment(
@ -414,7 +414,7 @@ public class HdfsDataSegmentPusherTest
) )
); );
Bucket bucket = new Bucket(4711, new DateTime(2012, 07, 10, 5, 30), 4712); Bucket bucket = new Bucket(4711, new DateTime(2012, 07, 10, 5, 30, ISOChronology.getInstanceUTC()), 4712);
Path path = JobHelper.makeFileNamePath( Path path = JobHelper.makeFileNamePath(
new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()), new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()),
new DistributedFileSystem(), new DistributedFileSystem(),
@ -524,7 +524,7 @@ public class HdfsDataSegmentPusherTest
) )
); );
Bucket bucket = new Bucket(4711, new DateTime(2012, 07, 10, 5, 30), 4712); Bucket bucket = new Bucket(4711, new DateTime(2012, 07, 10, 5, 30, ISOChronology.getInstanceUTC()), 4712);
Path path = JobHelper.makeFileNamePath( Path path = JobHelper.makeFileNamePath(
new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()), new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()),
new LocalFileSystem(), new LocalFileSystem(),

View File

@ -183,7 +183,7 @@ public class QuantileSqlAggregator implements SqlAggregator
} }
} else { } else {
final ExpressionVirtualColumn virtualColumn = input.toVirtualColumn( final ExpressionVirtualColumn virtualColumn = input.toVirtualColumn(
String.format("%s:v", name), StringUtils.format("%s:v", name),
ValueType.FLOAT, ValueType.FLOAT,
plannerContext.getExprMacroTable() plannerContext.getExprMacroTable()
); );

View File

@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import io.druid.collections.StupidPool; import io.druid.collections.StupidPool;
import io.druid.java.util.common.DateTimes;
import io.druid.query.QueryPlus; import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner; import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryRunnerTestHelper;
@ -40,7 +41,6 @@ import io.druid.query.topn.TopNQueryQueryToolChest;
import io.druid.query.topn.TopNQueryRunnerFactory; import io.druid.query.topn.TopNQueryRunnerFactory;
import io.druid.query.topn.TopNResultValue; import io.druid.query.topn.TopNResultValue;
import io.druid.segment.TestHelper; import io.druid.segment.TestHelper;
import org.joda.time.DateTime;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.junit.runners.Parameterized; import org.junit.runners.Parameterized;
@ -146,7 +146,7 @@ public class ApproximateHistogramTopNQueryTest
List<Result<TopNResultValue>> expectedResults = Collections.singletonList( List<Result<TopNResultValue>> expectedResults = Collections.singletonList(
new Result<TopNResultValue>( new Result<TopNResultValue>(
new DateTime("2011-01-12T00:00:00.000Z"), DateTimes.of("2011-01-12T00:00:00.000Z"),
new TopNResultValue( new TopNResultValue(
Arrays.<Map<String, Object>>asList( Arrays.<Map<String, Object>>asList(
ImmutableMap.<String, Object>builder() ImmutableMap.<String, Object>builder()

View File

@ -55,6 +55,7 @@ import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.task.AbstractTask; import io.druid.indexing.common.task.AbstractTask;
import io.druid.indexing.common.task.RealtimeIndexTask; import io.druid.indexing.common.task.RealtimeIndexTask;
import io.druid.indexing.common.task.TaskResource; import io.druid.indexing.common.task.TaskResource;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.ISE; import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequence;
@ -266,7 +267,7 @@ public class KafkaIndexTask extends AbstractTask implements ChatHandler
public TaskStatus run(final TaskToolbox toolbox) throws Exception public TaskStatus run(final TaskToolbox toolbox) throws Exception
{ {
log.info("Starting up!"); log.info("Starting up!");
startTime = DateTime.now(); startTime = DateTimes.nowUtc();
mapper = toolbox.getObjectMapper(); mapper = toolbox.getObjectMapper();
status = Status.STARTING; status = Status.STARTING;

View File

@ -65,6 +65,7 @@ import io.druid.indexing.overlord.TaskRunnerWorkItem;
import io.druid.indexing.overlord.TaskStorage; import io.druid.indexing.overlord.TaskStorage;
import io.druid.indexing.overlord.supervisor.Supervisor; import io.druid.indexing.overlord.supervisor.Supervisor;
import io.druid.indexing.overlord.supervisor.SupervisorReport; import io.druid.indexing.overlord.supervisor.SupervisorReport;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.IAE; import io.druid.java.util.common.IAE;
import io.druid.java.util.common.ISE; import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
@ -349,7 +350,7 @@ public class KafkaSupervisor implements Supervisor
} }
} }
); );
firstRunTime = DateTime.now().plus(ioConfig.getStartDelay()); firstRunTime = DateTimes.nowUtc().plus(ioConfig.getStartDelay());
scheduledExec.scheduleAtFixedRate( scheduledExec.scheduleAtFixedRate(
buildRunTask(), buildRunTask(),
ioConfig.getStartDelay().getMillis(), ioConfig.getStartDelay().getMillis(),
@ -666,7 +667,7 @@ public class KafkaSupervisor implements Supervisor
if (taskInfoProvider.getTaskLocation(entry.getKey()).equals(TaskLocation.unknown())) { if (taskInfoProvider.getTaskLocation(entry.getKey()).equals(TaskLocation.unknown())) {
killTask(entry.getKey()); killTask(entry.getKey());
} else { } else {
entry.getValue().startTime = new DateTime(0); entry.getValue().startTime = DateTimes.EPOCH;
} }
} }
} }
@ -969,7 +970,7 @@ public class KafkaSupervisor implements Supervisor
TaskGroup newTaskGroup = new TaskGroup(ImmutableMap.copyOf(startingPartitions), Optional.<DateTime>absent(), Optional.<DateTime>absent()); TaskGroup newTaskGroup = new TaskGroup(ImmutableMap.copyOf(startingPartitions), Optional.<DateTime>absent(), Optional.<DateTime>absent());
newTaskGroup.tasks.put(taskId, new TaskData()); newTaskGroup.tasks.put(taskId, new TaskData());
newTaskGroup.completionTimeout = DateTime.now().plus(ioConfig.getCompletionTimeout()); newTaskGroup.completionTimeout = DateTimes.nowUtc().plus(ioConfig.getCompletionTimeout());
taskGroupList.add(newTaskGroup); taskGroupList.add(newTaskGroup);
} }
@ -1052,7 +1053,7 @@ public class KafkaSupervisor implements Supervisor
TaskGroup group = entry.getValue(); TaskGroup group = entry.getValue();
// find the longest running task from this group // find the longest running task from this group
DateTime earliestTaskStart = DateTime.now(); DateTime earliestTaskStart = DateTimes.nowUtc();
for (TaskData taskData : group.tasks.values()) { for (TaskData taskData : group.tasks.values()) {
if (earliestTaskStart.isAfter(taskData.startTime)) { if (earliestTaskStart.isAfter(taskData.startTime)) {
earliestTaskStart = taskData.startTime; earliestTaskStart = taskData.startTime;
@ -1075,7 +1076,7 @@ public class KafkaSupervisor implements Supervisor
if (endOffsets != null) { if (endOffsets != null) {
// set a timeout and put this group in pendingCompletionTaskGroups so that it can be monitored for completion // set a timeout and put this group in pendingCompletionTaskGroups so that it can be monitored for completion
group.completionTimeout = DateTime.now().plus(ioConfig.getCompletionTimeout()); group.completionTimeout = DateTimes.nowUtc().plus(ioConfig.getCompletionTimeout());
pendingCompletionTaskGroups.putIfAbsent(groupId, Lists.<TaskGroup>newCopyOnWriteArrayList()); pendingCompletionTaskGroups.putIfAbsent(groupId, Lists.<TaskGroup>newCopyOnWriteArrayList());
pendingCompletionTaskGroups.get(groupId).add(group); pendingCompletionTaskGroups.get(groupId).add(group);
@ -1362,11 +1363,11 @@ public class KafkaSupervisor implements Supervisor
log.info("Creating new task group [%d] for partitions %s", groupId, partitionGroups.get(groupId).keySet()); log.info("Creating new task group [%d] for partitions %s", groupId, partitionGroups.get(groupId).keySet());
Optional<DateTime> minimumMessageTime = (ioConfig.getLateMessageRejectionPeriod().isPresent() ? Optional.of( Optional<DateTime> minimumMessageTime = (ioConfig.getLateMessageRejectionPeriod().isPresent() ? Optional.of(
DateTime.now().minus(ioConfig.getLateMessageRejectionPeriod().get()) DateTimes.nowUtc().minus(ioConfig.getLateMessageRejectionPeriod().get())
) : Optional.<DateTime>absent()); ) : Optional.<DateTime>absent());
Optional<DateTime> maximumMessageTime = (ioConfig.getEarlyMessageRejectionPeriod().isPresent() ? Optional.of( Optional<DateTime> maximumMessageTime = (ioConfig.getEarlyMessageRejectionPeriod().isPresent() ? Optional.of(
DateTime.now().plus(ioConfig.getEarlyMessageRejectionPeriod().get()) DateTimes.nowUtc().plus(ioConfig.getEarlyMessageRejectionPeriod().get())
) : Optional.<DateTime>absent()); ) : Optional.<DateTime>absent());
taskGroups.put(groupId, new TaskGroup(generateStartingOffsetsForPartitionGroup(groupId), minimumMessageTime, maximumMessageTime)); taskGroups.put(groupId, new TaskGroup(generateStartingOffsetsForPartitionGroup(groupId), minimumMessageTime, maximumMessageTime));
@ -1626,7 +1627,7 @@ public class KafkaSupervisor implements Supervisor
Map<Integer, Long> partitionLag = getLagPerPartition(getHighestCurrentOffsets()); Map<Integer, Long> partitionLag = getLagPerPartition(getHighestCurrentOffsets());
KafkaSupervisorReport report = new KafkaSupervisorReport( KafkaSupervisorReport report = new KafkaSupervisorReport(
dataSource, dataSource,
DateTime.now(), DateTimes.nowUtc(),
ioConfig.getTopic(), ioConfig.getTopic(),
numPartitions, numPartitions,
ioConfig.getReplicas(), ioConfig.getReplicas(),
@ -1648,7 +1649,7 @@ public class KafkaSupervisor implements Supervisor
Long remainingSeconds = null; Long remainingSeconds = null;
if (startTime != null) { if (startTime != null) {
remainingSeconds = Math.max( remainingSeconds = Math.max(
0, ioConfig.getTaskDuration().getMillis() - (DateTime.now().getMillis() - startTime.getMillis()) 0, ioConfig.getTaskDuration().getMillis() - (System.currentTimeMillis() - startTime.getMillis())
) / 1000; ) / 1000;
} }
@ -1674,7 +1675,7 @@ public class KafkaSupervisor implements Supervisor
Map<Integer, Long> currentOffsets = entry.getValue().currentOffsets; Map<Integer, Long> currentOffsets = entry.getValue().currentOffsets;
Long remainingSeconds = null; Long remainingSeconds = null;
if (taskGroup.completionTimeout != null) { if (taskGroup.completionTimeout != null) {
remainingSeconds = Math.max(0, taskGroup.completionTimeout.getMillis() - DateTime.now().getMillis()) remainingSeconds = Math.max(0, taskGroup.completionTimeout.getMillis() - System.currentTimeMillis())
/ 1000; / 1000;
} }
@ -1822,7 +1823,7 @@ public class KafkaSupervisor implements Supervisor
try { try {
updateCurrentOffsets(); updateCurrentOffsets();
updateLatestOffsetsFromKafka(); updateLatestOffsetsFromKafka();
offsetsLastUpdated = DateTime.now(); offsetsLastUpdated = DateTimes.nowUtc();
} }
catch (Exception e) { catch (Exception e) {
log.warn(e, "Exception while getting current/latest offsets"); log.warn(e, "Exception while getting current/latest offsets");

View File

@ -24,9 +24,9 @@ import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
import io.druid.segment.indexing.IOConfig; import io.druid.segment.indexing.IOConfig;
import org.hamcrest.CoreMatchers; import org.hamcrest.CoreMatchers;
import org.joda.time.DateTime;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Rule; import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
@ -111,8 +111,8 @@ public class KafkaIOConfigTest
Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties()); Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties());
Assert.assertEquals(false, config.isUseTransaction()); Assert.assertEquals(false, config.isUseTransaction());
Assert.assertEquals(true, config.isPauseAfterRead()); Assert.assertEquals(true, config.isPauseAfterRead());
Assert.assertEquals(new DateTime("2016-05-31T12:00Z"), config.getMinimumMessageTime().get()); Assert.assertEquals(DateTimes.of("2016-05-31T12:00Z"), config.getMinimumMessageTime().get());
Assert.assertEquals(new DateTime("2016-05-31T14:00Z"), config.getMaximumMessageTime().get()); Assert.assertEquals(DateTimes.of("2016-05-31T14:00Z"), config.getMaximumMessageTime().get());
Assert.assertTrue("skipOffsetGaps", config.isSkipOffsetGaps()); Assert.assertTrue("skipOffsetGaps", config.isSkipOffsetGaps());
} }

View File

@ -35,6 +35,7 @@ import io.druid.indexing.common.TaskInfoProvider;
import io.druid.indexing.common.TaskLocation; import io.druid.indexing.common.TaskLocation;
import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskStatus;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.IAE; import io.druid.java.util.common.IAE;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import org.easymock.Capture; import org.easymock.Capture;
@ -346,7 +347,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
public void testGetStartTime() throws Exception public void testGetStartTime() throws Exception
{ {
client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2); client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2);
DateTime now = DateTime.now(); DateTime now = DateTimes.nowUtc();
Capture<Request> captured = Capture.newInstance(); Capture<Request> captured = Capture.newInstance();
expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3)
@ -789,7 +790,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
@Test @Test
public void testGetStartTimeAsync() throws Exception public void testGetStartTimeAsync() throws Exception
{ {
final DateTime now = DateTime.now(); final DateTime now = DateTimes.nowUtc();
final int numRequests = TEST_IDS.size(); final int numRequests = TEST_IDS.size();
Capture<Request> captured = Capture.newInstance(CaptureType.ALL); Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes();

View File

@ -78,7 +78,9 @@ import io.druid.indexing.test.TestDataSegmentAnnouncer;
import io.druid.indexing.test.TestDataSegmentKiller; import io.druid.indexing.test.TestDataSegmentKiller;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.CompressionUtils;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.ISE; import io.druid.java.util.common.ISE;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Sequences;
@ -128,7 +130,6 @@ import org.apache.curator.test.TestingCluster;
import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.ProducerRecord;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.joda.time.Period; import org.joda.time.Period;
import org.junit.After; import org.junit.After;
@ -420,7 +421,7 @@ public class KafkaIndexTaskTest
kafkaServer.consumerProperties(), kafkaServer.consumerProperties(),
true, true,
false, false,
new DateTime("2010"), DateTimes.of("2010"),
null, null,
false false
), ),
@ -477,7 +478,7 @@ public class KafkaIndexTaskTest
true, true,
false, false,
null, null,
new DateTime("2010"), DateTimes.of("2010"),
false false
), ),
null, null,
@ -1663,7 +1664,7 @@ public class KafkaIndexTaskTest
return FluentIterable.from( return FluentIterable.from(
metadataStorageCoordinator.getUsedSegmentsForInterval( metadataStorageCoordinator.getUsedSegmentsForInterval(
DATA_SCHEMA.getDataSource(), DATA_SCHEMA.getDataSource(),
new Interval("0000/3000") Intervals.of("0000/3000")
) )
).transform( ).transform(
new Function<DataSegment, SegmentDescriptor>() new Function<DataSegment, SegmentDescriptor>()
@ -1759,7 +1760,7 @@ public class KafkaIndexTaskTest
private SegmentDescriptor SD(final Task task, final String intervalString, final int partitionNum) private SegmentDescriptor SD(final Task task, final String intervalString, final int partitionNum)
{ {
final Interval interval = new Interval(intervalString); final Interval interval = Intervals.of(intervalString);
return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum); return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum);
} }
} }

View File

@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListenableFuture;
import io.druid.java.util.common.StringUtils;
import io.druid.data.input.impl.DimensionSchema; import io.druid.data.input.impl.DimensionSchema;
import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.JSONParseSpec;
@ -57,7 +56,9 @@ import io.druid.indexing.overlord.TaskRunnerListener;
import io.druid.indexing.overlord.TaskRunnerWorkItem; import io.druid.indexing.overlord.TaskRunnerWorkItem;
import io.druid.indexing.overlord.TaskStorage; import io.druid.indexing.overlord.TaskStorage;
import io.druid.indexing.overlord.supervisor.SupervisorReport; import io.druid.indexing.overlord.supervisor.SupervisorReport;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.ISE; import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory;
@ -607,7 +608,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED)) expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED))
.anyTimes(); .anyTimes();
expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTime.now())).anyTimes(); expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes();
expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata( new KafkaDataSourceMetadata(
null null
@ -696,7 +697,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
expect(taskStorage.getTask("id5")).andReturn(Optional.of(id3)).anyTimes(); expect(taskStorage.getTask("id5")).andReturn(Optional.of(id3)).anyTimes();
expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED)) expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED))
.anyTimes(); .anyTimes();
expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTime.now())).anyTimes(); expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes();
expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata( new KafkaDataSourceMetadata(
null null
@ -728,7 +729,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.<Task>of()).anyTimes(); expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.<Task>of()).anyTimes();
expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED)) expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED))
.anyTimes(); .anyTimes();
expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTime.now())).anyTimes(); expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes();
expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata( new KafkaDataSourceMetadata(
null null
@ -790,8 +791,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
supervisor = getSupervisor(2, 1, true, "PT1H", null, null, false); supervisor = getSupervisor(2, 1, true, "PT1H", null, null, false);
addSomeEvents(1); addSomeEvents(1);
DateTime now = DateTime.now(); DateTime now = DateTimes.nowUtc();
DateTime maxi = DateTime.now().plusMinutes(60); DateTime maxi = now.plusMinutes(60);
Task id1 = createKafkaIndexTask( Task id1 = createKafkaIndexTask(
"id1", "id1",
DATASOURCE, DATASOURCE,
@ -876,7 +877,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.<Task>of()).anyTimes(); expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.<Task>of()).anyTimes();
expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED)) expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED))
.anyTimes(); .anyTimes();
expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTime.now())).anyTimes(); expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes();
expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata( new KafkaDataSourceMetadata(
null null
@ -976,10 +977,10 @@ public class KafkaSupervisorTest extends EasyMockSupport
.andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)) .andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING))
.anyTimes(); .anyTimes();
expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0")))
.andReturn(Futures.immediateFuture(DateTime.now().minusMinutes(2))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2)))
.andReturn(Futures.immediateFuture(DateTime.now())); .andReturn(Futures.immediateFuture(DateTimes.nowUtc()));
expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1")))
.andReturn(Futures.immediateFuture(DateTime.now())) .andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.times(2); .times(2);
expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0")))
.andReturn(Futures.immediateFuture((Map<Integer, Long>) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) .andReturn(Futures.immediateFuture((Map<Integer, Long>) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)))
@ -1202,7 +1203,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
{ {
final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); final TaskLocation location1 = new TaskLocation("testHost", 1234, -1);
final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); final TaskLocation location2 = new TaskLocation("testHost2", 145, -1);
final DateTime startTime = new DateTime(); final DateTime startTime = DateTimes.nowUtc();
supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false);
addSomeEvents(6); addSomeEvents(6);
@ -1383,10 +1384,10 @@ public class KafkaSupervisorTest extends EasyMockSupport
.andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)) .andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING))
.anyTimes(); .anyTimes();
expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0")))
.andReturn(Futures.immediateFuture(DateTime.now().minusMinutes(2))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2)))
.andReturn(Futures.immediateFuture(DateTime.now())); .andReturn(Futures.immediateFuture(DateTimes.nowUtc()));
expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1")))
.andReturn(Futures.immediateFuture(DateTime.now())) .andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.times(2); .times(2);
expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0")))
.andReturn(Futures.<Map<Integer, Long>>immediateFailedFuture(new RuntimeException())).times(2); .andReturn(Futures.<Map<Integer, Long>>immediateFailedFuture(new RuntimeException())).times(2);
@ -1450,10 +1451,10 @@ public class KafkaSupervisorTest extends EasyMockSupport
.andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)) .andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING))
.anyTimes(); .anyTimes();
expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0")))
.andReturn(Futures.immediateFuture(DateTime.now().minusMinutes(2))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2)))
.andReturn(Futures.immediateFuture(DateTime.now())); .andReturn(Futures.immediateFuture(DateTimes.nowUtc()));
expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1")))
.andReturn(Futures.immediateFuture(DateTime.now())) .andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.times(2); .times(2);
expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0")))
.andReturn(Futures.immediateFuture((Map<Integer, Long>) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) .andReturn(Futures.immediateFuture((Map<Integer, Long>) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)))
@ -1508,7 +1509,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
{ {
final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); final TaskLocation location1 = new TaskLocation("testHost", 1234, -1);
final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); final TaskLocation location2 = new TaskLocation("testHost2", 145, -1);
final DateTime startTime = new DateTime(); final DateTime startTime = DateTimes.nowUtc();
supervisor = getSupervisor(2, 1, true, "PT1H", null, null, false); supervisor = getSupervisor(2, 1, true, "PT1H", null, null, false);
addSomeEvents(1); addSomeEvents(1);
@ -1697,7 +1698,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
{ {
final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); final TaskLocation location1 = new TaskLocation("testHost", 1234, -1);
final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); final TaskLocation location2 = new TaskLocation("testHost2", 145, -1);
final DateTime startTime = new DateTime(); final DateTime startTime = DateTimes.nowUtc();
supervisor = getSupervisor(2, 1, true, "PT1H", null, null, false); supervisor = getSupervisor(2, 1, true, "PT1H", null, null, false);
addSomeEvents(1); addSomeEvents(1);

View File

@ -19,7 +19,7 @@
package io.druid.server.lookup.namespace; package io.druid.server.lookup.namespace;
import io.druid.common.utils.JodaUtils; import io.druid.java.util.common.JodaUtils;
import io.druid.java.util.common.Pair; import io.druid.java.util.common.Pair;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.logger.Logger;

View File

@ -32,6 +32,7 @@ import io.druid.data.input.impl.StringDimensionSchema;
import io.druid.data.input.impl.TimestampSpec; import io.druid.data.input.impl.TimestampSpec;
import io.druid.java.util.common.parsers.ParseException; import io.druid.java.util.common.parsers.ParseException;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.chrono.ISOChronology;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -116,7 +117,7 @@ public class ProtobufInputRowParserTest
ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, "prototest.desc", "ProtoTestEvent"); ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, "prototest.desc", "ProtoTestEvent");
//create binary of proto test event //create binary of proto test event
DateTime dateTime = new DateTime(2012, 07, 12, 9, 30); DateTime dateTime = new DateTime(2012, 07, 12, 9, 30, ISOChronology.getInstanceUTC());
ProtoTestEventWrapper.ProtoTestEvent event = ProtoTestEventWrapper.ProtoTestEvent.newBuilder() ProtoTestEventWrapper.ProtoTestEvent event = ProtoTestEventWrapper.ProtoTestEvent.newBuilder()
.setDescription("description") .setDescription("description")
.setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE) .setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE)

View File

@ -27,11 +27,11 @@ import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.segment.loading.SegmentLoadingException; import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
@ -73,7 +73,7 @@ public class S3DataSegmentArchiverTest
.binaryVersion(1) .binaryVersion(1)
.dataSource("dataSource") .dataSource("dataSource")
.dimensions(ImmutableList.<String>of()) .dimensions(ImmutableList.<String>of())
.interval(Interval.parse("2015/2016")) .interval(Intervals.of("2015/2016"))
.version("version") .version("version")
.loadSpec(ImmutableMap.<String, Object>of( .loadSpec(ImmutableMap.<String, Object>of(
"type", "type",

View File

@ -31,6 +31,7 @@ import com.google.common.collect.Maps;
import com.google.common.collect.Ordering; import com.google.common.collect.Ordering;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.segment.loading.SegmentLoadingException; import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NumberedShardSpec; import io.druid.timeline.partition.NumberedShardSpec;
@ -42,7 +43,6 @@ import org.jets3t.service.StorageObjectsChunk;
import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import org.jets3t.service.model.S3Object; import org.jets3t.service.model.S3Object;
import org.jets3t.service.model.StorageObject; import org.jets3t.service.model.StorageObject;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.BeforeClass; import org.junit.BeforeClass;
@ -62,62 +62,47 @@ public class S3DataSegmentFinderTest
{ {
private static final ObjectMapper mapper = new DefaultObjectMapper(); private static final ObjectMapper mapper = new DefaultObjectMapper();
private static final DataSegment SEGMENT_1 = DataSegment.builder() private static final DataSegment SEGMENT_1 = DataSegment
.dataSource("wikipedia") .builder()
.interval( .dataSource("wikipedia")
new Interval( .interval(Intervals.of("2013-08-31T00:00:00.000Z/2013-09-01T00:00:00.000Z"))
"2013-08-31T00:00:00.000Z/2013-09-01T00:00:00.000Z" .version("2015-10-21T22:07:57.074Z")
) .loadSpec(
) ImmutableMap.<String, Object>of(
.version("2015-10-21T22:07:57.074Z") "type",
.loadSpec( "s3_zip",
ImmutableMap.<String, Object>of( "bucket",
"type", "bucket1",
"s3_zip", "key",
"bucket", "abc/somewhere/index.zip"
"bucket1", )
"key", )
"abc/somewhere/index.zip" .dimensions(ImmutableList.of("language", "page"))
) .metrics(ImmutableList.of("count"))
) .build();
.dimensions(ImmutableList.of("language", "page"))
.metrics(ImmutableList.of("count"))
.build();
private static final DataSegment SEGMENT_2 = DataSegment.builder(SEGMENT_1) private static final DataSegment SEGMENT_2 = DataSegment
.interval( .builder(SEGMENT_1)
new Interval( .interval(Intervals.of("2013-09-01T00:00:00.000Z/2013-09-02T00:00:00.000Z"))
"2013-09-01T00:00:00.000Z/2013-09-02T00:00:00.000Z" .build();
)
)
.build();
private static final DataSegment SEGMENT_3 = DataSegment.builder(SEGMENT_1) private static final DataSegment SEGMENT_3 = DataSegment
.interval( .builder(SEGMENT_1)
new Interval( .interval(Intervals.of("2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"))
"2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z" .version("2015-10-22T22:07:57.074Z")
) .build();
)
.version("2015-10-22T22:07:57.074Z")
.build();
private static final DataSegment SEGMENT_4_0 = DataSegment.builder(SEGMENT_1) private static final DataSegment SEGMENT_4_0 = DataSegment
.interval( .builder(SEGMENT_1)
new Interval( .interval(Intervals.of("2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"))
"2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z" .shardSpec(new NumberedShardSpec(0, 2))
) .build();
)
.shardSpec(new NumberedShardSpec(0, 2))
.build();
private static final DataSegment SEGMENT_4_1 = DataSegment.builder(SEGMENT_1) private static final DataSegment SEGMENT_4_1 = DataSegment
.interval( .builder(SEGMENT_1)
new Interval( .interval(Intervals.of("2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"))
"2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z" .shardSpec(new NumberedShardSpec(1, 2))
) .build();
)
.shardSpec(new NumberedShardSpec(1, 2))
.build();
@Rule @Rule
public final TemporaryFolder temporaryFolder = new TemporaryFolder(); public final TemporaryFolder temporaryFolder = new TemporaryFolder();

View File

@ -23,7 +23,7 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.MapUtils; import io.druid.java.util.common.MapUtils;
import io.druid.segment.loading.SegmentLoadingException; import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
@ -33,7 +33,6 @@ import org.jets3t.service.ServiceException;
import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import org.jets3t.service.model.S3Object; import org.jets3t.service.model.S3Object;
import org.jets3t.service.model.StorageObject; import org.jets3t.service.model.StorageObject;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -44,7 +43,7 @@ public class S3DataSegmentMoverTest
{ {
private static final DataSegment sourceSegment = new DataSegment( private static final DataSegment sourceSegment = new DataSegment(
"test", "test",
new Interval("2013-01-01/2013-01-02"), Intervals.of("2013-01-01/2013-01-02"),
"1", "1",
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(
"key", "key",
@ -119,7 +118,7 @@ public class S3DataSegmentMoverTest
S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client, new S3DataSegmentPusherConfig()); S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client, new S3DataSegmentPusherConfig());
mover.move(new DataSegment( mover.move(new DataSegment(
"test", "test",
new Interval("2013-01-01/2013-01-02"), Intervals.of("2013-01-01/2013-01-02"),
"1", "1",
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(
"key", "key",
@ -142,7 +141,7 @@ public class S3DataSegmentMoverTest
S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client, new S3DataSegmentPusherConfig()); S3DataSegmentMover mover = new S3DataSegmentMover(mockS3Client, new S3DataSegmentPusherConfig());
mover.move(new DataSegment( mover.move(new DataSegment(
"test", "test",
new Interval("2013-01-01/2013-01-02"), Intervals.of("2013-01-01/2013-01-02"),
"1", "1",
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(
"key", "key",

View File

@ -24,6 +24,7 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.io.Files; import com.google.common.io.Files;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
@ -32,7 +33,6 @@ import org.easymock.EasyMock;
import org.easymock.IAnswer; import org.easymock.IAnswer;
import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import org.jets3t.service.model.S3Object; import org.jets3t.service.model.S3Object;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Rule; import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
@ -103,7 +103,7 @@ public class S3DataSegmentPusherTest
DataSegment segmentToPush = new DataSegment( DataSegment segmentToPush = new DataSegment(
"foo", "foo",
new Interval("2015/2016"), Intervals.of("2015/2016"),
"0", "0",
Maps.<String, Object>newHashMap(), Maps.<String, Object>newHashMap(),
Lists.<String>newArrayList(), Lists.<String>newArrayList(),

View File

@ -24,6 +24,7 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import io.druid.data.input.MapBasedRow; import io.druid.data.input.MapBasedRow;
import io.druid.data.input.Row; import io.druid.data.input.Row;
import io.druid.java.util.common.DateTimes;
import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.PostAggregator; import io.druid.query.aggregation.PostAggregator;
@ -99,7 +100,7 @@ public class VarianceTestHelper extends QueryRunnerTestHelper
for (int i = 0; i < values.length; i++) { for (int i = 0; i < values.length; i++) {
theVals.put(names[i], values[i]); theVals.put(names[i], values[i]);
} }
DateTime ts = new DateTime(timestamp); DateTime ts = DateTimes.of(timestamp);
return new MapBasedRow(ts, theVals); return new MapBasedRow(ts, theVals);
} }
} }

View File

@ -20,6 +20,7 @@
package io.druid.query.aggregation.variance; package io.druid.query.aggregation.variance;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Sequences;
import io.druid.query.Druids; import io.druid.query.Druids;
import io.druid.query.QueryPlus; import io.druid.query.QueryPlus;
@ -31,7 +32,6 @@ import io.druid.query.timeseries.TimeseriesQuery;
import io.druid.query.timeseries.TimeseriesQueryRunnerTest; import io.druid.query.timeseries.TimeseriesQueryRunnerTest;
import io.druid.query.timeseries.TimeseriesResultValue; import io.druid.query.timeseries.TimeseriesResultValue;
import io.druid.segment.TestHelper; import io.druid.segment.TestHelper;
import org.joda.time.DateTime;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.junit.runners.Parameterized; import org.junit.runners.Parameterized;
@ -79,7 +79,7 @@ public class VarianceTimeseriesQueryTest
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList( List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
new Result<>( new Result<>(
new DateTime("2011-04-01"), DateTimes.of("2011-04-01"),
new TimeseriesResultValue( new TimeseriesResultValue(
VarianceTestHelper.of( VarianceTestHelper.of(
"rows", 13L, "rows", 13L,
@ -92,7 +92,7 @@ public class VarianceTimeseriesQueryTest
) )
), ),
new Result<>( new Result<>(
new DateTime("2011-04-02"), DateTimes.of("2011-04-02"),
new TimeseriesResultValue( new TimeseriesResultValue(
VarianceTestHelper.of( VarianceTestHelper.of(
"rows", 13L, "rows", 13L,

View File

@ -22,6 +22,7 @@ package io.druid.query.aggregation.variance;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequence;
import io.druid.query.QueryPlus; import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner; import io.druid.query.QueryRunner;
@ -38,7 +39,6 @@ import io.druid.query.topn.TopNQueryQueryToolChest;
import io.druid.query.topn.TopNQueryRunnerTest; import io.druid.query.topn.TopNQueryRunnerTest;
import io.druid.query.topn.TopNResultValue; import io.druid.query.topn.TopNResultValue;
import io.druid.segment.TestHelper; import io.druid.segment.TestHelper;
import org.joda.time.DateTime;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.junit.runners.Parameterized; import org.junit.runners.Parameterized;
@ -92,7 +92,7 @@ public class VarianceTopNQueryTest
List<Result<TopNResultValue>> expectedResults = Arrays.asList( List<Result<TopNResultValue>> expectedResults = Arrays.asList(
new Result<TopNResultValue>( new Result<TopNResultValue>(
new DateTime("2011-01-12T00:00:00.000Z"), DateTimes.of("2011-01-12T00:00:00.000Z"),
new TopNResultValue( new TopNResultValue(
Arrays.<Map<String, Object>>asList( Arrays.<Map<String, Object>>asList(
ImmutableMap.<String, Object>builder() ImmutableMap.<String, Object>builder()

View File

@ -25,6 +25,7 @@ import com.google.common.collect.Lists;
import com.google.common.hash.HashFunction; import com.google.common.hash.HashFunction;
import com.google.common.hash.Hashing; import com.google.common.hash.Hashing;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import org.apache.commons.codec.binary.Base64; import org.apache.commons.codec.binary.Base64;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Ignore; import org.junit.Ignore;
@ -36,13 +37,13 @@ import java.security.MessageDigest;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.Locale;
import java.util.Random; import java.util.Random;
/** /**
*/ */
public class HyperLogLogCollectorTest public class HyperLogLogCollectorTest
{ {
private static final Logger log = new Logger(HyperLogLogCollectorTest.class);
private final HashFunction fn = Hashing.murmur3_128(); private final HashFunction fn = Hashing.murmur3_128();
@ -118,15 +119,10 @@ public class HyperLogLogCollectorTest
int n = count; int n = count;
System.out.println("True cardinality " + n); log.info("True cardinality " + n);
System.out.println("Rolling buffer cardinality " + rolling.estimateCardinality()); log.info("Rolling buffer cardinality " + rolling.estimateCardinality());
System.out.println("Simple buffer cardinality " + simple.estimateCardinality()); log.info("Simple buffer cardinality " + simple.estimateCardinality());
System.out.println( log.info("Rolling cardinality estimate off by %4.1f%%", 100 * (1 - rolling.estimateCardinality() / n));
StringUtils.format(
"Rolling cardinality estimate off by %4.1f%%",
100 * (1 - rolling.estimateCardinality() / n)
)
);
Assert.assertEquals(n, simple.estimateCardinality(), n * 0.05); Assert.assertEquals(n, simple.estimateCardinality(), n * 0.05);
Assert.assertEquals(n, rolling.estimateCardinality(), n * 0.05); Assert.assertEquals(n, rolling.estimateCardinality(), n * 0.05);
@ -145,22 +141,13 @@ public class HyperLogLogCollectorTest
theCollector.add(fn.hashLong(count).asBytes()); theCollector.add(fn.hashLong(count).asBytes());
rolling.fold(theCollector); rolling.fold(theCollector);
} }
System.out.printf( log.info("testHighCardinalityRollingFold2 took %d ms", System.currentTimeMillis() - start);
Locale.ENGLISH,
"testHighCardinalityRollingFold2 took %d ms%n",
System.currentTimeMillis() - start
);
int n = count; int n = count;
System.out.println("True cardinality " + n); log.info("True cardinality " + n);
System.out.println("Rolling buffer cardinality " + rolling.estimateCardinality()); log.info("Rolling buffer cardinality " + rolling.estimateCardinality());
System.out.println( log.info("Rolling cardinality estimate off by %4.1f%%", 100 * (1 - rolling.estimateCardinality() / n));
StringUtils.format(
"Rolling cardinality estimate off by %4.1f%%",
100 * (1 - rolling.estimateCardinality() / n)
)
);
Assert.assertEquals(n, rolling.estimateCardinality(), n * 0.05); Assert.assertEquals(n, rolling.estimateCardinality(), n * 0.05);
} }
@ -843,9 +830,8 @@ public class HyperLogLogCollectorTest
error += errorThisTime; error += errorThisTime;
System.out.printf( log.info(
Locale.ENGLISH, "%,d ==? %,f in %,d millis. actual error[%,f%%], avg. error [%,f%%]",
"%,d ==? %,f in %,d millis. actual error[%,f%%], avg. error [%,f%%]%n",
numThings, numThings,
estimatedValue, estimatedValue,
System.currentTimeMillis() - startTime, System.currentTimeMillis() - startTime,

View File

@ -20,9 +20,8 @@
package io.druid.indexer; package io.druid.indexer;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.Pair; import io.druid.java.util.common.Pair;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -118,7 +117,7 @@ public class Bucket
{ {
ByteBuffer buf = ByteBuffer.wrap(keyBytes); ByteBuffer buf = ByteBuffer.wrap(keyBytes);
Bucket bucket = new Bucket(buf.getInt(), new DateTime(buf.getLong()), buf.getInt()); Bucket bucket = new Bucket(buf.getInt(), DateTimes.utc(buf.getLong()), buf.getInt());
byte[] bytesLeft = new byte[buf.remaining()]; byte[] bytesLeft = new byte[buf.remaining()];
buf.get(bytesLeft); buf.get(bytesLeft);

View File

@ -31,6 +31,7 @@ import com.google.common.io.Closeables;
import io.druid.data.input.InputRow; import io.druid.data.input.InputRow;
import io.druid.data.input.Rows; import io.druid.data.input.Rows;
import io.druid.hll.HyperLogLogCollector; import io.druid.hll.HyperLogLogCollector;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.ISE; import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.granularity.Granularity;
@ -257,14 +258,14 @@ public class DetermineHashedPartitionsJob implements Jobby
if (determineIntervals) { if (determineIntervals) {
interval = config.getGranularitySpec() interval = config.getGranularitySpec()
.getSegmentGranularity() .getSegmentGranularity()
.bucket(new DateTime(inputRow.getTimestampFromEpoch())); .bucket(DateTimes.utc(inputRow.getTimestampFromEpoch()));
if (!hyperLogLogs.containsKey(interval)) { if (!hyperLogLogs.containsKey(interval)) {
hyperLogLogs.put(interval, HyperLogLogCollector.makeLatestCollector()); hyperLogLogs.put(interval, HyperLogLogCollector.makeLatestCollector());
} }
} else { } else {
final Optional<Interval> maybeInterval = config.getGranularitySpec() final Optional<Interval> maybeInterval = config.getGranularitySpec()
.bucketInterval(new DateTime(inputRow.getTimestampFromEpoch())); .bucketInterval(DateTimes.utc(inputRow.getTimestampFromEpoch()));
if (!maybeInterval.isPresent()) { if (!maybeInterval.isPresent()) {
throw new ISE("WTF?! No bucket found for timestamp: %s", inputRow.getTimestampFromEpoch()); throw new ISE("WTF?! No bucket found for timestamp: %s", inputRow.getTimestampFromEpoch());
@ -324,7 +325,7 @@ public class DetermineHashedPartitionsJob implements Jobby
HyperLogLogCollector.makeCollector(ByteBuffer.wrap(value.getBytes(), 0, value.getLength())) HyperLogLogCollector.makeCollector(ByteBuffer.wrap(value.getBytes(), 0, value.getLength()))
); );
} }
Optional<Interval> intervalOptional = config.getGranularitySpec().bucketInterval(new DateTime(key.get())); Optional<Interval> intervalOptional = config.getGranularitySpec().bucketInterval(DateTimes.utc(key.get()));
if (!intervalOptional.isPresent()) { if (!intervalOptional.isPresent()) {
throw new ISE("WTF?! No bucket found for timestamp: %s", key.get()); throw new ISE("WTF?! No bucket found for timestamp: %s", key.get());

View File

@ -37,6 +37,7 @@ import io.druid.collections.CombiningIterable;
import io.druid.data.input.InputRow; import io.druid.data.input.InputRow;
import io.druid.data.input.Rows; import io.druid.data.input.Rows;
import io.druid.indexer.partitions.SingleDimensionPartitionsSpec; import io.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.ISE; import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.granularity.Granularity;
@ -67,6 +68,7 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.joda.time.chrono.ISOChronology;
import java.io.IOException; import java.io.IOException;
import java.io.OutputStream; import java.io.OutputStream;
@ -322,7 +324,7 @@ public class DeterminePartitionsJob implements Jobby
{ {
final List<Object> timeAndDims = HadoopDruidIndexerConfig.JSON_MAPPER.readValue(key.getBytes(), List.class); final List<Object> timeAndDims = HadoopDruidIndexerConfig.JSON_MAPPER.readValue(key.getBytes(), List.class);
final DateTime timestamp = new DateTime(timeAndDims.get(0)); final DateTime timestamp = new DateTime(timeAndDims.get(0), ISOChronology.getInstanceUTC());
final Map<String, Iterable<String>> dims = (Map<String, Iterable<String>>) timeAndDims.get(1); final Map<String, Iterable<String>> dims = (Map<String, Iterable<String>>) timeAndDims.get(1);
helper.emitDimValueCounts(context, timestamp, dims); helper.emitDimValueCounts(context, timestamp, dims);
@ -359,7 +361,7 @@ public class DeterminePartitionsJob implements Jobby
for (final String dim : inputRow.getDimensions()) { for (final String dim : inputRow.getDimensions()) {
dims.put(dim, inputRow.getDimension(dim)); dims.put(dim, inputRow.getDimension(dim));
} }
helper.emitDimValueCounts(context, new DateTime(inputRow.getTimestampFromEpoch()), dims); helper.emitDimValueCounts(context, DateTimes.utc(inputRow.getTimestampFromEpoch()), dims);
} }
} }
@ -566,7 +568,7 @@ public class DeterminePartitionsJob implements Jobby
{ {
final ByteBuffer groupKey = ByteBuffer.wrap(keyBytes.getGroupKey()); final ByteBuffer groupKey = ByteBuffer.wrap(keyBytes.getGroupKey());
groupKey.position(4); // Skip partition groupKey.position(4); // Skip partition
final DateTime bucket = new DateTime(groupKey.getLong()); final DateTime bucket = DateTimes.utc(groupKey.getLong());
final PeekingIterator<DimValueCount> iterator = Iterators.peekingIterator(combinedIterable.iterator()); final PeekingIterator<DimValueCount> iterator = Iterators.peekingIterator(combinedIterable.iterator());
log.info( log.info(

View File

@ -37,7 +37,6 @@ import com.google.inject.Binder;
import com.google.inject.Injector; import com.google.inject.Injector;
import com.google.inject.Key; import com.google.inject.Key;
import com.google.inject.Module; import com.google.inject.Module;
import io.druid.common.utils.JodaUtils;
import io.druid.data.input.InputRow; import io.druid.data.input.InputRow;
import io.druid.data.input.impl.InputRowParser; import io.druid.data.input.impl.InputRowParser;
import io.druid.guice.GuiceInjectors; import io.druid.guice.GuiceInjectors;
@ -46,6 +45,8 @@ import io.druid.guice.annotations.Self;
import io.druid.indexer.partitions.PartitionsSpec; import io.druid.indexer.partitions.PartitionsSpec;
import io.druid.indexer.path.PathSpec; import io.druid.indexer.path.PathSpec;
import io.druid.initialization.Initialization; import io.druid.initialization.Initialization;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.JodaUtils;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.granularity.Granularity;
import io.druid.java.util.common.guava.FunctionalIterable; import io.druid.java.util.common.guava.FunctionalIterable;
@ -410,9 +411,7 @@ public class HadoopDruidIndexerConfig
public Optional<Bucket> getBucket(InputRow inputRow) public Optional<Bucket> getBucket(InputRow inputRow)
{ {
final Optional<Interval> timeBucket = schema.getDataSchema().getGranularitySpec().bucketInterval( final Optional<Interval> timeBucket = schema.getDataSchema().getGranularitySpec().bucketInterval(
new DateTime( DateTimes.utc(inputRow.getTimestampFromEpoch())
inputRow.getTimestampFromEpoch()
)
); );
if (!timeBucket.isPresent()) { if (!timeBucket.isPresent()) {
return Optional.absent(); return Optional.absent();

View File

@ -22,13 +22,13 @@ package io.druid.indexer;
import io.druid.data.input.InputRow; import io.druid.data.input.InputRow;
import io.druid.data.input.impl.InputRowParser; import io.druid.data.input.impl.InputRowParser;
import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.StringInputRowParser;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.RE; import io.druid.java.util.common.RE;
import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.logger.Logger;
import io.druid.java.util.common.parsers.ParseException; import io.druid.java.util.common.parsers.ParseException;
import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.Mapper;
import org.joda.time.DateTime;
import java.io.IOException; import java.io.IOException;
@ -82,7 +82,7 @@ public abstract class HadoopDruidIndexerMapper<KEYOUT, VALUEOUT> extends Mapper<
} }
if (!granularitySpec.bucketIntervals().isPresent() if (!granularitySpec.bucketIntervals().isPresent()
|| granularitySpec.bucketInterval(new DateTime(inputRow.getTimestampFromEpoch())) || granularitySpec.bucketInterval(DateTimes.utc(inputRow.getTimestampFromEpoch()))
.isPresent()) { .isPresent()) {
innerMap(inputRow, value, context, reportParseExceptions); innerMap(inputRow, value, context, reportParseExceptions);
} }

View File

@ -27,9 +27,9 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import io.druid.indexer.partitions.HashedPartitionsSpec; import io.druid.indexer.partitions.HashedPartitionsSpec;
import io.druid.indexer.partitions.PartitionsSpec; import io.druid.indexer.partitions.PartitionsSpec;
import io.druid.java.util.common.DateTimes;
import io.druid.segment.IndexSpec; import io.druid.segment.IndexSpec;
import io.druid.segment.indexing.TuningConfig; import io.druid.segment.indexing.TuningConfig;
import org.joda.time.DateTime;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -50,7 +50,7 @@ public class HadoopTuningConfig implements TuningConfig
{ {
return new HadoopTuningConfig( return new HadoopTuningConfig(
null, null,
new DateTime().toString(), DateTimes.nowUtc().toString(),
DEFAULT_PARTITIONS_SPEC, DEFAULT_PARTITIONS_SPEC,
DEFAULT_SHARD_SPECS, DEFAULT_SHARD_SPECS,
DEFAULT_INDEX_SPEC, DEFAULT_INDEX_SPEC,
@ -115,7 +115,7 @@ public class HadoopTuningConfig implements TuningConfig
) )
{ {
this.workingPath = workingPath; this.workingPath = workingPath;
this.version = version == null ? new DateTime().toString() : version; this.version = version == null ? DateTimes.nowUtc().toString() : version;
this.partitionsSpec = partitionsSpec == null ? DEFAULT_PARTITIONS_SPEC : partitionsSpec; this.partitionsSpec = partitionsSpec == null ? DEFAULT_PARTITIONS_SPEC : partitionsSpec;
this.shardSpecs = shardSpecs == null ? DEFAULT_SHARD_SPECS : shardSpecs; this.shardSpecs = shardSpecs == null ? DEFAULT_SHARD_SPECS : shardSpecs;
this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec; this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec;

View File

@ -27,6 +27,7 @@ import com.google.common.io.ByteStreams;
import com.google.common.io.Files; import com.google.common.io.Files;
import com.google.common.io.OutputSupplier; import com.google.common.io.OutputSupplier;
import io.druid.indexer.updater.HadoopDruidConverterConfig; import io.druid.indexer.updater.HadoopDruidConverterConfig;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.FileUtils; import io.druid.java.util.common.FileUtils;
import io.druid.java.util.common.IAE; import io.druid.java.util.common.IAE;
import io.druid.java.util.common.IOE; import io.druid.java.util.common.IOE;
@ -51,7 +52,6 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.Progressable;
import org.joda.time.DateTime;
import java.io.BufferedOutputStream; import java.io.BufferedOutputStream;
import java.io.File; import java.io.File;
@ -622,10 +622,10 @@ public class JobHelper
log.info( log.info(
"File[%s / %s / %sB] existed, but wasn't the same as [%s / %s / %sB]", "File[%s / %s / %sB] existed, but wasn't the same as [%s / %s / %sB]",
finalIndexZipFile.getPath(), finalIndexZipFile.getPath(),
new DateTime(finalIndexZipFile.getModificationTime()), DateTimes.utc(finalIndexZipFile.getModificationTime()),
finalIndexZipFile.getLen(), finalIndexZipFile.getLen(),
zipFile.getPath(), zipFile.getPath(),
new DateTime(zipFile.getModificationTime()), DateTimes.utc(zipFile.getModificationTime()),
zipFile.getLen() zipFile.getLen()
); );
outputFS.delete(finalIndexZipFilePath, false); outputFS.delete(finalIndexZipFilePath, false);
@ -634,7 +634,7 @@ public class JobHelper
log.info( log.info(
"File[%s / %s / %sB] existed and will be kept", "File[%s / %s / %sB] existed and will be kept",
finalIndexZipFile.getPath(), finalIndexZipFile.getPath(),
new DateTime(finalIndexZipFile.getModificationTime()), DateTimes.utc(finalIndexZipFile.getModificationTime()),
finalIndexZipFile.getLen() finalIndexZipFile.getLen()
); );
needRename = false; needRename = false;

View File

@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import io.druid.common.utils.JodaUtils; import io.druid.java.util.common.JodaUtils;
import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.granularity.Granularity;
import io.druid.query.filter.DimFilter; import io.druid.query.filter.DimFilter;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;

View File

@ -24,10 +24,11 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Ordering; import com.google.common.collect.Ordering;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularity;
import io.druid.indexer.HadoopDruidIndexerConfig; import io.druid.indexer.HadoopDruidIndexerConfig;
import io.druid.indexer.hadoop.FSSpideringIterator; import io.druid.indexer.hadoop.FSSpideringIterator;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularity;
import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.guava.Comparators;
import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
@ -88,7 +89,7 @@ public class GranularUnprocessedPathSpec extends GranularityPathSpec
Set<Interval> bucketsToRun = Sets.newTreeSet(Comparators.intervals()); Set<Interval> bucketsToRun = Sets.newTreeSet(Comparators.intervals());
for (Map.Entry<Long, Long> entry : inputModifiedTimes.entrySet()) { for (Map.Entry<Long, Long> entry : inputModifiedTimes.entrySet()) {
DateTime timeBucket = new DateTime(entry.getKey()); DateTime timeBucket = DateTimes.utc(entry.getKey());
long mTime = entry.getValue(); long mTime = entry.getValue();
String bucketOutput = StringUtils.format( String bucketOutput = StringUtils.format(

View File

@ -168,7 +168,7 @@ public class GranularityPathSpec implements PathSpec
end = inputInterval.getEndMillis(); end = inputInterval.getEndMillis();
makeNew = true; makeNew = true;
} }
return makeNew ? new Interval(start, end) : interval; return makeNew ? new Interval(start, end, interval.getChronology()) : interval;
} }
} }

View File

@ -35,6 +35,7 @@ import io.druid.data.input.impl.StringInputRowParser;
import io.druid.data.input.impl.TimestampSpec; import io.druid.data.input.impl.TimestampSpec;
import io.druid.indexer.hadoop.WindowedDataSegment; import io.druid.indexer.hadoop.WindowedDataSegment;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
@ -71,8 +72,8 @@ public class BatchDeltaIngestionTest
private static final ObjectMapper MAPPER; private static final ObjectMapper MAPPER;
private static final IndexIO INDEX_IO; private static final IndexIO INDEX_IO;
private static final Interval INTERVAL_FULL = new Interval("2014-10-22T00:00:00Z/P1D"); private static final Interval INTERVAL_FULL = Intervals.of("2014-10-22T00:00:00Z/P1D");
private static final Interval INTERVAL_PARTIAL = new Interval("2014-10-22T00:00:00Z/PT2H"); private static final Interval INTERVAL_PARTIAL = Intervals.of("2014-10-22T00:00:00Z/PT2H");
private static final DataSegment SEGMENT; private static final DataSegment SEGMENT;
static { static {

View File

@ -21,10 +21,12 @@ package io.druid.indexer;
import com.google.common.primitives.Bytes; import com.google.common.primitives.Bytes;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.Pair; import io.druid.java.util.common.Pair;
import org.hamcrest.number.OrderingComparison; import org.hamcrest.number.OrderingComparison;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.chrono.ISOChronology;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
@ -39,7 +41,7 @@ public class BucketTest
@Before public void setUp() @Before public void setUp()
{ {
time = new DateTime(2014, 11, 24, 10, 30); time = new DateTime(2014, 11, 24, 10, 30, ISOChronology.getInstanceUTC());
shardNum = 1; shardNum = 1;
partitionNum = 1; partitionNum = 1;
bucket = new Bucket(shardNum, time, partitionNum); bucket = new Bucket(shardNum, time, partitionNum);
@ -80,10 +82,12 @@ public class BucketTest
bucket.equals(new Bucket(shardNum, time, partitionNum + 1))); bucket.equals(new Bucket(shardNum, time, partitionNum + 1)));
Assert.assertFalse("Objects do not have the same shardNum", Assert.assertFalse("Objects do not have the same shardNum",
bucket.equals(new Bucket(shardNum + 1, time, partitionNum))); bucket.equals(new Bucket(shardNum + 1, time, partitionNum)));
Assert.assertFalse("Objects do not have the same time", bucket.equals(new Bucket(shardNum, new DateTime(), partitionNum))); Assert.assertFalse(
"Objects do not have the same time",
bucket.equals(new Bucket(shardNum, DateTimes.nowUtc(), partitionNum))
);
Assert.assertFalse("Object do have NULL time", bucket.equals(new Bucket(shardNum, null, partitionNum))); Assert.assertFalse("Object do have NULL time", bucket.equals(new Bucket(shardNum, null, partitionNum)));
Assert.assertTrue("Objects must be the same", bucket.equals(new Bucket(shardNum, time, partitionNum))); Assert.assertTrue("Objects must be the same", bucket.equals(new Bucket(shardNum, time, partitionNum)));
} }
@Test public void testHashCode() @Test public void testHashCode()

View File

@ -27,12 +27,12 @@ import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.StringInputRowParser;
import io.druid.data.input.impl.TimestampSpec; import io.druid.data.input.impl.TimestampSpec;
import io.druid.indexer.partitions.HashedPartitionsSpec; import io.druid.indexer.partitions.HashedPartitionsSpec;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
@ -147,7 +147,7 @@ public class DetermineHashedPartitionsJobTest
new UniformGranularitySpec( new UniformGranularitySpec(
Granularities.DAY, Granularities.DAY,
Granularities.NONE, Granularities.NONE,
ImmutableList.of(new Interval(interval)) ImmutableList.of(Intervals.of(interval))
), ),
HadoopDruidIndexerConfig.JSON_MAPPER HadoopDruidIndexerConfig.JSON_MAPPER
), ),

View File

@ -27,6 +27,7 @@ import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.StringInputRowParser;
import io.druid.data.input.impl.TimestampSpec; import io.druid.data.input.impl.TimestampSpec;
import io.druid.indexer.partitions.SingleDimensionPartitionsSpec; import io.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory;
@ -34,7 +35,6 @@ import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import io.druid.timeline.partition.SingleDimensionShardSpec; import io.druid.timeline.partition.SingleDimensionShardSpec;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.joda.time.Interval;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -237,7 +237,7 @@ public class DeterminePartitionsJobTest
), ),
new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")}, new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")},
new UniformGranularitySpec( new UniformGranularitySpec(
Granularities.DAY, Granularities.NONE, ImmutableList.of(new Interval(interval)) Granularities.DAY, Granularities.NONE, ImmutableList.of(Intervals.of(interval))
), ),
HadoopDruidIndexerConfig.JSON_MAPPER HadoopDruidIndexerConfig.JSON_MAPPER
), ),

View File

@ -26,14 +26,14 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.MapBasedInputRow;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.HashBasedNumberedShardSpec;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -71,7 +71,7 @@ public class HadoopDruidIndexerConfigTest
new UniformGranularitySpec( new UniformGranularitySpec(
Granularities.MINUTE, Granularities.MINUTE,
Granularities.MINUTE, Granularities.MINUTE,
ImmutableList.of(new Interval("2010-01-01/P1D")) ImmutableList.of(Intervals.of("2010-01-01/P1D"))
), ),
jsonMapper jsonMapper
), ),
@ -80,7 +80,7 @@ public class HadoopDruidIndexerConfigTest
null, null,
null, null,
null, null,
ImmutableMap.of(new DateTime("2010-01-01T01:00:00").getMillis(), specs), ImmutableMap.of(DateTimes.of("2010-01-01T01:00:00").getMillis(), specs),
null, null,
null, null,
false, false,
@ -110,9 +110,9 @@ public class HadoopDruidIndexerConfigTest
"dim2", "dim2",
"4" "4"
); );
final long timestamp = new DateTime("2010-01-01T01:00:01").getMillis(); final long timestamp = DateTimes.of("2010-01-01T01:00:01").getMillis();
final Bucket expectedBucket = config.getBucket(new MapBasedInputRow(timestamp, dims, values)).get(); final Bucket expectedBucket = config.getBucket(new MapBasedInputRow(timestamp, dims, values)).get();
final long nextBucketTimestamp = Granularities.MINUTE.bucketEnd(new DateTime(timestamp)).getMillis(); final long nextBucketTimestamp = Granularities.MINUTE.bucketEnd(DateTimes.utc(timestamp)).getMillis();
// check that all rows having same set of dims and truncated timestamp hash to same bucket // check that all rows having same set of dims and truncated timestamp hash to same bucket
for (int i = 0; timestamp + i < nextBucketTimestamp; i++) { for (int i = 0; timestamp + i < nextBucketTimestamp; i++) {
Assert.assertEquals( Assert.assertEquals(
@ -134,7 +134,7 @@ public class HadoopDruidIndexerConfigTest
new UniformGranularitySpec( new UniformGranularitySpec(
Granularities.MINUTE, Granularities.MINUTE,
Granularities.MINUTE, Granularities.MINUTE,
ImmutableList.of(new Interval("2010-01-01/P1D")) ImmutableList.of(Intervals.of("2010-01-01/P1D"))
), ),
jsonMapper jsonMapper
), ),
@ -143,12 +143,12 @@ public class HadoopDruidIndexerConfigTest
null, null,
null, null,
null, null,
ImmutableMap.<Long, List<HadoopyShardSpec>>of(new DateTime("2010-01-01T01:00:00").getMillis(), ImmutableMap.<Long, List<HadoopyShardSpec>>of(DateTimes.of("2010-01-01T01:00:00").getMillis(),
Lists.newArrayList(new HadoopyShardSpec( Lists.newArrayList(new HadoopyShardSpec(
NoneShardSpec.instance(), NoneShardSpec.instance(),
1 1
)), )),
new DateTime("2010-01-01T02:00:00").getMillis(), DateTimes.of("2010-01-01T02:00:00").getMillis(),
Lists.newArrayList(new HadoopyShardSpec( Lists.newArrayList(new HadoopyShardSpec(
NoneShardSpec.instance(), NoneShardSpec.instance(),
2 2
@ -183,10 +183,10 @@ public class HadoopDruidIndexerConfigTest
"dim2", "dim2",
"4" "4"
); );
final long ts1 = new DateTime("2010-01-01T01:00:01").getMillis(); final long ts1 = DateTimes.of("2010-01-01T01:00:01").getMillis();
Assert.assertEquals(config.getBucket(new MapBasedInputRow(ts1, dims, values)).get().getShardNum(), 1); Assert.assertEquals(config.getBucket(new MapBasedInputRow(ts1, dims, values)).get().getShardNum(), 1);
final long ts2 = new DateTime("2010-01-01T02:00:01").getMillis(); final long ts2 = DateTimes.of("2010-01-01T02:00:01").getMillis();
Assert.assertEquals(config.getBucket(new MapBasedInputRow(ts2, dims, values)).get().getShardNum(), 2); Assert.assertEquals(config.getBucket(new MapBasedInputRow(ts2, dims, values)).get().getShardNum(), 2);
} }

View File

@ -30,12 +30,12 @@ import io.druid.indexer.partitions.PartitionsSpec;
import io.druid.indexer.partitions.SingleDimensionPartitionsSpec; import io.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import io.druid.indexer.updater.MetadataStorageUpdaterJobSpec; import io.druid.indexer.updater.MetadataStorageUpdaterJobSpec;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.granularity.PeriodGranularity; import io.druid.java.util.common.granularity.PeriodGranularity;
import io.druid.metadata.MetadataStorageConnectorConfig; import io.druid.metadata.MetadataStorageConnectorConfig;
import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.joda.time.DateTimeZone; import org.joda.time.DateTimeZone;
import org.joda.time.Interval;
import org.joda.time.Period; import org.joda.time.Period;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -78,7 +78,7 @@ public class HadoopIngestionSpecTest
Assert.assertEquals( Assert.assertEquals(
"getIntervals", "getIntervals",
Lists.newArrayList(new Interval("2012-01-01/P1D")), Lists.newArrayList(Intervals.of("2012-01-01/P1D")),
granularitySpec.getIntervals().get() granularitySpec.getIntervals().get()
); );

View File

@ -32,6 +32,7 @@ import io.druid.indexer.path.PathSpec;
import io.druid.indexer.path.StaticPathSpec; import io.druid.indexer.path.StaticPathSpec;
import io.druid.indexer.path.UsedSegmentLister; import io.druid.indexer.path.UsedSegmentLister;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.DataSchema;
@ -51,8 +52,8 @@ import java.util.Map;
public class HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest public class HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest
{ {
private final String testDatasource = "test"; private final String testDatasource = "test";
private final Interval testDatasourceInterval = new Interval("1970/3000"); private final Interval testDatasourceInterval = Intervals.of("1970/3000");
private final Interval testDatasourceIntervalPartial = new Interval("2050/3000"); private final Interval testDatasourceIntervalPartial = Intervals.of("2050/3000");
private final ObjectMapper jsonMapper; private final ObjectMapper jsonMapper;
public HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest() public HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest()
@ -65,7 +66,7 @@ public class HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest
private static final DataSegment SEGMENT = new DataSegment( private static final DataSegment SEGMENT = new DataSegment(
"test1", "test1",
Interval.parse("2000/3000"), Intervals.of("2000/3000"),
"ver", "ver",
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(
"type", "local", "type", "local",
@ -233,9 +234,7 @@ public class HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest
new UniformGranularitySpec( new UniformGranularitySpec(
Granularities.DAY, Granularities.DAY,
null, null,
ImmutableList.of( ImmutableList.of(Intervals.of("2010-01-01/P1D"))
new Interval("2010-01-01/P1D")
)
), ),
jsonMapper jsonMapper
), ),

View File

@ -22,13 +22,15 @@ package io.druid.indexer;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import io.druid.java.util.common.StringUtils;
import io.druid.data.input.InputRow; import io.druid.data.input.InputRow;
import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.MapBasedInputRow;
import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.StringInputRowParser;
import io.druid.data.input.impl.TimeAndDimsParseSpec; import io.druid.data.input.impl.TimeAndDimsParseSpec;
import io.druid.data.input.impl.TimestampSpec; import io.druid.data.input.impl.TimestampSpec;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory;
@ -40,8 +42,6 @@ import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.mapreduce.Reducer; import org.apache.hadoop.mapreduce.Reducer;
import org.easymock.Capture; import org.easymock.Capture;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -80,7 +80,7 @@ public class IndexGeneratorCombinerTest
new HyperUniquesAggregatorFactory("unique_hosts", "host") new HyperUniquesAggregatorFactory("unique_hosts", "host")
}, },
new UniformGranularitySpec( new UniformGranularitySpec(
Granularities.DAY, Granularities.NONE, ImmutableList.of(Interval.parse("2010/2011")) Granularities.DAY, Granularities.NONE, ImmutableList.of(Intervals.of("2010/2011"))
), ),
HadoopDruidIndexerConfig.JSON_MAPPER HadoopDruidIndexerConfig.JSON_MAPPER
), ),
@ -136,7 +136,7 @@ public class IndexGeneratorCombinerTest
{ {
long timestamp = System.currentTimeMillis(); long timestamp = System.currentTimeMillis();
Bucket bucket = new Bucket(0, new DateTime(timestamp), 0); Bucket bucket = new Bucket(0, DateTimes.utc(timestamp), 0);
SortableBytes keySortableBytes = new SortableBytes( SortableBytes keySortableBytes = new SortableBytes(
bucket.toGroupKey(), bucket.toGroupKey(),
new byte[0] new byte[0]
@ -195,7 +195,7 @@ public class IndexGeneratorCombinerTest
{ {
long timestamp = System.currentTimeMillis(); long timestamp = System.currentTimeMillis();
Bucket bucket = new Bucket(0, new DateTime(timestamp), 0); Bucket bucket = new Bucket(0, DateTimes.utc(timestamp), 0);
SortableBytes keySortableBytes = new SortableBytes( SortableBytes keySortableBytes = new SortableBytes(
bucket.toGroupKey(), bucket.toGroupKey(),
new byte[0] new byte[0]

View File

@ -31,6 +31,7 @@ import io.druid.data.input.impl.InputRowParser;
import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.JSONParseSpec;
import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.StringInputRowParser;
import io.druid.data.input.impl.TimestampSpec; import io.druid.data.input.impl.TimestampSpec;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.RE; import io.druid.java.util.common.RE;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
@ -423,7 +424,7 @@ public class IndexGeneratorJobTest
this.useCombiner = useCombiner; this.useCombiner = useCombiner;
this.partitionType = partitionType; this.partitionType = partitionType;
this.shardInfoForEachSegment = shardInfoForEachSegment; this.shardInfoForEachSegment = shardInfoForEachSegment;
this.interval = new Interval(interval); this.interval = Intervals.of(interval);
this.data = data; this.data = data;
this.inputFormatName = inputFormatName; this.inputFormatName = inputFormatName;
this.inputRowParser = inputRowParser; this.inputRowParser = inputRowParser;

View File

@ -25,6 +25,7 @@ import io.druid.data.input.impl.CSVParseSpec;
import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.StringInputRowParser;
import io.druid.data.input.impl.TimestampSpec; import io.druid.data.input.impl.TimestampSpec;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory;
@ -60,7 +61,7 @@ public class JobHelperTest
private HadoopDruidIndexerConfig config; private HadoopDruidIndexerConfig config;
private File tmpDir; private File tmpDir;
private File dataFile; private File dataFile;
private Interval interval = new Interval("2014-10-22T00:00:00Z/P1D"); private Interval interval = Intervals.of("2014-10-22T00:00:00Z/P1D");
@Before @Before
public void setup() throws Exception public void setup() throws Exception
@ -155,7 +156,7 @@ public class JobHelperTest
{ {
DataSegment segment = new DataSegment( DataSegment segment = new DataSegment(
"test1", "test1",
Interval.parse("2000/3000"), Intervals.of("2000/3000"),
"ver", "ver",
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(
"type", "google", "type", "google",

View File

@ -22,6 +22,7 @@ package io.druid.indexer.hadoop;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import io.druid.java.util.common.Intervals;
import io.druid.query.filter.SelectorDimFilter; import io.druid.query.filter.SelectorDimFilter;
import io.druid.segment.TestHelper; import io.druid.segment.TestHelper;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
@ -40,7 +41,7 @@ public class DatasourceIngestionSpecTest
@Test @Test
public void testSingleIntervalSerde() throws Exception public void testSingleIntervalSerde() throws Exception
{ {
Interval interval = Interval.parse("2014/2015"); Interval interval = Intervals.of("2014/2015");
DatasourceIngestionSpec expected = new DatasourceIngestionSpec( DatasourceIngestionSpec expected = new DatasourceIngestionSpec(
"test", "test",
@ -74,7 +75,7 @@ public class DatasourceIngestionSpecTest
DatasourceIngestionSpec.class DatasourceIngestionSpec.class
); );
List<Interval> intervals = ImmutableList.of(Interval.parse("2014/2015"), Interval.parse("2016/2017")); List<Interval> intervals = ImmutableList.of(Intervals.of("2014/2015"), Intervals.of("2016/2017"));
DatasourceIngestionSpec expected = new DatasourceIngestionSpec( DatasourceIngestionSpec expected = new DatasourceIngestionSpec(
"test", "test",
@ -119,7 +120,7 @@ public class DatasourceIngestionSpecTest
ImmutableList.of( ImmutableList.of(
new DataSegment( new DataSegment(
"test", "test",
Interval.parse("2014/2017"), Intervals.of("2014/2017"),
"v0", "v0",
null, null,
null, null,
@ -152,7 +153,7 @@ public class DatasourceIngestionSpecTest
DatasourceIngestionSpec actual = MAPPER.readValue(jsonStr, DatasourceIngestionSpec.class); DatasourceIngestionSpec actual = MAPPER.readValue(jsonStr, DatasourceIngestionSpec.class);
Assert.assertEquals( Assert.assertEquals(
new DatasourceIngestionSpec("test", Interval.parse("2014/2015"), null, null, null, null, null, false), new DatasourceIngestionSpec("test", Intervals.of("2014/2015"), null, null, null, null, null, false),
actual actual
); );
} }

View File

@ -29,6 +29,7 @@ import com.google.common.collect.Sets;
import com.google.common.io.Files; import com.google.common.io.Files;
import io.druid.indexer.JobHelper; import io.druid.indexer.JobHelper;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.BlockLocation;
@ -42,7 +43,6 @@ import org.apache.hadoop.mapred.TextInputFormat;
import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.JobContext;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Rule; import org.junit.Rule;
@ -74,7 +74,7 @@ public class DatasourceInputFormatTest
WindowedDataSegment.of( WindowedDataSegment.of(
new DataSegment( new DataSegment(
"test1", "test1",
Interval.parse("2000/3000"), Intervals.of("2000/3000"),
"ver", "ver",
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(
"type", "local", "type", "local",
@ -90,7 +90,7 @@ public class DatasourceInputFormatTest
WindowedDataSegment.of( WindowedDataSegment.of(
new DataSegment( new DataSegment(
"test2", "test2",
Interval.parse("2050/3000"), Intervals.of("2050/3000"),
"ver", "ver",
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(
"type", "hdfs", "type", "hdfs",
@ -106,7 +106,7 @@ public class DatasourceInputFormatTest
WindowedDataSegment.of( WindowedDataSegment.of(
new DataSegment( new DataSegment(
"test3", "test3",
Interval.parse("2030/3000"), Intervals.of("2030/3000"),
"ver", "ver",
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(
"type", "hdfs", "type", "hdfs",
@ -287,7 +287,7 @@ public class DatasourceInputFormatTest
WindowedDataSegment.of( WindowedDataSegment.of(
new DataSegment( new DataSegment(
"test1", "test1",
Interval.parse("2000/3000"), Intervals.of("2000/3000"),
"ver", "ver",
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(
"type", "local", "type", "local",

View File

@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.io.ByteArrayDataOutput; import com.google.common.io.ByteArrayDataOutput;
import com.google.common.io.ByteStreams; import com.google.common.io.ByteStreams;
import io.druid.java.util.common.Intervals;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -39,13 +40,13 @@ public class DatasourceInputSplitTest
@Test @Test
public void testSerde() throws Exception public void testSerde() throws Exception
{ {
Interval interval = Interval.parse("2000/3000"); Interval interval = Intervals.of("2000/3000");
DatasourceInputSplit expected = new DatasourceInputSplit( DatasourceInputSplit expected = new DatasourceInputSplit(
Lists.newArrayList( Lists.newArrayList(
new WindowedDataSegment( new WindowedDataSegment(
new DataSegment( new DataSegment(
"test", "test",
Interval.parse("2000/3000"), Intervals.of("2000/3000"),
"ver", "ver",
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(
"type", "local", "type", "local",

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -36,7 +37,7 @@ public class WindowedDataSegmentTest
private static final ObjectMapper MAPPER = new DefaultObjectMapper(); private static final ObjectMapper MAPPER = new DefaultObjectMapper();
private static final DataSegment SEGMENT = new DataSegment( private static final DataSegment SEGMENT = new DataSegment(
"test1", "test1",
Interval.parse("2000/3000"), Intervals.of("2000/3000"),
"ver", "ver",
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(
"type", "local", "type", "local",
@ -65,7 +66,7 @@ public class WindowedDataSegmentTest
@Test @Test
public void testSerdePartialWindow() throws IOException public void testSerdePartialWindow() throws IOException
{ {
final Interval partialInterval = new Interval("2500/3000"); final Interval partialInterval = Intervals.of("2500/3000");
final WindowedDataSegment windowedDataSegment = new WindowedDataSegment(SEGMENT, partialInterval); final WindowedDataSegment windowedDataSegment = new WindowedDataSegment(SEGMENT, partialInterval);
final WindowedDataSegment roundTrip = MAPPER.readValue( final WindowedDataSegment roundTrip = MAPPER.readValue(
MAPPER.writeValueAsBytes(windowedDataSegment), MAPPER.writeValueAsBytes(windowedDataSegment),

View File

@ -44,6 +44,7 @@ import io.druid.indexer.hadoop.WindowedDataSegment;
import io.druid.initialization.Initialization; import io.druid.initialization.Initialization;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.ISE; import io.druid.java.util.common.ISE;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory;
@ -56,7 +57,6 @@ import io.druid.timeline.partition.NoneShardSpec;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Job;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -74,7 +74,7 @@ public class DatasourcePathSpecTest
{ {
this.ingestionSpec = new DatasourceIngestionSpec( this.ingestionSpec = new DatasourceIngestionSpec(
"test", "test",
Interval.parse("2000/3000"), Intervals.of("2000/3000"),
null, null,
null, null,
null, null,
@ -87,7 +87,7 @@ public class DatasourcePathSpecTest
WindowedDataSegment.of( WindowedDataSegment.of(
new DataSegment( new DataSegment(
ingestionSpec.getDataSource(), ingestionSpec.getDataSource(),
Interval.parse("2000/3000"), Intervals.of("2000/3000"),
"ver", "ver",
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(
"type", "local", "type", "local",
@ -103,7 +103,7 @@ public class DatasourcePathSpecTest
WindowedDataSegment.of( WindowedDataSegment.of(
new DataSegment( new DataSegment(
ingestionSpec.getDataSource(), ingestionSpec.getDataSource(),
Interval.parse("2050/3000"), Intervals.of("2050/3000"),
"ver", "ver",
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(
"type", "hdfs", "type", "hdfs",
@ -278,7 +278,7 @@ public class DatasourcePathSpecTest
new LongSumAggregatorFactory("visited_sum", "visited") new LongSumAggregatorFactory("visited_sum", "visited")
}, },
new UniformGranularitySpec( new UniformGranularitySpec(
Granularities.DAY, Granularities.NONE, ImmutableList.of(Interval.parse("2000/3000")) Granularities.DAY, Granularities.NONE, ImmutableList.of(Intervals.of("2000/3000"))
), ),
HadoopDruidIndexerConfig.JSON_MAPPER HadoopDruidIndexerConfig.JSON_MAPPER
), ),

View File

@ -29,6 +29,7 @@ import io.druid.indexer.HadoopIOConfig;
import io.druid.indexer.HadoopIngestionSpec; import io.druid.indexer.HadoopIngestionSpec;
import io.druid.indexer.HadoopTuningConfig; import io.druid.indexer.HadoopTuningConfig;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.granularity.Granularity;
@ -40,7 +41,6 @@ import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.joda.time.DateTimeZone; import org.joda.time.DateTimeZone;
import org.joda.time.Interval;
import org.joda.time.Period; import org.joda.time.Period;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
@ -151,7 +151,7 @@ public class GranularityPathSpecTest
new UniformGranularitySpec( new UniformGranularitySpec(
Granularities.DAY, Granularities.DAY,
Granularities.MINUTE, Granularities.MINUTE,
ImmutableList.of(new Interval("2015-11-06T00:00Z/2015-11-07T00:00Z")) ImmutableList.of(Intervals.of("2015-11-06T00:00Z/2015-11-07T00:00Z"))
), ),
jsonMapper jsonMapper
), ),
@ -202,7 +202,7 @@ public class GranularityPathSpecTest
new UniformGranularitySpec( new UniformGranularitySpec(
Granularities.DAY, Granularities.DAY,
Granularities.ALL, Granularities.ALL,
ImmutableList.of(new Interval("2015-01-01T11Z/2015-01-02T05Z")) ImmutableList.of(Intervals.of("2015-01-01T11Z/2015-01-02T05Z"))
), ),
jsonMapper jsonMapper
), ),

View File

@ -42,6 +42,7 @@ import io.druid.indexer.JobHelper;
import io.druid.indexer.Jobby; import io.druid.indexer.Jobby;
import io.druid.indexer.SQLMetadataStorageUpdaterJobHandler; import io.druid.indexer.SQLMetadataStorageUpdaterJobHandler;
import io.druid.java.util.common.FileUtils; import io.druid.java.util.common.FileUtils;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.metadata.MetadataSegmentManagerConfig; import io.druid.metadata.MetadataSegmentManagerConfig;
import io.druid.metadata.MetadataStorageConnectorConfig; import io.druid.metadata.MetadataStorageConnectorConfig;
@ -103,7 +104,7 @@ public class HadoopConverterJobTest
private Supplier<MetadataStorageTablesConfig> metadataStorageTablesConfigSupplier; private Supplier<MetadataStorageTablesConfig> metadataStorageTablesConfigSupplier;
private DerbyConnector connector; private DerbyConnector connector;
private final Interval interval = Interval.parse("2011-01-01T00:00:00.000Z/2011-05-01T00:00:00.000Z"); private final Interval interval = Intervals.of("2011-01-01T00:00:00.000Z/2011-05-01T00:00:00.000Z");
@After @After
public void tearDown() public void tearDown()

View File

@ -23,9 +23,9 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.segment.IndexSpec; import io.druid.segment.IndexSpec;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Rule; import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
@ -44,7 +44,7 @@ public class HadoopDruidConverterConfigTest
{ {
final HadoopDruidConverterConfig config = new HadoopDruidConverterConfig( final HadoopDruidConverterConfig config = new HadoopDruidConverterConfig(
"datasource", "datasource",
Interval.parse("2000/2010"), Intervals.of("2000/2010"),
new IndexSpec(), new IndexSpec(),
ImmutableList.<DataSegment>of(), ImmutableList.<DataSegment>of(),
true, true,

Some files were not shown because too many files have changed in this diff Show More