Skip to content

Commit 484bc09

Browse files
authored
Fix DateTimeUtils init order bug
1 parent 0ecaf37 commit 484bc09

File tree

10 files changed

+89
-41
lines changed

10 files changed

+89
-41
lines changed

integration-test/src/test/java/org/apache/iotdb/relational/it/query/old/builtinfunction/scalar/IoTDBCastFunctionTableIT.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1338,6 +1338,7 @@ public void testTryCastWithTextSource() {
13381338

13391339
@Test
13401340
public void testDateOutOfRange() {
1341+
DateTimeUtils.initTimestampPrecision();
13411342
tableAssertTestFail(
13421343
String.format(
13431344
"select CAST((s1 + %d) AS TIMESTAMP) from dateType where time = 1",

integration-test/src/test/java/org/apache/iotdb/relational/it/query/old/builtinfunction/scalar/IoTDBFormatFunctionTableIT.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -145,6 +145,7 @@ public void testNumberFormat() {
145145

146146
@Test
147147
public void testTimestampFormat() {
148+
DateTimeUtils.initTimestampPrecision();
148149
tableResultSetEqualTest(
149150
"SELECT FORMAT('%1$tF %1$tT', s1) FROM timestamp_table",
150151
new String[] {"_col0"},

integration-test/src/test/java/org/apache/iotdb/relational/it/query/old/query/IoTDBArithmeticTableIT.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -347,6 +347,7 @@ public void testDivisionByZero() {
347347

348348
@Test
349349
public void testDateOutOfRange() {
350+
DateTimeUtils.initTimestampPrecision();
350351
tableAssertTestFail(
351352
String.format(
352353
"select date + %s from table2 where time = 1",

integration-test/src/test/java/org/apache/iotdb/relational/it/query/view/old/query/IoTDBArithmeticTableViewIT.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -362,6 +362,7 @@ public void testDivisionByZero() {
362362

363363
@Test
364364
public void testDateOutOfRange() {
365+
DateTimeUtils.initTimestampPrecision();
365366
tableAssertTestFail(
366367
String.format(
367368
"select date + %s from table2 where time = 1",

iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanGraphPrinter.java

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020
package org.apache.iotdb.db.queryengine.plan.planner.plan.node;
2121

2222
import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
23+
import org.apache.iotdb.commons.conf.CommonDescriptor;
2324
import org.apache.iotdb.commons.partition.DataPartition;
2425
import org.apache.iotdb.commons.path.PartialPath;
2526
import org.apache.iotdb.db.queryengine.plan.analyze.TemplatedInfo;
@@ -99,7 +100,6 @@
99100
import java.util.Optional;
100101

101102
import static com.google.common.base.Preconditions.checkArgument;
102-
import static org.apache.iotdb.db.utils.DateTimeUtils.TIMESTAMP_PRECISION;
103103

104104
public class PlanGraphPrinter extends PlanVisitor<List<String>, PlanGraphPrinter.GraphContext> {
105105

@@ -837,7 +837,10 @@ public List<String> visitGapFill(GapFillNode node, GraphContext context) {
837837
if (node.getMonthDuration() != 0) {
838838
boxValue.add(String.format("Interval: %smo", node.getMonthDuration()));
839839
} else {
840-
boxValue.add(String.format("Interval: %s" + TIMESTAMP_PRECISION, node.getNonMonthDuration()));
840+
boxValue.add(
841+
String.format(
842+
"Interval: %s" + CommonDescriptor.getInstance().getConfig().getTimestampPrecision(),
843+
node.getNonMonthDuration()));
841844
}
842845
boxValue.add(String.format("GapFillColumn: %s", node.getGapFillColumn()));
843846
if (!node.getGapFillGroupingKeys().isEmpty()) {

iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/scalar/DateBinFunctionColumnTransformer.java

Lines changed: 13 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919

2020
package org.apache.iotdb.db.queryengine.transformation.dag.column.unary.scalar;
2121

22+
import org.apache.iotdb.commons.conf.CommonDescriptor;
2223
import org.apache.iotdb.db.queryengine.transformation.dag.column.ColumnTransformer;
2324
import org.apache.iotdb.db.queryengine.transformation.dag.column.unary.UnaryColumnTransformer;
2425

@@ -32,8 +33,6 @@
3233
import java.time.temporal.ChronoUnit;
3334
import java.util.concurrent.TimeUnit;
3435

35-
import static org.apache.iotdb.db.utils.DateTimeUtils.TIMESTAMP_PRECISION;
36-
3736
public class DateBinFunctionColumnTransformer extends UnaryColumnTransformer {
3837

3938
private static final long NANOSECONDS_IN_MILLISECOND = 1_000_000;
@@ -62,7 +61,7 @@ public DateBinFunctionColumnTransformer(
6261
private static LocalDateTime convertToLocalDateTime(long timestamp, ZoneId zoneId) {
6362
Instant instant;
6463

65-
switch (TIMESTAMP_PRECISION) {
64+
switch (CommonDescriptor.getInstance().getConfig().getTimestampPrecision()) {
6665
case "ms":
6766
instant = Instant.ofEpochMilli(timestamp);
6867
break;
@@ -77,7 +76,9 @@ private static LocalDateTime convertToLocalDateTime(long timestamp, ZoneId zoneI
7776
TimeUnit.NANOSECONDS.toSeconds(timestamp), timestamp % 1_000_000_000);
7877
break;
7978
default:
80-
throw new IllegalArgumentException("Unsupported precision: " + TIMESTAMP_PRECISION);
79+
throw new IllegalArgumentException(
80+
"Unsupported precision: "
81+
+ CommonDescriptor.getInstance().getConfig().getTimestampPrecision());
8182
}
8283

8384
return LocalDateTime.ofInstant(instant, zoneId);
@@ -89,7 +90,7 @@ private static long convertToTimestamp(LocalDateTime dateTime, ZoneId zoneId) {
8990
// Get the nanoseconds section
9091
long nanoAdjustment = dateTime.getNano();
9192

92-
switch (TIMESTAMP_PRECISION) {
93+
switch (CommonDescriptor.getInstance().getConfig().getTimestampPrecision()) {
9394
case "ms":
9495
return epochMilliSecond + nanoAdjustment / NANOSECONDS_IN_MILLISECOND;
9596
case "us":
@@ -98,20 +99,24 @@ private static long convertToTimestamp(LocalDateTime dateTime, ZoneId zoneId) {
9899
case "ns":
99100
return TimeUnit.MILLISECONDS.toNanos(epochMilliSecond) + nanoAdjustment;
100101
default:
101-
throw new IllegalArgumentException("Unknown precision: " + TIMESTAMP_PRECISION);
102+
throw new IllegalArgumentException(
103+
"Unknown precision: "
104+
+ CommonDescriptor.getInstance().getConfig().getTimestampPrecision());
102105
}
103106
}
104107

105108
private static long getNanoTimeStamp(long timestamp) {
106-
switch (TIMESTAMP_PRECISION) {
109+
switch (CommonDescriptor.getInstance().getConfig().getTimestampPrecision()) {
107110
case "ms":
108111
return TimeUnit.MILLISECONDS.toNanos(timestamp);
109112
case "us":
110113
return TimeUnit.MICROSECONDS.toNanos(timestamp);
111114
case "ns":
112115
return timestamp;
113116
default:
114-
throw new IllegalArgumentException("Unknown precision: " + TIMESTAMP_PRECISION);
117+
throw new IllegalArgumentException(
118+
"Unknown precision: "
119+
+ CommonDescriptor.getInstance().getConfig().getTimestampPrecision());
115120
}
116121
}
117122

iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/scalar/ExtractTransformer.java

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -31,10 +31,10 @@
3131
import java.util.function.Function;
3232

3333
import static java.time.temporal.ChronoField.ALIGNED_WEEK_OF_YEAR;
34-
import static org.apache.iotdb.db.utils.DateTimeUtils.EXTRACT_TIMESTAMP_MS_PART;
35-
import static org.apache.iotdb.db.utils.DateTimeUtils.EXTRACT_TIMESTAMP_NS_PART;
36-
import static org.apache.iotdb.db.utils.DateTimeUtils.EXTRACT_TIMESTAMP_US_PART;
3734
import static org.apache.iotdb.db.utils.DateTimeUtils.convertToZonedDateTime;
35+
import static org.apache.iotdb.db.utils.DateTimeUtils.getExtractTimestampMsPartFunction;
36+
import static org.apache.iotdb.db.utils.DateTimeUtils.getExtractTimestampNsPartFunction;
37+
import static org.apache.iotdb.db.utils.DateTimeUtils.getExtractTimestampUsPartFunction;
3838

3939
public class ExtractTransformer extends UnaryColumnTransformer {
4040
private final Function<Long, Long> evaluateFunction;
@@ -72,11 +72,11 @@ public static Function<Long, Long> constructEvaluateFunction(Extract.Field field
7272
case SECOND:
7373
return timestamp -> (long) convertToZonedDateTime(timestamp, zoneId).getSecond();
7474
case MS:
75-
return EXTRACT_TIMESTAMP_MS_PART;
75+
return getExtractTimestampMsPartFunction();
7676
case US:
77-
return EXTRACT_TIMESTAMP_US_PART;
77+
return getExtractTimestampUsPartFunction();
7878
case NS:
79-
return EXTRACT_TIMESTAMP_NS_PART;
79+
return getExtractTimestampNsPartFunction();
8080
default:
8181
throw new UnsupportedOperationException("Unexpected extract field: " + field);
8282
}

iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNode.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -137,6 +137,7 @@
137137
import static org.apache.iotdb.commons.conf.IoTDBConstant.DEFAULT_CLUSTER_NAME;
138138
import static org.apache.iotdb.commons.utils.StatusUtils.retrieveExitStatusCode;
139139
import static org.apache.iotdb.db.conf.IoTDBStartCheck.PROPERTIES_FILE_NAME;
140+
import static org.apache.iotdb.db.utils.DateTimeUtils.initTimestampPrecision;
140141

141142
public class DataNode extends ServerCommandLine implements DataNodeMBean {
142143

@@ -392,6 +393,9 @@ private void pullAndCheckSystemConfigurations() throws StartupException {
392393
} catch (Exception e) {
393394
throw new StartupException(e.getMessage());
394395
}
396+
397+
// init
398+
initTimestampPrecision();
395399
long endTime = System.currentTimeMillis();
396400
logger.info(
397401
"Successfully pull system configurations from ConfigNode-leader, which takes {} ms",

iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/DateTimeUtils.java

Lines changed: 50 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -59,12 +59,11 @@ private DateTimeUtils() {
5959
// forbidding instantiation
6060
}
6161

62-
public static final String TIMESTAMP_PRECISION =
63-
CommonDescriptor.getInstance().getConfig().getTimestampPrecision();
62+
private static String timestampPrecision;
6463

6564
public static long correctPrecision(long millis) {
6665
try {
67-
switch (TIMESTAMP_PRECISION) {
66+
switch (timestampPrecision) {
6867
case "us":
6968
case "microsecond":
7069
return Math.multiplyExact(millis, 1_000L);
@@ -80,44 +79,70 @@ public static long correctPrecision(long millis) {
8079
throw new IoTDBRuntimeException(
8180
String.format(
8281
"Timestamp overflow, Millisecond: %s , Timestamp precision: %s",
83-
millis, TIMESTAMP_PRECISION),
82+
millis, timestampPrecision),
8483
NUMERIC_VALUE_OUT_OF_RANGE.getStatusCode(),
8584
true);
8685
}
8786
}
8887

89-
public static final Function<Long, Long> CAST_TIMESTAMP_TO_MS;
90-
public static final Function<Long, Long> EXTRACT_TIMESTAMP_MS_PART;
91-
public static final Function<Long, Long> EXTRACT_TIMESTAMP_US_PART;
92-
public static final Function<Long, Long> EXTRACT_TIMESTAMP_NS_PART;
88+
private static Function<Long, Long> castTimestampToMs;
89+
private static Function<Long, Long> extractTimestampMsPart;
90+
private static Function<Long, Long> extractTimestampUsPart;
91+
private static Function<Long, Long> extractTimestampNsPart;
9392

94-
static {
95-
switch (CommonDescriptor.getInstance().getConfig().getTimestampPrecision()) {
93+
private static void updateFunction() {
94+
switch (timestampPrecision) {
9695
case "us":
9796
case "microsecond":
98-
CAST_TIMESTAMP_TO_MS = timestamp -> timestamp / 1000;
99-
EXTRACT_TIMESTAMP_MS_PART = timestamp -> Math.floorMod(timestamp, 1000_000L) / 1000;
100-
EXTRACT_TIMESTAMP_US_PART = timestamp -> Math.floorMod(timestamp, 1000L);
101-
EXTRACT_TIMESTAMP_NS_PART = timestamp -> 0L;
97+
castTimestampToMs = timestamp -> timestamp / 1000;
98+
extractTimestampMsPart = timestamp -> Math.floorMod(timestamp, 1000_000L) / 1000;
99+
extractTimestampUsPart = timestamp -> Math.floorMod(timestamp, 1000L);
100+
extractTimestampNsPart = timestamp -> 0L;
102101
break;
103102
case "ns":
104103
case "nanosecond":
105-
CAST_TIMESTAMP_TO_MS = timestamp -> timestamp / 1000000;
106-
EXTRACT_TIMESTAMP_MS_PART = timestamp -> Math.floorMod(timestamp, 1000_000_000L) / 1000_000;
107-
EXTRACT_TIMESTAMP_US_PART = timestamp -> Math.floorMod(timestamp, 1000_000L) / 1000;
108-
EXTRACT_TIMESTAMP_NS_PART = timestamp -> Math.floorMod(timestamp, 1000L);
104+
castTimestampToMs = timestamp -> timestamp / 1000000;
105+
extractTimestampMsPart = timestamp -> Math.floorMod(timestamp, 1000_000_000L) / 1000_000;
106+
extractTimestampUsPart = timestamp -> Math.floorMod(timestamp, 1000_000L) / 1000;
107+
extractTimestampNsPart = timestamp -> Math.floorMod(timestamp, 1000L);
109108
break;
110109
case "ms":
111110
case "millisecond":
112111
default:
113-
CAST_TIMESTAMP_TO_MS = timestamp -> timestamp;
114-
EXTRACT_TIMESTAMP_MS_PART = timestamp -> Math.floorMod(timestamp, 1000L);
115-
EXTRACT_TIMESTAMP_US_PART = timestamp -> 0L;
116-
EXTRACT_TIMESTAMP_NS_PART = timestamp -> 0L;
112+
castTimestampToMs = timestamp -> timestamp;
113+
extractTimestampMsPart = timestamp -> Math.floorMod(timestamp, 1000L);
114+
extractTimestampUsPart = timestamp -> 0L;
115+
extractTimestampNsPart = timestamp -> 0L;
117116
break;
118117
}
119118
}
120119

120+
public static Function<Long, Long> getExtractTimestampMsPartFunction() {
121+
if (extractTimestampMsPart == null) {
122+
throw new IllegalArgumentException("ExtractTimestampMsPart is null");
123+
}
124+
return extractTimestampMsPart;
125+
}
126+
127+
public static Function<Long, Long> getExtractTimestampUsPartFunction() {
128+
if (extractTimestampUsPart == null) {
129+
throw new IllegalArgumentException("ExtractTimestampUsPart is null");
130+
}
131+
return extractTimestampUsPart;
132+
}
133+
134+
public static Function<Long, Long> getExtractTimestampNsPartFunction() {
135+
if (extractTimestampNsPart == null) {
136+
throw new IllegalArgumentException("ExtractTimestampNsPart is null");
137+
}
138+
return extractTimestampNsPart;
139+
}
140+
141+
public static void initTimestampPrecision() {
142+
timestampPrecision = CommonDescriptor.getInstance().getConfig().getTimestampPrecision();
143+
updateFunction();
144+
}
145+
121146
private static final DateTimeFormatter DATE_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd");
122147
public static final DateTimeFormatter ISO_LOCAL_DATE_WIDTH_1_2;
123148

@@ -787,12 +812,12 @@ public static String convertLongToDate(
787812
}
788813

789814
public static LocalDate convertToLocalDate(long timestamp, ZoneId zoneId) {
790-
timestamp = CAST_TIMESTAMP_TO_MS.apply(timestamp);
815+
timestamp = castTimestampToMs.apply(timestamp);
791816
return Instant.ofEpochMilli(timestamp).atZone(zoneId).toLocalDate();
792817
}
793818

794819
public static ZonedDateTime convertToZonedDateTime(long timestamp, ZoneId zoneId) {
795-
timestamp = CAST_TIMESTAMP_TO_MS.apply(timestamp);
820+
timestamp = castTimestampToMs.apply(timestamp);
796821
return ZonedDateTime.ofInstant(Instant.ofEpochMilli(timestamp), zoneId);
797822
}
798823

@@ -924,7 +949,7 @@ public static Long parseDateTimeExpressionToLong(String dateExpression, ZoneId z
924949
parser1.getInterpreter().setPredictionMode(PredictionMode.SLL);
925950
parser1.removeErrorListeners();
926951
parser1.addErrorListener(SqlParseError.INSTANCE);
927-
return astVisitor.parseDateExpression(parser1.dateExpression(), TIMESTAMP_PRECISION);
952+
return astVisitor.parseDateExpression(parser1.dateExpression(), timestampPrecision);
928953
}
929954

930955
public static Integer parseDateExpressionToInt(String dateExpression) {

iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/ExtractExpressionTest.java

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LongLiteral;
2424

2525
import com.google.common.collect.ImmutableMap;
26+
import org.junit.BeforeClass;
2627
import org.junit.Test;
2728

2829
import static org.apache.iotdb.db.queryengine.plan.relational.analyzer.TestUtils.assertAnalyzeSemanticException;
@@ -31,8 +32,14 @@
3132
import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.output;
3233
import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.project;
3334
import static org.apache.iotdb.db.queryengine.plan.relational.planner.assertions.PlanMatchPattern.tableScan;
35+
import static org.apache.iotdb.db.utils.DateTimeUtils.initTimestampPrecision;
3436

3537
public class ExtractExpressionTest {
38+
@BeforeClass
39+
public static void setUp() {
40+
initTimestampPrecision();
41+
}
42+
3643
@Test
3744
public void pushDownTest() {
3845
PlanTester planTester = new PlanTester();

0 commit comments

Comments
 (0)