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

View File

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

View File

@ -91,7 +91,7 @@ public class DataSegmentUtils
return new SegmentIdentifierParts(
dataSource,
new Interval(start.getMillis(), end.getMillis()),
new Interval(start, end),
version,
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.module.SimpleModule;
import com.fasterxml.jackson.databind.ser.std.ToStringSerializer;
import io.druid.java.util.common.Intervals;
import org.joda.time.Interval;
import java.io.IOException;
@ -61,7 +62,7 @@ public class TestObjectMapper extends ObjectMapper
JsonParser jsonParser, DeserializationContext deserializationContext
) throws IOException, JsonProcessingException
{
return new Interval(jsonParser.getText());
return Intervals.of(jsonParser.getText());
}
}
);

View File

@ -19,19 +19,18 @@
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.Test;
import com.google.common.collect.ImmutableMap;
public class MapBasedRowTest
{
@Test
public void testGetLongMetricFromString()
{
MapBasedRow row = new MapBasedRow(
new DateTime(),
DateTimes.nowUtc(),
ImmutableMap.<String, Object>builder()
.put("k0", "-1.2")
.put("k1", "1.23")

View File

@ -27,8 +27,8 @@ import com.google.common.collect.Lists;
import io.druid.TestObjectMapper;
import io.druid.data.input.ByteBufferInputRowParser;
import io.druid.data.input.InputRow;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.StringUtils;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Test;
@ -63,7 +63,7 @@ public class InputRowParserSerdeTest
Assert.assertEquals(ImmutableList.of("foo", "bar"), parsed.getDimensions());
Assert.assertEquals(ImmutableList.of("x"), parsed.getDimension("foo"));
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
@ -79,7 +79,7 @@ public class InputRowParserSerdeTest
Assert.assertEquals(ImmutableList.of("foo", "bar"), parsed.getDimensions());
Assert.assertEquals(ImmutableList.of("x"), parsed.getDimension("foo"));
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("456"), parsed.getDimension("metA"));
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\"]}";
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 io.druid.data.input.Firehose;
import io.druid.data.input.Row;
import io.druid.java.util.common.DateTimes;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.filefilter.TrueFileFilter;
import org.hamcrest.CoreMatchers;
import org.joda.time.DateTime;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
@ -124,7 +124,7 @@ public class PrefetchableTextFilesFirehoseFactoryTest
for (int i = 0; i < 10; i++) {
for (int j = 0; j < 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(j), row.getDimension("b").get(0));
}

View File

@ -20,6 +20,7 @@
package io.druid.data.input.impl;
import com.google.common.collect.ImmutableMap;
import io.druid.java.util.common.DateTimes;
import org.joda.time.DateTime;
import org.joda.time.format.ISODateTimeFormat;
import org.junit.Assert;
@ -32,7 +33,7 @@ public class TimestampSpecTest
{
TimestampSpec spec = new TimestampSpec("TIMEstamp", "yyyy-MM-dd", null);
Assert.assertEquals(
new DateTime("2014-03-01"),
DateTimes.of("2014-03-01"),
spec.extractTimestamp(ImmutableMap.<String, Object>of("TIMEstamp", "2014-03-01"))
);
}
@ -40,9 +41,9 @@ public class TimestampSpecTest
@Test
public void testExtractTimestampWithMissingTimestampColumn() throws Exception
{
TimestampSpec spec = new TimestampSpec(null, null, new DateTime(0));
TimestampSpec spec = new TimestampSpec(null, null, DateTimes.EPOCH);
Assert.assertEquals(
new DateTime("1970-01-01"),
DateTimes.of("1970-01-01"),
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 io.druid.TestObjectMapper;
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.PartitionChunk;
import io.druid.timeline.partition.ShardSpec;
import io.druid.timeline.partition.ShardSpecLookup;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
@ -90,7 +91,7 @@ public class DataSegmentTest
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");
DataSegment segment = new DataSegment(
@ -150,8 +151,8 @@ public class DataSegmentTest
{
final DataSegment segment = DataSegment.builder()
.dataSource("foo")
.interval(new Interval("2012-01-01/2012-01-02"))
.version(new DateTime("2012-01-01T11:22:33.444Z").toString())
.interval(Intervals.of("2012-01-01/2012-01-02"))
.version(DateTimes.of("2012-01-01T11:22:33.444Z").toString())
.shardSpec(NoneShardSpec.instance())
.build();
@ -166,8 +167,8 @@ public class DataSegmentTest
{
final DataSegment segment = DataSegment.builder()
.dataSource("foo")
.interval(new Interval("2012-01-01/2012-01-02"))
.version(new DateTime("2012-01-01T11:22:33.444Z").toString())
.interval(Intervals.of("2012-01-01/2012-01-02"))
.version(DateTimes.of("2012-01-01T11:22:33.444Z").toString())
.shardSpec(getShardSpec(0))
.build();
@ -182,8 +183,8 @@ public class DataSegmentTest
{
final DataSegment segment = DataSegment.builder()
.dataSource("foo")
.interval(new Interval("2012-01-01/2012-01-02"))
.version(new DateTime("2012-01-01T11:22:33.444Z").toString())
.interval(Intervals.of("2012-01-01/2012-01-02"))
.version(DateTimes.of("2012-01-01T11:22:33.444Z").toString())
.shardSpec(getShardSpec(7))
.build();
@ -198,8 +199,8 @@ public class DataSegmentTest
{
final DataSegment segment = DataSegment.builder()
.dataSource("foo")
.interval(new Interval("2012-01-01/2012-01-02"))
.version(new DateTime("2012-01-01T11:22:33.444Z").toString())
.interval(Intervals.of("2012-01-01/2012-01-02"))
.version(DateTimes.of("2012-01-01T11:22:33.444Z").toString())
.build();
final DataSegment segment2 = mapper.readValue(mapper.writeValueAsString(segment), DataSegment.class);
@ -240,7 +241,7 @@ public class DataSegmentTest
{
return DataSegment.builder()
.dataSource(dataSource)
.interval(new Interval(interval))
.interval(Intervals.of(interval))
.version(version)
.size(1)
.build();

View File

@ -19,8 +19,8 @@
package io.druid.timeline;
import io.druid.java.util.common.Intervals;
import io.druid.timeline.DataSegmentUtils.SegmentIdentifierParts;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
@ -32,19 +32,19 @@ public class DataSegmentUtilsTest
public void testBasic()
{
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(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(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(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(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
}
@ -53,19 +53,19 @@ public class DataSegmentUtilsTest
public void testDataSourceWithUnderscore1()
{
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(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(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(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(desc, DataSegmentUtils.valueOf(datasource, desc.toString()));
}
@ -74,28 +74,28 @@ public class DataSegmentUtilsTest
public void testDataSourceWithUnderscore2()
{
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(
"datasource_2015-01-01T00:00:00.000Z_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()));
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_2015-01-01T00:00:00.000Z_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()));
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-01T00:00:00.000Z_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00.000Z_ver",
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_2015-01-01T00:00:00.000Z_2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z_ver",
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.BenchmarkSchemaInfo;
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.guava.Sequence;
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.timeline.DataSegment;
import io.druid.timeline.partition.LinearShardSpec;
import org.joda.time.Interval;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@ -89,7 +89,7 @@ public class ExpressionBenchmark
BenchmarkColumnSchema.makeNormal("y", ValueType.FLOAT, false, 1, 0d, 0d, 10000d, false)
),
ImmutableList.of(),
new Interval("2000/P1D"),
Intervals.of("2000/P1D"),
false
);

View File

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

View File

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

View File

@ -25,7 +25,7 @@ import com.google.common.io.Files;
import io.druid.benchmark.datagen.BenchmarkSchemaInfo;
import io.druid.benchmark.datagen.BenchmarkSchemas;
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.java.util.common.granularity.Granularities;
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.partition.LinearShardSpec;
import org.apache.commons.io.FileUtils;
import org.joda.time.Interval;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@ -124,7 +123,7 @@ public class SqlBenchmark
groupByQuery = GroupByQuery
.builder()
.setDataSource("foo")
.setInterval(new Interval(JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT))
.setInterval(Intervals.ETERNITY)
.setDimensions(
Arrays.<DimensionSpec>asList(
new DefaultDimensionSpec("dimZipf", "d0"),

View File

@ -30,6 +30,7 @@ import io.druid.concurrent.Execs;
import io.druid.data.input.InputRow;
import io.druid.hll.HyperLogLogHash;
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.guava.Sequence;
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.serde.ComplexMetrics;
import org.apache.commons.io.FileUtils;
import org.joda.time.Interval;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@ -214,7 +214,7 @@ public class TimeseriesBenchmark
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<>();
LongSumAggregatorFactory lsaf = new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential");
queryAggs.add(lsaf);

View File

@ -19,6 +19,7 @@
package io.druid.server.coordinator;
import io.druid.java.util.common.DateTimes;
import io.druid.timeline.DataSegment;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -39,7 +40,7 @@ import java.util.concurrent.TimeUnit;
@State(Scope.Benchmark)
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 DataSegment segment;

View File

@ -22,8 +22,8 @@ package io.druid.benchmark;
import io.druid.benchmark.datagen.BenchmarkColumnSchema;
import io.druid.benchmark.datagen.BenchmarkDataGenerator;
import io.druid.data.input.InputRow;
import io.druid.java.util.common.Intervals;
import io.druid.segment.column.ValueType;
import org.joda.time.Interval;
import org.junit.Test;
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++) {
InputRow row = dataGenerator.nextRow();
//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++) {
InputRow row = dataGenerator2.nextRow();
//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.JsonProperty;
import com.google.common.base.Preconditions;
import io.druid.java.util.common.DateTimes;
import org.joda.time.DateTime;
/**
@ -50,7 +51,7 @@ public class AuditEntry
this.key = key;
this.type = type;
this.auditInfo = authorInfo;
this.auditTime = auditTime == null ? DateTime.now() : auditTime;
this.auditTime = auditTime == null ? DateTimes.nowUtc() : auditTime;
this.payload = payload;
}
@ -151,7 +152,7 @@ public class AuditEntry
this.key = null;
this.auditInfo = null;
this.payload = null;
this.auditTime = DateTime.now();
this.auditTime = DateTimes.nowUtc();
}
public Builder key(String key)

View File

@ -30,7 +30,9 @@ import org.joda.time.format.ISODateTimeFormat;
import java.util.List;
/**
* Do NOT remove "unused" members in this class. They are used by generated Antlr
*/
@SuppressWarnings("unused")
interface Function
{
String name();
@ -1024,7 +1026,7 @@ interface Function
}
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();
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.Ordering;
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.guava.Comparators;
import io.druid.timeline.partition.ImmutablePartitionHolder;
@ -288,7 +288,7 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
}
Interval lower = completePartitionsTimeline.floorKey(
new Interval(interval.getStartMillis(), JodaUtils.MAX_INSTANT)
new Interval(interval.getStart(), DateTimes.MAX)
);
if (lower == null || !lower.overlaps(interval)) {

View File

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

View File

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

View File

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

View File

@ -25,8 +25,8 @@ import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Ordering;
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.Intervals;
import io.druid.java.util.common.Pair;
import io.druid.timeline.partition.ImmutablePartitionHolder;
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-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(
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(
Arrays.asList(
@ -104,7 +104,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-04-03/2011-04-06", "1", 3),
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(
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(
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(
Arrays.asList(
@ -125,7 +125,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-04-03/2011-04-06", "1", 3),
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(
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(
Arrays.asList(
@ -142,7 +142,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-04-02/2011-04-03", "1", 2),
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(
@ -150,7 +150,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-04-02T18/2011-04-03", "1", 2),
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(
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
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(
Arrays.asList(
createExpected("2011-05-01/2011-05-03", "2", 7),
createExpected("2011-05-03/2011-05-04", "3", 8),
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(
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(
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(
Arrays.asList(
createExpected("2011-05-01/2011-05-02", "1", 6),
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
public void testInsertInWrongOrder() throws Exception
{
DateTime overallStart = new DateTime().minus(Hours.TWO);
DateTime overallStart = DateTimes.nowUtc().minus(Hours.TWO);
Assert.assertTrue(
"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(
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(
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(
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(
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(
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(
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(
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-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));
assertValues(
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());
add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(10, 20, 1, 61));
assertValues(
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());
@ -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());
}
@ -368,10 +368,10 @@ public class VersionedIntervalTimelineTest
testIncompletePartitionDoesNotOvershadow();
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(
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());
}
@ -384,18 +384,18 @@ public class VersionedIntervalTimelineTest
Collections.singletonList(
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(
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(
Collections.singletonList(
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);
@ -403,18 +403,18 @@ public class VersionedIntervalTimelineTest
Collections.singletonList(
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(
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(
Collections.singletonList(
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(
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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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-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(
Collections.singletonList(
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-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(
Arrays.asList(
@ -1202,7 +1202,7 @@ public class VersionedIntervalTimelineTest
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-10/2011-01-14", "2", 3);
timeline.remove(new Interval("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-02/2011-01-03"), "2", makeSingle(2));
timeline.remove(Intervals.of("2011-01-10/2011-01-14"), "2", makeSingle(3));
assertValues(
Collections.singletonList(
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-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(
Arrays.asList(
@ -1244,7 +1244,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-10/2011-01-15", "2", 2),
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-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);
assertValues(
Collections.singletonList(
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(
"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(
"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-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(
Arrays.asList(
@ -1297,7 +1297,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-05/2011-01-10", "2", 2),
createExpected("2011-01-10/2011-01-15", "3", 3)
),
timeline.lookup(new Interval(new DateTime(0), new DateTime(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);
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
@ -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));
@ -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(
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-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(new Interval("2011-04-01/2011-04-05"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-06"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-07"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-08"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-09"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("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-03"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-05"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-06"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-07"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-08"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-09"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-10"), "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(new Interval("2011-04-05/2011-04-07"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-08"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-09"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-06"), "1"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-07"), "1"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-08"), "1"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-09"), "1"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-06"), "2"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-07"), "2"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-08"), "2"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-09"), "2"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "1"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "1"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "1"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "1"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "2"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "2"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "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(new Interval("2011-04-06/2011-04-08"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-09"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-10"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-30"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-07"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-08"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-09"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-10"), "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(new Interval("2011-04-07/2011-04-09"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-07/2011-04-10"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-07/2011-04-30"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-08"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-09"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-10"), "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.assertFalse(timeline.isOvershadowed(new Interval("2011-04-08/2011-04-10"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-08/2011-04-30"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-08/2011-04-09"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-08/2011-04-10"), "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(new Interval("2011-04-09/2011-04-15"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-09/2011-04-17"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("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-10"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-15"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-17"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-19"), "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(new Interval("2011-04-15/2011-04-17"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-15/2011-04-18"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-15/2011-04-19"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-15/2011-04-20"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-15/2011-04-30"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-16"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-17"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-18"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-19"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-20"), "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(new Interval("2011-04-21/2011-04-22"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-19/2011-04-20"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-21/2011-04-22"), "0"));
}
@Test
@ -1645,77 +1645,77 @@ public class VersionedIntervalTimelineTest
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(new Interval("2011-04-01/2011-04-05"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-06"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-07"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-08"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-09"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-01/2011-04-10"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("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-03"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-05"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-06"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-07"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-08"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-09"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-10"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-11"), "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(new Interval("2011-04-05/2011-04-07"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-08"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-09"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("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-06"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "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(new Interval("2011-04-05/2011-04-07"), "12"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-08"), "12"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-09"), "12"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("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-06"), "12"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "12"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "12"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "12"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "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(new Interval("2011-04-05/2011-04-07"), "13"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-08"), "13"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-09"), "13"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("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-06"), "13"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "13"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "13"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "13"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "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(new Interval("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-12"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-15"), "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(new Interval("2011-04-05/2011-04-18"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-19"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-05/2011-04-20"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("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-17"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-18"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-19"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-20"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-21"), "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(new Interval("2011-04-06/2011-04-08"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-09"), "0"));
Assert.assertTrue(timeline.isOvershadowed(new Interval("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-07"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-08"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-09"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-10"), "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(new Interval("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-12"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-15"), "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(new Interval("2011-04-06/2011-04-18"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-19"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-06/2011-04-20"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("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-17"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-18"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-19"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-20"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-21"), "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(new Interval("2011-04-12/2011-04-16"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-15"), "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(new Interval("2011-04-12/2011-04-18"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-12/2011-04-19"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("2011-04-12/2011-04-20"), "0"));
Assert.assertFalse(timeline.isOvershadowed(new Interval("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-17"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-18"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-19"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-20"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-21"), "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.assertFalse(timeline.isOvershadowed(new Interval("2011-04-21/2011-04-22"), "0"));
Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-21"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-21/2011-04-22"), "0"));
}
private Pair<Interval, Pair<String, PartitionHolder<Integer>>> createExpected(
@ -1738,7 +1738,7 @@ public class VersionedIntervalTimelineTest
)
{
return Pair.of(
new Interval(intervalString),
Intervals.of(intervalString),
Pair.of(version, new PartitionHolder<Integer>(values))
);
}
@ -1750,17 +1750,17 @@ public class VersionedIntervalTimelineTest
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)
{
add(new Interval(interval), version, makeSingle(value));
add(interval, version, makeSingle(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)

View File

@ -22,6 +22,7 @@ package io.druid.emitter.ambari.metrics;
import com.google.common.collect.Maps;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
import junitparams.JUnitParamsRunner;
import junitparams.Parameters;
import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
@ -44,7 +45,7 @@ public class WhiteListBasedDruidToTimelineEventConverterTest
new DefaultObjectMapper()
);
private ServiceMetricEvent event;
private final DateTime createdTime = new DateTime();
private final DateTime createdTime = DateTimes.nowUtc();
private final String hostname = "testHost:8080";
private final String serviceName = "historical";
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.microsoft.azure.storage.StorageException;
import io.druid.java.util.common.Intervals;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import org.easymock.EasyMockSupport;
import org.joda.time.Interval;
import org.junit.Before;
import org.junit.Test;
@ -44,7 +44,7 @@ public class AzureDataSegmentKillerTest extends EasyMockSupport
private static final DataSegment dataSegment = new DataSegment(
"test",
new Interval("2015-04-12/2015-04-13"),
Intervals.of("2015-04-12/2015-04-13"),
"1",
ImmutableMap.<String, Object>of("containerName", containerName, "blobPath", blobPath),
null,

View File

@ -22,11 +22,11 @@ package io.druid.storage.azure;
import com.google.common.collect.ImmutableMap;
import com.microsoft.azure.storage.StorageException;
import io.druid.java.util.common.FileUtils;
import io.druid.java.util.common.Intervals;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import org.easymock.EasyMockSupport;
import org.joda.time.Interval;
import org.junit.Before;
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 DataSegment dataSegment = new DataSegment(
"test",
new Interval("2015-04-12/2015-04-13"),
Intervals.of("2015-04-12/2015-04-13"),
"1",
ImmutableMap.<String, Object>of("containerName", containerName, "blobPath", blobPath),
null,

View File

@ -26,12 +26,12 @@ import com.google.common.collect.Maps;
import com.google.common.io.Files;
import com.microsoft.azure.storage.StorageException;
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.StringUtils;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import org.easymock.EasyMockSupport;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
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 DataSegment dataSegment = new DataSegment(
"test",
new Interval("2015-04-12/2015-04-13"),
Intervals.of("2015-04-12/2015-04-13"),
"1",
ImmutableMap.<String, Object>of("containerName", containerName, "blobPath", blobPath),
null,
@ -94,7 +94,7 @@ public class AzureDataSegmentPusherTest extends EasyMockSupport
DataSegment segmentToPush = new DataSegment(
"foo",
new Interval("2015/2016"),
Intervals.of("2015/2016"),
"0",
Maps.<String, Object>newHashMap(),
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.io.Files;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import org.easymock.EasyMock;
import org.jclouds.io.Payload;
import org.jclouds.openstack.swift.v1.features.ObjectApi;
import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
@ -74,7 +74,7 @@ public class CloudFilesDataSegmentPusherTest
DataSegment segmentToPush = new DataSegment(
"foo",
new Interval("2015/2016"),
Intervals.of("2015/2016"),
"0",
Maps.<String, Object>newHashMap(),
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.Lists;
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.guava.Sequences;
import io.druid.query.Druids;
@ -61,7 +62,7 @@ public class DistinctCountTimeseriesQueryTest
String visitor_id = "visitor_id";
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();
index.add(
new MapBasedInputRow(

View File

@ -22,9 +22,9 @@ package io.druid.query.aggregation.distinctcount;
import com.google.common.base.Supplier;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import io.druid.collections.StupidPool;
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.guava.Sequences;
import io.druid.query.QueryRunnerTestHelper;
@ -79,7 +79,7 @@ public class DistinctCountTopNQueryTest
String visitor_id = "visitor_id";
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();
index.add(
new MapBasedInputRow(

View File

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

View File

@ -21,11 +21,11 @@ package io.druid.storage.google;
import com.google.common.collect.ImmutableMap;
import io.druid.java.util.common.FileUtils;
import io.druid.java.util.common.Intervals;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import org.easymock.EasyMockSupport;
import org.joda.time.Interval;
import org.junit.Test;
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 DataSegment dataSegment = new DataSegment(
"test",
new Interval("2015-04-12/2015-04-13"),
Intervals.of("2015-04-12/2015-04-13"),
"1",
ImmutableMap.<String, Object>of("bucket", bucket, "path", path),
null,

View File

@ -25,11 +25,11 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.io.Files;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import org.easymock.EasyMock;
import org.easymock.EasyMockSupport;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
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 DataSegment dataSegment = new DataSegment(
"test",
new Interval("2015-04-12/2015-04-13"),
Intervals.of("2015-04-12/2015-04-13"),
"1",
ImmutableMap.<String, Object>of("bucket", bucket, "path", path),
null,
@ -87,7 +87,7 @@ public class GoogleDataSegmentPusherTest extends EasyMockSupport
DataSegment segmentToPush = new DataSegment(
"foo",
new Interval("2015/2016"),
Intervals.of("2015/2016"),
"0",
Maps.<String, Object>newHashMap(),
Lists.<String>newArrayList(),

View File

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

View File

@ -18,9 +18,10 @@
*/
package io.druid.data.input.orc;
import io.druid.java.util.common.StringUtils;
import io.druid.data.input.MapBasedInputRow;
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.fs.Path;
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.TypeDescription;
import org.apache.orc.Writer;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
@ -101,7 +101,7 @@ public class DruidOrcInputFormatTest
MapBasedInputRow row = (MapBasedInputRow) parser.parse(data);
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(col1, row.getEvent().get("col1"));
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.initialization.Initialization;
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.serde2.io.HiveDecimalWritable;
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.LongWritable;
import org.apache.hadoop.io.Text;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@ -167,7 +167,7 @@ public class OrcHadoopInputRowParserTest
oi.setStructFieldData(struct, oi.getStructFieldRef("col6"), null);
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("col2", ImmutableList.of("foo", "bar"), row.getRaw("col2"));
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.Maps;
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.InputRowParser;
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.JobHelper;
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.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
@ -117,7 +118,7 @@ public class OrcIndexGeneratorJobTest
"2014102212,i.example.com,963",
"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 outputRoot;
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.ParseSpec;
import io.druid.data.input.impl.TimestampSpec;
import io.druid.java.util.common.DateTimes;
import org.apache.avro.LogicalType;
import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
@ -92,7 +93,7 @@ public class ParquetHadoopInputRowParser implements InputRowParser<GenericRecord
if (logicalType instanceof LogicalTypes.Date) {
int daysSinceEpoch = (Integer) genericRecordAsMap.get(timestampSpec.getTimestampColumn());
dateTime = new DateTime(TimeUnit.DAYS.toMillis(daysSinceEpoch));
dateTime = DateTimes.utc(TimeUnit.DAYS.toMillis(daysSinceEpoch));
} else {
// Fall back to a binary format that will be parsed using joda-time
dateTime = timestampSpec.extractTimestamp(genericRecordAsMap);

View File

@ -38,6 +38,7 @@ import java.util.Date;
import java.util.List;
import java.util.Locale;
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())));
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")));
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.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.guava.Sequence;
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.io.CharSource;
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.guava.MergeSequence;
import io.druid.java.util.common.guava.Sequence;
@ -43,7 +44,6 @@ import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang.StringUtils;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.AfterClass;
import org.junit.Assert;
@ -145,7 +145,7 @@ public class MultiSegmentScanQueryTest
private static IncrementalIndex newIndex(String minTimeStamp, int maxRowCount)
{
final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
.withMinTimestamp(new DateTime(minTimeStamp).getMillis())
.withMinTimestamp(DateTimes.of(minTimeStamp).getMillis())
.withQueryGranularity(Granularities.HOUR)
.withMetrics(TestIndex.METRIC_AGGS)
.build();

View File

@ -25,7 +25,9 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.ObjectArrays;
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.Intervals;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.DefaultGenericQueryMetricsFactory;
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.spec.LegacySegmentSpec;
import io.druid.query.spec.QuerySegmentSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -94,7 +94,7 @@ public class ScanQueryRunnerTest
};
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);
@ -506,7 +506,7 @@ public class ScanQueryRunnerTest
event.put(
specs[0],
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("DOUBLE") ? Double.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 io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.TableDataSource;
import io.druid.query.spec.LegacySegmentSpec;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
@ -58,7 +58,7 @@ public class ScanQuerySpecTest
ScanQuery query = new ScanQuery(
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,
0,
3,

View File

@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableList;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.emitter.statsd.DimensionConverter;
import io.druid.emitter.statsd.StatsDMetric;
import org.joda.time.DateTime;
import io.druid.java.util.common.DateTimes;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
@ -46,7 +46,7 @@ public class DimensionConverterTest
.setDimension("remoteAddress", "194.0.90.2")
.setDimension("id", "ID")
.setDimension("context", "{context}")
.build(new DateTime(), "query/time", 10)
.build(DateTimes.nowUtc(), "query/time", 10)
.build("broker", "brokerHost1");
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 io.druid.emitter.statsd.StatsDEmitter;
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.replay;
import static org.easymock.EasyMock.verify;
import org.joda.time.DateTime;
import org.junit.Test;
/**
*/
public class StatsDEmitterTest
@ -47,7 +46,7 @@ public class StatsDEmitterTest
replay(client);
emitter.emit(new ServiceMetricEvent.Builder()
.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")
);
verify(client);
@ -75,7 +74,7 @@ public class StatsDEmitterTest
.setDimension("remoteAddress", "194.0.90.2")
.setDimension("id", "ID")
.setDimension("context", "{context}")
.build(new DateTime(), "query/time", 10)
.build(DateTimes.nowUtc(), "query/time", 10)
.build("broker", "brokerHost1")
);
verify(client);
@ -103,7 +102,7 @@ public class StatsDEmitterTest
.setDimension("remoteAddress", "194.0.90.2")
.setDimension("id", "ID")
.setDimension("context", "{context}")
.build(new DateTime(), "query/time", 10)
.build(DateTimes.nowUtc(), "query/time", 10)
.build("broker", "brokerHost1")
);
verify(client);

View File

@ -21,8 +21,9 @@ package io.druid.query.aggregation;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.primitives.Longs;
import io.druid.java.util.common.StringUtils;
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 org.joda.time.DateTime;
@ -116,7 +117,7 @@ public class TimestampAggregatorFactory extends AggregatorFactory
@Override
public Object finalizeComputation(Object object)
{
return new DateTime((long) object);
return DateTimes.utc((long) object);
}
@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.TimestampSpec;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.Druids;
import io.druid.query.QueryPlus;
@ -46,7 +47,6 @@ import io.druid.query.select.SelectQueryRunnerFactory;
import io.druid.query.select.SelectResultValue;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexSchema;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -85,7 +85,7 @@ public class MapVirtualColumnTest
);
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();
final IncrementalIndex index = new IncrementalIndex.Builder()
.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.EncoderFactory;
import org.joda.time.DateTime;
import org.joda.time.chrono.ISOChronology;
import org.junit.Before;
import org.junit.Test;
import org.schemarepo.InMemoryRepository;
@ -75,7 +76,7 @@ public class AvroStreamInputRowParserTest
public static final float SOME_FLOAT_VALUE = 0.23555f;
public static final int SOME_INT_VALUE = 1;
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 TimeAndDimsParseSpec PARSE_SPEC = new TimeAndDimsParseSpec(
new TimestampSpec("timestamp", "millis", null),

View File

@ -26,6 +26,7 @@ import com.google.common.collect.Lists;
import com.google.common.io.Files;
import io.druid.data.input.MapBasedRow;
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.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
@ -221,7 +222,7 @@ public class SketchAggregationWithSimpleDataTest
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("sketchEstimatePostAgg"), 0.01);
@ -249,7 +250,7 @@ public class SketchAggregationWithSimpleDataTest
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());
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()));
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("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 io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.IOE;
import io.druid.java.util.common.Intervals;
import io.druid.storage.hdfs.HdfsDataSegmentFinder;
import io.druid.timeline.DataSegment;
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.Path;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.joda.time.Interval;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
@ -56,13 +56,10 @@ public class HdfsDataSegmentFinderTest
private static final ObjectMapper mapper = new DefaultObjectMapper();
private static final String DESCRIPTOR_JSON = "descriptor.json";
private static final String INDEX_ZIP = "index.zip";
private static final DataSegment SEGMENT_1 = DataSegment.builder()
private static final DataSegment SEGMENT_1 = DataSegment
.builder()
.dataSource("wikipedia")
.interval(
new Interval(
"2013-08-31T00:00:00.000Z/2013-09-01T00:00:00.000Z"
)
)
.interval(Intervals.of("2013-08-31T00:00:00.000Z/2013-09-01T00:00:00.000Z"))
.version("2015-10-21T22:07:57.074Z")
.loadSpec(
ImmutableMap.<String, Object>of(
@ -76,48 +73,33 @@ public class HdfsDataSegmentFinderTest
.metrics(ImmutableList.of("count"))
.build();
private static final DataSegment SEGMENT_2 = DataSegment.builder(SEGMENT_1)
.interval(
new Interval(
"2013-09-01T00:00:00.000Z/2013-09-02T00:00:00.000Z"
)
)
private static final DataSegment SEGMENT_2 = DataSegment
.builder(SEGMENT_1)
.interval(Intervals.of("2013-09-01T00:00:00.000Z/2013-09-02T00:00:00.000Z"))
.build();
private static final DataSegment SEGMENT_3 = DataSegment.builder(SEGMENT_1)
.interval(
new Interval(
"2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"
)
)
private static final DataSegment SEGMENT_3 = DataSegment
.builder(SEGMENT_1)
.interval(Intervals.of("2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"))
.version("2015-10-22T22:07:57.074Z")
.build();
private static final DataSegment SEGMENT_4_0 = DataSegment.builder(SEGMENT_1)
.interval(
new Interval(
"2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"
)
)
private static final DataSegment SEGMENT_4_0 = DataSegment
.builder(SEGMENT_1)
.interval(Intervals.of("2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"))
.shardSpec(new NumberedShardSpec(0, 2))
.build();
private static final DataSegment SEGMENT_4_1 = DataSegment.builder(SEGMENT_1)
.interval(
new Interval(
"2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"
)
)
private static final DataSegment SEGMENT_4_1 = DataSegment
.builder(SEGMENT_1)
.interval(Intervals.of("2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"))
.shardSpec(new NumberedShardSpec(1, 2))
.build();
private static final DataSegment SEGMENT_5 = DataSegment.builder()
private static final DataSegment SEGMENT_5 = DataSegment
.builder()
.dataSource("wikipedia")
.interval(
new Interval(
"2013-09-03T00:00:00.000Z/2013-09-04T00:00:00.000Z"
)
)
.interval(Intervals.of("2013-09-03T00:00:00.000Z/2013-09-04T00:00:00.000Z"))
.version("2015-10-21T22:07:57.074Z")
.loadSpec(
ImmutableMap.<String, Object>of(

View File

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

View File

@ -20,7 +20,6 @@
package io.druid.storage.hdfs;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.InjectableValues;
@ -42,6 +41,7 @@ import io.druid.indexer.HadoopIngestionSpec;
import io.druid.indexer.JobHelper;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.jackson.GranularityModule;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.StringUtils;
import io.druid.segment.loading.LocalDataSegmentPusher;
import io.druid.segment.loading.LocalDataSegmentPusherConfig;
@ -57,6 +57,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.TaskType;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.joda.time.chrono.ISOChronology;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
@ -147,7 +148,7 @@ public class HdfsDataSegmentPusherTest
DataSegment segmentToPush = new DataSegment(
"foo",
new Interval("2015/2016"),
Intervals.of("2015/2016"),
"0",
Maps.<String, Object>newHashMap(),
Lists.<String>newArrayList(),
@ -230,7 +231,7 @@ public class HdfsDataSegmentPusherTest
for (int i = 0; i < numberOfSegments; i++) {
segments[i] = new DataSegment(
"foo",
new Interval("2015/2016"),
Intervals.of("2015/2016"),
"0",
Maps.<String, Object>newHashMap(),
Lists.<String>newArrayList(),
@ -337,11 +338,10 @@ public class HdfsDataSegmentPusherTest
Interval.class, new StdDeserializer<Interval>(Interval.class)
{
@Override
public Interval deserialize(
JsonParser jsonParser, DeserializationContext deserializationContext
) throws IOException, JsonProcessingException
public Interval deserialize(JsonParser jsonParser, DeserializationContext deserializationContext)
throws IOException
{
return new Interval(jsonParser.getText());
return Intervals.of(jsonParser.getText());
}
}
);
@ -353,7 +353,7 @@ public class HdfsDataSegmentPusherTest
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");
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(
new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()),
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(
new Path(cfg.getSchema().getIOConfig().getSegmentOutputPath()),
new LocalFileSystem(),

View File

@ -183,7 +183,7 @@ public class QuantileSqlAggregator implements SqlAggregator
}
} else {
final ExpressionVirtualColumn virtualColumn = input.toVirtualColumn(
String.format("%s:v", name),
StringUtils.format("%s:v", name),
ValueType.FLOAT,
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.Lists;
import io.druid.collections.StupidPool;
import io.druid.java.util.common.DateTimes;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
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.TopNResultValue;
import io.druid.segment.TestHelper;
import org.joda.time.DateTime;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
@ -146,7 +146,7 @@ public class ApproximateHistogramTopNQueryTest
List<Result<TopNResultValue>> expectedResults = Collections.singletonList(
new Result<TopNResultValue>(
new DateTime("2011-01-12T00:00:00.000Z"),
DateTimes.of("2011-01-12T00:00:00.000Z"),
new TopNResultValue(
Arrays.<Map<String, Object>>asList(
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.RealtimeIndexTask;
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.StringUtils;
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
{
log.info("Starting up!");
startTime = DateTime.now();
startTime = DateTimes.nowUtc();
mapper = toolbox.getObjectMapper();
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.supervisor.Supervisor;
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.ISE;
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(
buildRunTask(),
ioConfig.getStartDelay().getMillis(),
@ -666,7 +667,7 @@ public class KafkaSupervisor implements Supervisor
if (taskInfoProvider.getTaskLocation(entry.getKey()).equals(TaskLocation.unknown())) {
killTask(entry.getKey());
} 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());
newTaskGroup.tasks.put(taskId, new TaskData());
newTaskGroup.completionTimeout = DateTime.now().plus(ioConfig.getCompletionTimeout());
newTaskGroup.completionTimeout = DateTimes.nowUtc().plus(ioConfig.getCompletionTimeout());
taskGroupList.add(newTaskGroup);
}
@ -1052,7 +1053,7 @@ public class KafkaSupervisor implements Supervisor
TaskGroup group = entry.getValue();
// find the longest running task from this group
DateTime earliestTaskStart = DateTime.now();
DateTime earliestTaskStart = DateTimes.nowUtc();
for (TaskData taskData : group.tasks.values()) {
if (earliestTaskStart.isAfter(taskData.startTime)) {
earliestTaskStart = taskData.startTime;
@ -1075,7 +1076,7 @@ public class KafkaSupervisor implements Supervisor
if (endOffsets != null) {
// 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.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());
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> maximumMessageTime = (ioConfig.getEarlyMessageRejectionPeriod().isPresent() ? Optional.of(
DateTime.now().plus(ioConfig.getEarlyMessageRejectionPeriod().get())
DateTimes.nowUtc().plus(ioConfig.getEarlyMessageRejectionPeriod().get())
) : Optional.<DateTime>absent());
taskGroups.put(groupId, new TaskGroup(generateStartingOffsetsForPartitionGroup(groupId), minimumMessageTime, maximumMessageTime));
@ -1626,7 +1627,7 @@ public class KafkaSupervisor implements Supervisor
Map<Integer, Long> partitionLag = getLagPerPartition(getHighestCurrentOffsets());
KafkaSupervisorReport report = new KafkaSupervisorReport(
dataSource,
DateTime.now(),
DateTimes.nowUtc(),
ioConfig.getTopic(),
numPartitions,
ioConfig.getReplicas(),
@ -1648,7 +1649,7 @@ public class KafkaSupervisor implements Supervisor
Long remainingSeconds = null;
if (startTime != null) {
remainingSeconds = Math.max(
0, ioConfig.getTaskDuration().getMillis() - (DateTime.now().getMillis() - startTime.getMillis())
0, ioConfig.getTaskDuration().getMillis() - (System.currentTimeMillis() - startTime.getMillis())
) / 1000;
}
@ -1674,7 +1675,7 @@ public class KafkaSupervisor implements Supervisor
Map<Integer, Long> currentOffsets = entry.getValue().currentOffsets;
Long remainingSeconds = 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;
}
@ -1822,7 +1823,7 @@ public class KafkaSupervisor implements Supervisor
try {
updateCurrentOffsets();
updateLatestOffsetsFromKafka();
offsetsLastUpdated = DateTime.now();
offsetsLastUpdated = DateTimes.nowUtc();
}
catch (Exception e) {
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.google.common.collect.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
import io.druid.segment.indexing.IOConfig;
import org.hamcrest.CoreMatchers;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
@ -111,8 +111,8 @@ public class KafkaIOConfigTest
Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties());
Assert.assertEquals(false, config.isUseTransaction());
Assert.assertEquals(true, config.isPauseAfterRead());
Assert.assertEquals(new DateTime("2016-05-31T12:00Z"), config.getMinimumMessageTime().get());
Assert.assertEquals(new DateTime("2016-05-31T14:00Z"), config.getMaximumMessageTime().get());
Assert.assertEquals(DateTimes.of("2016-05-31T12:00Z"), config.getMinimumMessageTime().get());
Assert.assertEquals(DateTimes.of("2016-05-31T14:00Z"), config.getMaximumMessageTime().get());
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.TaskStatus;
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.StringUtils;
import org.easymock.Capture;
@ -346,7 +347,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
public void testGetStartTime() throws Exception
{
client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2);
DateTime now = DateTime.now();
DateTime now = DateTimes.nowUtc();
Capture<Request> captured = Capture.newInstance();
expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3)
@ -789,7 +790,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
@Test
public void testGetStartTimeAsync() throws Exception
{
final DateTime now = DateTime.now();
final DateTime now = DateTimes.nowUtc();
final int numRequests = TEST_IDS.size();
Capture<Request> captured = Capture.newInstance(CaptureType.ALL);
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.jackson.DefaultObjectMapper;
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.Intervals;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularities;
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.ProducerRecord;
import org.easymock.EasyMock;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.joda.time.Period;
import org.junit.After;
@ -420,7 +421,7 @@ public class KafkaIndexTaskTest
kafkaServer.consumerProperties(),
true,
false,
new DateTime("2010"),
DateTimes.of("2010"),
null,
false
),
@ -477,7 +478,7 @@ public class KafkaIndexTaskTest
true,
false,
null,
new DateTime("2010"),
DateTimes.of("2010"),
false
),
null,
@ -1663,7 +1664,7 @@ public class KafkaIndexTaskTest
return FluentIterable.from(
metadataStorageCoordinator.getUsedSegmentsForInterval(
DATA_SCHEMA.getDataSource(),
new Interval("0000/3000")
Intervals.of("0000/3000")
)
).transform(
new Function<DataSegment, SegmentDescriptor>()
@ -1759,7 +1760,7 @@ public class KafkaIndexTaskTest
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);
}
}

View File

@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.Futures;
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.DimensionsSpec;
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.TaskStorage;
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.StringUtils;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory;
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(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED))
.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(
new KafkaDataSourceMetadata(
null
@ -696,7 +697,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
expect(taskStorage.getTask("id5")).andReturn(Optional.of(id3)).anyTimes();
expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED))
.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(
new KafkaDataSourceMetadata(
null
@ -728,7 +729,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.<Task>of()).anyTimes();
expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED))
.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(
new KafkaDataSourceMetadata(
null
@ -790,8 +791,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
supervisor = getSupervisor(2, 1, true, "PT1H", null, null, false);
addSomeEvents(1);
DateTime now = DateTime.now();
DateTime maxi = DateTime.now().plusMinutes(60);
DateTime now = DateTimes.nowUtc();
DateTime maxi = now.plusMinutes(60);
Task id1 = createKafkaIndexTask(
"id1",
DATASOURCE,
@ -876,7 +877,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.<Task>of()).anyTimes();
expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED))
.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(
new KafkaDataSourceMetadata(
null
@ -976,10 +977,10 @@ public class KafkaSupervisorTest extends EasyMockSupport
.andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING))
.anyTimes();
expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0")))
.andReturn(Futures.immediateFuture(DateTime.now().minusMinutes(2)))
.andReturn(Futures.immediateFuture(DateTime.now()));
.andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2)))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()));
expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1")))
.andReturn(Futures.immediateFuture(DateTime.now()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.times(2);
expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0")))
.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 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);
addSomeEvents(6);
@ -1383,10 +1384,10 @@ public class KafkaSupervisorTest extends EasyMockSupport
.andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING))
.anyTimes();
expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0")))
.andReturn(Futures.immediateFuture(DateTime.now().minusMinutes(2)))
.andReturn(Futures.immediateFuture(DateTime.now()));
.andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2)))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()));
expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1")))
.andReturn(Futures.immediateFuture(DateTime.now()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.times(2);
expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0")))
.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))
.anyTimes();
expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0")))
.andReturn(Futures.immediateFuture(DateTime.now().minusMinutes(2)))
.andReturn(Futures.immediateFuture(DateTime.now()));
.andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2)))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()));
expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1")))
.andReturn(Futures.immediateFuture(DateTime.now()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.times(2);
expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0")))
.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 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);
addSomeEvents(1);
@ -1697,7 +1698,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
{
final TaskLocation location1 = new TaskLocation("testHost", 1234, -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);
addSomeEvents(1);

View File

@ -19,7 +19,7 @@
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.StringUtils;
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.java.util.common.parsers.ParseException;
import org.joda.time.DateTime;
import org.joda.time.chrono.ISOChronology;
import org.junit.Before;
import org.junit.Test;
@ -116,7 +117,7 @@ public class ProtobufInputRowParserTest
ProtobufInputRowParser parser = new ProtobufInputRowParser(parseSpec, "prototest.desc", "ProtoTestEvent");
//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()
.setDescription("description")
.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.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment;
import org.easymock.EasyMock;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
@ -73,7 +73,7 @@ public class S3DataSegmentArchiverTest
.binaryVersion(1)
.dataSource("dataSource")
.dimensions(ImmutableList.<String>of())
.interval(Interval.parse("2015/2016"))
.interval(Intervals.of("2015/2016"))
.version("version")
.loadSpec(ImmutableMap.<String, Object>of(
"type",

View File

@ -31,6 +31,7 @@ import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment;
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.model.S3Object;
import org.jets3t.service.model.StorageObject;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
@ -62,13 +62,10 @@ public class S3DataSegmentFinderTest
{
private static final ObjectMapper mapper = new DefaultObjectMapper();
private static final DataSegment SEGMENT_1 = DataSegment.builder()
private static final DataSegment SEGMENT_1 = DataSegment
.builder()
.dataSource("wikipedia")
.interval(
new Interval(
"2013-08-31T00:00:00.000Z/2013-09-01T00:00:00.000Z"
)
)
.interval(Intervals.of("2013-08-31T00:00:00.000Z/2013-09-01T00:00:00.000Z"))
.version("2015-10-21T22:07:57.074Z")
.loadSpec(
ImmutableMap.<String, Object>of(
@ -84,38 +81,26 @@ public class S3DataSegmentFinderTest
.metrics(ImmutableList.of("count"))
.build();
private static final DataSegment SEGMENT_2 = DataSegment.builder(SEGMENT_1)
.interval(
new Interval(
"2013-09-01T00:00:00.000Z/2013-09-02T00:00:00.000Z"
)
)
private static final DataSegment SEGMENT_2 = DataSegment
.builder(SEGMENT_1)
.interval(Intervals.of("2013-09-01T00:00:00.000Z/2013-09-02T00:00:00.000Z"))
.build();
private static final DataSegment SEGMENT_3 = DataSegment.builder(SEGMENT_1)
.interval(
new Interval(
"2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"
)
)
private static final DataSegment SEGMENT_3 = DataSegment
.builder(SEGMENT_1)
.interval(Intervals.of("2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"))
.version("2015-10-22T22:07:57.074Z")
.build();
private static final DataSegment SEGMENT_4_0 = DataSegment.builder(SEGMENT_1)
.interval(
new Interval(
"2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"
)
)
private static final DataSegment SEGMENT_4_0 = DataSegment
.builder(SEGMENT_1)
.interval(Intervals.of("2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"))
.shardSpec(new NumberedShardSpec(0, 2))
.build();
private static final DataSegment SEGMENT_4_1 = DataSegment.builder(SEGMENT_1)
.interval(
new Interval(
"2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"
)
)
private static final DataSegment SEGMENT_4_1 = DataSegment
.builder(SEGMENT_1)
.interval(Intervals.of("2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z"))
.shardSpec(new NumberedShardSpec(1, 2))
.build();

View File

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

View File

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

View File

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

View File

@ -20,6 +20,7 @@
package io.druid.query.aggregation.variance;
import com.google.common.collect.Lists;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.Druids;
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.TimeseriesResultValue;
import io.druid.segment.TestHelper;
import org.joda.time.DateTime;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
@ -79,7 +79,7 @@ public class VarianceTimeseriesQueryTest
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
new Result<>(
new DateTime("2011-04-01"),
DateTimes.of("2011-04-01"),
new TimeseriesResultValue(
VarianceTestHelper.of(
"rows", 13L,
@ -92,7 +92,7 @@ public class VarianceTimeseriesQueryTest
)
),
new Result<>(
new DateTime("2011-04-02"),
DateTimes.of("2011-04-02"),
new TimeseriesResultValue(
VarianceTestHelper.of(
"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.Iterables;
import com.google.common.collect.Lists;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.guava.Sequence;
import io.druid.query.QueryPlus;
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.TopNResultValue;
import io.druid.segment.TestHelper;
import org.joda.time.DateTime;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
@ -92,7 +92,7 @@ public class VarianceTopNQueryTest
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
new Result<TopNResultValue>(
new DateTime("2011-01-12T00:00:00.000Z"),
DateTimes.of("2011-01-12T00:00:00.000Z"),
new TopNResultValue(
Arrays.<Map<String, Object>>asList(
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.Hashing;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import org.apache.commons.codec.binary.Base64;
import org.junit.Assert;
import org.junit.Ignore;
@ -36,13 +37,13 @@ import java.security.MessageDigest;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Locale;
import java.util.Random;
/**
*/
public class HyperLogLogCollectorTest
{
private static final Logger log = new Logger(HyperLogLogCollectorTest.class);
private final HashFunction fn = Hashing.murmur3_128();
@ -118,15 +119,10 @@ public class HyperLogLogCollectorTest
int n = count;
System.out.println("True cardinality " + n);
System.out.println("Rolling buffer cardinality " + rolling.estimateCardinality());
System.out.println("Simple buffer cardinality " + simple.estimateCardinality());
System.out.println(
StringUtils.format(
"Rolling cardinality estimate off by %4.1f%%",
100 * (1 - rolling.estimateCardinality() / n)
)
);
log.info("True cardinality " + n);
log.info("Rolling buffer cardinality " + rolling.estimateCardinality());
log.info("Simple buffer cardinality " + simple.estimateCardinality());
log.info("Rolling cardinality estimate off by %4.1f%%", 100 * (1 - rolling.estimateCardinality() / n));
Assert.assertEquals(n, simple.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());
rolling.fold(theCollector);
}
System.out.printf(
Locale.ENGLISH,
"testHighCardinalityRollingFold2 took %d ms%n",
System.currentTimeMillis() - start
);
log.info("testHighCardinalityRollingFold2 took %d ms", System.currentTimeMillis() - start);
int n = count;
System.out.println("True cardinality " + n);
System.out.println("Rolling buffer cardinality " + rolling.estimateCardinality());
System.out.println(
StringUtils.format(
"Rolling cardinality estimate off by %4.1f%%",
100 * (1 - rolling.estimateCardinality() / n)
)
);
log.info("True cardinality " + n);
log.info("Rolling buffer cardinality " + rolling.estimateCardinality());
log.info("Rolling cardinality estimate off by %4.1f%%", 100 * (1 - rolling.estimateCardinality() / n));
Assert.assertEquals(n, rolling.estimateCardinality(), n * 0.05);
}
@ -843,9 +830,8 @@ public class HyperLogLogCollectorTest
error += errorThisTime;
System.out.printf(
Locale.ENGLISH,
"%,d ==? %,f in %,d millis. actual error[%,f%%], avg. error [%,f%%]%n",
log.info(
"%,d ==? %,f in %,d millis. actual error[%,f%%], avg. error [%,f%%]",
numThings,
estimatedValue,
System.currentTimeMillis() - startTime,

View File

@ -20,9 +20,8 @@
package io.druid.indexer;
import com.google.common.annotations.VisibleForTesting;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.Pair;
import org.joda.time.DateTime;
import java.nio.ByteBuffer;
@ -118,7 +117,7 @@ public class Bucket
{
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()];
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.Rows;
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.StringUtils;
import io.druid.java.util.common.granularity.Granularity;
@ -257,14 +258,14 @@ public class DetermineHashedPartitionsJob implements Jobby
if (determineIntervals) {
interval = config.getGranularitySpec()
.getSegmentGranularity()
.bucket(new DateTime(inputRow.getTimestampFromEpoch()));
.bucket(DateTimes.utc(inputRow.getTimestampFromEpoch()));
if (!hyperLogLogs.containsKey(interval)) {
hyperLogLogs.put(interval, HyperLogLogCollector.makeLatestCollector());
}
} else {
final Optional<Interval> maybeInterval = config.getGranularitySpec()
.bucketInterval(new DateTime(inputRow.getTimestampFromEpoch()));
.bucketInterval(DateTimes.utc(inputRow.getTimestampFromEpoch()));
if (!maybeInterval.isPresent()) {
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()))
);
}
Optional<Interval> intervalOptional = config.getGranularitySpec().bucketInterval(new DateTime(key.get()));
Optional<Interval> intervalOptional = config.getGranularitySpec().bucketInterval(DateTimes.utc(key.get()));
if (!intervalOptional.isPresent()) {
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.Rows;
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.StringUtils;
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.joda.time.DateTime;
import org.joda.time.Interval;
import org.joda.time.chrono.ISOChronology;
import java.io.IOException;
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 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);
helper.emitDimValueCounts(context, timestamp, dims);
@ -359,7 +361,7 @@ public class DeterminePartitionsJob implements Jobby
for (final String dim : inputRow.getDimensions()) {
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());
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());
log.info(

View File

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

View File

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

View File

@ -27,9 +27,9 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import io.druid.indexer.partitions.HashedPartitionsSpec;
import io.druid.indexer.partitions.PartitionsSpec;
import io.druid.java.util.common.DateTimes;
import io.druid.segment.IndexSpec;
import io.druid.segment.indexing.TuningConfig;
import org.joda.time.DateTime;
import java.util.List;
import java.util.Map;
@ -50,7 +50,7 @@ public class HadoopTuningConfig implements TuningConfig
{
return new HadoopTuningConfig(
null,
new DateTime().toString(),
DateTimes.nowUtc().toString(),
DEFAULT_PARTITIONS_SPEC,
DEFAULT_SHARD_SPECS,
DEFAULT_INDEX_SPEC,
@ -115,7 +115,7 @@ public class HadoopTuningConfig implements TuningConfig
)
{
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.shardSpecs = shardSpecs == null ? DEFAULT_SHARD_SPECS : shardSpecs;
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.OutputSupplier;
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.IAE;
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.security.UserGroupInformation;
import org.apache.hadoop.util.Progressable;
import org.joda.time.DateTime;
import java.io.BufferedOutputStream;
import java.io.File;
@ -622,10 +622,10 @@ public class JobHelper
log.info(
"File[%s / %s / %sB] existed, but wasn't the same as [%s / %s / %sB]",
finalIndexZipFile.getPath(),
new DateTime(finalIndexZipFile.getModificationTime()),
DateTimes.utc(finalIndexZipFile.getModificationTime()),
finalIndexZipFile.getLen(),
zipFile.getPath(),
new DateTime(zipFile.getModificationTime()),
DateTimes.utc(zipFile.getModificationTime()),
zipFile.getLen()
);
outputFS.delete(finalIndexZipFilePath, false);
@ -634,7 +634,7 @@ public class JobHelper
log.info(
"File[%s / %s / %sB] existed and will be kept",
finalIndexZipFile.getPath(),
new DateTime(finalIndexZipFile.getModificationTime()),
DateTimes.utc(finalIndexZipFile.getModificationTime()),
finalIndexZipFile.getLen()
);
needRename = false;

View File

@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
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.query.filter.DimFilter;
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.Ordering;
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.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.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.hadoop.fs.FileStatus;
@ -88,7 +89,7 @@ public class GranularUnprocessedPathSpec extends GranularityPathSpec
Set<Interval> bucketsToRun = Sets.newTreeSet(Comparators.intervals());
for (Map.Entry<Long, Long> entry : inputModifiedTimes.entrySet()) {
DateTime timeBucket = new DateTime(entry.getKey());
DateTime timeBucket = DateTimes.utc(entry.getKey());
long mTime = entry.getValue();
String bucketOutput = StringUtils.format(

View File

@ -168,7 +168,7 @@ public class GranularityPathSpec implements PathSpec
end = inputInterval.getEndMillis();
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.indexer.hadoop.WindowedDataSegment;
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.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory;
@ -71,8 +72,8 @@ public class BatchDeltaIngestionTest
private static final ObjectMapper MAPPER;
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_PARTIAL = new Interval("2014-10-22T00:00:00Z/PT2H");
private static final Interval INTERVAL_FULL = Intervals.of("2014-10-22T00:00:00Z/P1D");
private static final Interval INTERVAL_PARTIAL = Intervals.of("2014-10-22T00:00:00Z/PT2H");
private static final DataSegment SEGMENT;
static {

View File

@ -21,10 +21,12 @@ package io.druid.indexer;
import com.google.common.primitives.Bytes;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.Pair;
import org.hamcrest.number.OrderingComparison;
import org.joda.time.DateTime;
import org.joda.time.chrono.ISOChronology;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@ -39,7 +41,7 @@ public class BucketTest
@Before public void setUp()
{
time = new DateTime(2014, 11, 24, 10, 30);
time = new DateTime(2014, 11, 24, 10, 30, ISOChronology.getInstanceUTC());
shardNum = 1;
partitionNum = 1;
bucket = new Bucket(shardNum, time, partitionNum);
@ -80,10 +82,12 @@ public class BucketTest
bucket.equals(new Bucket(shardNum, time, partitionNum + 1)));
Assert.assertFalse("Objects do not have the same shardNum",
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.assertTrue("Objects must be the same", bucket.equals(new Bucket(shardNum, time, partitionNum)));
}
@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.TimestampSpec;
import io.druid.indexer.partitions.HashedPartitionsSpec;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -147,7 +147,7 @@ public class DetermineHashedPartitionsJobTest
new UniformGranularitySpec(
Granularities.DAY,
Granularities.NONE,
ImmutableList.of(new Interval(interval))
ImmutableList.of(Intervals.of(interval))
),
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.TimestampSpec;
import io.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory;
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.timeline.partition.SingleDimensionShardSpec;
import org.apache.commons.io.FileUtils;
import org.joda.time.Interval;
import org.junit.After;
import org.junit.Assert;
import org.junit.Test;
@ -237,7 +237,7 @@ public class DeterminePartitionsJobTest
),
new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")},
new UniformGranularitySpec(
Granularities.DAY, Granularities.NONE, ImmutableList.of(new Interval(interval))
Granularities.DAY, Granularities.NONE, ImmutableList.of(Intervals.of(interval))
),
HadoopDruidIndexerConfig.JSON_MAPPER
),

View File

@ -26,14 +26,14 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import io.druid.data.input.MapBasedInputRow;
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.query.aggregation.AggregatorFactory;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import io.druid.timeline.partition.HashBasedNumberedShardSpec;
import io.druid.timeline.partition.NoneShardSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
@ -71,7 +71,7 @@ public class HadoopDruidIndexerConfigTest
new UniformGranularitySpec(
Granularities.MINUTE,
Granularities.MINUTE,
ImmutableList.of(new Interval("2010-01-01/P1D"))
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
),
jsonMapper
),
@ -80,7 +80,7 @@ public class HadoopDruidIndexerConfigTest
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,
false,
@ -110,9 +110,9 @@ public class HadoopDruidIndexerConfigTest
"dim2",
"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 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
for (int i = 0; timestamp + i < nextBucketTimestamp; i++) {
Assert.assertEquals(
@ -134,7 +134,7 @@ public class HadoopDruidIndexerConfigTest
new UniformGranularitySpec(
Granularities.MINUTE,
Granularities.MINUTE,
ImmutableList.of(new Interval("2010-01-01/P1D"))
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
),
jsonMapper
),
@ -143,12 +143,12 @@ public class HadoopDruidIndexerConfigTest
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(
NoneShardSpec.instance(),
1
)),
new DateTime("2010-01-01T02:00:00").getMillis(),
DateTimes.of("2010-01-01T02:00:00").getMillis(),
Lists.newArrayList(new HadoopyShardSpec(
NoneShardSpec.instance(),
2
@ -183,10 +183,10 @@ public class HadoopDruidIndexerConfigTest
"dim2",
"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);
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);
}

View File

@ -30,12 +30,12 @@ import io.druid.indexer.partitions.PartitionsSpec;
import io.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import io.druid.indexer.updater.MetadataStorageUpdaterJobSpec;
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.PeriodGranularity;
import io.druid.metadata.MetadataStorageConnectorConfig;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.joda.time.DateTimeZone;
import org.joda.time.Interval;
import org.joda.time.Period;
import org.junit.Assert;
import org.junit.Test;
@ -78,7 +78,7 @@ public class HadoopIngestionSpecTest
Assert.assertEquals(
"getIntervals",
Lists.newArrayList(new Interval("2012-01-01/P1D")),
Lists.newArrayList(Intervals.of("2012-01-01/P1D")),
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.UsedSegmentLister;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.indexing.DataSchema;
@ -51,8 +52,8 @@ import java.util.Map;
public class HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest
{
private final String testDatasource = "test";
private final Interval testDatasourceInterval = new Interval("1970/3000");
private final Interval testDatasourceIntervalPartial = new Interval("2050/3000");
private final Interval testDatasourceInterval = Intervals.of("1970/3000");
private final Interval testDatasourceIntervalPartial = Intervals.of("2050/3000");
private final ObjectMapper jsonMapper;
public HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest()
@ -65,7 +66,7 @@ public class HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest
private static final DataSegment SEGMENT = new DataSegment(
"test1",
Interval.parse("2000/3000"),
Intervals.of("2000/3000"),
"ver",
ImmutableMap.<String, Object>of(
"type", "local",
@ -233,9 +234,7 @@ public class HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(
new Interval("2010-01-01/P1D")
)
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
),
jsonMapper
),

View File

@ -22,13 +22,15 @@ package io.druid.indexer;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import io.druid.java.util.common.StringUtils;
import io.druid.data.input.InputRow;
import io.druid.data.input.MapBasedInputRow;
import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.StringInputRowParser;
import io.druid.data.input.impl.TimeAndDimsParseSpec;
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.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
@ -40,8 +42,6 @@ import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.mapreduce.Reducer;
import org.easymock.Capture;
import org.easymock.EasyMock;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@ -80,7 +80,7 @@ public class IndexGeneratorCombinerTest
new HyperUniquesAggregatorFactory("unique_hosts", "host")
},
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
),
@ -136,7 +136,7 @@ public class IndexGeneratorCombinerTest
{
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(
bucket.toGroupKey(),
new byte[0]
@ -195,7 +195,7 @@ public class IndexGeneratorCombinerTest
{
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(
bucket.toGroupKey(),
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.StringInputRowParser;
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.StringUtils;
import io.druid.java.util.common.granularity.Granularities;
@ -423,7 +424,7 @@ public class IndexGeneratorJobTest
this.useCombiner = useCombiner;
this.partitionType = partitionType;
this.shardInfoForEachSegment = shardInfoForEachSegment;
this.interval = new Interval(interval);
this.interval = Intervals.of(interval);
this.data = data;
this.inputFormatName = inputFormatName;
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.StringInputRowParser;
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.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
@ -60,7 +61,7 @@ public class JobHelperTest
private HadoopDruidIndexerConfig config;
private File tmpDir;
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
public void setup() throws Exception
@ -155,7 +156,7 @@ public class JobHelperTest
{
DataSegment segment = new DataSegment(
"test1",
Interval.parse("2000/3000"),
Intervals.of("2000/3000"),
"ver",
ImmutableMap.<String, Object>of(
"type", "google",

View File

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

View File

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

View File

@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.io.ByteArrayDataOutput;
import com.google.common.io.ByteStreams;
import io.druid.java.util.common.Intervals;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import org.joda.time.Interval;
@ -39,13 +40,13 @@ public class DatasourceInputSplitTest
@Test
public void testSerde() throws Exception
{
Interval interval = Interval.parse("2000/3000");
Interval interval = Intervals.of("2000/3000");
DatasourceInputSplit expected = new DatasourceInputSplit(
Lists.newArrayList(
new WindowedDataSegment(
new DataSegment(
"test",
Interval.parse("2000/3000"),
Intervals.of("2000/3000"),
"ver",
ImmutableMap.<String, Object>of(
"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.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import org.joda.time.Interval;
@ -36,7 +37,7 @@ public class WindowedDataSegmentTest
private static final ObjectMapper MAPPER = new DefaultObjectMapper();
private static final DataSegment SEGMENT = new DataSegment(
"test1",
Interval.parse("2000/3000"),
Intervals.of("2000/3000"),
"ver",
ImmutableMap.<String, Object>of(
"type", "local",
@ -65,7 +66,7 @@ public class WindowedDataSegmentTest
@Test
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 roundTrip = MAPPER.readValue(
MAPPER.writeValueAsBytes(windowedDataSegment),

View File

@ -44,6 +44,7 @@ import io.druid.indexer.hadoop.WindowedDataSegment;
import io.druid.initialization.Initialization;
import io.druid.jackson.DefaultObjectMapper;
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.query.aggregation.AggregatorFactory;
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.mapreduce.Job;
import org.easymock.EasyMock;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
@ -74,7 +74,7 @@ public class DatasourcePathSpecTest
{
this.ingestionSpec = new DatasourceIngestionSpec(
"test",
Interval.parse("2000/3000"),
Intervals.of("2000/3000"),
null,
null,
null,
@ -87,7 +87,7 @@ public class DatasourcePathSpecTest
WindowedDataSegment.of(
new DataSegment(
ingestionSpec.getDataSource(),
Interval.parse("2000/3000"),
Intervals.of("2000/3000"),
"ver",
ImmutableMap.<String, Object>of(
"type", "local",
@ -103,7 +103,7 @@ public class DatasourcePathSpecTest
WindowedDataSegment.of(
new DataSegment(
ingestionSpec.getDataSource(),
Interval.parse("2050/3000"),
Intervals.of("2050/3000"),
"ver",
ImmutableMap.<String, Object>of(
"type", "hdfs",
@ -278,7 +278,7 @@ public class DatasourcePathSpecTest
new LongSumAggregatorFactory("visited_sum", "visited")
},
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
),

View File

@ -29,6 +29,7 @@ import io.druid.indexer.HadoopIOConfig;
import io.druid.indexer.HadoopIngestionSpec;
import io.druid.indexer.HadoopTuningConfig;
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.granularity.Granularities;
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.security.UserGroupInformation;
import org.joda.time.DateTimeZone;
import org.joda.time.Interval;
import org.joda.time.Period;
import org.junit.After;
import org.junit.Assert;
@ -151,7 +151,7 @@ public class GranularityPathSpecTest
new UniformGranularitySpec(
Granularities.DAY,
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
),
@ -202,7 +202,7 @@ public class GranularityPathSpecTest
new UniformGranularitySpec(
Granularities.DAY,
Granularities.ALL,
ImmutableList.of(new Interval("2015-01-01T11Z/2015-01-02T05Z"))
ImmutableList.of(Intervals.of("2015-01-01T11Z/2015-01-02T05Z"))
),
jsonMapper
),

View File

@ -42,6 +42,7 @@ import io.druid.indexer.JobHelper;
import io.druid.indexer.Jobby;
import io.druid.indexer.SQLMetadataStorageUpdaterJobHandler;
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.metadata.MetadataSegmentManagerConfig;
import io.druid.metadata.MetadataStorageConnectorConfig;
@ -103,7 +104,7 @@ public class HadoopConverterJobTest
private Supplier<MetadataStorageTablesConfig> metadataStorageTablesConfigSupplier;
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
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.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import io.druid.segment.IndexSpec;
import io.druid.timeline.DataSegment;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
@ -44,7 +44,7 @@ public class HadoopDruidConverterConfigTest
{
final HadoopDruidConverterConfig config = new HadoopDruidConverterConfig(
"datasource",
Interval.parse("2000/2010"),
Intervals.of("2000/2010"),
new IndexSpec(),
ImmutableList.<DataSegment>of(),
true,

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