mirror of https://github.com/apache/druid.git
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:
parent
fa87eaa6e8
commit
cbd1902db8
|
@ -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 +
|
||||
'}';
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -91,7 +91,7 @@ public class DataSegmentUtils
|
|||
|
||||
return new SegmentIdentifierParts(
|
||||
dataSource,
|
||||
new Interval(start.getMillis(), end.getMillis()),
|
||||
new Interval(start, end),
|
||||
version,
|
||||
trail
|
||||
);
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
);
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -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"))
|
||||
);
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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
|
||||
);
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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"),
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
|
@ -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)
|
||||
|
|
|
@ -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)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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"));
|
||||
|
|
|
@ -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"));
|
||||
|
|
|
@ -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[][][]{{
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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}";
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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]) :
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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<>();
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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),
|
||||
|
|
|
@ -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"));
|
||||
}
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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
|
||||
),
|
||||
|
|
|
@ -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
|
||||
),
|
||||
|
|
|
@ -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);
|
||||
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
|
||||
|
|
|
@ -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
|
||||
),
|
||||
|
|
|
@ -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]
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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),
|
||||
|
|
|
@ -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
|
||||
),
|
||||
|
|
|
@ -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
|
||||
),
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue