Use both Joad Ids and Java IDs as Timezone to string readers (#5349)

* Use both Joad Ids and Java IDs as Timezone to string readers

Change-Id: Ieb5c18559879f3f3a0104912ce2f0a354ad0aac3

* move the function to DateTimes and add org.joda.time.DateTimeZone#forID as part of forbidden api

Change-Id: Iff97fa044758019ed0c231587d10e31a9cc18da0

* exclude class and remove other usage

Change-Id: Ib458c2caaa1865535767e1009fbf017a92c8f615

* remove it from test classes

Change-Id: I9b576324f6c7e17a74bd8b13879232c9a8cd40b4

* remove unused

Change-Id: If1c5b70c26c2b7c83c20434cb72b2060653f5052
This commit is contained in:
Slim 2018-02-06 03:04:11 -08:00 committed by Nishant Bangarwa
parent 9a62b02cb7
commit 37c09ce3f8
25 changed files with 78 additions and 56 deletions

View File

@ -8,6 +8,7 @@ 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.DateTimeZone#forID(java.lang.String)
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)

View File

@ -27,6 +27,7 @@ import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.StringInputRowParser;
import io.druid.data.input.impl.TimestampSpec;
import io.druid.indexer.partitions.HashedPartitionsSpec;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.granularity.Granularity;
@ -35,7 +36,6 @@ 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.DateTimeZone;
import org.joda.time.Interval;
import org.joda.time.Period;
import org.junit.Assert;
@ -115,7 +115,7 @@ public class DetermineHashedPartitionsJobTest
0,
1,
first,
new PeriodGranularity(new Period("P1D"), null, DateTimeZone.forID("America/Los_Angeles"))
new PeriodGranularity(new Period("P1D"), null, DateTimes.inferTzfromString("America/Los_Angeles"))
}
}
);

View File

@ -30,12 +30,12 @@ import io.druid.indexer.partitions.PartitionsSpec;
import io.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import io.druid.indexer.updater.MetadataStorageUpdaterJobSpec;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
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.Period;
import org.junit.Assert;
import org.junit.Test;
@ -118,7 +118,7 @@ public class HadoopIngestionSpecTest
Assert.assertEquals(
"getSegmentGranularity",
new PeriodGranularity(new Period("PT1H"), null, DateTimeZone.forID("America/Los_Angeles")),
new PeriodGranularity(new Period("PT1H"), null, DateTimes.inferTzfromString("America/Los_Angeles")),
granularitySpec.getSegmentGranularity()
);
}

View File

@ -21,6 +21,7 @@ package io.druid.tests.indexer;
import com.google.common.base.Throwables;
import com.google.inject.Inject;
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.logger.Logger;
@ -173,7 +174,7 @@ public class ITKafkaIndexingServiceTest extends AbstractIndexerTest
new StringSerializer()
);
DateTimeZone zone = DateTimeZone.forID("UTC");
DateTimeZone zone = DateTimes.inferTzfromString("UTC");
// format for putting into events
DateTimeFormatter event_fmt = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'Z'");

View File

@ -21,6 +21,7 @@ package io.druid.tests.indexer;
import com.google.common.base.Throwables;
import com.google.inject.Inject;
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.logger.Logger;
@ -153,7 +154,7 @@ public class ITKafkaTest extends AbstractIndexerTest
new StringSerializer()
);
DateTimeZone zone = DateTimeZone.forID("UTC");
DateTimeZone zone = DateTimes.inferTzfromString("UTC");
// format for putting into events
DateTimeFormatter event_fmt = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'Z'");

View File

@ -21,10 +21,13 @@ package io.druid.java.util.common;
import org.joda.time.Chronology;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.chrono.ISOChronology;
import org.joda.time.format.DateTimeFormatter;
import org.joda.time.format.ISODateTimeFormat;
import java.util.TimeZone;
public final class DateTimes
{
public static final DateTime EPOCH = utc(0);
@ -38,6 +41,17 @@ public final class DateTimes
ISODateTimeFormat.dateTimeParser().withOffsetParsed()
);
public static DateTimeZone inferTzfromString(String tzId)
{
try {
return DateTimeZone.forID(tzId);
}
catch (IllegalArgumentException e) {
// also support Java timezone strings
return DateTimeZone.forTimeZone(TimeZone.getTimeZone(tzId));
}
}
/**
* Simple wrapper class to enforce UTC Chronology in formatter. Specifically, it will use
* {@link DateTimeFormatter#withChronology(Chronology)} to set the chronology to

View File

@ -957,6 +957,7 @@
</signaturesFiles>
<excludes>
<exclude>io/druid/java/util/common/DateTimes$UtcFormatter.class</exclude>
<exclude>io/druid/java/util/common/DateTimes.class</exclude>
</excludes>
</configuration>
<executions>

View File

@ -29,6 +29,7 @@ import com.fasterxml.jackson.databind.SerializerProvider;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.fasterxml.jackson.databind.ser.std.ToStringSerializer;
import com.google.common.base.Throwables;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.guava.Accumulator;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Yielder;
@ -36,7 +37,6 @@ import org.joda.time.DateTimeZone;
import java.io.IOException;
import java.nio.ByteOrder;
import java.util.TimeZone;
/**
*/
@ -57,13 +57,7 @@ public class DruidDefaultSerializersModule extends SimpleModule
throws IOException
{
String tzId = jp.getText();
try {
return DateTimeZone.forID(tzId);
}
catch (IllegalArgumentException e) {
// also support Java timezone strings
return DateTimeZone.forTimeZone(TimeZone.getTimeZone(tzId));
}
return DateTimes.inferTzfromString(tzId);
}
}
);

View File

@ -19,6 +19,7 @@
package io.druid.query.expression;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.granularity.PeriodGranularity;
import io.druid.math.expr.Expr;
@ -46,7 +47,7 @@ public class ExprUtils
}
final Object literalValue = timeZoneArg.getLiteralValue();
return literalValue == null ? DateTimeZone.UTC : DateTimeZone.forID((String) literalValue);
return literalValue == null ? DateTimeZone.UTC : DateTimes.inferTzfromString((String) literalValue);
}
public static PeriodGranularity toPeriodGranularity(
@ -64,7 +65,7 @@ public class ExprUtils
timeZone = null;
} else {
final String value = timeZoneArg.eval(bindings).asString();
timeZone = value != null ? DateTimeZone.forID(value) : null;
timeZone = value != null ? DateTimes.inferTzfromString(value) : null;
}
if (originArg == null) {
@ -77,4 +78,5 @@ public class ExprUtils
return new PeriodGranularity(period, origin, timeZone);
}
}

View File

@ -54,7 +54,7 @@ public class TimestampParseExprMacro implements ExprMacroTable.ExprMacro
final DateTimeZone timeZone;
if (args.size() > 2 && args.get(2).getLiteralValue() != null) {
timeZone = DateTimeZone.forID((String) args.get(2).getLiteralValue());
timeZone = DateTimes.inferTzfromString((String) args.get(2).getLiteralValue());
} else {
timeZone = DateTimeZone.UTC;
}

View File

@ -315,7 +315,7 @@ public class QueryGranularityTest
@Test
public void testPeriodDaylightSaving() throws Exception
{
final DateTimeZone tz = DateTimeZone.forID("America/Los_Angeles");
final DateTimeZone tz = DateTimes.inferTzfromString("America/Los_Angeles");
final DateTime baseTime = new DateTime("2012-11-04T00:00:00", tz);
assertSameInterval(
Lists.newArrayList(
@ -361,7 +361,7 @@ public class QueryGranularityTest
@Test
public void testIterableMonth() throws Exception
{
final DateTimeZone tz = DateTimeZone.forID("America/Los_Angeles");
final DateTimeZone tz = DateTimes.inferTzfromString("America/Los_Angeles");
final DateTime baseTime = new DateTime("2012-11-03T10:00:00", tz);
assertSameInterval(
Lists.newArrayList(
@ -378,7 +378,7 @@ public class QueryGranularityTest
@Test
public void testIterableWeek() throws Exception
{
final DateTimeZone tz = DateTimeZone.forID("America/Los_Angeles");
final DateTimeZone tz = DateTimes.inferTzfromString("America/Los_Angeles");
final DateTime baseTime = new DateTime("2012-11-03T10:00:00", tz);
assertSameInterval(
Lists.newArrayList(
@ -405,7 +405,7 @@ public class QueryGranularityTest
@Test
public void testPeriodTruncateDays() throws Exception
{
final DateTimeZone tz = DateTimeZone.forID("America/Los_Angeles");
final DateTimeZone tz = DateTimes.inferTzfromString("America/Los_Angeles");
final DateTime origin = DateTimes.of("2012-01-02T05:00:00.000-08:00");
PeriodGranularity periodOrigin = new PeriodGranularity(
new Period("P2D"),
@ -485,7 +485,7 @@ public class QueryGranularityTest
public void testCompoundPeriodTruncate() throws Exception
{
{
final DateTimeZone tz = DateTimeZone.forID("America/Los_Angeles");
final DateTimeZone tz = DateTimes.inferTzfromString("America/Los_Angeles");
final DateTime origin = DateTimes.of("2012-01-02T05:00:00.000-08:00");
PeriodGranularity periodOrigin = new PeriodGranularity(
new Period("P1M2D"),
@ -530,7 +530,7 @@ public class QueryGranularityTest
}
{
final DateTimeZone tz = DateTimeZone.forID("America/Los_Angeles");
final DateTimeZone tz = DateTimes.inferTzfromString("America/Los_Angeles");
final DateTime origin = DateTimes.of("2012-01-02T05:00:00.000-08:00");
PeriodGranularity periodOrigin = new PeriodGranularity(
new Period("PT12H5M"),
@ -666,13 +666,13 @@ public class QueryGranularityTest
Assert.assertEquals(new PeriodGranularity(
new Period("P1D"),
DateTimes.EPOCH,
DateTimeZone.forID("America/Los_Angeles")
DateTimes.inferTzfromString("America/Los_Angeles")
), gran);
PeriodGranularity expected = new PeriodGranularity(
new Period("P1D"),
DateTimes.of("2012-01-01"),
DateTimeZone.forID("America/Los_Angeles")
DateTimes.inferTzfromString("America/Los_Angeles")
);
String jsonOut = mapper.writeValueAsString(expected);

View File

@ -75,7 +75,7 @@ public class TimewarpOperatorTest
Assert.assertEquals(
tOffset,
t.plus(testOperator.computeOffset(t.getMillis(), DateTimeZone.forID("America/Los_Angeles")))
t.plus(testOperator.computeOffset(t.getMillis(), DateTimes.inferTzfromString("America/Los_Angeles")))
);
}
}
@ -226,7 +226,7 @@ public class TimewarpOperatorTest
Druids.newTimeseriesQueryBuilder()
.dataSource("dummy")
.intervals("2014-07-31T-07/2014-08-05T-07")
.granularity(new PeriodGranularity(new Period("P1D"), null, DateTimeZone.forID("America/Los_Angeles")))
.granularity(new PeriodGranularity(new Period("P1D"), null, DateTimes.inferTzfromString("America/Los_Angeles")))
.aggregators(Arrays.<AggregatorFactory>asList(new CountAggregatorFactory("count")))
.build();
@ -286,7 +286,7 @@ public class TimewarpOperatorTest
Druids.newTimeseriesQueryBuilder()
.dataSource("dummy")
.intervals("2014-07-31T-07/2014-08-05T-07")
.granularity(new PeriodGranularity(new Period("P1D"), null, DateTimeZone.forID("America/Phoenix")))
.granularity(new PeriodGranularity(new Period("P1D"), null, DateTimes.inferTzfromString("America/Phoenix")))
.aggregators(Arrays.<AggregatorFactory>asList(new CountAggregatorFactory("count")))
.build();

View File

@ -89,7 +89,7 @@ public class TimeFormatExtractionFnTest
{
TimeFormatExtractionFn fn = new TimeFormatExtractionFn(
"'In Berlin ist es schon 'EEEE",
DateTimeZone.forID("Europe/Berlin"),
DateTimes.inferTzfromString("Europe/Berlin"),
"de",
null,
false
@ -101,7 +101,7 @@ public class TimeFormatExtractionFnTest
Assert.assertEquals("In Berlin ist es schon Sonntag", fn.apply(timestamps[4]));
Assert.assertEquals("In Berlin ist es schon Dienstag", fn.apply(timestamps[5]));
testSerde(fn, "'In Berlin ist es schon 'EEEE", DateTimeZone.forID("Europe/Berlin"), "de", Granularities.NONE);
testSerde(fn, "'In Berlin ist es schon 'EEEE", DateTimes.inferTzfromString("Europe/Berlin"), "de", Granularities.NONE);
}
public void testSerde(
@ -149,7 +149,7 @@ public class TimeFormatExtractionFnTest
{
TimeFormatExtractionFn fn = new TimeFormatExtractionFn(
"'In Berlin ist es schon 'EEEE",
DateTimeZone.forID("Europe/Berlin"),
DateTimes.inferTzfromString("Europe/Berlin"),
"de",
null,
false
@ -157,7 +157,7 @@ public class TimeFormatExtractionFnTest
TimeFormatExtractionFn fn2 = new TimeFormatExtractionFn(
"'In Berlin ist es schon 'EEEE",
DateTimeZone.forID("Europe/Berlin"),
DateTimes.inferTzfromString("Europe/Berlin"),
"de",
null,
true
@ -165,7 +165,7 @@ public class TimeFormatExtractionFnTest
TimeFormatExtractionFn fn3 = new TimeFormatExtractionFn(
"'In Berlin ist es schon 'EEEE",
DateTimeZone.forID("Europe/Berlin"),
DateTimes.inferTzfromString("Europe/Berlin"),
"de",
null,
true

View File

@ -2304,7 +2304,7 @@ public class GroupByQueryRunnerTest
@Test
public void testGroupByWithTimeZone()
{
DateTimeZone tz = DateTimeZone.forID("America/Los_Angeles");
DateTimeZone tz = DateTimes.inferTzfromString("America/Los_Angeles");
GroupByQuery query = GroupByQuery.builder()
.setDataSource(QueryRunnerTestHelper.dataSource)

View File

@ -454,7 +454,7 @@ public class TimeseriesQueryRunnerTest
new PeriodGranularity(
new Period("P1D"),
null,
DateTimeZone.forID("America/Los_Angeles")
DateTimes.inferTzfromString("America/Los_Angeles")
)
)
.descending(descending)
@ -462,13 +462,13 @@ public class TimeseriesQueryRunnerTest
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
new Result<>(
new DateTime("2011-03-31", DateTimeZone.forID("America/Los_Angeles")),
new DateTime("2011-03-31", DateTimes.inferTzfromString("America/Los_Angeles")),
new TimeseriesResultValue(
ImmutableMap.<String, Object>of("rows", 13L, "idx", 6619L)
)
),
new Result<>(
new DateTime("2011-04-01T", DateTimeZone.forID("America/Los_Angeles")),
new DateTime("2011-04-01T", DateTimes.inferTzfromString("America/Los_Angeles")),
new TimeseriesResultValue(
ImmutableMap.<String, Object>of("rows", 13L, "idx", 5827L)
)
@ -559,7 +559,7 @@ public class TimeseriesQueryRunnerTest
new PeriodGranularity(
new Period("P7D"),
null,
DateTimeZone.forID("America/Los_Angeles")
DateTimes.inferTzfromString("America/Los_Angeles")
)
)
.intervals(
@ -581,13 +581,13 @@ public class TimeseriesQueryRunnerTest
List<Result<TimeseriesResultValue>> expectedResults1 = Arrays.asList(
new Result<>(
new DateTime("2011-01-06T00:00:00.000-08:00", DateTimeZone.forID("America/Los_Angeles")),
new DateTime("2011-01-06T00:00:00.000-08:00", DateTimes.inferTzfromString("America/Los_Angeles")),
new TimeseriesResultValue(
ImmutableMap.<String, Object>of("rows", 13L, "idx", 6071L)
)
),
new Result<>(
new DateTime("2011-01-13T00:00:00.000-08:00", DateTimeZone.forID("America/Los_Angeles")),
new DateTime("2011-01-13T00:00:00.000-08:00", DateTimes.inferTzfromString("America/Los_Angeles")),
new TimeseriesResultValue(
ImmutableMap.<String, Object>of("rows", 91L, "idx", 33382L)
)

View File

@ -50,7 +50,6 @@ import io.druid.query.search.ContainsSearchQuerySpec;
import io.druid.segment.IndexBuilder;
import io.druid.segment.StorageAdapter;
import io.druid.segment.column.Column;
import org.joda.time.DateTimeZone;
import org.junit.AfterClass;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -225,7 +224,13 @@ public class TimeFilteringTest extends BaseFilterTest
@Test
public void testTimeFilterWithTimeFormatExtractionFn()
{
ExtractionFn exfn = new TimeFormatExtractionFn("EEEE", DateTimeZone.forID("America/New_York"), "en", null, false);
ExtractionFn exfn = new TimeFormatExtractionFn(
"EEEE",
DateTimes.inferTzfromString("America/New_York"),
"en",
null,
false
);
assertFilterMatches(
new SelectorDimFilter(Column.TIME_COLUMN_NAME, "Wednesday", exfn),
ImmutableList.<String>of("0", "1", "2", "3", "4", "5")

View File

@ -255,7 +255,7 @@ public class CachingClusteredClientTest
private static final DimFilter DIM_FILTER = null;
private static final List<PostAggregator> RENAMED_POST_AGGS = ImmutableList.of();
private static final Granularity GRANULARITY = Granularities.DAY;
private static final DateTimeZone TIMEZONE = DateTimeZone.forID("America/Los_Angeles");
private static final DateTimeZone TIMEZONE = DateTimes.inferTzfromString("America/Los_Angeles");
private static final Granularity PT1H_TZ_GRANULARITY = new PeriodGranularity(new Period("PT1H"), null, TIMEZONE);
private static final String TOP_DIM = "a_dim";

View File

@ -28,7 +28,6 @@ import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.granularity.PeriodGranularity;
import org.joda.time.DateTimeZone;
import org.joda.time.Interval;
import org.joda.time.Period;
import org.joda.time.chrono.ISOChronology;
@ -240,7 +239,7 @@ public class UniformGranularityTest
public void testPeriodSegmentGranularity()
{
final GranularitySpec spec = new UniformGranularitySpec(
new PeriodGranularity(new Period("P1D"), null, DateTimeZone.forID("America/Los_Angeles")),
new PeriodGranularity(new Period("P1D"), null, DateTimes.inferTzfromString("America/Los_Angeles")),
null,
Lists.newArrayList(
Intervals.of("2012-01-08T00-08:00/2012-01-11T00-08:00"),
@ -260,7 +259,7 @@ public class UniformGranularityTest
actualIntervals.add(interval.toDurationMillis());
}
final ISOChronology chrono = ISOChronology.getInstance(DateTimeZone.forID("America/Los_Angeles"));
final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzfromString("America/Los_Angeles"));
final ArrayList<Long> expectedIntervals = Lists.newArrayList(
new Interval("2012-01-01/2012-01-02", chrono).toDurationMillis(),

View File

@ -20,6 +20,7 @@
package io.druid.sql.calcite.expression.builtin;
import com.google.common.collect.ImmutableList;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.StringUtils;
import io.druid.query.expression.TimestampExtractExprMacro;
import io.druid.sql.calcite.expression.DruidExpression;
@ -88,7 +89,7 @@ public class TimeExtractOperatorConversion implements SqlOperatorConversion
);
final DateTimeZone timeZone = call.getOperands().size() > 2 && !RexLiteral.isNullLiteral(call.getOperands().get(2))
? DateTimeZone.forID(RexLiteral.stringValue(call.getOperands().get(2)))
? DateTimes.inferTzfromString(RexLiteral.stringValue(call.getOperands().get(2)))
: plannerContext.getTimeZone();
return applyTimeExtract(timeExpression, unit, timeZone);

View File

@ -21,6 +21,7 @@ package io.druid.sql.calcite.expression.builtin;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.granularity.PeriodGranularity;
import io.druid.math.expr.ExprMacroTable;
import io.druid.query.expression.TimestampFloorExprMacro;
@ -140,7 +141,7 @@ public class TimeFloorOperatorConversion implements SqlOperatorConversion
: null;
final DateTimeZone timeZone =
operands.size() > 3 && !RexLiteral.isNullLiteral(operands.get(3))
? DateTimeZone.forID(RexLiteral.stringValue(operands.get(3)))
? DateTimes.inferTzfromString(RexLiteral.stringValue(operands.get(3)))
: plannerContext.getTimeZone();
final PeriodGranularity granularity = new PeriodGranularity(period, origin, timeZone);
return applyTimestampFloor(druidExpressions.get(0), granularity, plannerContext.getExprMacroTable());

View File

@ -20,6 +20,7 @@
package io.druid.sql.calcite.expression.builtin;
import com.google.common.collect.ImmutableList;
import io.druid.java.util.common.DateTimes;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.expression.OperatorConversions;
@ -72,7 +73,7 @@ public class TimeFormatOperatorConversion implements SqlOperatorConversion
? RexLiteral.stringValue(call.getOperands().get(1))
: "yyyy-MM-dd'T'HH:mm:ss.SSSZZ";
final DateTimeZone timeZone = call.getOperands().size() > 2 && !RexLiteral.isNullLiteral(call.getOperands().get(2))
? DateTimeZone.forID(RexLiteral.stringValue(call.getOperands().get(2)))
? DateTimes.inferTzfromString(RexLiteral.stringValue(call.getOperands().get(2)))
: plannerContext.getTimeZone();
return DruidExpression.fromFunctionCall(

View File

@ -22,6 +22,7 @@ package io.druid.sql.calcite.planner;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import io.druid.java.util.common.DateTimes;
import io.druid.math.expr.ExprMacroTable;
import io.druid.server.security.AuthenticationResult;
import io.druid.server.security.AuthorizerMapper;
@ -98,7 +99,7 @@ public class PlannerContext
}
if (tzParam != null) {
timeZone = DateTimeZone.forID(String.valueOf(tzParam));
timeZone = DateTimes.inferTzfromString(String.valueOf(tzParam));
} else {
timeZone = DateTimeZone.UTC;
}

View File

@ -256,7 +256,7 @@ public class DruidAvaticaHandlerTest
"SELECT __time, CAST(__time AS DATE) AS t2 FROM druid.foo LIMIT 1"
);
final DateTimeZone timeZone = DateTimeZone.forID("America/Los_Angeles");
final DateTimeZone timeZone = DateTimes.inferTzfromString("America/Los_Angeles");
final DateTime localDateTime = new DateTime("2000-01-01T00Z", timeZone);
final List<Map<String, Object>> resultRows = getRows(resultSet);

View File

@ -5418,7 +5418,7 @@ public class CalciteQueryTest
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(QSS(Filtration.eternity()))
.granularity(new PeriodGranularity(Period.months(1), null, DateTimeZone.forID(LOS_ANGELES)))
.granularity(new PeriodGranularity(Period.months(1), null, DateTimes.inferTzfromString(LOS_ANGELES)))
.aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
.context(TIMESERIES_CONTEXT_LOS_ANGELES)
.build()
@ -5602,7 +5602,7 @@ public class CalciteQueryTest
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(QSS(Filtration.eternity()))
.granularity(new PeriodGranularity(Period.months(1), null, DateTimeZone.forID(LOS_ANGELES)))
.granularity(new PeriodGranularity(Period.months(1), null, DateTimes.inferTzfromString(LOS_ANGELES)))
.aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
@ -5633,7 +5633,7 @@ public class CalciteQueryTest
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(QSS(Filtration.eternity()))
.granularity(new PeriodGranularity(Period.months(1), null, DateTimeZone.forID(LOS_ANGELES)))
.granularity(new PeriodGranularity(Period.months(1), null, DateTimes.inferTzfromString(LOS_ANGELES)))
.aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
.context(TIMESERIES_CONTEXT_LOS_ANGELES)
.build()
@ -6507,7 +6507,7 @@ public class CalciteQueryTest
// Generate timestamps for expected results
private static long T(final String timeString, final String timeZoneString)
{
final DateTimeZone timeZone = DateTimeZone.forID(timeZoneString);
final DateTimeZone timeZone = DateTimes.inferTzfromString(timeZoneString);
return Calcites.jodaToCalciteTimestamp(new DateTime(timeString, timeZone), timeZone);
}

View File

@ -65,7 +65,7 @@ import java.util.Map;
public class ExpressionsTest
{
private static final DateTimeZone LOS_ANGELES = DateTimeZone.forID("America/Los_Angeles");
private static final DateTimeZone LOS_ANGELES = DateTimes.inferTzfromString("America/Los_Angeles");
private final PlannerContext plannerContext = PlannerContext.create(
CalciteTests.createOperatorTable(),