Add org.joda.time.DateTime.parse() to forbidden APIs (#4857)

* Added org.joda.time.DateTime#(java.lang.String) to forbidden API.

* Added org.joda.time.DateTime#(java.lang.String, org.joda.time.format.DateTimeFormatter) to forbidden API.

* Add additional APIs that may create DateTime with default time zone

* Add helper function that accepts formatter to parse String.

* Add additional forbidden APIs

* Replace existing usage of forbidden APIs

* Use wrapper class to enforce Chronology on DateTimeFormatter.

* Creates constant UtcFormatter for constant ISODateTimeFormat.
This commit is contained in:
Goh Wei Xiang 2017-09-27 15:46:44 -07:00 committed by Roman Leventov
parent c3fbe5158d
commit 2c30d5ba55
20 changed files with 140 additions and 72 deletions

View File

@ -21,13 +21,12 @@ package io.druid.timeline;
import com.google.common.base.Function;
import io.druid.guice.annotations.PublicApi;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.joda.time.format.DateTimeFormatter;
import org.joda.time.format.ISODateTimeFormat;
import java.util.Objects;
@ -83,11 +82,9 @@ public class DataSegmentUtils
return null;
}
DateTimeFormatter formatter = ISODateTimeFormat.dateTime();
try {
DateTime start = formatter.parseDateTime(splits[0]);
DateTime end = formatter.parseDateTime(splits[1]);
DateTime start = DateTimes.ISO_DATE_TIME.parse(splits[0]);
DateTime end = DateTimes.ISO_DATE_TIME.parse(splits[1]);
String version = splits[2];
String trail = splits.length > 3 ? join(splits, DataSegment.delimiter, 3, splits.length) : null;

View File

@ -61,9 +61,11 @@ public class TimestampSpecTest
};
TimestampSpec spec = new TimestampSpec("TIMEstamp", DATE_FORMAT, null);
DateTimes.UtcFormatter formatter = DateTimes.wrapFormatter(ISODateTimeFormat.dateHourMinuteSecond());
for (String date : dates) {
DateTime dateTime = spec.extractTimestamp(ImmutableMap.<String, Object>of("TIMEstamp", date));
DateTime expectedDateTime = ISODateTimeFormat.dateHourMinuteSecond().parseDateTime(date);
DateTime expectedDateTime = formatter.parse(date);
Assert.assertEquals(expectedDateTime, dateTime);
}
}

View File

@ -1,4 +1,4 @@
@defaultMessage Uses default time zone
@defaultMessage Uses default time zone, use factory methods from DateTimes and Intervals classes instead
org.joda.time.DateTime#<init>()
org.joda.time.DateTime#<init>(long)
org.joda.time.DateTime#<init>(java.lang.Object)
@ -6,11 +6,17 @@ 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.DateTime#parse(java.lang.String)
org.joda.time.DateTime#parse(java.lang.String, org.joda.time.format.DateTimeFormatter)
org.joda.time.base.BaseDateTime#<init>()
org.joda.time.base.BaseDateTime#<init>(long)
org.joda.time.base.BaseDateTime#<init>(int, int, int, int, int, int, int)
org.joda.time.Instant#toDateTime()
org.joda.time.Instant#toDateTimeISO()
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#toDateTimeISO()
org.joda.time.base.AbstractInstant#toMutableDateTime()
org.joda.time.base.AbstractInstant#toMutableDateTimeISO()
org.joda.time.LocalDateTime#<init>()
@ -40,6 +46,31 @@ 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)
org.joda.time.MutableDateTime#<init>()
org.joda.time.MutableDateTime#<init>(long)
org.joda.time.MutableDateTime#<init>(java.lang.Object)
org.joda.time.MutableDateTime#<init>(int, int, int, int, int, int, int)
org.joda.time.MutableDateTime#now()
org.joda.time.MutableDateTime#parse(java.lang.String)
org.joda.time.MutableDateTime#parse(java.lang.String, org.joda.time.format.DateTimeFormatter)
org.joda.time.MutableInterval#<init>()
org.joda.time.MutableInterval#<init>(long, long)
org.joda.time.MutableInterval#<init>(java.lang.Object)
org.joda.time.MutableInterval#parse(java.lang.String)
@defaultMessage Uses default time zone, use DateTimes.UtcFormatter to parse.
org.joda.time.format.DateTimeFormatter#parseInto(org.joda.time.ReadWritableInstant, java.lang.String, int)
org.joda.time.format.DateTimeFormatter#parseDateTime(java.lang.String)
org.joda.time.format.DateTimeFormatter#parseMutableDateTime(java.lang.String)
@defaultMessage Do not use Chronology instance with default time zone.
org.joda.time.chrono.BuddhistChronology#getInstance()
org.joda.time.chrono.CopticChronology#getInstance()
org.joda.time.chrono.EthiopicChronology#getInstance()
org.joda.time.chrono.GJChronology#getInstance()
org.joda.time.chrono.GregorianChronology#getInstance()
org.joda.time.chrono.IslamicChronology#getInstance()
org.joda.time.chrono.ISOChronology#getInstance()
@defaultMessage Doesn't handle edge cases where the start of day isn't midnight.
org.joda.time.LocalDate#toDateTimeAtMidnight()

View File

@ -20,12 +20,11 @@
package io.druid.math.expr;
import com.google.common.base.Strings;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.StringUtils;
import org.joda.time.DateTime;
import org.joda.time.format.DateTimeFormat;
import org.joda.time.format.DateTimeFormatter;
import org.joda.time.format.ISODateTimeFormat;
import java.util.List;
@ -824,17 +823,17 @@ interface Function
throw new IAE("first argument should be string type but got %s type", value.type());
}
DateTimeFormatter formatter = ISODateTimeFormat.dateOptionalTimeParser();
DateTimes.UtcFormatter formatter = DateTimes.ISO_DATE_OPTIONAL_TIME;
if (args.size() > 1) {
ExprEval format = args.get(1).eval(bindings);
if (format.type() != ExprType.STRING) {
throw new IAE("second argument should be string type but got %s type", format.type());
}
formatter = DateTimeFormat.forPattern(format.asString());
formatter = DateTimes.wrapFormatter(DateTimeFormat.forPattern(format.asString()));
}
DateTime date;
try {
date = DateTime.parse(value.asString(), formatter);
date = formatter.parse(value.asString());
}
catch (IllegalArgumentException e) {
throw new IAE(e, "invalid value %s", value.asString());

View File

@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.io.Resources;
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;
@ -32,7 +33,6 @@ import io.druid.query.Result;
import io.druid.query.select.SelectResultValue;
import io.druid.segment.ColumnSelectorFactory;
import org.easymock.EasyMock;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
@ -65,8 +65,8 @@ public class TimestampAggregationSelectTest
{
return Iterables.transform(
ImmutableList.of(
ImmutableList.of("timeMin", "tmin", TimestampMinAggregatorFactory.class, DateTime.parse("2011-01-12T01:00:00.000Z").getMillis()),
ImmutableList.of("timeMax", "tmax", TimestampMaxAggregatorFactory.class, DateTime.parse("2011-01-31T01:00:00.000Z").getMillis())
ImmutableList.of("timeMin", "tmin", TimestampMinAggregatorFactory.class, DateTimes.of("2011-01-12T01:00:00.000Z").getMillis()),
ImmutableList.of("timeMax", "tmax", TimestampMaxAggregatorFactory.class, DateTimes.of("2011-01-31T01:00:00.000Z").getMillis())
),
new Function<List<?>, Object[]>()
{

View File

@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
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;
@ -63,8 +64,8 @@ public class TimestampGroupByAggregationTest
final List<Object[]> constructors = Lists.newArrayList();
final List<List<Object>> partialConstructors = ImmutableList.<List<Object>>of(
ImmutableList.<Object>of("timeMin", "tmin", "time_min", TimestampMinAggregatorFactory.class, DateTime.parse("2011-01-12T01:00:00.000Z")),
ImmutableList.<Object>of("timeMax", "tmax", "time_max", TimestampMaxAggregatorFactory.class, DateTime.parse("2011-01-31T01:00:00.000Z"))
ImmutableList.<Object>of("timeMin", "tmin", "time_min", TimestampMinAggregatorFactory.class, DateTimes.of("2011-01-12T01:00:00.000Z")),
ImmutableList.<Object>of("timeMax", "tmax", "time_max", TimestampMaxAggregatorFactory.class, DateTimes.of("2011-01-31T01:00:00.000Z"))
);
for (final List<Object> partialConstructor : partialConstructors) {

View File

@ -31,6 +31,7 @@ import com.yahoo.sketches.theta.Union;
import com.yahoo.sketches.theta.UpdateSketch;
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;
@ -42,7 +43,6 @@ import io.druid.query.groupby.GroupByQueryConfig;
import io.druid.query.groupby.GroupByQueryRunnerTest;
import io.druid.query.groupby.epinephelinae.GrouperTestUtil;
import io.druid.query.groupby.epinephelinae.TestColumnSelectorFactory;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
@ -106,7 +106,7 @@ public class SketchAggregationTest
Assert.assertEquals(1, results.size());
Assert.assertEquals(
new MapBasedRow(
DateTime.parse("2014-10-19T00:00:00.000Z"),
DateTimes.of("2014-10-19T00:00:00.000Z"),
ImmutableMap
.<String, Object>builder()
.put("sids_sketch_count", 50.0)
@ -148,7 +148,7 @@ public class SketchAggregationTest
Assert.assertEquals(
ImmutableList.of(
new MapBasedRow(
DateTime.parse("2014-10-19T00:00:00.000Z"),
DateTimes.of("2014-10-19T00:00:00.000Z"),
ImmutableMap
.<String, Object>builder()
.put("product", "product_3")
@ -161,7 +161,7 @@ public class SketchAggregationTest
.build()
),
new MapBasedRow(
DateTime.parse("2014-10-19T00:00:00.000Z"),
DateTimes.of("2014-10-19T00:00:00.000Z"),
ImmutableMap
.<String, Object>builder()
.put("product", "product_1")
@ -174,7 +174,7 @@ public class SketchAggregationTest
.build()
),
new MapBasedRow(
DateTime.parse("2014-10-19T00:00:00.000Z"),
DateTimes.of("2014-10-19T00:00:00.000Z"),
ImmutableMap
.<String, Object>builder()
.put("product", "product_2")
@ -187,7 +187,7 @@ public class SketchAggregationTest
.build()
),
new MapBasedRow(
DateTime.parse("2014-10-19T00:00:00.000Z"),
DateTimes.of("2014-10-19T00:00:00.000Z"),
ImmutableMap
.<String, Object>builder()
.put("product", "product_4")
@ -200,7 +200,7 @@ public class SketchAggregationTest
.build()
),
new MapBasedRow(
DateTime.parse("2014-10-19T00:00:00.000Z"),
DateTimes.of("2014-10-19T00:00:00.000Z"),
ImmutableMap
.<String, Object>builder()
.put("product", "product_5")
@ -346,7 +346,7 @@ public class SketchAggregationTest
Assert.assertEquals(
ImmutableList.of(
new MapBasedRow(
DateTime.parse("2014-10-19T00:00:00.000Z"),
DateTimes.of("2014-10-19T00:00:00.000Z"),
ImmutableMap
.<String, Object>builder()
.put("product", "product_1")

View File

@ -38,7 +38,6 @@ import io.druid.query.select.SelectResultValue;
import io.druid.query.timeseries.TimeseriesResultValue;
import io.druid.query.topn.DimensionAndMetricValueExtractor;
import io.druid.query.topn.TopNResultValue;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
@ -136,7 +135,7 @@ public class SketchAggregationWithSimpleDataTest
Assert.assertEquals(
ImmutableList.of(
new MapBasedRow(
DateTime.parse("2014-10-19T00:00:00.000Z"),
DateTimes.of("2014-10-19T00:00:00.000Z"),
ImmutableMap
.<String, Object>builder()
.put("product", "product_3")
@ -149,7 +148,7 @@ public class SketchAggregationWithSimpleDataTest
.build()
),
new MapBasedRow(
DateTime.parse("2014-10-19T00:00:00.000Z"),
DateTimes.of("2014-10-19T00:00:00.000Z"),
ImmutableMap
.<String, Object>builder()
.put("product", "product_1")
@ -162,7 +161,7 @@ public class SketchAggregationWithSimpleDataTest
.build()
),
new MapBasedRow(
DateTime.parse("2014-10-19T00:00:00.000Z"),
DateTimes.of("2014-10-19T00:00:00.000Z"),
ImmutableMap
.<String, Object>builder()
.put("product", "product_2")
@ -175,7 +174,7 @@ public class SketchAggregationWithSimpleDataTest
.build()
),
new MapBasedRow(
DateTime.parse("2014-10-19T00:00:00.000Z"),
DateTimes.of("2014-10-19T00:00:00.000Z"),
ImmutableMap
.<String, Object>builder()
.put("product", "product_4")
@ -188,7 +187,7 @@ public class SketchAggregationWithSimpleDataTest
.build()
),
new MapBasedRow(
DateTime.parse("2014-10-19T00:00:00.000Z"),
DateTimes.of("2014-10-19T00:00:00.000Z"),
ImmutableMap
.<String, Object>builder()
.put("product", "product_5")

View File

@ -25,6 +25,7 @@ import com.google.common.io.Files;
import com.yahoo.sketches.theta.Sketches;
import com.yahoo.sketches.theta.UpdateSketch;
import io.druid.data.input.MapBasedRow;
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;
@ -37,7 +38,6 @@ import io.druid.query.groupby.GroupByQueryConfig;
import io.druid.query.groupby.GroupByQueryRunnerTest;
import io.druid.query.groupby.epinephelinae.GrouperTestUtil;
import io.druid.query.groupby.epinephelinae.TestColumnSelectorFactory;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
@ -100,7 +100,7 @@ public class OldApiSketchAggregationTest
Assert.assertEquals(1, results.size());
Assert.assertEquals(
new MapBasedRow(
DateTime.parse("2014-10-19T00:00:00.000Z"),
DateTimes.of("2014-10-19T00:00:00.000Z"),
ImmutableMap
.<String, Object>builder()
.put("sketch_count", 50.0)
@ -132,7 +132,7 @@ public class OldApiSketchAggregationTest
Assert.assertEquals(1, results.size());
Assert.assertEquals(
new MapBasedRow(
DateTime.parse("2014-10-19T00:00:00.000Z"),
DateTimes.of("2014-10-19T00:00:00.000Z"),
ImmutableMap
.<String, Object>builder()
.put("sids_sketch_count", 50.0)

View File

@ -28,6 +28,7 @@ import com.google.inject.Module;
import com.google.inject.name.Names;
import io.druid.guice.GuiceInjectors;
import io.druid.initialization.Initialization;
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.io.Closer;
@ -45,7 +46,6 @@ import org.I0Itec.zkclient.ZkClient;
import org.I0Itec.zkclient.exception.ZkException;
import org.apache.curator.test.TestingServer;
import org.apache.zookeeper.CreateMode;
import org.joda.time.DateTime;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@ -134,7 +134,7 @@ public class TestKafkaExtractionCluster
kafkaConfig = new KafkaConfig(serverProperties);
final long time = DateTime.parse("2015-01-01").getMillis();
final long time = DateTimes.of("2015-01-01").getMillis();
kafkaServer = new KafkaServer(
kafkaConfig,
new Time()

View File

@ -35,6 +35,7 @@ import io.druid.data.input.impl.StringInputRowParser;
import io.druid.data.input.impl.TimestampSpec;
import io.druid.indexer.hadoop.WindowedDataSegment;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularities;
@ -53,7 +54,6 @@ import io.druid.segment.realtime.firehose.WindowedStorageAdapter;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.HashBasedNumberedShardSpec;
import org.apache.commons.io.FileUtils;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Rule;
@ -127,19 +127,19 @@ public class BatchDeltaIngestionTest
List<ImmutableMap<String, Object>> expectedRows = ImmutableList.of(
ImmutableMap.<String, Object>of(
"time", DateTime.parse("2014-10-22T00:00:00.000Z"),
"time", DateTimes.of("2014-10-22T00:00:00.000Z"),
"host", ImmutableList.of("a.example.com"),
"visited_sum", 100L,
"unique_hosts", 1.0d
),
ImmutableMap.<String, Object>of(
"time", DateTime.parse("2014-10-22T01:00:00.000Z"),
"time", DateTimes.of("2014-10-22T01:00:00.000Z"),
"host", ImmutableList.of("b.example.com"),
"visited_sum", 150L,
"unique_hosts", 1.0d
),
ImmutableMap.<String, Object>of(
"time", DateTime.parse("2014-10-22T02:00:00.000Z"),
"time", DateTimes.of("2014-10-22T02:00:00.000Z"),
"host", ImmutableList.of("c.example.com"),
"visited_sum", 200L,
"unique_hosts", 1.0d
@ -173,13 +173,13 @@ public class BatchDeltaIngestionTest
List<ImmutableMap<String, Object>> expectedRows = ImmutableList.of(
ImmutableMap.<String, Object>of(
"time", DateTime.parse("2014-10-22T00:00:00.000Z"),
"time", DateTimes.of("2014-10-22T00:00:00.000Z"),
"host", ImmutableList.of("a.example.com"),
"visited_sum", 100L,
"unique_hosts", 1.0d
),
ImmutableMap.<String, Object>of(
"time", DateTime.parse("2014-10-22T01:00:00.000Z"),
"time", DateTimes.of("2014-10-22T01:00:00.000Z"),
"host", ImmutableList.of("b.example.com"),
"visited_sum", 150L,
"unique_hosts", 1.0d
@ -250,19 +250,19 @@ public class BatchDeltaIngestionTest
List<ImmutableMap<String, Object>> expectedRows = ImmutableList.of(
ImmutableMap.<String, Object>of(
"time", DateTime.parse("2014-10-22T00:00:00.000Z"),
"time", DateTimes.of("2014-10-22T00:00:00.000Z"),
"host", ImmutableList.of("a.example.com"),
"visited_sum", 190L,
"unique_hosts", 1.0d
),
ImmutableMap.<String, Object>of(
"time", DateTime.parse("2014-10-22T01:00:00.000Z"),
"time", DateTimes.of("2014-10-22T01:00:00.000Z"),
"host", ImmutableList.of("b.example.com"),
"visited_sum", 175L,
"unique_hosts", 1.0d
),
ImmutableMap.<String, Object>of(
"time", DateTime.parse("2014-10-22T02:00:00.000Z"),
"time", DateTimes.of("2014-10-22T02:00:00.000Z"),
"host", ImmutableList.of("c.example.com"),
"visited_sum", 270L,
"unique_hosts", 1.0d

View File

@ -25,13 +25,13 @@ import com.google.common.collect.Lists;
import io.druid.data.input.InputRow;
import io.druid.indexer.HadoopDruidIndexerConfig;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
import io.druid.timeline.DataSegment;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.easymock.EasyMock;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Test;
@ -98,19 +98,19 @@ public class DatasourceRecordReaderTest
{
List<ImmutableMap<String, Object>> expectedRows = ImmutableList.of(
ImmutableMap.<String, Object>of(
"time", DateTime.parse("2014-10-22T00:00:00.000Z"),
"time", DateTimes.of("2014-10-22T00:00:00.000Z"),
"host", ImmutableList.of("a.example.com"),
"visited_sum", 100L,
"unique_hosts", 1.0d
),
ImmutableMap.<String, Object>of(
"time", DateTime.parse("2014-10-22T01:00:00.000Z"),
"time", DateTimes.of("2014-10-22T01:00:00.000Z"),
"host", ImmutableList.of("b.example.com"),
"visited_sum", 150L,
"unique_hosts", 1.0d
),
ImmutableMap.<String, Object>of(
"time", DateTime.parse("2014-10-22T02:00:00.000Z"),
"time", DateTimes.of("2014-10-22T02:00:00.000Z"),
"host", ImmutableList.of("c.example.com"),
"visited_sum", 200L,
"unique_hosts", 1.0d

View File

@ -19,8 +19,11 @@
package io.druid.java.util.common;
import org.joda.time.Chronology;
import org.joda.time.DateTime;
import org.joda.time.chrono.ISOChronology;
import org.joda.time.format.DateTimeFormatter;
import org.joda.time.format.ISODateTimeFormat;
public final class DateTimes
{
@ -28,6 +31,42 @@ public final class DateTimes
public static final DateTime MAX = utc(JodaUtils.MAX_INSTANT);
public static final DateTime MIN = utc(JodaUtils.MIN_INSTANT);
public static final UtcFormatter ISO_DATE_TIME = wrapFormatter(ISODateTimeFormat.dateTime());
public static final UtcFormatter ISO_DATE_OPTIONAL_TIME = wrapFormatter(ISODateTimeFormat.dateOptionalTimeParser());
public static final UtcFormatter ISO_DATE_OR_TIME = wrapFormatter(ISODateTimeFormat.dateTimeParser());
public static final UtcFormatter ISO_DATE_OR_TIME_WITH_OFFSET = wrapFormatter(
ISODateTimeFormat.dateTimeParser().withOffsetParsed()
);
/**
* Simple wrapper class to enforce UTC Chronology in formatter. Specifically, it will use
* {@link DateTimeFormatter#withChronology(Chronology)} to set the chronology to
* {@link ISOChronology#getInstanceUTC()} on the wrapped {@link DateTimeFormatter}.
*/
public static class UtcFormatter
{
private final DateTimeFormatter innerFormatter;
private UtcFormatter(final DateTimeFormatter innerFormatter)
{
this.innerFormatter = innerFormatter.withChronology(ISOChronology.getInstanceUTC());
}
public DateTime parse(final String instant)
{
return innerFormatter.parseDateTime(instant);
}
}
/**
* Creates a {@link UtcFormatter} that wraps around a {@link DateTimeFormatter}.
* @param formatter inner {@link DateTimeFormatter} used to parse {@link String}
*/
public static UtcFormatter wrapFormatter(final DateTimeFormatter formatter)
{
return new UtcFormatter(formatter);
}
public static DateTime utc(long instant)
{
return new DateTime(instant, ISOChronology.getInstanceUTC());

View File

@ -42,7 +42,7 @@ public class TimestampParser
{
if (format.equalsIgnoreCase("auto")) {
// Could be iso or millis
final DateTimeFormatter parser = createAutoParser();
final DateTimes.UtcFormatter parser = DateTimes.wrapFormatter(createAutoParser());
return (String input) -> {
Preconditions.checkArgument(!Strings.isNullOrEmpty(input), "null timestamp");
@ -59,7 +59,7 @@ public class TimestampParser
}
}
return new DateTime(parser.parseDateTime(input), timeZone);
return parser.parse(input).withZone(timeZone);
}
}
@ -87,10 +87,10 @@ public class TimestampParser
};
} else {
try {
final DateTimeFormatter formatter = DateTimeFormat.forPattern(format);
final DateTimes.UtcFormatter formatter = DateTimes.wrapFormatter(DateTimeFormat.forPattern(format));
return input -> {
Preconditions.checkArgument(!Strings.isNullOrEmpty(input), "null timestamp");
return formatter.parseDateTime(ParserUtils.stripQuotes(input));
return formatter.parse(ParserUtils.stripQuotes(input));
};
}
catch (Exception e) {

View File

@ -940,6 +940,9 @@
<signaturesFile>${session.executionRootDirectory}/codestyle/joda-time-forbidden-apis.txt</signaturesFile>
<signaturesFile>${session.executionRootDirectory}/codestyle/druid-forbidden-apis.txt</signaturesFile>
</signaturesFiles>
<excludes>
<exclude>io/druid/java/util/common/DateTimes$UtcFormatter.class</exclude>
</excludes>
</configuration>
<executions>
<execution>

View File

@ -36,7 +36,6 @@ import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.joda.time.Interval;
import org.joda.time.Period;
import org.joda.time.format.ISODateTimeFormat;
import java.io.IOException;
@ -108,9 +107,7 @@ class JodaStuff
return null;
}
// make sure to preserve time zone information when parsing timestamps
return ISODateTimeFormat.dateTimeParser()
.withOffsetParsed()
.parseDateTime(str);
return DateTimes.ISO_DATE_OR_TIME_WITH_OFFSET.parse(str);
}
throw ctxt.mappingException(getValueClass());
}

View File

@ -19,14 +19,13 @@
package io.druid.query.expression;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.IAE;
import io.druid.math.expr.Expr;
import io.druid.math.expr.ExprEval;
import io.druid.math.expr.ExprMacroTable;
import org.joda.time.DateTimeZone;
import org.joda.time.format.DateTimeFormat;
import org.joda.time.format.DateTimeFormatter;
import org.joda.time.format.ISODateTimeFormat;
import javax.annotation.Nonnull;
import java.util.List;
@ -56,9 +55,10 @@ public class TimestampParseExprMacro implements ExprMacroTable.ExprMacro
timeZone = DateTimeZone.UTC;
}
final DateTimeFormatter formatter = formatString == null
? ISODateTimeFormat.dateTimeParser()
: DateTimeFormat.forPattern(formatString).withZone(timeZone);
final DateTimes.UtcFormatter formatter =
formatString == null
? DateTimes.ISO_DATE_OR_TIME
: DateTimes.wrapFormatter(DateTimeFormat.forPattern(formatString).withZone(timeZone));
class TimestampParseExpr implements Expr
{
@ -67,7 +67,7 @@ public class TimestampParseExprMacro implements ExprMacroTable.ExprMacro
public ExprEval eval(final ObjectBinding bindings)
{
try {
return ExprEval.of(formatter.parseDateTime(arg.eval(bindings).asString()).getMillis());
return ExprEval.of(formatter.parse(arg.eval(bindings).asString()).getMillis());
}
catch (IllegalArgumentException e) {
// Catch exceptions potentially thrown by formatter.parseDateTime. Our docs say that unparseable timestamps

View File

@ -23,9 +23,10 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.io.ByteSource;
import com.google.common.io.Files;
import io.druid.java.util.common.JodaUtils;
import io.druid.data.input.InputRow;
import io.druid.data.input.MapBasedInputRow;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.JodaUtils;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.segment.data.CompressedObjectStrategy;
@ -34,7 +35,6 @@ import io.druid.segment.data.ConciseBitmapSerdeFactory;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.commons.io.FileUtils;
import org.joda.time.DateTime;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@ -56,7 +56,7 @@ public class IndexMergerV9CompatibilityTest
{
@Rule
public final CloserRule closer = new CloserRule(false);
private static final long TIMESTAMP = DateTime.parse("2014-01-01").getMillis();
private static final long TIMESTAMP = DateTimes.of("2014-01-01").getMillis();
private static final AggregatorFactory[] DEFAULT_AGG_FACTORIES = new AggregatorFactory[]{
new CountAggregatorFactory(
"count"

View File

@ -47,7 +47,6 @@ import io.druid.timeline.DataSegment;
import io.druid.timeline.TimelineObjectHolder;
import io.druid.timeline.VersionedIntervalTimeline;
import io.druid.timeline.partition.PartitionChunk;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.joda.time.Interval;
import org.skife.jdbi.v2.BaseResultSetMapper;
@ -574,8 +573,8 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager
protected Interval mapInternal(int index, Map<String, Object> row)
{
return new Interval(
DateTime.parse((String) row.get("start")),
DateTime.parse((String) row.get("end"))
DateTimes.of((String) row.get("start")),
DateTimes.of((String) row.get("end"))
);
}
}

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Charsets;
import com.google.common.collect.ImmutableMap;
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.jackson.JacksonUtils;
import io.druid.query.BaseQuery;
@ -63,7 +64,7 @@ public class LoggingRequestLoggerTest
private static final ByteArrayOutputStream baos = new ByteArrayOutputStream();
private static Appender appender;
final DateTime timestamp = DateTime.parse("2016-01-01T00:00:00Z");
final DateTime timestamp = DateTimes.of("2016-01-01T00:00:00Z");
final String remoteAddr = "some.host.tld";
final Map<String, Object> queryContext = ImmutableMap.<String, Object>of("foo", "bar");
final Query query = new FakeQuery(