From f0192dceaefe60467a54fc3129e62b6092be0df3 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Tue, 5 Nov 2019 16:25:27 +0800 Subject: [PATCH 01/29] [FLINK-14599][table-planner-blink] Add setTimestamp/getTimestamp interface to TypeGetterSetters/VectorizedColumnBatch and writeTimestamp interface to BinaryWriter --- .../dataformat/AbstractBinaryWriter.java | 25 +++++++++++ .../flink/table/dataformat/BinaryArray.java | 43 +++++++++++++++++++ .../table/dataformat/BinaryArrayWriter.java | 1 - .../flink/table/dataformat/BinaryRow.java | 43 +++++++++++++++++++ .../flink/table/dataformat/BinaryWriter.java | 8 +++- .../flink/table/dataformat/ColumnarRow.java | 10 +++++ .../flink/table/dataformat/GenericArray.java | 10 +++++ .../flink/table/dataformat/JoinedRow.java | 18 ++++++++ .../flink/table/dataformat/NestedRow.java | 42 ++++++++++++++++++ .../table/dataformat/ObjectArrayRow.java | 10 +++++ .../table/dataformat/TypeGetterSetters.java | 20 ++++++++- .../flink/table/dataformat/UpdatableRow.java | 10 +++++ .../vector/VectorizedColumnBatch.java | 25 +++++++++++ .../flink/table/dataformat/BaseRowTest.java | 33 +++++++++++--- .../table/dataformat/BinaryArrayTest.java | 39 +++++++++++++++++ .../flink/table/dataformat/BinaryRowTest.java | 35 +++++++++++++++ 16 files changed, 363 insertions(+), 9 deletions(-) diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/AbstractBinaryWriter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/AbstractBinaryWriter.java index 6499528de45c9..52a0c69f8f56f 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/AbstractBinaryWriter.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/AbstractBinaryWriter.java @@ -181,6 +181,31 @@ public void writeDecimal(int pos, Decimal value, int precision) { } } + @Override + public void writeTimestamp(int pos, SqlTimestamp value, int precision) { + if (SqlTimestamp.isCompact(precision)) { + assert 0 == value.getNanoOfMillisecond(); + writeLong(pos, value.getMillisecond()); + } else { + ensureCapacity(12); + + // zero-out the bytes + segment.putLong(cursor, 0L); + segment.putInt(cursor + 8, 0); + + if (value == null) { + setNullBit(pos); + setOffsetAndSize(pos, cursor, 0); + } else { + segment.putLong(cursor, value.getMillisecond()); + segment.putInt(cursor + 8, value.getNanoOfMillisecond()); + setOffsetAndSize(pos, cursor, 12); + } + + cursor += 12; + } + } + private void zeroBytes(int offset, int size) { for (int i = offset; i < offset + size; i++) { segment.put(i, (byte) 0); diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java index 523d96aced0d4..8f32827257659 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java @@ -191,6 +191,22 @@ public Decimal getDecimal(int pos, int precision, int scale) { return Decimal.readDecimalFieldFromSegments(segments, offset, offsetAndSize, precision, scale); } + @Override + public SqlTimestamp getTimestamp(int pos, int precision) { + assertIndexIsValid(pos); + + if (SqlTimestamp.isCompact(precision)) { + return SqlTimestamp.fromEpochMillis(segments[0].getLong(getElementOffset(pos, 8))); + } + + int fieldOffset = getElementOffset(pos, 8); + final long offsetAndSize = SegmentsUtil.getLong(segments, fieldOffset); + final int subOffset = (int) (offsetAndSize >> 32); + final long millisecond = SegmentsUtil.getLong(segments, offset + subOffset); + final int nanoOfMillisecond = SegmentsUtil.getInt(segments, offset + subOffset + 8); + return SqlTimestamp.fromEpochMillis(millisecond, nanoOfMillisecond); + } + @Override public BinaryGeneric getGeneric(int pos) { assertIndexIsValid(pos); @@ -361,6 +377,33 @@ public void setDecimal(int pos, Decimal value, int precision) { } } + @Override + public void setTimestamp(int pos, SqlTimestamp value, int precision) { + assertIndexIsValid(pos); + + if (SqlTimestamp.isCompact(precision)) { + setLong(pos, value.getMillisecond()); + } else { + int fieldOffset = getElementOffset(pos, 8); + int cursor = (int) (SegmentsUtil.getLong(segments, fieldOffset) >>> 32); + assert cursor > 0 : "invalid cursor " + cursor; + + // zero-out the bytes + SegmentsUtil.setLong(segments, offset + cursor, 0L); + SegmentsUtil.setInt(segments, offset + cursor + 8, 0); + + if (value == null) { + setNullAt(pos); + SegmentsUtil.setLong(segments, fieldOffset, ((long) cursor) << 32); + } else { + // write millisecond and nanoOfMillisecond to the variable length portion. + SegmentsUtil.setLong(segments, offset + cursor, value.getMillisecond()); + SegmentsUtil.setInt(segments, offset + cursor + 8, value.getNanoOfMillisecond()); + setLong(pos, ((long) cursor << 32) | 12L); + } + } + } + public boolean anyNull() { for (int i = offset + 4; i < elementOffset; i += 4) { if (SegmentsUtil.getInt(segments, i) != 0) { diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArrayWriter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArrayWriter.java index ad97e3533ade2..9686340bcbbce 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArrayWriter.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArrayWriter.java @@ -125,7 +125,6 @@ public void setNullAt(int pos, LogicalType type) { setNullInt(pos); break; case BIGINT: - case TIMESTAMP_WITHOUT_TIME_ZONE: case TIMESTAMP_WITH_LOCAL_TIME_ZONE: case INTERVAL_DAY_TIME: setNullLong(pos); diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRow.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRow.java index 75f7e24cb0887..e5b22de424e50 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRow.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRow.java @@ -212,6 +212,33 @@ public void setDecimal(int pos, Decimal value, int precision) { } } + @Override + public void setTimestamp(int pos, SqlTimestamp value, int precision) { + assertIndexIsValid(pos); + + if (SqlTimestamp.isCompact(precision)) { + setLong(pos, value.getMillisecond()); + } else { + int fieldOffset = getFieldOffset(pos); + int cursor = (int) (segments[0].getLong(fieldOffset) >>> 32); + assert cursor > 0 : "invalid cursor " + cursor; + + // zero-out the bytes + SegmentsUtil.setLong(segments, offset + cursor, 0L); + SegmentsUtil.setInt(segments, offset + cursor + 8, 0); + + if (value == null) { + setNullAt(pos); + segments[0].putLong(fieldOffset, ((long) cursor) << 32); + } else { + // write millisecond and nanoOfMillisecond to the variable length portion. + SegmentsUtil.setLong(segments, offset + cursor, value.getMillisecond()); + SegmentsUtil.setInt(segments, offset + cursor + 8, value.getNanoOfMillisecond()); + setLong(pos, ((long) cursor << 32) | 12L); + } + } + } + @Override public void setBoolean(int pos, boolean value) { assertIndexIsValid(pos); @@ -304,6 +331,22 @@ public Decimal getDecimal(int pos, int precision, int scale) { return Decimal.readDecimalFieldFromSegments(segments, offset, offsetAndSize, precision, scale); } + @Override + public SqlTimestamp getTimestamp(int pos, int precision) { + assertIndexIsValid(pos); + + if (SqlTimestamp.isCompact(precision)) { + return SqlTimestamp.fromEpochMillis(segments[0].getLong(getFieldOffset(pos))); + } + + int fieldOffset = getFieldOffset(pos); + final long offsetAndSize = segments[0].getLong(fieldOffset); + final int subOffset = (int) (offsetAndSize >> 32); + final long millisecond = SegmentsUtil.getLong(segments, offset + subOffset); + final int nanoOfMillisecond = SegmentsUtil.getInt(segments, offset + subOffset + 8); + return SqlTimestamp.fromEpochMillis(millisecond, nanoOfMillisecond); + } + @Override public BinaryGeneric getGeneric(int pos) { assertIndexIsValid(pos); diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryWriter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryWriter.java index 3a559b7768fed..c1f2da3f92fcd 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryWriter.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryWriter.java @@ -24,6 +24,7 @@ import org.apache.flink.table.runtime.typeutils.BinaryGenericSerializer; import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.TimestampType; /** * Writer to write a composite data format, like row, array. @@ -63,6 +64,8 @@ public interface BinaryWriter { void writeDecimal(int pos, Decimal value, int precision); + void writeTimestamp(int pos, SqlTimestamp value, int precision); + void writeArray(int pos, BaseArray value, BaseArraySerializer serializer); void writeMap(int pos, BaseMap value, BaseMapSerializer serializer); @@ -94,8 +97,11 @@ static void write(BinaryWriter writer, int pos, case INTERVAL_YEAR_MONTH: writer.writeInt(pos, (int) o); break; - case BIGINT: case TIMESTAMP_WITHOUT_TIME_ZONE: + TimestampType timestampType = (TimestampType) type; + writer.writeTimestamp(pos, (SqlTimestamp) o, timestampType.getPrecision()); + break; + case BIGINT: case TIMESTAMP_WITH_LOCAL_TIME_ZONE: case INTERVAL_DAY_TIME: writer.writeLong(pos, (long) o); diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/ColumnarRow.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/ColumnarRow.java index 3fed637e5ab32..d536521878552 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/ColumnarRow.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/ColumnarRow.java @@ -116,6 +116,11 @@ public Decimal getDecimal(int ordinal, int precision, int scale) { return vectorizedColumnBatch.getDecimal(rowId, ordinal, precision, scale); } + @Override + public SqlTimestamp getTimestamp(int ordinal, int precision) { + return vectorizedColumnBatch.getTimestamp(rowId, ordinal, precision); + } + @Override public BinaryGeneric getGeneric(int pos) { throw new UnsupportedOperationException("GenericType is not supported."); @@ -196,6 +201,11 @@ public void setDecimal(int ordinal, Decimal value, int precision) { throw new UnsupportedOperationException("Not support the operation!"); } + @Override + public void setTimestamp(int ordinal, SqlTimestamp value, int precision) { + throw new UnsupportedOperationException("Not support the operation!"); + } + @Override public boolean equals(Object o) { throw new UnsupportedOperationException( diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/GenericArray.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/GenericArray.java index 94077be599676..5b05ffc229f5d 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/GenericArray.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/GenericArray.java @@ -197,6 +197,11 @@ public Decimal getDecimal(int pos, int precision, int scale) { return (Decimal) getObject(pos); } + @Override + public SqlTimestamp getTimestamp(int pos, int precision) { + return (SqlTimestamp) getObject(pos); + } + @Override public BinaryGeneric getGeneric(int pos) { return (BinaryGeneric) getObject(pos); @@ -285,6 +290,11 @@ public void setDecimal(int pos, Decimal value, int precision) { setObject(pos, value); } + @Override + public void setTimestamp(int pos, SqlTimestamp value, int precision) { + setObject(pos, value); + } + public Object getObject(int pos) { return ((Object[]) arr)[pos]; } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/JoinedRow.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/JoinedRow.java index e3023a8c09310..b739ada9e6400 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/JoinedRow.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/JoinedRow.java @@ -135,6 +135,15 @@ public Decimal getDecimal(int i, int precision, int scale) { } } + @Override + public SqlTimestamp getTimestamp(int i, int precision) { + if (i < row1.getArity()) { + return row1.getTimestamp(i, precision); + } else { + return row2.getTimestamp(i - row1.getArity(), precision); + } + } + @Override public BinaryGeneric getGeneric(int i) { if (i < row1.getArity()) { @@ -270,6 +279,15 @@ public void setDecimal(int i, Decimal value, int precision) { } } + @Override + public void setTimestamp(int i, SqlTimestamp value, int precision) { + if (i < row1.getArity()) { + row1.setTimestamp(i, value, precision); + } else { + row2.setTimestamp(i - row1.getArity(), value, precision); + } + } + @Override public boolean equals(Object o) { throw new UnsupportedOperationException( diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/NestedRow.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/NestedRow.java index f31a138daf7af..1a8f1ccc38fd4 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/NestedRow.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/NestedRow.java @@ -144,6 +144,32 @@ public void setDecimal(int pos, Decimal value, int precision) { } } + @Override + public void setTimestamp(int pos, SqlTimestamp value, int precision) { + assertIndexIsValid(pos); + + if (SqlTimestamp.isCompact(precision)) { + setLong(pos, value.getMillisecond()); + } else { + int fieldOffset = getFieldOffset(pos); + int cursor = (int) (SegmentsUtil.getLong(segments, fieldOffset) >>> 32); + assert cursor > 0 : "invalid cursor " + cursor; + + // zero-out the bytes + SegmentsUtil.setLong(segments, offset + cursor, 0L); + SegmentsUtil.setInt(segments, offset + cursor + 8, 0); + + if (value == null) { + setNullAt(pos); + SegmentsUtil.setLong(segments, fieldOffset, ((long) cursor) << 32); + } else { + SegmentsUtil.setLong(segments, offset + cursor, value.getMillisecond()); + SegmentsUtil.setInt(segments, offset + cursor + 8, value.getNanoOfMillisecond()); + setLong(pos, ((long) cursor << 32) | 12L); + } + } + } + @Override public void setBoolean(int pos, boolean value) { assertIndexIsValid(pos); @@ -242,6 +268,22 @@ public Decimal getDecimal(int pos, int precision, int scale) { return Decimal.readDecimalFieldFromSegments(segments, offset, offsetAndSize, precision, scale); } + @Override + public SqlTimestamp getTimestamp(int pos, int precision) { + assertIndexIsValid(pos); + + if (SqlTimestamp.isCompact(precision)) { + return SqlTimestamp.fromEpochMillis(SegmentsUtil.getLong(segments, getFieldOffset(pos))); + } + + int fieldOffset = getFieldOffset(pos); + final long offsetAndSize = SegmentsUtil.getLong(segments, fieldOffset); + final int subOffset = (int) (offsetAndSize >> 32); + final long millisecond = SegmentsUtil.getLong(segments, offset + subOffset); + final int nanoOfMillisecond = SegmentsUtil.getInt(segments, offset + subOffset + 8); + return SqlTimestamp.fromEpochMillis(millisecond, nanoOfMillisecond); + } + @Override public BinaryGeneric getGeneric(int pos) { assertIndexIsValid(pos); diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/ObjectArrayRow.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/ObjectArrayRow.java index b6f4799727bfd..1c10bc6d3f2a1 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/ObjectArrayRow.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/ObjectArrayRow.java @@ -65,6 +65,11 @@ public void setDecimal(int i, Decimal value, int precision) { this.fields[i] = value; } + @Override + public void setTimestamp(int ordinal, SqlTimestamp value, int precision) { + this.fields[ordinal] = value; + } + @Override public BinaryString getString(int ordinal) { return (BinaryString) this.fields[ordinal]; @@ -90,6 +95,11 @@ public Decimal getDecimal(int ordinal, int precision, int scale) { return (Decimal) this.fields[ordinal]; } + @Override + public SqlTimestamp getTimestamp(int ordinal, int precision) { + return (SqlTimestamp) this.fields[ordinal]; + } + @Override public BinaryGeneric getGeneric(int ordinal) { return (BinaryGeneric) this.fields[ordinal]; diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/TypeGetterSetters.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/TypeGetterSetters.java index c2af1999b456a..a6658bf4e4d1b 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/TypeGetterSetters.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/TypeGetterSetters.java @@ -20,6 +20,7 @@ import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimestampType; /** * Provide type specialized getters and setters to reduce if/else and eliminate box and unbox. @@ -90,6 +91,11 @@ public interface TypeGetterSetters { */ Decimal getDecimal(int ordinal, int precision, int scale); + /** + * Get Timestamp value, internal format is SqlTimestamp. + */ + SqlTimestamp getTimestamp(int ordinal, int precision); + /** * Get generic value, internal format is BinaryGeneric. */ @@ -160,6 +166,16 @@ public interface TypeGetterSetters { */ void setDecimal(int i, Decimal value, int precision); + /** + * Set Timestamp value. + * + *

Note: + * If precision is compact: can call setNullAt when SqlTimestamp value is null. + * Otherwise: can not call setNullAt when SqlTimestamp value is null, must call + * setTimestamp(ordinal, null, precision) because we need to update var-length-part. + */ + void setTimestamp(int ordinal, SqlTimestamp value, int precision); + static Object get(TypeGetterSetters row, int ordinal, LogicalType type) { switch (type.getTypeRoot()) { case BOOLEAN: @@ -173,8 +189,10 @@ static Object get(TypeGetterSetters row, int ordinal, LogicalType type) { case TIME_WITHOUT_TIME_ZONE: case INTERVAL_YEAR_MONTH: return row.getInt(ordinal); - case BIGINT: case TIMESTAMP_WITHOUT_TIME_ZONE: + TimestampType timestampType = (TimestampType) type; + return row.getTimestamp(ordinal, timestampType.getPrecision()); + case BIGINT: case TIMESTAMP_WITH_LOCAL_TIME_ZONE: case INTERVAL_DAY_TIME: return row.getLong(ordinal); diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/UpdatableRow.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/UpdatableRow.java index e67aa38dff2f0..5d916c548261f 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/UpdatableRow.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/UpdatableRow.java @@ -106,6 +106,11 @@ public Decimal getDecimal(int ordinal, int precision, int scale) { return updated[ordinal] ? (Decimal) fields[ordinal] : row.getDecimal(ordinal, precision, scale); } + @Override + public SqlTimestamp getTimestamp(int ordinal, int precision) { + return updated[ordinal] ? (SqlTimestamp) fields[ordinal] : row.getTimestamp(ordinal, precision); + } + @Override public BinaryGeneric getGeneric(int ordinal) { return updated[ordinal] ? (BinaryGeneric) fields[ordinal] : row.getGeneric(ordinal); @@ -171,6 +176,11 @@ public void setDecimal(int ordinal, Decimal value, int precision) { setField(ordinal, value); } + @Override + public void setTimestamp(int ordinal, SqlTimestamp value, int precision) { + setField(ordinal, value); + } + public void setField(int ordinal, Object value) { updated[ordinal] = true; fields[ordinal] = value; diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatch.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatch.java index 122d35916196a..9a27a63dcc9ae 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatch.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatch.java @@ -19,6 +19,7 @@ package org.apache.flink.table.dataformat.vector; import org.apache.flink.table.dataformat.Decimal; +import org.apache.flink.table.dataformat.SqlTimestamp; import org.apache.flink.table.dataformat.vector.BytesColumnVector.Bytes; import java.io.Serializable; @@ -132,4 +133,28 @@ public Decimal getDecimal(int rowId, int colId, int precision, int scale) { return Decimal.fromUnscaledBytes(precision, scale, bytes); } } + + public SqlTimestamp getTimestamp(int rowId, int colId, int precision) { + if (isNullAt(rowId, colId)) { + return null; + } + + if (SqlTimestamp.isCompact(precision)) { + return SqlTimestamp.fromEpochMillis(getLong(rowId, colId)); + } else { + byte[] bytes = getBytes(rowId, colId); + assert bytes.length == 12; + long l = 0; + for (int i = 0; i < 8; i++) { + l <<= 8; + l |= (bytes[i] & (0xff)); + } + int n = 0; + for (int i = 8; i < 12; i++) { + n <<= 8; + n |= (bytes[i] & (0xff)); + } + return SqlTimestamp.fromEpochMillis(l, n); + } + } } diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BaseRowTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BaseRowTest.java index 82019d04b2d9e..afdda414c6050 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BaseRowTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BaseRowTest.java @@ -30,6 +30,8 @@ import org.junit.Test; import java.math.BigDecimal; +import java.sql.Timestamp; +import java.time.LocalDateTime; import static org.apache.flink.table.utils.BinaryGenericAsserter.equivalent; import static org.junit.Assert.assertArrayEquals; @@ -52,6 +54,8 @@ public class BaseRowTest { private BinaryRow underRow; private byte[] bytes; private BinaryGenericSerializer genericSerializer; + private SqlTimestamp sqlTimestamp1; + private SqlTimestamp sqlTimestamp2; @Before public void before() { @@ -76,6 +80,8 @@ public void before() { writer.complete(); } bytes = new byte[] {1, 5, 6}; + sqlTimestamp1 = SqlTimestamp.fromEpochMillis(123L); + sqlTimestamp2 = SqlTimestamp.fromLocalDateTime(LocalDateTime.of(1969, 1, 1, 0, 0, 0, 123456789)); } @Test @@ -89,11 +95,11 @@ public void testNestedRow() { BinaryRowWriter writer = new BinaryRowWriter(row); writer.writeRow(0, getBinaryRow(), null); writer.complete(); - testAll(row.getRow(0, 16)); + testAll(row.getRow(0, 18)); } private BinaryRow getBinaryRow() { - BinaryRow row = new BinaryRow(16); + BinaryRow row = new BinaryRow(18); BinaryRowWriter writer = new BinaryRowWriter(row); writer.writeBoolean(0, true); writer.writeByte(1, (byte) 1); @@ -111,12 +117,14 @@ private BinaryRow getBinaryRow() { DataTypes.INT().getLogicalType(), DataTypes.INT().getLogicalType(), null)); writer.writeRow(14, underRow, new BaseRowSerializer(null, RowType.of(new IntType(), new IntType()))); writer.writeBinary(15, bytes); + writer.writeTimestamp(16, sqlTimestamp1, 3); + writer.writeTimestamp(17, sqlTimestamp2, 9); return row; } @Test public void testGenericRow() { - GenericRow row = new GenericRow(16); + GenericRow row = new GenericRow(18); row.setField(0, true); row.setField(1, (byte) 1); row.setField(2, (short) 2); @@ -133,12 +141,14 @@ public void testGenericRow() { row.setField(13, map); row.setField(14, underRow); row.setField(15, bytes); + row.setField(16, sqlTimestamp1); + row.setField(17, sqlTimestamp2); testAll(row); } @Test public void testBoxedWrapperRow() { - BoxedWrapperRow row = new BoxedWrapperRow(16); + BoxedWrapperRow row = new BoxedWrapperRow(18); row.setBoolean(0, true); row.setByte(1, (byte) 1); row.setShort(2, (short) 2); @@ -154,6 +164,8 @@ public void testBoxedWrapperRow() { row.setNonPrimitiveValue(13, map); row.setNonPrimitiveValue(14, underRow); row.setNonPrimitiveValue(15, bytes); + row.setNonPrimitiveValue(16, sqlTimestamp1); + row.setNonPrimitiveValue(17, sqlTimestamp2); testAll(row); } @@ -166,7 +178,7 @@ public void testJoinedRow() { row1.setField(3, 3); row1.setField(4, (long) 4); - GenericRow row2 = new GenericRow(11); + GenericRow row2 = new GenericRow(13); row2.setField(0, (float) 5); row2.setField(1, (double) 6); row2.setField(2, (char) 7); @@ -178,11 +190,13 @@ public void testJoinedRow() { row2.setField(8, map); row2.setField(9, underRow); row2.setField(10, bytes); + row2.setField(11, sqlTimestamp1); + row2.setField(12, sqlTimestamp2); testAll(new JoinedRow(row1, row2)); } private void testAll(BaseRow row) { - assertEquals(16, row.getArity()); + assertEquals(18, row.getArity()); // test header assertEquals(0, row.getHeader()); @@ -206,6 +220,8 @@ private void testAll(BaseRow row) { assertEquals(15, row.getRow(14, 2).getInt(0)); assertEquals(16, row.getRow(14, 2).getInt(1)); assertArrayEquals(bytes, row.getBinary(15)); + assertEquals(sqlTimestamp1, row.getTimestamp(16, 3)); + assertEquals(sqlTimestamp2, row.getTimestamp(17, 9)); // test set row.setBoolean(0, false); @@ -227,6 +243,11 @@ private void testAll(BaseRow row) { row.setDecimal(11, Decimal.fromBigDecimal(new BigDecimal(12), 20, 0), 20); assertEquals(Decimal.fromBigDecimal(new BigDecimal(12), 20, 0), row.getDecimal(11, 20, 0)); + row.setTimestamp(16, SqlTimestamp.fromEpochMillis(456L), 3); + assertEquals(SqlTimestamp.fromEpochMillis(456L), row.getTimestamp(16, 3)); + row.setTimestamp(17, SqlTimestamp.fromTimestamp(Timestamp.valueOf("1970-01-01 00:00:00.123456789")), 9); + assertEquals(SqlTimestamp.fromTimestamp(Timestamp.valueOf("1970-01-01 00:00:00.123456789")), row.getTimestamp(17, 9)); + // test null assertFalse(row.isNullAt(0)); row.setNullAt(0); diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryArrayTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryArrayTest.java index 9de5cdfa259cc..73c2fb750f094 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryArrayTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryArrayTest.java @@ -35,6 +35,8 @@ import org.junit.Test; import java.math.BigDecimal; +import java.sql.Timestamp; +import java.time.LocalDateTime; import static org.apache.flink.table.dataformat.BinaryString.fromString; import static org.apache.flink.table.utils.BinaryGenericAsserter.equivalent; @@ -502,4 +504,41 @@ public void testBinary() { Assert.assertArrayEquals(bytes1, array.getBinary(0)); Assert.assertArrayEquals(bytes2, array.getBinary(1)); } + + @Test + public void testSqlTimestamp() { + BinaryArray array = new BinaryArray(); + BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); + + // 1. compact + { + final int precision = 3; + writer.reset(); + writer.writeTimestamp(0, SqlTimestamp.fromEpochMillis(123L), precision); + writer.setNullAt(1); + writer.complete(); + + assertEquals("1970-01-01T00:00:00.123", array.getTimestamp(0, 3).toString()); + assertTrue(array.isNullAt(1)); + array.setTimestamp(0, SqlTimestamp.fromEpochMillis(-123L), precision); + assertEquals("1969-12-31T23:59:59.877", array.getTimestamp(0, 3).toString()); + } + + // 2. not compact + { + final int precision = 9; + SqlTimestamp sqlTimestamp1 = SqlTimestamp.fromLocalDateTime(LocalDateTime.of(1970, 1, 1, 0, 0, 0, 123456789)); + SqlTimestamp sqlTimestamp2 = SqlTimestamp.fromTimestamp(Timestamp.valueOf("1969-01-01 00:00:00.123456789")); + + writer.reset(); + writer.writeTimestamp(0, sqlTimestamp1, precision); + writer.writeTimestamp(1, null, precision); + writer.complete(); + + assertEquals("1970-01-01T00:00:00.123456789", array.getTimestamp(0, precision).toString()); + assertTrue(array.isNullAt(1)); + array.setTimestamp(0, sqlTimestamp2, precision); + assertEquals("1969-01-01T00:00:00.123456789", array.getTimestamp(0, precision).toString()); + } + } } diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryRowTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryRowTest.java index 3a90aaa178452..c2a8e0c00dce0 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryRowTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryRowTest.java @@ -1001,4 +1001,39 @@ private void testSerializerPagesInternal(BinaryRow row24, BinaryRow row160) thro assertEquals(row160, rets.get(1)); } } + + @Test + public void testSqlTimestamp() { + // 1. compact + { + final int precision = 3; + BinaryRow row = new BinaryRow(2); + BinaryRowWriter writer = new BinaryRowWriter(row); + writer.writeTimestamp(0, SqlTimestamp.fromEpochMillis(123L), precision); + writer.setNullAt(1); + writer.complete(); + + assertEquals("1970-01-01T00:00:00.123", row.getTimestamp(0, 3).toString()); + assertTrue(row.isNullAt(1)); + row.setTimestamp(0, SqlTimestamp.fromEpochMillis(-123L), precision); + assertEquals("1969-12-31T23:59:59.877", row.getTimestamp(0, 3).toString()); + } + + // 2. not compact + { + final int precision = 9; + SqlTimestamp sqlTimestamp1 = SqlTimestamp.fromLocalDateTime(LocalDateTime.of(1969, 1, 1, 0, 0, 0, 123456789)); + SqlTimestamp sqlTimestamp2 = SqlTimestamp.fromTimestamp(Timestamp.valueOf("1970-01-01 00:00:00.123456789")); + BinaryRow row = new BinaryRow(2); + BinaryRowWriter writer = new BinaryRowWriter(row); + writer.writeTimestamp(0, sqlTimestamp1, precision); + writer.writeTimestamp(1, null, precision); + writer.complete(); + + assertEquals("1969-01-01T00:00:00.123456789", row.getTimestamp(0, precision).toString()); + assertTrue(row.isNullAt(1)); + row.setTimestamp(0, sqlTimestamp2, precision); + assertEquals("1970-01-01T00:00:00.123456789", row.getTimestamp(0, precision).toString()); + } + } } From 90df5e43c9430210877198b5bae89ab9110550b7 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Tue, 5 Nov 2019 17:10:53 +0800 Subject: [PATCH 02/29] [FLINK-14599][table-planner-blink] Use SqlTimestamp as internal representation of Timestamp type --- .../planner/plan/utils/RexNodeExtractor.scala | 3 +- .../runtime/batch/sql/CalcITCase.scala | 2 +- .../planner/utils/DateTimeTestUtil.scala | 6 +-- .../dataformat/DataFormatConverters.java | 40 +++++++++++-------- .../types/ClassLogicalTypeConverter.java | 4 +- .../runtime/types/InternalSerializers.java | 6 ++- 6 files changed, 38 insertions(+), 23 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala index f178eb1154c12..b7a81515dc41a 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala @@ -21,6 +21,7 @@ package org.apache.flink.table.planner.plan.utils import org.apache.flink.table.api.TableException import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, FunctionLookup, UnresolvedIdentifier} import org.apache.flink.table.dataformat.DataFormatConverters.{LocalDateConverter, LocalDateTimeConverter, LocalTimeConverter} +import org.apache.flink.table.dataformat.SqlTimestamp import org.apache.flink.table.expressions._ import org.apache.flink.table.expressions.utils.ApiExpressionUtils._ import org.apache.flink.table.functions.BuiltInFunctionDefinitions.{AND, CAST, OR} @@ -338,7 +339,7 @@ class RexNodeToExpressionConverter( case TIMESTAMP_WITHOUT_TIME_ZONE => val v = literal.getValueAs(classOf[java.lang.Long]) - LocalDateTimeConverter.INSTANCE.toExternal(v) + LocalDateTimeConverter.INSTANCE.toExternal(SqlTimestamp.fromEpochMillis(v)) case TIMESTAMP_WITH_LOCAL_TIME_ZONE => val v = literal.getValueAs(classOf[java.lang.Long]) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala index 3bb462dc29f72..c37fe8297e282 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala @@ -1007,7 +1007,7 @@ class CalcITCase extends BatchTestBase { val table = parseQuery("SELECT CURRENT_TIMESTAMP FROM testTable WHERE a = TRUE") val result = executeQuery(table) val ts1 = LocalDateTimeConverter.INSTANCE.toInternal( - result.toList.head.getField(0).asInstanceOf[LocalDateTime]) + result.toList.head.getField(0).asInstanceOf[LocalDateTime]).getMillisecond val ts2 = System.currentTimeMillis() diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/DateTimeTestUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/DateTimeTestUtil.scala index f4e9578c86221..d2ec594da6849 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/DateTimeTestUtil.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/DateTimeTestUtil.scala @@ -19,10 +19,9 @@ package org.apache.flink.table.planner.utils import org.apache.flink.table.dataformat.DataFormatConverters.{LocalDateConverter, LocalDateTimeConverter, LocalTimeConverter} - +import org.apache.flink.table.dataformat.SqlTimestamp import org.apache.calcite.avatica.util.DateTimeUtils import org.apache.calcite.avatica.util.DateTimeUtils.dateStringToUnixDate - import java.time.{LocalDate, LocalDateTime, LocalTime} object DateTimeTestUtil { @@ -47,7 +46,8 @@ object DateTimeTestUtil { if (s == null) { null } else { - LocalDateTimeConverter.INSTANCE.toExternal(DateTimeUtils.timestampStringToUnixDate(s)) + LocalDateTimeConverter.INSTANCE.toExternal( + SqlTimestamp.fromEpochMillis(DateTimeUtils.timestampStringToUnixDate(s))) } } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java index 1185457b5c25e..6339d8ca3c164 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java @@ -691,27 +691,31 @@ LocalTime toExternalImpl(BaseRow row, int column) { /** * Converter for LocalDateTime. */ - public static final class LocalDateTimeConverter extends DataFormatConverter { + public static final class LocalDateTimeConverter extends DataFormatConverter { private static final long serialVersionUID = 1L; - public static final LocalDateTimeConverter INSTANCE = new LocalDateTimeConverter(); + public static final LocalDateTimeConverter INSTANCE = new LocalDateTimeConverter(3); - private LocalDateTimeConverter() {} + private final int precision; + + private LocalDateTimeConverter(int precision) { + this.precision = precision; + } @Override - Long toInternalImpl(LocalDateTime value) { - return SqlDateTimeUtils.localDateTimeToUnixTimestamp(value); + SqlTimestamp toInternalImpl(LocalDateTime value) { + return SqlTimestamp.fromLocalDateTime(value); } @Override - LocalDateTime toExternalImpl(Long value) { - return SqlDateTimeUtils.unixTimestampToLocalDateTime(value); + LocalDateTime toExternalImpl(SqlTimestamp value) { + return value.toLocalDateTime(); } @Override LocalDateTime toExternalImpl(BaseRow row, int column) { - return toExternalImpl(row.getLong(column)); + return toExternalImpl(row.getTimestamp(column, precision)); } } @@ -799,27 +803,31 @@ Time toExternalImpl(BaseRow row, int column) { /** * Converter for timestamp. */ - public static final class TimestampConverter extends DataFormatConverter { + public static final class TimestampConverter extends DataFormatConverter { private static final long serialVersionUID = -779956524906131757L; - public static final TimestampConverter INSTANCE = new TimestampConverter(); + public static final TimestampConverter INSTANCE = new TimestampConverter(3); - private TimestampConverter() {} + private final int precision; + + private TimestampConverter(int precision) { + this.precision = precision; + } @Override - Long toInternalImpl(Timestamp value) { - return SqlDateTimeUtils.timestampToInternal(value); + SqlTimestamp toInternalImpl(Timestamp value) { + return SqlTimestamp.fromTimestamp(value); } @Override - Timestamp toExternalImpl(Long value) { - return SqlDateTimeUtils.internalToTimestamp(value); + Timestamp toExternalImpl(SqlTimestamp value) { + return value.toTimestamp(); } @Override Timestamp toExternalImpl(BaseRow row, int column) { - return toExternalImpl(row.getLong(column)); + return toExternalImpl(row.getTimestamp(column, precision)); } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/ClassLogicalTypeConverter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/ClassLogicalTypeConverter.java index 7c7ad74248d76..c2de26a0d52c5 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/ClassLogicalTypeConverter.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/ClassLogicalTypeConverter.java @@ -24,6 +24,7 @@ import org.apache.flink.table.dataformat.BinaryGeneric; import org.apache.flink.table.dataformat.BinaryString; import org.apache.flink.table.dataformat.Decimal; +import org.apache.flink.table.dataformat.SqlTimestamp; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.utils.TypeConversions; @@ -53,8 +54,9 @@ public static Class getInternalClassForType(LogicalType type) { case TIME_WITHOUT_TIME_ZONE: case INTERVAL_YEAR_MONTH: return Integer.class; - case BIGINT: case TIMESTAMP_WITHOUT_TIME_ZONE: + return SqlTimestamp.class; + case BIGINT: case TIMESTAMP_WITH_LOCAL_TIME_ZONE: case INTERVAL_DAY_TIME: return Long.class; diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/InternalSerializers.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/InternalSerializers.java index 928215e98d728..2dcd840eec7b9 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/InternalSerializers.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/InternalSerializers.java @@ -34,6 +34,7 @@ import org.apache.flink.table.runtime.typeutils.BinaryGenericSerializer; import org.apache.flink.table.runtime.typeutils.BinaryStringSerializer; import org.apache.flink.table.runtime.typeutils.DecimalSerializer; +import org.apache.flink.table.runtime.typeutils.SqlTimestampSerializer; import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.IntType; @@ -41,6 +42,7 @@ import org.apache.flink.table.types.logical.MapType; import org.apache.flink.table.types.logical.MultisetType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimestampType; import org.apache.flink.table.types.logical.TypeInformationAnyType; /** @@ -61,8 +63,10 @@ public static TypeSerializer create(LogicalType type, ExecutionConfig config) { case TIME_WITHOUT_TIME_ZONE: case INTERVAL_YEAR_MONTH: return IntSerializer.INSTANCE; - case BIGINT: case TIMESTAMP_WITHOUT_TIME_ZONE: + TimestampType timestampType = (TimestampType) type; + return new SqlTimestampSerializer(timestampType.getPrecision()); + case BIGINT: case TIMESTAMP_WITH_LOCAL_TIME_ZONE: case INTERVAL_DAY_TIME: return LongSerializer.INSTANCE; From 22fa48063fd588b2b83868ec02550ec28b2b47a7 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Tue, 5 Nov 2019 17:36:06 +0800 Subject: [PATCH 03/29] [FLINK-14599][table-planner-blink] Use SqlTimestamp in CodeGenUtils --- .../table/planner/codegen/CodeGenUtils.scala | 36 +++++++++++-------- 1 file changed, 22 insertions(+), 14 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala index 2d2b43c55254f..f4e3d6fb763e2 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala @@ -97,6 +97,8 @@ object CodeGenUtils { val STRING_UTIL: String = className[BinaryStringUtil] + val SQL_TIMESTAMP_TERM: String = className[SqlTimestamp] + // ---------------------------------------------------------------------------------------- private val nameCounter = new AtomicInteger @@ -133,7 +135,7 @@ object CodeGenUtils { case DATE => "int" case TIME_WITHOUT_TIME_ZONE => "int" - case TIMESTAMP_WITHOUT_TIME_ZONE | TIMESTAMP_WITH_LOCAL_TIME_ZONE => "long" + case TIMESTAMP_WITH_LOCAL_TIME_ZONE => "long" case INTERVAL_YEAR_MONTH => "int" case INTERVAL_DAY_TIME => "long" @@ -151,7 +153,7 @@ object CodeGenUtils { case DATE => className[JInt] case TIME_WITHOUT_TIME_ZONE => className[JInt] - case TIMESTAMP_WITHOUT_TIME_ZONE | TIMESTAMP_WITH_LOCAL_TIME_ZONE => className[JLong] + case TIMESTAMP_WITH_LOCAL_TIME_ZONE => className[JLong] case INTERVAL_YEAR_MONTH => className[JInt] case INTERVAL_DAY_TIME => className[JLong] @@ -162,6 +164,7 @@ object CodeGenUtils { case ARRAY => className[BaseArray] case MULTISET | MAP => className[BaseMap] case ROW => className[BaseRow] + case TIMESTAMP_WITHOUT_TIME_ZONE => className[SqlTimestamp] case ANY => className[BinaryGeneric[_]] } @@ -190,7 +193,7 @@ object CodeGenUtils { case VARCHAR | CHAR => s"$BINARY_STRING.EMPTY_UTF8" case DATE | TIME_WITHOUT_TIME_ZONE => "-1" - case TIMESTAMP_WITHOUT_TIME_ZONE | TIMESTAMP_WITH_LOCAL_TIME_ZONE => "-1L" + case TIMESTAMP_WITH_LOCAL_TIME_ZONE => "-1L" case INTERVAL_YEAR_MONTH => "-1" case INTERVAL_DAY_TIME => "-1L" @@ -223,7 +226,8 @@ object CodeGenUtils { case DECIMAL => s"$term.hashCode()" case DATE => s"${className[JInt]}.hashCode($term)" case TIME_WITHOUT_TIME_ZONE => s"${className[JInt]}.hashCode($term)" - case TIMESTAMP_WITHOUT_TIME_ZONE | TIMESTAMP_WITH_LOCAL_TIME_ZONE => + case TIMESTAMP_WITHOUT_TIME_ZONE => s"$term.hashCode()" + case TIMESTAMP_WITH_LOCAL_TIME_ZONE => s"${className[JLong]}.hashCode($term)" case INTERVAL_YEAR_MONTH => s"${className[JInt]}.hashCode($term)" case INTERVAL_DAY_TIME => s"${className[JLong]}.hashCode($term)" @@ -414,8 +418,10 @@ object CodeGenUtils { // temporal types case DATE => s"$rowTerm.getInt($indexTerm)" case TIME_WITHOUT_TIME_ZONE => s"$rowTerm.getInt($indexTerm)" - case TIMESTAMP_WITHOUT_TIME_ZONE | - TIMESTAMP_WITH_LOCAL_TIME_ZONE => s"$rowTerm.getLong($indexTerm)" + case TIMESTAMP_WITHOUT_TIME_ZONE => + val dt = t.asInstanceOf[TimestampType] + s"$rowTerm.getTimestamp($indexTerm, ${dt.getPrecision})" + case TIMESTAMP_WITH_LOCAL_TIME_ZONE => s"$rowTerm.getLong($indexTerm)" case INTERVAL_YEAR_MONTH => s"$rowTerm.getInt($indexTerm)" case INTERVAL_DAY_TIME => s"$rowTerm.getLong($indexTerm)" @@ -539,8 +545,10 @@ object CodeGenUtils { case BOOLEAN => s"$binaryRowTerm.setBoolean($index, $fieldValTerm)" case DATE => s"$binaryRowTerm.setInt($index, $fieldValTerm)" case TIME_WITHOUT_TIME_ZONE => s"$binaryRowTerm.setInt($index, $fieldValTerm)" - case TIMESTAMP_WITHOUT_TIME_ZONE | - TIMESTAMP_WITH_LOCAL_TIME_ZONE => s"$binaryRowTerm.setLong($index, $fieldValTerm)" + case TIMESTAMP_WITHOUT_TIME_ZONE => + val dt = t.asInstanceOf[TimestampType] + s"$binaryRowTerm.setTimestamp($index, $fieldValTerm, ${dt.getPrecision})" + case TIMESTAMP_WITH_LOCAL_TIME_ZONE => s"$binaryRowTerm.setLong($index, $fieldValTerm)" case INTERVAL_YEAR_MONTH => s"$binaryRowTerm.setInt($index, $fieldValTerm)" case INTERVAL_DAY_TIME => s"$binaryRowTerm.setLong($index, $fieldValTerm)" case DECIMAL => @@ -568,8 +576,7 @@ object CodeGenUtils { case BOOLEAN => s"$rowTerm.setBoolean($indexTerm, $fieldTerm)" case DATE => s"$rowTerm.setInt($indexTerm, $fieldTerm)" case TIME_WITHOUT_TIME_ZONE => s"$rowTerm.setInt($indexTerm, $fieldTerm)" - case TIMESTAMP_WITHOUT_TIME_ZONE | - TIMESTAMP_WITH_LOCAL_TIME_ZONE => s"$rowTerm.setLong($indexTerm, $fieldTerm)" + case TIMESTAMP_WITH_LOCAL_TIME_ZONE => s"$rowTerm.setLong($indexTerm, $fieldTerm)" case INTERVAL_YEAR_MONTH => s"$rowTerm.setInt($indexTerm, $fieldTerm)" case INTERVAL_DAY_TIME => s"$rowTerm.setLong($indexTerm, $fieldTerm)" case _ => s"$rowTerm.setNonPrimitiveValue($indexTerm, $fieldTerm)" @@ -590,8 +597,7 @@ object CodeGenUtils { case DOUBLE => s"$arrayTerm.setNullDouble($index)" case TIME_WITHOUT_TIME_ZONE => s"$arrayTerm.setNullInt($index)" case DATE => s"$arrayTerm.setNullInt($index)" - case TIMESTAMP_WITHOUT_TIME_ZONE | - TIMESTAMP_WITH_LOCAL_TIME_ZONE => s"$arrayTerm.setNullLong($index)" + case TIMESTAMP_WITH_LOCAL_TIME_ZONE => s"$arrayTerm.setNullLong($index)" case INTERVAL_YEAR_MONTH => s"$arrayTerm.setNullInt($index)" case INTERVAL_DAY_TIME => s"$arrayTerm.setNullLong($index)" case _ => s"$arrayTerm.setNullLong($index)" @@ -640,8 +646,10 @@ object CodeGenUtils { s"$writerTerm.writeDecimal($indexTerm, $fieldValTerm, ${dt.getPrecision})" case DATE => s"$writerTerm.writeInt($indexTerm, $fieldValTerm)" case TIME_WITHOUT_TIME_ZONE => s"$writerTerm.writeInt($indexTerm, $fieldValTerm)" - case TIMESTAMP_WITHOUT_TIME_ZONE | - TIMESTAMP_WITH_LOCAL_TIME_ZONE => s"$writerTerm.writeLong($indexTerm, $fieldValTerm)" + case TIMESTAMP_WITHOUT_TIME_ZONE => + val dt = t.asInstanceOf[TimestampType] + s"$writerTerm.writeTimestamp($indexTerm, $fieldValTerm, ${dt.getPrecision})" + case TIMESTAMP_WITH_LOCAL_TIME_ZONE => s"$writerTerm.writeLong($indexTerm, $fieldValTerm)" case INTERVAL_YEAR_MONTH => s"$writerTerm.writeInt($indexTerm, $fieldValTerm)" case INTERVAL_DAY_TIME => s"$writerTerm.writeLong($indexTerm, $fieldValTerm)" From 88082bbbd4a1c1357eb122d48726a48c1f31d2a5 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Tue, 5 Nov 2019 18:14:32 +0800 Subject: [PATCH 04/29] [FLINK-14599][table-planner-blink] Fix extract() built-in function for Timestamp type --- .../table/planner/codegen/calls/ExtractCallGen.scala | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ExtractCallGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ExtractCallGen.scala index 6d64d801b1dce..ed6c2c346790e 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ExtractCallGen.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ExtractCallGen.scala @@ -53,7 +53,7 @@ class ExtractCallGen(method: Method) (terms) => s""" |${qualifyMethod(method)}(${terms.head}, - | ${terms(1)} / ${TimeUnit.DAY.multiplier.intValue()}) + | ${terms(1)}.getMillisecond() / ${TimeUnit.DAY.multiplier.intValue()}) |""".stripMargin } @@ -71,19 +71,23 @@ class ExtractCallGen(method: Method) generateCallIfArgsNotNull(ctx, returnType, operands) { (terms) => { val factor = getFactor(unit) + val longTerm = tpe.getTypeRoot match { + case LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE => s"${terms(1)}.getMillisecond()" + case _ => s"${terms(1)}" + } unit match { case TimeUnit.QUARTER => s""" - |((${terms(1)} % $factor) - 1) / ${unit.multiplier.intValue()} + 1 + |(($longTerm % $factor) - 1) / ${unit.multiplier.intValue()} + 1 |""".stripMargin case _ => if (factor == 1) { s""" - |${terms(1)} / ${unit.multiplier.intValue()} + |$longTerm / ${unit.multiplier.intValue()} |""".stripMargin } else { s""" - |(${terms(1)} % $factor) / ${unit.multiplier.intValue()} + |($longTerm % $factor) / ${unit.multiplier.intValue()} |""".stripMargin } } From e6d75bc3bb8cb9b7ad57b0b2f0dcdbe740aab74e Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Tue, 5 Nov 2019 19:12:02 +0800 Subject: [PATCH 05/29] [FLINK-14599][table-planner-blink] Fix casting from Timestamp to String and vice versa --- .../table/planner/codegen/calls/ScalarOperatorGens.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala index 06d89043018b6..5916376303160 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala @@ -2169,7 +2169,8 @@ object ScalarOperatorGens { s"${qualifyMethod(BuiltInMethods.STRING_TO_TIME)}($operandTerm.toString())" case TIMESTAMP_WITHOUT_TIME_ZONE => s""" - |${qualifyMethod(BuiltInMethods.STRING_TO_TIMESTAMP)}($operandTerm.toString()) + |${SQL_TIMESTAMP_TERM}.fromEpochMillis( + | ${qualifyMethod(BuiltInMethods.STRING_TO_TIMESTAMP)}($operandTerm.toString())) |""".stripMargin case _ => throw new UnsupportedOperationException } @@ -2184,7 +2185,8 @@ object ScalarOperatorGens { case TIME_WITHOUT_TIME_ZONE => s"${qualifyMethod(BuiltInMethods.UNIX_TIME_TO_STRING)}($operandTerm)" case TIMESTAMP_WITHOUT_TIME_ZONE => // including rowtime indicator - s"${qualifyMethod(BuiltInMethods.TIMESTAMP_TO_STRING)}($operandTerm, 3)" + val longTerm = s"$operandTerm.getMillisecond()" + s"${qualifyMethod(BuiltInMethods.TIMESTAMP_TO_STRING)}($longTerm, 3)" case TIMESTAMP_WITH_LOCAL_TIME_ZONE => val method = qualifyMethod(BuiltInMethods.TIMESTAMP_TO_STRING_TIME_ZONE) val zone = ctx.addReusableTimeZone() From 0ec0b46a7165269e925ecfc2e6b58b3d49a5c1e5 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Tue, 5 Nov 2019 19:16:58 +0800 Subject: [PATCH 06/29] [FLINK-14599][table-planner-blink] Fix floor()/ceil() built-in function for Timestamp type --- .../codegen/calls/FloorCeilCallGen.scala | 39 ++++++++++++++----- 1 file changed, 30 insertions(+), 9 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/FloorCeilCallGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/FloorCeilCallGen.scala index d511957c11e4d..38e5b66e8cfb8 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/FloorCeilCallGen.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/FloorCeilCallGen.scala @@ -18,7 +18,7 @@ package org.apache.flink.table.planner.codegen.calls -import org.apache.flink.table.planner.codegen.CodeGenUtils.{getEnum, primitiveTypeTermForType, qualifyMethod} +import org.apache.flink.table.planner.codegen.CodeGenUtils.{getEnum, primitiveTypeTermForType, qualifyMethod, SQL_TIMESTAMP_TERM} import org.apache.flink.table.planner.codegen.GenerateUtils.generateCallIfArgsNotNull import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, GeneratedExpression} import org.apache.flink.table.types.logical.{LogicalType, LogicalTypeRoot} @@ -78,15 +78,36 @@ class FloorCeilCallGen( // for Unix Date / Unix Time case YEAR | MONTH => - s""" - |($internalType) ${qualifyMethod(temporalMethod.get)}(${terms(1)}, ${terms.head}) - |""".stripMargin + operand.resultType.getTypeRoot match { + case LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE => + val longTerm = s"${terms.head}.getMillisecond()" + s""" + |$SQL_TIMESTAMP_TERM.fromEpochMillis( + | ${qualifyMethod(temporalMethod.get)}(${terms(1)}, $longTerm)) + """.stripMargin + case _ => + s""" + |($internalType) ${qualifyMethod(temporalMethod.get)}( + | ${terms(1)}, ${terms.head}) + |""".stripMargin + } case _ => - s""" - |${qualifyMethod(arithmeticMethod)}( - | ($internalType) ${terms.head}, - | ($internalType) ${unit.startUnit.multiplier.intValue()}) - |""".stripMargin + operand.resultType.getTypeRoot match { + case LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE => + val longTerm = s"${terms.head}.getMillisecond()" + s""" + |$SQL_TIMESTAMP_TERM.fromEpochMillis(${qualifyMethod(arithmeticMethod)}( + | $longTerm, + | (long) ${unit.startUnit.multiplier.intValue()})) + """.stripMargin + case _ => + s""" + |${qualifyMethod(arithmeticMethod)}( + | ($internalType) ${terms.head}, + | ($internalType) ${unit.startUnit.multiplier.intValue()}) + |""".stripMargin + } + } } } From d44a523670406d84d7ebc0d6c794079d38f991ee Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Tue, 5 Nov 2019 19:35:21 +0800 Subject: [PATCH 07/29] [FLINK-14599][table-planner-blink] Fix time interval arithmetic for Timestamp type --- .../codegen/calls/ScalarOperatorGens.scala | 43 +++++++++++++++---- 1 file changed, 34 insertions(+), 9 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala index 5916376303160..095d2fdec3374 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala @@ -174,8 +174,8 @@ object ScalarOperatorGens { (l, r) => s"$l $op ((int) ($r / ${MILLIS_PER_DAY}L))" } case TIMESTAMP_WITHOUT_TIME_ZONE => - generateOperatorIfNotNull(ctx, new TimestampType(), left, right) { - (l, r) => s"($l * ${MILLIS_PER_DAY}L) $op $r" + generateOperatorIfNotNull(ctx, new TimestampType(3), left, right) { + (l, r) => s"$SQL_TIMESTAMP_TERM.fromEpochMillis(($l * ${MILLIS_PER_DAY}L) $op $r)" } } @@ -191,12 +191,21 @@ object ScalarOperatorGens { case (TIMESTAMP_WITHOUT_TIME_ZONE, INTERVAL_DAY_TIME) => generateOperatorIfNotNull(ctx, left.resultType, left, right) { - (l, r) => s"$l $op $r" + (l, r) => { + val leftTerm = s"$l.getMillisecond()" + s"$SQL_TIMESTAMP_TERM.fromEpochMillis($leftTerm $op $r)" + } } case (TIMESTAMP_WITHOUT_TIME_ZONE, INTERVAL_YEAR_MONTH) => generateOperatorIfNotNull(ctx, left.resultType, left, right) { - (l, r) => s"${qualifyMethod(BuiltInMethod.ADD_MONTHS.method)}($l, $op($r))" + (l, r) => { + val leftTerm = s"$l.getMillisecond()" + s""" + |$SQL_TIMESTAMP_TERM.fromEpochMillis( + | ${qualifyMethod(BuiltInMethod.ADD_MONTHS.method)}($leftTerm, $op($r))) + """.stripMargin + } } // minus arithmetic of time points (i.e. for TIMESTAMPDIFF) @@ -207,11 +216,23 @@ object ScalarOperatorGens { generateOperatorIfNotNull(ctx, resultType, left, right) { (ll, rr) => (left.resultType.getTypeRoot, right.resultType.getTypeRoot) match { case (TIMESTAMP_WITHOUT_TIME_ZONE, DATE) => + val leftTerm = s"$ll.getMillisecond()" s"${qualifyMethod(BuiltInMethod.SUBTRACT_MONTHS.method)}" + - s"($ll, $rr * ${MILLIS_PER_DAY}L)" + s"($leftTerm, $rr * ${MILLIS_PER_DAY}L)" case (DATE, TIMESTAMP_WITHOUT_TIME_ZONE) => + val rightTerm = s"$rr.getMillisecond()" s"${qualifyMethod(BuiltInMethod.SUBTRACT_MONTHS.method)}" + - s"($ll * ${MILLIS_PER_DAY}L, $rr)" + s"($ll * ${MILLIS_PER_DAY}L, $rightTerm)" + case (TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE) => + val leftTerm = s"$ll.getMillisecond()" + val rightTerm = s"$rr.getMillisecond()" + s"${qualifyMethod(BuiltInMethod.SUBTRACT_MONTHS.method)}($leftTerm, $rightTerm)" + case (TIMESTAMP_WITHOUT_TIME_ZONE, _) => + val leftTerm = s"$ll.getMillisecond()" + s"${qualifyMethod(BuiltInMethod.SUBTRACT_MONTHS.method)}($leftTerm, $rr)" + case (_, TIMESTAMP_WITHOUT_TIME_ZONE) => + val rightTerm = s"$rr.getMillisecond()" + s"${qualifyMethod(BuiltInMethod.SUBTRACT_MONTHS.method)}($ll, $rightTerm)" case _ => s"${qualifyMethod(BuiltInMethod.SUBTRACT_MONTHS.method)}($ll, $rr)" } @@ -221,13 +242,17 @@ object ScalarOperatorGens { generateOperatorIfNotNull(ctx, resultType, left, right) { (ll, rr) => (left.resultType.getTypeRoot, right.resultType.getTypeRoot) match { case (TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE) => - s"$ll $op $rr" + val leftTerm = s"$ll.getMillisecond()" + val rightTerm = s"$rr.getMillisecond()" + s"$leftTerm $op $rightTerm" case (DATE, DATE) => s"($ll * ${MILLIS_PER_DAY}L) $op ($rr * ${MILLIS_PER_DAY}L)" case (TIMESTAMP_WITHOUT_TIME_ZONE, DATE) => - s"$ll $op ($rr * ${MILLIS_PER_DAY}L)" + val leftTerm = s"$ll.getMillisecond()" + s"$leftTerm $op ($rr * ${MILLIS_PER_DAY}L)" case (DATE, TIMESTAMP_WITHOUT_TIME_ZONE) => - s"($ll * ${MILLIS_PER_DAY}L) $op $rr" + val rightTerm = s"$rr.getMillisecond()" + s"($ll * ${MILLIS_PER_DAY}L) $op $rightTerm" } } } From a7ff4db3dba0e37fbab62136b8daa878e1d847d3 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 6 Nov 2019 09:59:54 +0800 Subject: [PATCH 08/29] [FLINK-14599][table-planner-blink] Fix cast/reinterpret/compare built-in function for Timestamp type --- .../codegen/calls/ScalarOperatorGens.scala | 68 +++++++++++++------ 1 file changed, 47 insertions(+), 21 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala index 095d2fdec3374..8e5dd7c2b54cf 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala @@ -498,6 +498,13 @@ object ScalarOperatorGens { else if (isNumeric(left.resultType) && isNumeric(right.resultType)) { (leftTerm, rightTerm) => s"$leftTerm $operator $rightTerm" } + + // both sides are timestamp + else if (isTimestamp(left.resultType) && isTimestamp(right.resultType)) { + (leftTerm, rightTerm) => + s"$leftTerm.compareTo($rightTerm) $operator 0" + } + // both sides are temporal of same type else if (isTemporal(left.resultType) && isInteroperable(left.resultType, right.resultType)) { @@ -779,10 +786,8 @@ object ScalarOperatorGens { // Interval Months -> Long case (DATE, INTEGER) | (TIME_WITHOUT_TIME_ZONE, INTEGER) | - (TIMESTAMP_WITHOUT_TIME_ZONE, BIGINT) | (INTEGER, DATE) | (INTEGER, TIME_WITHOUT_TIME_ZONE) | - (BIGINT, TIMESTAMP_WITHOUT_TIME_ZONE) | (INTEGER, INTERVAL_YEAR_MONTH) | (BIGINT, INTERVAL_DAY_TIME) | (INTERVAL_YEAR_MONTH, INTEGER) | @@ -792,6 +797,16 @@ object ScalarOperatorGens { (INTERVAL_YEAR_MONTH, BIGINT) => internalExprCasting(operand, targetType) + case (TIMESTAMP_WITHOUT_TIME_ZONE, BIGINT) => + generateUnaryOperatorIfNotNull(ctx, targetType, operand) { + operandTerm => s"$operandTerm.getMillisecond()" + } + + case (BIGINT, TIMESTAMP_WITHOUT_TIME_ZONE) => + generateUnaryOperatorIfNotNull(ctx, targetType, operand) { + operandTerm => s"$SQL_TIMESTAMP_TERM.fromEpochMillis($operandTerm)" + } + case (from, to) => throw new CodeGenException(s"Unsupported reinterpret from '$from' to '$to'.") } @@ -815,7 +830,7 @@ object ScalarOperatorGens { generateUnaryOperatorIfNotNull(ctx, targetType, operand) { operandTerm => val timeZone = ctx.addReusableTimeZone() - s"$method($operandTerm, $timeZone)" + s"$method($SQL_TIMESTAMP_TERM.fromEpochMillis($operandTerm), $timeZone)" } case (TIMESTAMP_WITH_LOCAL_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE) => @@ -823,7 +838,7 @@ object ScalarOperatorGens { generateUnaryOperatorIfNotNull(ctx, targetType, operand) { operandTerm => val zone = ctx.addReusableTimeZone() - s"$method($operandTerm, $zone)" + s"$SQL_TIMESTAMP_TERM.fromEpochMillis($method($operandTerm, $zone))" } // identity casting @@ -958,7 +973,8 @@ object ScalarOperatorGens { resultNullable = true) { operandTerm => s""" - |${qualifyMethod(BuiltInMethods.STRING_TO_TIMESTAMP)}($operandTerm.toString()) + |$SQL_TIMESTAMP_TERM.fromEpochMillis( + | ${qualifyMethod(BuiltInMethods.STRING_TO_TIMESTAMP)}($operandTerm.toString())) """.stripMargin } @@ -1002,7 +1018,8 @@ object ScalarOperatorGens { // DECIMAL -> Timestamp case (DECIMAL, TIMESTAMP_WITHOUT_TIME_ZONE) => generateUnaryOperatorIfNotNull(ctx, targetType, operand) { - operandTerm => s"$DECIMAL_TERM.castToTimestamp($operandTerm)" + operandTerm => + s"$SQL_TIMESTAMP_TERM.fromEpochMillis($DECIMAL_TERM.castToTimestamp($operandTerm))" } // NUMERIC TYPE -> Boolean @@ -1022,7 +1039,10 @@ object ScalarOperatorGens { case (DATE, TIMESTAMP_WITHOUT_TIME_ZONE) => generateUnaryOperatorIfNotNull(ctx, targetType, operand) { operandTerm => - s"$operandTerm * ${classOf[DateTimeUtils].getCanonicalName}.MILLIS_PER_DAY" + s""" + |$SQL_TIMESTAMP_TERM.fromEpochMillis( + | $operandTerm * ${classOf[DateTimeUtils].getCanonicalName}.MILLIS_PER_DAY) + """.stripMargin } // Timestamp -> Date @@ -1030,14 +1050,16 @@ object ScalarOperatorGens { val targetTypeTerm = primitiveTypeTermForType(targetType) generateUnaryOperatorIfNotNull(ctx, targetType, operand) { operandTerm => - s"($targetTypeTerm) ($operandTerm / " + - s"${classOf[DateTimeUtils].getCanonicalName}.MILLIS_PER_DAY)" + s""" + |($targetTypeTerm) ($operandTerm.getMillisecond() / + | ${classOf[DateTimeUtils].getCanonicalName}.MILLIS_PER_DAY) + """.stripMargin } // Time -> Timestamp case (TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE) => generateUnaryOperatorIfNotNull(ctx, targetType, operand) { - operandTerm => s"$operandTerm" + operandTerm => s"$SQL_TIMESTAMP_TERM.fromEpochMillis($operandTerm)" } // Timestamp -> Time @@ -1045,7 +1067,7 @@ object ScalarOperatorGens { val targetTypeTerm = primitiveTypeTermForType(targetType) generateUnaryOperatorIfNotNull(ctx, targetType, operand) { operandTerm => - s"($targetTypeTerm) ($operandTerm % " + + s"($targetTypeTerm) ($operandTerm.getMillisecond() % " + s"${classOf[DateTimeUtils].getCanonicalName}.MILLIS_PER_DAY)" } @@ -1085,8 +1107,12 @@ object ScalarOperatorGens { case (TIMESTAMP_WITHOUT_TIME_ZONE, DECIMAL) => val dt = targetType.asInstanceOf[DecimalType] generateUnaryOperatorIfNotNull(ctx, targetType, operand) { - operandTerm => s"$DECIMAL_TERM.castFrom" + - s"(((double) ($operandTerm / 1000.0)), ${dt.getPrecision}, ${dt.getScale})" + operandTerm => + s""" + |$DECIMAL_TERM.castFrom( + | ((double) ($operandTerm.getMillisecond() / 1000.0)), + | ${dt.getPrecision}, ${dt.getScale}) + """.stripMargin } // Tinyint -> Timestamp @@ -1098,7 +1124,7 @@ object ScalarOperatorGens { (INTEGER, TIMESTAMP_WITHOUT_TIME_ZONE) | (BIGINT, TIMESTAMP_WITHOUT_TIME_ZONE) => generateUnaryOperatorIfNotNull(ctx, targetType, operand) { - operandTerm => s"(((long) $operandTerm) * 1000)" + operandTerm => s"$SQL_TIMESTAMP_TERM.fromEpochMillis(((long) $operandTerm) * 1000)" } // Float -> Timestamp @@ -1106,43 +1132,43 @@ object ScalarOperatorGens { case (FLOAT, TIMESTAMP_WITHOUT_TIME_ZONE) | (DOUBLE, TIMESTAMP_WITHOUT_TIME_ZONE) => generateUnaryOperatorIfNotNull(ctx, targetType, operand) { - operandTerm => s"((long) ($operandTerm * 1000))" + operandTerm => s"$SQL_TIMESTAMP_TERM.fromEpochMillis((long) ($operandTerm * 1000))" } // Timestamp -> Tinyint case (TIMESTAMP_WITHOUT_TIME_ZONE, TINYINT) => generateUnaryOperatorIfNotNull(ctx, targetType, operand) { - operandTerm => s"((byte) ($operandTerm / 1000))" + operandTerm => s"((byte) ($operandTerm.getMillisecond() / 1000))" } // Timestamp -> Smallint case (TIMESTAMP_WITHOUT_TIME_ZONE, SMALLINT) => generateUnaryOperatorIfNotNull(ctx, targetType, operand) { - operandTerm => s"((short) ($operandTerm / 1000))" + operandTerm => s"((short) ($operandTerm.getMillisecond() / 1000))" } // Timestamp -> Int case (TIMESTAMP_WITHOUT_TIME_ZONE, INTEGER) => generateUnaryOperatorIfNotNull(ctx, targetType, operand) { - operandTerm => s"((int) ($operandTerm / 1000))" + operandTerm => s"((int) ($operandTerm.getMillisecond() / 1000))" } // Timestamp -> BigInt case (TIMESTAMP_WITHOUT_TIME_ZONE, BIGINT) => generateUnaryOperatorIfNotNull(ctx, targetType, operand) { - operandTerm => s"((long) ($operandTerm / 1000))" + operandTerm => s"((long) ($operandTerm.getMillisecond() / 1000))" } // Timestamp -> Float case (TIMESTAMP_WITHOUT_TIME_ZONE, FLOAT) => generateUnaryOperatorIfNotNull(ctx, targetType, operand) { - operandTerm => s"((float) ($operandTerm / 1000.0))" + operandTerm => s"((float) ($operandTerm.getMillisecond() / 1000.0))" } // Timestamp -> Double case (TIMESTAMP_WITHOUT_TIME_ZONE, DOUBLE) => generateUnaryOperatorIfNotNull(ctx, targetType, operand) { - operandTerm => s"((double) ($operandTerm / 1000.0))" + operandTerm => s"((double) ($operandTerm.getMillisecond() / 1000.0))" } // internal temporal casting From 7cc6cf4ea8ab2de0d9bdadee75c26c8dd617f218 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 6 Nov 2019 10:52:20 +0800 Subject: [PATCH 09/29] [FLINK-14599][table-planner-blink] Fix Timestamp literal and fixup casting Timestamp to Timestamp with local time zone --- .../table/planner/codegen/ExpressionReducer.scala | 13 ++++++++++--- .../flink/table/planner/codegen/GenerateUtils.scala | 9 ++++++++- .../planner/codegen/calls/ScalarOperatorGens.scala | 2 +- 3 files changed, 19 insertions(+), 5 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala index a50a5f41751e4..b6112aba07e7c 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala @@ -23,19 +23,17 @@ import org.apache.flink.configuration.Configuration import org.apache.flink.metrics.MetricGroup import org.apache.flink.table.api.{TableConfig, TableException} import org.apache.flink.table.dataformat.BinaryStringUtil.safeToString -import org.apache.flink.table.dataformat.{BinaryString, Decimal, GenericRow} +import org.apache.flink.table.dataformat.{BinaryString, Decimal, GenericRow, SqlTimestamp} import org.apache.flink.table.functions.{FunctionContext, UserDefinedFunction} import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.codegen.FunctionCodeGenerator.generateFunction import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import org.apache.flink.table.runtime.functions.SqlDateTimeUtils import org.apache.flink.table.types.logical.RowType - import org.apache.calcite.avatica.util.ByteString import org.apache.calcite.rex.{RexBuilder, RexExecutor, RexNode} import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.commons.lang3.StringEscapeUtils - import java.io.File import java.util.TimeZone @@ -172,6 +170,15 @@ class ExpressionReducer( } reducedValues.add(maySkipNullLiteralReduce(rexBuilder, value, unreduced)) reducedIdx += 1 + case SqlTypeName.TIMESTAMP => + val reducedValue = reduced.getField(reducedIdx) + val value = if (reducedValue != null) { + Long.box(reducedValue.asInstanceOf[SqlTimestamp].getMillisecond) + } else { + reducedValue + } + reducedValues.add(maySkipNullLiteralReduce(rexBuilder, value, unreduced)) + reducedIdx += 1 case _ => val reducedValue = reduced.getField(reducedIdx) // RexBuilder handle double literal incorrectly, convert it into BigDecimal manually diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala index 3656ca069cab5..7b62455da09d7 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala @@ -370,8 +370,15 @@ object GenerateUtils { generateNonNullLiteral(literalType, literalValue.toString, literalValue) case TIMESTAMP_WITHOUT_TIME_ZONE => + // TODO: support Timestamp(3) now + val fieldTerm = newName("timestamp") val millis = literalValue.asInstanceOf[Long] - generateNonNullLiteral(literalType, millis + "L", millis) + val fieldTimestamp = + s""" + |$SQL_TIMESTAMP_TERM $fieldTerm = $SQL_TIMESTAMP_TERM.fromEpochMillis(${millis}L); + """.stripMargin + ctx.addReusableMember(fieldTimestamp) + generateNonNullLiteral(literalType, fieldTerm, literalType) case TIMESTAMP_WITH_LOCAL_TIME_ZONE => val millis = unixTimestampToLocalDateTime(literalValue.asInstanceOf[Long]) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala index 8e5dd7c2b54cf..3e994acd22f70 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala @@ -830,7 +830,7 @@ object ScalarOperatorGens { generateUnaryOperatorIfNotNull(ctx, targetType, operand) { operandTerm => val timeZone = ctx.addReusableTimeZone() - s"$method($SQL_TIMESTAMP_TERM.fromEpochMillis($operandTerm), $timeZone)" + s"$method($operandTerm.getMillisecond(), $timeZone)" } case (TIMESTAMP_WITH_LOCAL_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE) => From 886eac47c2973b0ff0840379fc12061ff94dcb13 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 6 Nov 2019 11:22:24 +0800 Subject: [PATCH 10/29] [FLINK-14599][table-planner-blink] Fix count distinct on timestamp fields --- .../flink/table/types/logical/TimestampType.java | 3 ++- .../table/planner/plan/utils/AggregateUtil.scala | 6 ++++-- .../runtime/types/TypeInfoDataTypeConverter.java | 15 +++++++++++++++ 3 files changed, 21 insertions(+), 3 deletions(-) diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/TimestampType.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/TimestampType.java index 87f062d456cb4..dfeefbc77e5d1 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/TimestampType.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/TimestampType.java @@ -58,7 +58,8 @@ public final class TimestampType extends LogicalType { private static final Set INPUT_OUTPUT_CONVERSION = conversionSet( java.sql.Timestamp.class.getName(), - java.time.LocalDateTime.class.getName()); + java.time.LocalDateTime.class.getName(), + "org.apache.flink.table.dataformat.SqlTimestamp"); private static final Class DEFAULT_CONVERSION = java.time.LocalDateTime.class; diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala index 973a6dab82fa3..90ac53c47c84d 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala @@ -20,7 +20,7 @@ package org.apache.flink.table.planner.plan.utils import org.apache.flink.api.common.typeinfo.Types import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.api.{DataTypes, TableConfig, TableException} -import org.apache.flink.table.dataformat.{BaseRow, BinaryString, Decimal} +import org.apache.flink.table.dataformat.{BaseRow, BinaryString, Decimal, SqlTimestamp} import org.apache.flink.table.dataview.MapViewTypeInfo import org.apache.flink.table.expressions.ExpressionUtils.extractValue import org.apache.flink.table.expressions._ @@ -494,7 +494,9 @@ object AggregateUtil extends Enumeration { case DATE => DataTypes.INT case TIME_WITHOUT_TIME_ZONE => DataTypes.INT - case TIMESTAMP_WITHOUT_TIME_ZONE => DataTypes.BIGINT + case TIMESTAMP_WITHOUT_TIME_ZONE => + val dt = argTypes(0).asInstanceOf[TimestampType] + DataTypes.TIMESTAMP(dt.getPrecision).bridgedTo(classOf[SqlTimestamp]) case INTERVAL_YEAR_MONTH => DataTypes.INT case INTERVAL_DAY_TIME => DataTypes.BIGINT diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoDataTypeConverter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoDataTypeConverter.java index b08f72d59f187..750cc5f6e9199 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoDataTypeConverter.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoDataTypeConverter.java @@ -29,6 +29,7 @@ import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.dataformat.BinaryString; import org.apache.flink.table.dataformat.Decimal; +import org.apache.flink.table.dataformat.SqlTimestamp; import org.apache.flink.table.dataview.MapViewTypeInfo; import org.apache.flink.table.functions.AggregateFunctionDefinition; import org.apache.flink.table.functions.TableFunctionDefinition; @@ -36,6 +37,7 @@ import org.apache.flink.table.runtime.typeutils.BigDecimalTypeInfo; import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo; import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo; +import org.apache.flink.table.runtime.typeutils.SqlTimestampTypeInfo; import org.apache.flink.table.types.CollectionDataType; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.FieldsDataType; @@ -43,11 +45,14 @@ import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimestampKind; +import org.apache.flink.table.types.logical.TimestampType; import org.apache.flink.table.types.utils.TypeConversions; import org.apache.flink.table.typeutils.TimeIntervalTypeInfo; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; +import java.time.LocalDateTime; import java.util.HashMap; import java.util.Map; @@ -98,6 +103,16 @@ public static TypeInformation fromDataTypeToTypeInfo(DataType dataType) { } LogicalType logicalType = fromDataTypeToLogicalType(dataType); switch (logicalType.getTypeRoot()) { + case TIMESTAMP_WITHOUT_TIME_ZONE: + TimestampType timestampType = (TimestampType) logicalType; + if (timestampType.getKind() == TimestampKind.REGULAR) { + return clazz == SqlTimestamp.class ? + new SqlTimestampTypeInfo(timestampType.getPrecision()) : + (clazz == LocalDateTime.class ? + Types.LOCAL_DATE_TIME : Types.SQL_TIMESTAMP); + } else { + return TypeConversions.fromDataTypeToLegacyInfo(dataType); + } case DECIMAL: DecimalType decimalType = (DecimalType) logicalType; return clazz == Decimal.class ? From a207224fb1d87bf7d04ac0b74cf698728272bc1c Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 6 Nov 2019 11:32:29 +0800 Subject: [PATCH 11/29] [FLINK-14599][table-planner-blink] Fix timestampdiff built-in function for Timestamp type --- .../codegen/calls/TimestampDiffCallGen.scala | 26 +++++++++++++++---- 1 file changed, 21 insertions(+), 5 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/TimestampDiffCallGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/TimestampDiffCallGen.scala index d9c746da6acef..5503cafe1b8a6 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/TimestampDiffCallGen.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/TimestampDiffCallGen.scala @@ -42,11 +42,22 @@ class TimestampDiffCallGen extends CallGenerator { TimeUnit.MONTH | TimeUnit.QUARTER => (operands(1).resultType.getTypeRoot, operands(2).resultType.getTypeRoot) match { + case (TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE) => + generateCallIfArgsNotNull(ctx, new IntType(), operands) { + terms => + val leftTerm = s"${terms(1)}.getMillisecond()" + val rightTerm = s"${terms(2)}.getMillisecond()" + s""" + |${qualifyMethod(BuiltInMethod.SUBTRACT_MONTHS.method)}( + | $leftTerm, $rightTerm) / ${unit.multiplier.intValue()} + """.stripMargin + } case (TIMESTAMP_WITHOUT_TIME_ZONE, DATE) => generateCallIfArgsNotNull(ctx, new IntType(), operands) { terms => + val leftTerm = s"${terms(1)}.getMillisecond()" s""" - |${qualifyMethod(BuiltInMethod.SUBTRACT_MONTHS.method)}(${terms(1)}, + |${qualifyMethod(BuiltInMethod.SUBTRACT_MONTHS.method)}($leftTerm, | ${terms(2)} * ${MILLIS_PER_DAY}L) / ${unit.multiplier.intValue()} |""".stripMargin } @@ -54,9 +65,10 @@ class TimestampDiffCallGen extends CallGenerator { case (DATE, TIMESTAMP_WITHOUT_TIME_ZONE) => generateCallIfArgsNotNull(ctx, new IntType(), operands) { terms => + val rightTerm = s"${terms(2)}.getMillisecond()" s""" |${qualifyMethod(BuiltInMethod.SUBTRACT_MONTHS.method)}( - |${terms(1)} * ${MILLIS_PER_DAY}L, ${terms(2)}) / ${unit.multiplier.intValue()} + |${terms(1)} * ${MILLIS_PER_DAY}L, $rightTerm) / ${unit.multiplier.intValue()} |""".stripMargin } @@ -79,16 +91,19 @@ class TimestampDiffCallGen extends CallGenerator { case (TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE) => generateCallIfArgsNotNull(ctx, new IntType(), operands) { terms => + val leftTerm = s"${terms(1)}.getMillisecond()" + val rightTerm = s"${terms(2)}.getMillisecond()" s""" - |(int)((${terms(1)} - ${terms(2)}) / ${unit.multiplier.intValue()}) + |(int)(($leftTerm - $rightTerm) / ${unit.multiplier.intValue()}) |""".stripMargin } case (TIMESTAMP_WITHOUT_TIME_ZONE, DATE) => generateCallIfArgsNotNull(ctx, new IntType(), operands) { terms => + val leftTerm = s"${terms(1)}.getMillisecond()" s""" - |(int)((${terms(1)} - + |(int)(($leftTerm - | ${terms(2)} * ${MILLIS_PER_DAY}L) / ${unit.multiplier.intValue()}) |""".stripMargin } @@ -96,9 +111,10 @@ class TimestampDiffCallGen extends CallGenerator { case (DATE, TIMESTAMP_WITHOUT_TIME_ZONE) => generateCallIfArgsNotNull(ctx, new IntType(), operands) { terms => + val rightTerm = s"${terms(2)}.getMillisecond()" s""" |(int)((${terms(1)} * ${MILLIS_PER_DAY}L - - | ${terms(2)}) / ${unit.multiplier.intValue()}) + | $rightTerm) / ${unit.multiplier.intValue()}) |""".stripMargin } From 77d1789ffbdd50d119f7df1cc602c52c54814d6b Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 6 Nov 2019 12:07:09 +0800 Subject: [PATCH 12/29] [FLINK-14599][table-planner-blink] Fix toTimestamp built-in function --- .../planner/codegen/calls/MethodCallGen.scala | 27 ++++++++++++------- .../flink/table/dataformat/SqlTimestamp.java | 19 +++++++++++++ 2 files changed, 36 insertions(+), 10 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/MethodCallGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/MethodCallGen.scala index c442509c3eb61..6c9fb11607af6 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/MethodCallGen.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/MethodCallGen.scala @@ -18,11 +18,10 @@ package org.apache.flink.table.planner.codegen.calls -import org.apache.flink.table.planner.codegen.CodeGenUtils.{BINARY_STRING, qualifyMethod} +import org.apache.flink.table.planner.codegen.CodeGenUtils.{BINARY_STRING, qualifyMethod, SQL_TIMESTAMP_TERM} import org.apache.flink.table.planner.codegen.GenerateUtils.generateCallIfArgsNotNull import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, GeneratedExpression} -import org.apache.flink.table.types.logical.LogicalType - +import org.apache.flink.table.types.logical.{LogicalType, LogicalTypeRoot} import java.lang.reflect.Method import java.util.TimeZone @@ -34,13 +33,17 @@ class MethodCallGen(method: Method) extends CallGenerator { returnType: LogicalType): GeneratedExpression = { generateCallIfArgsNotNull(ctx, returnType, operands, !method.getReturnType.isPrimitive) { originalTerms => { - val terms = originalTerms.zip(method.getParameterTypes).map { case (term, clazz) => - // convert the BinaryString parameter to String if the method parameter accept String - if (clazz == classOf[String]) { - s"$term.toString()" - } else { - term - } + val terms = originalTerms.zipWithIndex.zip(method.getParameterTypes).map { + case ((term, i), clazz) => + // convert the BinaryString parameter to String if the method parameter accept String + if (clazz == classOf[String]) { + s"$term.toString()" + } else if ((clazz == classOf[Long] || clazz == classOf[java.lang.Long]) && + operands(i).resultType.getTypeRoot == LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE) { + s"$term.getMillisecond()" + } else { + term + } } // generate method invoke code and adapt when it's a time zone related function @@ -60,6 +63,10 @@ class MethodCallGen(method: Method) extends CallGenerator { // convert String to BinaryString if the return type is String if (method.getReturnType == classOf[String]) { s"$BINARY_STRING.fromString($call)" + } else if ((method.getReturnType == classOf[Long] + || method.getReturnType == classOf[java.lang.Long]) && + returnType.getTypeRoot == LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE) { + s"$SQL_TIMESTAMP_TERM.fromEpochMillis($call)" } else { call } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/SqlTimestamp.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/SqlTimestamp.java index 53892c4d069e1..c9b5a53a9990d 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/SqlTimestamp.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/SqlTimestamp.java @@ -117,6 +117,25 @@ public static SqlTimestamp fromEpochMillis(long millisecond, int nanoOfMilliseco return new SqlTimestamp(millisecond, nanoOfMillisecond); } + /** + * Obtains an instance of {@code SqlTimestamp} from a millisecond. + * + *

This returns a {@code SqlTimestamp} with the specified millisecond. + * The nanoOfMillisecond field will be set to zero. + * + * @param millisecond the number of milliseconds since January 1, 1970, 00:00:00 GMT + * A negative number is the number of milliseconds before + * January 1, 1970, 00:00:00 GMT + * @return an instance of {@code SqlTimestamp} + */ + public static SqlTimestamp fromEpochMillis(Long millisecond) { + if (millisecond == null) { + return null; + } + return new SqlTimestamp(millisecond, 0); + } + + /** * Convert this {@code SqlTimestmap} object to a {@link Timestamp}. * From 4a83ff633cbce8c9c26267a9a3e20fe30f23bda3 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 6 Nov 2019 13:02:20 +0800 Subject: [PATCH 13/29] [FLINK-14599][table-planner-blink] Fix current_timestamp built-in function --- .../planner/codegen/CodeGeneratorContext.scala | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala index 8c47cb7b556fd..c52e43073a7a7 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala @@ -410,7 +410,8 @@ class CodeGeneratorContext(val tableConfig: TableConfig) { val fieldTerm = s"timestamp" val field = s""" - |final long $fieldTerm = java.lang.System.currentTimeMillis(); + |final $SQL_TIMESTAMP_TERM $fieldTerm = + | $SQL_TIMESTAMP_TERM.fromEpochMillis(java.lang.System.currentTimeMillis()); |""".stripMargin reusablePerRecordStatements.add(field) fieldTerm @@ -431,7 +432,7 @@ class CodeGeneratorContext(val tableConfig: TableConfig) { // adopted from org.apache.calcite.runtime.SqlFunctions.currentTime() val field = s""" - |$fieldTerm = (int) ($timestamp % ${DateTimeUtils.MILLIS_PER_DAY}); + |$fieldTerm = (int) ($timestamp.getMillisecond() % ${DateTimeUtils.MILLIS_PER_DAY}); |if (time < 0) { | time += ${DateTimeUtils.MILLIS_PER_DAY}; |} @@ -449,12 +450,14 @@ class CodeGeneratorContext(val tableConfig: TableConfig) { val timestamp = addReusableTimestamp() // declaration - reusableMemberStatements.add(s"private long $fieldTerm;") + reusableMemberStatements.add(s"private $SQL_TIMESTAMP_TERM $fieldTerm;") // assignment val field = s""" - |$fieldTerm = $timestamp + java.util.TimeZone.getDefault().getOffset($timestamp); + |$fieldTerm = $SQL_TIMESTAMP_TERM.fromEpochMillis( + | $timestamp.getMillisecond() + + | java.util.TimeZone.getDefault().getOffset($timestamp.getMillisecond())); |""".stripMargin reusablePerRecordStatements.add(field) fieldTerm @@ -475,7 +478,7 @@ class CodeGeneratorContext(val tableConfig: TableConfig) { // adopted from org.apache.calcite.runtime.SqlFunctions.localTime() val field = s""" - |$fieldTerm = (int) ($localtimestamp % ${DateTimeUtils.MILLIS_PER_DAY}); + |$fieldTerm = (int) ($localtimestamp.getMillisecond() % ${DateTimeUtils.MILLIS_PER_DAY}); |""".stripMargin reusablePerRecordStatements.add(field) fieldTerm @@ -497,7 +500,7 @@ class CodeGeneratorContext(val tableConfig: TableConfig) { // adopted from org.apache.calcite.runtime.SqlFunctions.currentDate() val field = s""" - |$fieldTerm = (int) ($timestamp / ${DateTimeUtils.MILLIS_PER_DAY}); + |$fieldTerm = (int) ($timestamp.getMillisecond() / ${DateTimeUtils.MILLIS_PER_DAY}); |if ($time < 0) { | $fieldTerm -= 1; |} From 1e4b5f41e8618a428767ec33e6418961dac343e8 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 6 Nov 2019 14:10:34 +0800 Subject: [PATCH 14/29] [FLINK-14599][table-planner-blink] Fix udx which use long as Timestamp type in parameters or result --- .../codegen/calls/ScalarFunctionCallGen.scala | 29 +++++++++++++++---- .../utils/UserDefinedFunctionUtils.scala | 5 +++- 2 files changed, 27 insertions(+), 7 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarFunctionCallGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarFunctionCallGen.scala index f1deb87d11cf2..5aef2fedf307a 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarFunctionCallGen.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarFunctionCallGen.scala @@ -18,6 +18,9 @@ package org.apache.flink.table.planner.codegen.calls +import java.sql.Timestamp +import java.time.LocalDateTime + import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.GenericTypeInfo import org.apache.flink.table.dataformat.DataFormatConverters @@ -29,7 +32,7 @@ import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, GenerateUti import org.apache.flink.table.planner.functions.utils.UserDefinedFunctionUtils import org.apache.flink.table.planner.functions.utils.UserDefinedFunctionUtils._ import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType -import org.apache.flink.table.types.logical.LogicalType +import org.apache.flink.table.types.logical.{BigIntType, LogicalType, LogicalTypeRoot} import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType /** @@ -66,7 +69,15 @@ class ScalarFunctionCallGen(scalarFunction: ScalarFunction) extends CallGenerato boxedTypeTermForType(returnType) } val resultTerm = ctx.addReusableLocalVariable(resultTypeTerm, "result") - val evalResult = s"$functionReference.eval(${parameters.map(_.resultTerm).mkString(", ")})" + val evalResult = + if (returnType.getTypeRoot == LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE) { + s""" + |$SQL_TIMESTAMP_TERM.fromEpochMillis( + | $functionReference.eval(${parameters.map(_.resultTerm).mkString(", ")})) + """.stripMargin + } else { + s"$functionReference.eval(${parameters.map(_.resultTerm).mkString(", ")})" + } val resultExternalType = UserDefinedFunctionUtils.getResultTypeOfScalarFunction( scalarFunction, arguments, operandTypes) val setResult = { @@ -136,13 +147,19 @@ object ScalarFunctionCallGen { } parameterClasses.zipWithIndex.zip(operands).map { case ((paramClass, i), operandExpr) => + var newOperandExpr = operandExpr + if (operandExpr.resultType.getTypeRoot == LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE + && (paramClass == classOf[Long] || paramClass == classOf[java.lang.Long])) { + val targetType = new BigIntType(operandExpr.resultType.isNullable) + newOperandExpr = ScalarOperatorGens.generateReinterpret(ctx, operandExpr, targetType) + } if (paramClass.isPrimitive) { - operandExpr + newOperandExpr } else { val externalResultTerm = genToExternalIfNeeded( - ctx, signatureTypes(i), operandExpr.resultTerm) - val exprOrNull = s"${operandExpr.nullTerm} ? null : ($externalResultTerm)" - operandExpr.copy(resultTerm = exprOrNull) + ctx, signatureTypes(i), newOperandExpr.resultTerm) + val exprOrNull = s"${newOperandExpr.nullTerm} ? null : ($externalResultTerm)" + newOperandExpr.copy(resultTerm = exprOrNull) } } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/UserDefinedFunctionUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/UserDefinedFunctionUtils.scala index f1f782dca1f73..575a0cd4a1189 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/UserDefinedFunctionUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/UserDefinedFunctionUtils.scala @@ -32,7 +32,7 @@ import org.apache.flink.table.runtime.types.ClassLogicalTypeConverter.{getDefaul import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.{fromDataTypeToLogicalType, fromLogicalTypeToDataType} import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType -import org.apache.flink.table.runtime.typeutils.TypeCheckUtils.isAny +import org.apache.flink.table.runtime.typeutils.TypeCheckUtils.{isAny, isLong, isTimestamp} import org.apache.flink.table.types.DataType import org.apache.flink.table.types.logical.{LogicalType, LogicalTypeRoot, RowType} import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType @@ -749,6 +749,9 @@ object UserDefinedFunctionUtils { val paraInternalType = fromDataTypeToLogicalType(parameterType) if (isAny(internal) && isAny(paraInternalType)) { getDefaultExternalClassForType(internal) == getDefaultExternalClassForType(paraInternalType) + } else if ((isTimestamp(internal) && isLong(paraInternalType)) + || (isLong(internal) && isTimestamp(paraInternalType))) { + true } else { // There is a special equal to GenericType. We need rewrite type extract to BaseRow etc... paraInternalType == internal || From f79164e0b68db625eecf589a2d96d81dd10ac414 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 6 Nov 2019 14:17:51 +0800 Subject: [PATCH 15/29] [FLINK-14599][table-planner-blink] Fix LookupJoinTest::testInvalidLookupTableFunction() --- .../planner/plan/stream/sql/join/LookupJoinTest.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.scala index 5bf012b943835..10eac476ededc 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.scala @@ -144,7 +144,7 @@ class LookupJoinTest extends TableTestBase with Serializable { "SELECT * FROM T AS T JOIN temporalTable2 " + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id AND T.b = D.name AND T.ts = D.ts", "Expected: eval(java.lang.Integer, org.apache.flink.table.dataformat.BinaryString, " + - "java.lang.Long) \n" + + "org.apache.flink.table.dataformat.SqlTimestamp) \n" + "Actual: eval(java.lang.Integer, java.lang.String, java.time.LocalDateTime)", classOf[TableException] ) @@ -178,7 +178,8 @@ class LookupJoinTest extends TableTestBase with Serializable { "SELECT * FROM T AS T JOIN temporalTable7 " + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id AND T.b = D.name AND T.ts = D.ts", "Expected: eval(java.util.concurrent.CompletableFuture, " + - "java.lang.Integer, org.apache.flink.table.dataformat.BinaryString, java.lang.Long) \n" + + "java.lang.Integer, org.apache.flink.table.dataformat.BinaryString, " + + "org.apache.flink.table.dataformat.SqlTimestamp) \n" + "Actual: eval(java.lang.Integer, org.apache.flink.table.dataformat.BinaryString, " + "java.time.LocalDateTime)", classOf[TableException] @@ -190,7 +191,8 @@ class LookupJoinTest extends TableTestBase with Serializable { "SELECT * FROM T AS T JOIN temporalTable8 " + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id AND T.b = D.name AND T.ts = D.ts", "Expected: eval(java.util.concurrent.CompletableFuture, " + - "java.lang.Integer, org.apache.flink.table.dataformat.BinaryString, java.lang.Long) \n" + + "java.lang.Integer, org.apache.flink.table.dataformat.BinaryString, " + + "org.apache.flink.table.dataformat.SqlTimestamp) \n" + "Actual: eval(java.util.concurrent.CompletableFuture, " + "java.lang.Integer, java.lang.String, java.time.LocalDateTime)", classOf[TableException] @@ -208,7 +210,8 @@ class LookupJoinTest extends TableTestBase with Serializable { "SELECT * FROM T AS T JOIN temporalTable10 " + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id AND T.b = D.name AND T.ts = D.ts", "Expected: eval(java.util.concurrent.CompletableFuture, " + - "java.lang.Integer, org.apache.flink.table.dataformat.BinaryString, java.lang.Long) \n" + + "java.lang.Integer, org.apache.flink.table.dataformat.BinaryString, " + + "org.apache.flink.table.dataformat.SqlTimestamp) \n" + "Actual: eval(org.apache.flink.streaming.api.functions.async.ResultFuture, " + "java.lang.Integer, org.apache.flink.table.dataformat.BinaryString, java.lang.Long)", classOf[TableException] From e76461e463608a09d9776561352bb0f077761245 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 6 Nov 2019 14:37:56 +0800 Subject: [PATCH 16/29] [FLINK-14599][table-planner-blink] Fix comparator of Timestamp type --- .../table/planner/codegen/GenerateUtils.scala | 3 +-- .../planner/codegen/sort/SortCodeGenerator.scala | 15 ++++++++++----- .../table/runtime/operators/sort/SortUtil.java | 10 ++++++++++ 3 files changed, 21 insertions(+), 7 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala index 7b62455da09d7..c25ad73c25a2e 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala @@ -662,8 +662,7 @@ object GenerateUtils { leftTerm: String, rightTerm: String): String = t.getTypeRoot match { case BOOLEAN => s"($leftTerm == $rightTerm ? 0 : ($leftTerm ? 1 : -1))" - case DATE | TIME_WITHOUT_TIME_ZONE | TIMESTAMP_WITHOUT_TIME_ZONE | - TIMESTAMP_WITH_LOCAL_TIME_ZONE => + case DATE | TIME_WITHOUT_TIME_ZONE | TIMESTAMP_WITH_LOCAL_TIME_ZONE => s"($leftTerm > $rightTerm ? 1 : $leftTerm < $rightTerm ? -1 : 0)" case _ if PlannerTypeUtils.isPrimitive(t) => s"($leftTerm > $rightTerm ? 1 : $leftTerm < $rightTerm ? -1 : 0)" diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/sort/SortCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/sort/SortCodeGenerator.scala index bf1e259ad1e36..410ea6daeb56d 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/sort/SortCodeGenerator.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/sort/SortCodeGenerator.scala @@ -19,14 +19,14 @@ package org.apache.flink.table.planner.codegen.sort import org.apache.flink.table.api.TableConfig -import org.apache.flink.table.dataformat.{BinaryRow, Decimal} +import org.apache.flink.table.dataformat.{BinaryRow, Decimal, SqlTimestamp} import org.apache.flink.table.planner.codegen.CodeGenUtils.{BASE_ROW, SEGMENT, newName} import org.apache.flink.table.planner.codegen.Indenter.toISC import org.apache.flink.table.runtime.generated.{GeneratedNormalizedKeyComputer, GeneratedRecordComparator, NormalizedKeyComputer, RecordComparator} import org.apache.flink.table.runtime.operators.sort.SortUtil import org.apache.flink.table.runtime.types.PlannerTypeUtils import org.apache.flink.table.types.logical.LogicalTypeRoot._ -import org.apache.flink.table.types.logical.{DecimalType, LogicalType} +import org.apache.flink.table.types.logical.{DecimalType, LogicalType, TimestampType} import scala.collection.mutable @@ -389,6 +389,8 @@ class SortCodeGenerator( t match { case dt: DecimalType => s"get$prefix($index, ${dt.getPrecision}, ${dt.getScale})" + case dt: TimestampType => + s"get$prefix($index, ${dt.getPrecision})" case _ => s"get$prefix($index)" } @@ -415,7 +417,7 @@ class SortCodeGenerator( case DECIMAL => "Decimal" case DATE => "Int" case TIME_WITHOUT_TIME_ZONE => "Int" - case TIMESTAMP_WITHOUT_TIME_ZONE => "Long" + case TIMESTAMP_WITHOUT_TIME_ZONE => "Timestamp" case INTERVAL_YEAR_MONTH => "Int" case INTERVAL_DAY_TIME => "Long" case _ => null @@ -437,7 +439,9 @@ class SortCodeGenerator( t.getTypeRoot match { case _ if PlannerTypeUtils.isPrimitive(t) => true case VARCHAR | CHAR | VARBINARY | BINARY | - DATE | TIME_WITHOUT_TIME_ZONE | TIMESTAMP_WITHOUT_TIME_ZONE => true + DATE | TIME_WITHOUT_TIME_ZONE => true + case TIMESTAMP_WITHOUT_TIME_ZONE => + SqlTimestamp.isCompact(t.asInstanceOf[TimestampType].getPrecision) case DECIMAL => Decimal.isCompact(t.asInstanceOf[DecimalType].getPrecision) case _ => false } @@ -452,7 +456,8 @@ class SortCodeGenerator( case FLOAT => 4 case DOUBLE => 8 case BIGINT => 8 - case TIMESTAMP_WITHOUT_TIME_ZONE => 8 + case TIMESTAMP_WITHOUT_TIME_ZONE + if SqlTimestamp.isCompact(t.asInstanceOf[TimestampType].getPrecision) => 8 case INTERVAL_YEAR_MONTH => 4 case INTERVAL_DAY_TIME => 8 case DATE => 4 diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SortUtil.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SortUtil.java index 0494f2e60db1b..233410ea89583 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SortUtil.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SortUtil.java @@ -22,6 +22,7 @@ import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.table.dataformat.BinaryString; import org.apache.flink.table.dataformat.Decimal; +import org.apache.flink.table.dataformat.SqlTimestamp; import java.nio.ByteOrder; @@ -135,6 +136,15 @@ public static void putBinaryNormalizedKey( } } + /** + * Support the compact precision SqlTimestamp. + */ + public static void putTimestampNormalizedKey( + SqlTimestamp value, MemorySegment target, int offset, int numBytes) { + assert value.getNanoOfMillisecond() == 0; + putLongNormalizedKey(value.getMillisecond(), target, offset, numBytes); + } + public static int compareBinary(byte[] a, byte[] b) { return compareBinary(a, 0, a.length, b, 0, b.length); } From 89e2ab24681e17c8c168ed42b93ba01157e198c6 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 6 Nov 2019 15:07:23 +0800 Subject: [PATCH 17/29] [FLINK-14599][table-planner-blink] Fix ROWTIME/PROCTIME and window props in window --- .../table/planner/codegen/GenerateUtils.scala | 15 ++++++++---- .../agg/AggsHandlerCodeGenerator.scala | 23 +++++++++++++------ .../agg/batch/WindowCodeGenerator.scala | 15 ++++++------ .../stream/StreamExecTableSourceScan.scala | 2 +- .../runtime/stream/sql/AggregateITCase.scala | 6 ++--- .../runtime/utils/StreamingTestBase.scala | 21 ----------------- .../match/RowtimeProcessFunction.java | 2 +- 7 files changed, 37 insertions(+), 47 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala index c25ad73c25a2e..fdda96c4d4cdf 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala @@ -445,10 +445,13 @@ object GenerateUtils { def generateProctimeTimestamp( ctx: CodeGeneratorContext, contextTerm: String): GeneratedExpression = { - val resultTerm = ctx.addReusableLocalVariable("long", "result") + val resultType = new TimestampType(3) + val resultTypeTerm = primitiveTypeTermForType(resultType) + val resultTerm = ctx.addReusableLocalVariable(resultTypeTerm, "result") val resultCode = s""" - |$resultTerm = $contextTerm.timerService().currentProcessingTime(); + |$resultTerm = $SQL_TIMESTAMP_TERM.fromEpochMillis( + | $contextTerm.timerService().currentProcessingTime()); |""".stripMargin.trim // the proctime has been materialized, so it's TIMESTAMP now, not PROCTIME_INDICATOR GeneratedExpression(resultTerm, NEVER_NULL, resultCode, new TimestampType(3)) @@ -462,13 +465,15 @@ object GenerateUtils { def generateRowtimeAccess( ctx: CodeGeneratorContext, contextTerm: String): GeneratedExpression = { + val resultType = new TimestampType(true, TimestampKind.ROWTIME, 3) + val resultTypeTerm = primitiveTypeTermForType(resultType) val Seq(resultTerm, nullTerm) = ctx.addReusableLocalVariables( - ("Long", "result"), + (resultTypeTerm, "result"), ("boolean", "isNull")) val accessCode = s""" - |$resultTerm = $contextTerm.timestamp(); + |$resultTerm = $SQL_TIMESTAMP_TERM.fromEpochMillis($contextTerm.timestamp()); |if ($resultTerm == null) { | throw new RuntimeException("Rowtime timestamp is null. Please make sure that a " + | "proper TimestampAssigner is defined and the stream environment uses the EventTime " + @@ -481,7 +486,7 @@ object GenerateUtils { resultTerm, nullTerm, accessCode, - new TimestampType(true, TimestampKind.ROWTIME, 3)) + resultType) } /** diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/AggsHandlerCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/AggsHandlerCodeGenerator.scala index 9ab0b3b09c85a..c6111af6dbe66 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/AggsHandlerCodeGenerator.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/AggsHandlerCodeGenerator.scala @@ -941,20 +941,29 @@ class AggsHandlerCodeGenerator( windowProperties: Seq[PlannerWindowProperty]): Seq[GeneratedExpression] = { windowProperties.map { case w: PlannerWindowStart => - // return a Timestamp(Internal is long) + // return a Timestamp(Internal is SqlTimestamp) GeneratedExpression( - s"$NAMESPACE_TERM.getStart()", "false", "", w.resultType) + s"$SQL_TIMESTAMP_TERM.fromEpochMillis($NAMESPACE_TERM.getStart())", + "false", + "", + w.resultType) case w: PlannerWindowEnd => - // return a Timestamp(Internal is long) + // return a Timestamp(Internal is SqlTimestamp) GeneratedExpression( - s"$NAMESPACE_TERM.getEnd()", "false", "", w.resultType) + s"$SQL_TIMESTAMP_TERM.fromEpochMillis($NAMESPACE_TERM.getEnd())", + "false", + "", + w.resultType) case r: PlannerRowtimeAttribute => - // return a rowtime, use long as internal type + // return a rowtime, use SqlTimestamp as internal type GeneratedExpression( - s"$NAMESPACE_TERM.getEnd() - 1", "false", "", r.resultType) + s"$SQL_TIMESTAMP_TERM.fromEpochMillis($NAMESPACE_TERM.getEnd() - 1)", + "false", + "", + r.resultType) case p: PlannerProctimeAttribute => // ignore this property, it will be null at the position later - GeneratedExpression("-1L", "true", "", p.resultType) + GeneratedExpression(s"$SQL_TIMESTAMP_TERM.fromEpochMillis(-1L)", "true", "", p.resultType) } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/WindowCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/WindowCodeGenerator.scala index 5ff275ed4305a..84f9fb04de818 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/WindowCodeGenerator.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/WindowCodeGenerator.scala @@ -26,7 +26,7 @@ import org.apache.flink.table.functions.{AggregateFunction, UserDefinedFunction} import org.apache.flink.table.planner.JLong import org.apache.flink.table.planner.calcite.FlinkRelBuilder.PlannerNamedWindowProperty import org.apache.flink.table.planner.calcite.FlinkTypeFactory -import org.apache.flink.table.planner.codegen.CodeGenUtils.{BINARY_ROW, boxedTypeTermForType, newName} +import org.apache.flink.table.planner.codegen.CodeGenUtils.{BINARY_ROW, SQL_TIMESTAMP_TERM, boxedTypeTermForType, newName} import org.apache.flink.table.planner.codegen.GenerateUtils.generateFieldAccess import org.apache.flink.table.planner.codegen.GeneratedExpression.{NEVER_NULL, NO_CODE} import org.apache.flink.table.planner.codegen.OperatorCodeGenerator.generateCollect @@ -47,7 +47,6 @@ import org.apache.flink.table.runtime.util.RowIterator import org.apache.flink.table.types.logical.LogicalTypeRoot.INTERVAL_DAY_TIME import org.apache.flink.table.types.logical._ import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasRoot - import org.apache.calcite.avatica.util.DateTimeUtils import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.AggregateCall @@ -632,17 +631,17 @@ abstract class WindowCodeGenerator( // get assigned window start timestamp def windowProps(size: Expression) = { val (startWValue, endWValue, rowTimeValue) = ( - s"$currentWindowTerm.getStart()", - s"$currentWindowTerm.getEnd()", - s"$currentWindowTerm.maxTimestamp()") + s"$SQL_TIMESTAMP_TERM.fromEpochMillis($currentWindowTerm.getStart())", + s"$SQL_TIMESTAMP_TERM.fromEpochMillis($currentWindowTerm.getEnd())", + s"$SQL_TIMESTAMP_TERM.fromEpochMillis($currentWindowTerm.maxTimestamp())") val start = if (startPos.isDefined) { - s"$propTerm.setLong($lastPos + ${startPos.get}, $startWValue);" + s"$propTerm.setTimestamp($lastPos + ${startPos.get}, $startWValue, 3);" } else "" val end = if (endPos.isDefined) { - s"$propTerm.setLong($lastPos + ${endPos.get}, $endWValue);" + s"$propTerm.setTimestamp($lastPos + ${endPos.get}, $endWValue, 3);" } else "" val rowTime = if (rowTimePos.isDefined) { - s"$propTerm.setLong($lastPos + ${rowTimePos.get}, $rowTimeValue);" + s"$propTerm.setTimestamp($lastPos + ${rowTimePos.get}, $rowTimeValue, 3);" } else "" (start, end, rowTime) } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecTableSourceScan.scala index 92ce7b48192c9..e258e1c143c50 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecTableSourceScan.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecTableSourceScan.scala @@ -273,7 +273,7 @@ private class PeriodicWatermarkAssignerWrapper( override def getCurrentWatermark: Watermark = assigner.getWatermark override def extractTimestamp(row: BaseRow, previousElementTimestamp: Long): Long = { - val timestamp: Long = row.getLong(timeFieldIdx) + val timestamp: Long = row.getTimestamp(timeFieldIdx, 3).getMillisecond assigner.nextTimestamp(timestamp) 0L } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala index 88e7aad7d4476..0d3417af3d8c4 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala @@ -517,10 +517,8 @@ class AggregateITCase( case (a, b, c, d, e) => (b, a, c, d, e) }).assignTimestampsAndWatermarks( new TimestampAndWatermarkWithOffset[(Long, Int, Int, String, Long)](0L)) - .toTable(tEnv, 'rowtime, 'a, 'c, 'd, 'e) + .toTable(tEnv, 'rowtime.rowtime, 'a, 'c, 'd, 'e) tEnv.registerTable("MyTable", t) - val sourceTable = tEnv.scan("MyTable") - addTableWithWatermark("MyTable1", sourceTable, "rowtime", 0) val innerSql = """ @@ -528,7 +526,7 @@ class AggregateITCase( | SUM(DISTINCT e) b, | MIN(DISTINCT e) c, | COUNT(DISTINCT e) d - |FROM MyTable1 + |FROM MyTable |GROUP BY a, TUMBLE(rowtime, INTERVAL '0.005' SECOND) """.stripMargin diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingTestBase.scala index 6996ada1c7279..9a01b040d49ce 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingTestBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingTestBase.scala @@ -57,25 +57,4 @@ class StreamingTestBase extends AbstractTestBase { val setting = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build() this.tEnv = StreamTableEnvironment.create(env, setting) } - - def addTableWithWatermark( - tableName: String, - sourceTable: Table, - rowtimeField: String, - offset: Long): Unit = { - val sourceRel = TableTestUtil.toRelNode(sourceTable) - val rowtimeFieldIdx = sourceRel.getRowType.getFieldNames.indexOf(rowtimeField) - if (rowtimeFieldIdx < 0) { - throw new TableException(s"$rowtimeField does not exist, please check it") - } - val watermarkAssigner = new LogicalWatermarkAssigner( - sourceRel.getCluster, - sourceRel.getTraitSet, - sourceRel, - Some(rowtimeFieldIdx), - Option(offset) - ) - val queryOperation = new PlannerQueryOperation(watermarkAssigner) - tEnv.registerTable(tableName, TableTestUtil.createTable(tEnv, queryOperation)) - } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/RowtimeProcessFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/RowtimeProcessFunction.java index 229db2111efd8..824c6cbdeb08e 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/RowtimeProcessFunction.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/RowtimeProcessFunction.java @@ -44,7 +44,7 @@ public RowtimeProcessFunction(int rowtimeIdx, TypeInformation returnTyp @Override public void processElement(BaseRow value, Context ctx, Collector out) throws Exception { - long timestamp = value.getLong(rowtimeIdx); + long timestamp = value.getTimestamp(rowtimeIdx, 3).getMillisecond(); ((TimestampedCollector) out).setAbsoluteTimestamp(timestamp); out.collect(value); } From d3b1bc9c182fa6a4346459a3d9c70ac83d60128d Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 6 Nov 2019 15:34:44 +0800 Subject: [PATCH 18/29] [FLINK-14599][table-planner-blink] Fix PROCTIME in match recognize --- .../flink/table/planner/codegen/MatchCodeGenerator.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala index 4e492b4b973e5..31c0c0d01de36 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala @@ -388,13 +388,15 @@ class MatchCodeGenerator( } private def generateProctimeTimestamp(): GeneratedExpression = { - val resultTerm = ctx.addReusableLocalVariable("long", "result") + val resultType = new TimestampType(3) + val resultTypeTerm = primitiveTypeTermForType(resultType) + val resultTerm = ctx.addReusableLocalVariable(resultTypeTerm, "result") val resultCode = s""" - |$resultTerm = $contextTerm.currentProcessingTime(); + |$resultTerm = $SQL_TIMESTAMP_TERM.fromEpochMillis($contextTerm.currentProcessingTime()); |""".stripMargin.trim // the proctime has been materialized, so it's TIMESTAMP now, not PROCTIME_INDICATOR - GeneratedExpression(resultTerm, NEVER_NULL, resultCode, new TimestampType(3)) + GeneratedExpression(resultTerm, NEVER_NULL, resultCode, resultType) } /** From 287c4a9d8452011ada4bb5d0669647fb632fa315 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 6 Nov 2019 16:12:59 +0800 Subject: [PATCH 19/29] [FLINK-14599][table-planner-blink] Introduce LegacyTimestampTypeInfo and LegacyLocalDateTimeTypeInfo to hold precision on conversion --- .../LegacyLocalDateTimeTypeInfo.java | 62 +++++++++++++++++++ .../typeutils/LegacyTimestampTypeInfo.java | 62 +++++++++++++++++++ 2 files changed, 124 insertions(+) create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/LegacyLocalDateTimeTypeInfo.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/LegacyTimestampTypeInfo.java diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/LegacyLocalDateTimeTypeInfo.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/LegacyLocalDateTimeTypeInfo.java new file mode 100644 index 0000000000000..0f110c3f5545d --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/LegacyLocalDateTimeTypeInfo.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.runtime.typeutils; + +import org.apache.flink.api.common.typeinfo.LocalTimeTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.base.LocalDateTimeComparator; +import org.apache.flink.api.common.typeutils.base.LocalDateTimeSerializer; + +import java.time.LocalDateTime; + +/** + * {@link TypeInformation} for {@link LocalDateTime}. + * + *

The difference between Types.LOCAL_DATE_TIME is this TypeInformation holds a precision + */ +public class LegacyLocalDateTimeTypeInfo extends LocalTimeTypeInfo { + + private final int precision; + + public LegacyLocalDateTimeTypeInfo(int precision) { + super( + LocalDateTime.class, + LocalDateTimeSerializer.INSTANCE, + LocalDateTimeComparator.class); + this.precision = precision; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof LegacyLocalDateTimeTypeInfo)) { + return false; + } + LegacyLocalDateTimeTypeInfo that = (LegacyLocalDateTimeTypeInfo) obj; + return this.precision == that.precision; + } + + @Override + public String toString() { + return String.format("Timestamp(%d)", precision); + } + + public int getPrecision() { + return precision; + } +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/LegacyTimestampTypeInfo.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/LegacyTimestampTypeInfo.java new file mode 100644 index 0000000000000..ca657fb2306a0 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/LegacyTimestampTypeInfo.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.runtime.typeutils; + +import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.base.SqlTimestampComparator; + +import java.sql.Timestamp; + +/** + * {@link TypeInformation} for {@link Timestamp}. + * + *

The difference between Types.SQL_TIMESTAMP is this TypeInformation holds a precision + */ +public class LegacyTimestampTypeInfo extends SqlTimeTypeInfo { + + private final int precision; + + @SuppressWarnings("unchecked") + public LegacyTimestampTypeInfo(int precision) { + super( + Timestamp.class, + org.apache.flink.api.common.typeutils.base.SqlTimestampSerializer.INSTANCE, + (Class) SqlTimestampComparator.class); + this.precision = precision; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof LegacyTimestampTypeInfo)) { + return false; + } + LegacyTimestampTypeInfo that = (LegacyTimestampTypeInfo) obj; + return this.precision == that.precision; + } + + @Override + public String toString() { + return String.format("Timestamp(%d)", precision); + } + + public int getPrecision() { + return precision; + } +} From 52cad966b5b77d28e373b91ae87088c70f8839fe Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 6 Nov 2019 18:04:13 +0800 Subject: [PATCH 20/29] [FLINK-14599][table-planner-blink] Support precision of Timestamp type in table source --- .../runtime/batch/sql/TableSourceITCase.scala | 61 ++++++++++++++----- .../dataformat/DataFormatConverters.java | 8 +++ .../types/LogicalTypeDataTypeConverter.java | 9 +++ .../types/TypeInfoDataTypeConverter.java | 10 ++- 4 files changed, 72 insertions(+), 16 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TableSourceITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TableSourceITCase.scala index f3b7e6dbe410c..362a9b6a4a757 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TableSourceITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TableSourceITCase.scala @@ -27,11 +27,14 @@ import org.apache.flink.table.planner.runtime.utils.{BatchTestBase, TestData} import org.apache.flink.table.planner.utils.{TestDataTypeTableSource, TestFileInputFormatTableSource, TestFilterableTableSource, TestInputFormatTableSource, TestNestedProjectableTableSource, TestPartitionableSourceFactory, TestProjectableTableSource, TestTableSources} import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter import org.apache.flink.types.Row - import org.junit.{Before, Test} - import java.io.FileWriter import java.lang.{Boolean => JBool, Integer => JInt, Long => JLong} +import java.math.{BigDecimal => JDecimal} +import java.sql.Timestamp +import java.time.LocalDateTime + +import scala.collection.mutable class TableSourceITCase extends BatchTestBase { @@ -225,28 +228,58 @@ class TableSourceITCase extends BatchTestBase { } @Test - def testDecimalSource(): Unit = { + def testMultiTypeSource(): Unit = { val tableSchema = TableSchema.builder().fields( - Array("a", "b", "c", "d"), + Array("a", "b", "c", "d", "e", "f"), Array( DataTypes.INT(), DataTypes.DECIMAL(5, 2), DataTypes.VARCHAR(5), - DataTypes.CHAR(5))).build() + DataTypes.CHAR(5), + DataTypes.TIMESTAMP(9).bridgedTo(classOf[LocalDateTime]), + DataTypes.TIMESTAMP(6).bridgedTo(classOf[Timestamp]) + ) + ).build() + + val ints = List(1, 2, 3, 4, null) + val decimals = List( + new JDecimal(5.1), new JDecimal(6.1), new JDecimal(7.1), new JDecimal(8.123), null) + val varchars = List("1", "12", "123", "1234", null) + val chars = List("1", "12", "123", "1234", null) + val datetimes = List( + LocalDateTime.of(1969, 1, 1, 0, 0, 0, 123456789), + LocalDateTime.of(1970, 1, 1, 0, 0, 0, 123456000), + LocalDateTime.of(1971, 1, 1, 0, 0, 0, 123000000), + LocalDateTime.of(1972, 1, 1, 0, 0, 0, 0), + null) + // TODO: The data type is Timestamp(6), we may need to truncate the last 3 digits + // of the fractional seconds + val timestamps = List( + Timestamp.valueOf("1969-01-01 00:00:00.123456789"), + Timestamp.valueOf("1970-01-01 00:00:00.123456"), + Timestamp.valueOf("1971-01-01 00:00:00.123"), + Timestamp.valueOf("1972-01-01 00:00:00"), + null + ) + + val data = new mutable.MutableList[Row] + + for (i <- ints.indices) { + data += row(ints(i), decimals(i), varchars(i), chars(i), datetimes(i), timestamps(i)) + } + val tableSource = new TestDataTypeTableSource( tableSchema, - Seq( - row(1, new java.math.BigDecimal(5.1), "1", "1"), - row(2, new java.math.BigDecimal(6.1), "12", "12"), - row(3, new java.math.BigDecimal(7.1), "123", "123") - )) + data.seq) tEnv.registerTableSource("MyInputFormatTable", tableSource) checkResult( - "SELECT a, b, c, d FROM MyInputFormatTable", + "SELECT a, b, c, d, e, f FROM MyInputFormatTable", Seq( - row(1, "5.10", "1", "1"), - row(2, "6.10", "12", "12"), - row(3, "7.10", "123", "123")) + row(1, "5.10", "1", "1", "1969-01-01T00:00:00.123456789", "1969-01-01T00:00:00.123456789"), + row(2, "6.10", "12", "12", "1970-01-01T00:00:00.123456", "1970-01-01T00:00:00.123456"), + row(3, "7.10", "123", "123", "1971-01-01T00:00:00.123", "1971-01-01T00:00:00.123"), + row(4, "8.12", "1234", "1234", "1972-01-01T00:00", "1972-01-01T00:00"), + row(null, null, null, null, null, null)) ) } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java index 6339d8ca3c164..a389972b0a899 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java @@ -35,6 +35,8 @@ import org.apache.flink.table.runtime.typeutils.BigDecimalTypeInfo; import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo; import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo; +import org.apache.flink.table.runtime.typeutils.LegacyLocalDateTimeTypeInfo; +import org.apache.flink.table.runtime.typeutils.LegacyTimestampTypeInfo; import org.apache.flink.table.types.CollectionDataType; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.KeyValueDataType; @@ -238,6 +240,12 @@ public static DataFormatConverter getConverterForDataType(DataType originDataTyp } else if (typeInfo instanceof BigDecimalTypeInfo) { BigDecimalTypeInfo decimalType = (BigDecimalTypeInfo) typeInfo; return new BigDecimalConverter(decimalType.precision(), decimalType.scale()); + } else if (typeInfo instanceof LegacyLocalDateTimeTypeInfo) { + LegacyLocalDateTimeTypeInfo dateTimeType = (LegacyLocalDateTimeTypeInfo) typeInfo; + return new LocalDateTimeConverter(dateTimeType.getPrecision()); + } else if (typeInfo instanceof LegacyTimestampTypeInfo) { + LegacyTimestampTypeInfo timestampType = (LegacyTimestampTypeInfo) typeInfo; + return new TimestampConverter(timestampType.getPrecision()); } if (clazz == BinaryGeneric.class) { diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/LogicalTypeDataTypeConverter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/LogicalTypeDataTypeConverter.java index 2df059fe58b5f..74d67ef30f72b 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/LogicalTypeDataTypeConverter.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/LogicalTypeDataTypeConverter.java @@ -27,6 +27,8 @@ import org.apache.flink.table.runtime.typeutils.BigDecimalTypeInfo; import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo; import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo; +import org.apache.flink.table.runtime.typeutils.LegacyLocalDateTimeTypeInfo; +import org.apache.flink.table.runtime.typeutils.LegacyTimestampTypeInfo; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.DecimalType; @@ -35,6 +37,7 @@ import org.apache.flink.table.types.logical.MapType; import org.apache.flink.table.types.logical.MultisetType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimestampType; import org.apache.flink.table.types.logical.TypeInformationAnyType; import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor; import org.apache.flink.table.types.utils.TypeConversions; @@ -98,6 +101,12 @@ protected LogicalType defaultMethod(LogicalType logicalType) { } else if (typeInfo instanceof BigDecimalTypeInfo) { BigDecimalTypeInfo decimalType = (BigDecimalTypeInfo) typeInfo; return new DecimalType(decimalType.precision(), decimalType.scale()); + } else if (typeInfo instanceof LegacyLocalDateTimeTypeInfo) { + LegacyLocalDateTimeTypeInfo dateTimeType = (LegacyLocalDateTimeTypeInfo) typeInfo; + return new TimestampType(dateTimeType.getPrecision()); + } else if (typeInfo instanceof LegacyTimestampTypeInfo) { + LegacyTimestampTypeInfo timstampType = (LegacyTimestampTypeInfo) typeInfo; + return new TimestampType(timstampType.getPrecision()); } else { return new TypeInformationAnyType<>(typeInfo); } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoDataTypeConverter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoDataTypeConverter.java index 750cc5f6e9199..cbaf55fb1f789 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoDataTypeConverter.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoDataTypeConverter.java @@ -37,6 +37,8 @@ import org.apache.flink.table.runtime.typeutils.BigDecimalTypeInfo; import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo; import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo; +import org.apache.flink.table.runtime.typeutils.LegacyLocalDateTimeTypeInfo; +import org.apache.flink.table.runtime.typeutils.LegacyTimestampTypeInfo; import org.apache.flink.table.runtime.typeutils.SqlTimestampTypeInfo; import org.apache.flink.table.types.CollectionDataType; import org.apache.flink.table.types.DataType; @@ -105,11 +107,15 @@ public static TypeInformation fromDataTypeToTypeInfo(DataType dataType) { switch (logicalType.getTypeRoot()) { case TIMESTAMP_WITHOUT_TIME_ZONE: TimestampType timestampType = (TimestampType) logicalType; + int precision = timestampType.getPrecision(); if (timestampType.getKind() == TimestampKind.REGULAR) { return clazz == SqlTimestamp.class ? - new SqlTimestampTypeInfo(timestampType.getPrecision()) : + new SqlTimestampTypeInfo(precision) : (clazz == LocalDateTime.class ? - Types.LOCAL_DATE_TIME : Types.SQL_TIMESTAMP); + ((3 == precision) ? + Types.LOCAL_DATE_TIME : new LegacyLocalDateTimeTypeInfo(precision)) : + ((3 == precision) ? + Types.SQL_TIMESTAMP : new LegacyTimestampTypeInfo(precision))); } else { return TypeConversions.fromDataTypeToLegacyInfo(dataType); } From 11222cd12598d8b0f4052b022400b108f34af2ba Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Fri, 8 Nov 2019 18:53:05 +0800 Subject: [PATCH 21/29] [FLINK-14599][table-planner-blink] Support precision of Timestamp type for timestamp literals --- .../converter/ExpressionConverter.java | 25 ++++++++- .../planner/calcite/FlinkTypeFactory.scala | 9 ++-- .../planner/calcite/FlinkTypeSystem.scala | 3 ++ .../planner/codegen/ExprCodeGenerator.scala | 9 +++- .../planner/codegen/ExpressionReducer.scala | 13 ++++- .../table/planner/codegen/GenerateUtils.scala | 54 +++++++++++++++++-- .../codegen/calls/ScalarOperatorGens.scala | 24 ++++++--- .../PlannerExpressionConverter.scala | 6 --- .../planner/expressions/ArrayTypeTest.scala | 44 +++++++++++++++ .../planner/expressions/MapTypeTest.scala | 25 +++++++++ .../planner/expressions/RowTypeTest.scala | 5 ++ .../expressions/TemporalTypesTest.scala | 29 +++++++++- .../utils/ExpressionTestBase.scala | 6 +++ .../plan/utils/RexNodeExtractorTest.scala | 6 +-- .../runtime/batch/sql/CalcITCase.scala | 4 +- .../runtime/functions/SqlDateTimeUtils.java | 2 +- 16 files changed, 231 insertions(+), 33 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java index cdd236ae714e7..e2d99f56921e7 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java @@ -52,8 +52,11 @@ import org.apache.calcite.util.TimeString; import org.apache.calcite.util.TimestampString; import org.apache.calcite.util.TimestampWithTimeZoneString; +import org.apache.flink.table.types.logical.TimestampType; import java.math.BigDecimal; +import java.sql.Timestamp; +import java.time.LocalDateTime; import java.util.Arrays; import java.util.Calendar; import java.util.Date; @@ -135,8 +138,26 @@ public RexNode visit(ValueLiteralExpression valueLiteral) { return relBuilder.getRexBuilder().makeTimeLiteral(TimeString.fromCalendarFields( valueAsCalendar(extractValue(valueLiteral, java.sql.Time.class))), 0); case TIMESTAMP_WITHOUT_TIME_ZONE: - return relBuilder.getRexBuilder().makeTimestampLiteral(TimestampString.fromCalendarFields( - valueAsCalendar(extractValue(valueLiteral, java.sql.Timestamp.class))), 3); + TimestampType timestampType = (TimestampType) type; + Class clazz = valueLiteral.getOutputDataType().getConversionClass(); + LocalDateTime datetime = null; + if (clazz == LocalDateTime.class) { + datetime = valueLiteral.getValueAs(LocalDateTime.class) + .orElseThrow(() -> new TableException("Invalid literal.")); + } else if (clazz == Timestamp.class) { + datetime = valueLiteral.getValueAs(Timestamp.class) + .orElseThrow(() -> new TableException("Invalid literal.")).toLocalDateTime(); + } else { + throw new TableException("Invalid literal."); + } + return relBuilder.getRexBuilder().makeTimestampLiteral( + new TimestampString( + datetime.getYear(), + datetime.getMonthValue(), + datetime.getDayOfMonth(), + datetime.getHour(), + datetime.getMinute(), + datetime.getSecond()).withNanos(datetime.getNano()), timestampType.getPrecision()); case TIMESTAMP_WITH_LOCAL_TIME_ZONE: TimeZone timeZone = TimeZone.getTimeZone(((FlinkContext) ((FlinkRelBuilder) this.relBuilder) .getCluster().getPlanner().getContext()).getTableConfig().getLocalTimeZone()); diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala index 1c61c00a741fb..9d43161cdead2 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala @@ -133,7 +133,7 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) extends JavaTypeFactoryImp timestampType.getKind match { case TimestampKind.PROCTIME => createProctimeIndicatorType(true) case TimestampKind.ROWTIME => createRowtimeIndicatorType(true) - case TimestampKind.REGULAR => createSqlType(TIMESTAMP) + case TimestampKind.REGULAR => createSqlType(TIMESTAMP, timestampType.getPrecision) } case _ => seenTypes.get(t) match { @@ -424,11 +424,12 @@ object FlinkTypeFactory { // blink runner support precision 3, but for consistent with flink runner, we set to 0. new TimeType() case TIMESTAMP => - if (relDataType.getPrecision > 3) { + val precision = relDataType.getPrecision + if (precision > 9 || precision < 0) { throw new TableException( - s"TIMESTAMP precision is not supported: ${relDataType.getPrecision}") + s"TIMESTAMP precision is not supported: ${precision}") } - new TimestampType(3) + new TimestampType(precision) case TIMESTAMP_WITH_LOCAL_TIME_ZONE => if (relDataType.getPrecision > 3) { throw new TableException( diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeSystem.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeSystem.scala index 0258f28e919ba..01dc86162b61b 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeSystem.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeSystem.scala @@ -53,6 +53,9 @@ class FlinkTypeSystem extends RelDataTypeSystemImpl { case SqlTypeName.VARCHAR | SqlTypeName.CHAR | SqlTypeName.VARBINARY | SqlTypeName.BINARY => Int.MaxValue + // The maximal precision of TIMESTAMP is 3, change it to 9 to support nanoseconds precision + case SqlTypeName.TIMESTAMP => 9 + case _ => super.getMaxPrecision(typeName) } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala index f73abfcc85feb..3b3169b33487e 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala @@ -37,10 +37,10 @@ import org.apache.flink.table.runtime.typeutils.TypeCheckUtils import org.apache.flink.table.runtime.typeutils.TypeCheckUtils.{isNumeric, isTemporal, isTimeInterval} import org.apache.flink.table.types.logical._ import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo - import org.apache.calcite.rex._ import org.apache.calcite.sql.SqlOperator import org.apache.calcite.sql.`type`.{ReturnTypes, SqlTypeName} +import org.apache.calcite.util.TimestampString import scala.collection.JavaConversions._ @@ -388,7 +388,12 @@ class ExprCodeGenerator(ctx: CodeGeneratorContext, nullableInput: Boolean) override def visitLiteral(literal: RexLiteral): GeneratedExpression = { val resultType = FlinkTypeFactory.toLogicalType(literal.getType) - val value = literal.getValue3 + val value = resultType.getTypeRoot match { + case LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE => + literal.getValueAs(classOf[TimestampString]) + case _ => + literal.getValue3 + } generateLiteral(ctx, resultType, value) } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala index b6112aba07e7c..a1186094bc747 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala @@ -37,6 +37,8 @@ import org.apache.commons.lang3.StringEscapeUtils import java.io.File import java.util.TimeZone +import org.apache.calcite.util.TimestampString + import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer @@ -173,7 +175,16 @@ class ExpressionReducer( case SqlTypeName.TIMESTAMP => val reducedValue = reduced.getField(reducedIdx) val value = if (reducedValue != null) { - Long.box(reducedValue.asInstanceOf[SqlTimestamp].getMillisecond) + val dt = reducedValue.asInstanceOf[SqlTimestamp].toLocalDateTime + val timestampString = + new TimestampString( + dt.getYear, + dt.getMonthValue, + dt.getDayOfMonth, + dt.getHour, + dt.getMinute, + dt.getSecond) + timestampString.withNanos(dt.getNano) } else { reducedValue } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala index fdda96c4d4cdf..fbb8beb7732c0 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala @@ -31,13 +31,16 @@ import org.apache.flink.table.runtime.types.PlannerTypeUtils import org.apache.flink.table.runtime.typeutils.TypeCheckUtils.{isCharacterString, isReference, isTemporal} import org.apache.flink.table.types.logical.LogicalTypeRoot._ import org.apache.flink.table.types.logical._ - import org.apache.calcite.avatica.util.ByteString +import org.apache.calcite.util.TimestampString import org.apache.commons.lang3.StringEscapeUtils - import java.math.{BigDecimal => JBigDecimal} +import java.lang.{Integer => JInteger} + +import org.apache.flink.table.runtime.functions.SqlDateTimeUtils import scala.collection.mutable +import scala.math.pow /** * Utilities to generate code for general purpose. @@ -370,12 +373,53 @@ object GenerateUtils { generateNonNullLiteral(literalType, literalValue.toString, literalValue) case TIMESTAMP_WITHOUT_TIME_ZONE => - // TODO: support Timestamp(3) now + def getNanoOfMillisSinceEpoch(timestampString: TimestampString): Int = { + val v = timestampString.toString() + val length = v.length + val nanoOfSeconds = length match { + case 19 | 20 => 0 + case _ => + JInteger.valueOf(v.substring(20)) * pow(10, 9 - (length - 20)).intValue() + } + nanoOfSeconds % 1000000 + } + + // TODO: we copied the logical of TimestampString::getMillisSinceEpoch since the copied + // DateTimeUtils.ymdToJulian is wrong. + // SEE CALCITE-1884 + def getMillisInSecond(timestampString: TimestampString): Int = { + val v = timestampString.toString() + val length = v.length + val milliOfSeconds = length match { + case 19 => 0 + case 21 => JInteger.valueOf(v.substring(20)).intValue() * 100 + case 22 => JInteger.valueOf(v.substring(20)).intValue() * 10 + case 20 | 23 | _ => JInteger.valueOf(v.substring(20, 23)).intValue() + } + milliOfSeconds + } + + def getMillisSinceEpoch(timestampString: TimestampString): Long = { + val v = timestampString.toString() + val year = JInteger.valueOf(v.substring(0, 4)) + val month = JInteger.valueOf(v.substring(5, 7)) + val day = JInteger.valueOf(v.substring(8, 10)) + val h = JInteger.valueOf(v.substring(11, 13)) + val m = JInteger.valueOf(v.substring(14, 16)) + val s = JInteger.valueOf(v.substring(17, 19)) + val ms = getMillisInSecond(timestampString) + val d = SqlDateTimeUtils.ymdToJulian(year, month, day) + d * 86400000L + h * 3600000L + m * 60000L + s * 1000L + ms.toLong + } + val fieldTerm = newName("timestamp") - val millis = literalValue.asInstanceOf[Long] + val millis = literalValue.asInstanceOf[TimestampString].getMillisSinceEpoch + val nanoOfMillis = getNanoOfMillisSinceEpoch( + literalValue.asInstanceOf[TimestampString]) val fieldTimestamp = s""" - |$SQL_TIMESTAMP_TERM $fieldTerm = $SQL_TIMESTAMP_TERM.fromEpochMillis(${millis}L); + |$SQL_TIMESTAMP_TERM $fieldTerm = + | $SQL_TIMESTAMP_TERM.fromEpochMillis(${millis}L, $nanoOfMillis); """.stripMargin ctx.addReusableMember(fieldTimestamp) generateNonNullLiteral(literalType, fieldTerm, literalType) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala index 3e994acd22f70..c1ac5f877911d 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala @@ -974,8 +974,8 @@ object ScalarOperatorGens { operandTerm => s""" |$SQL_TIMESTAMP_TERM.fromEpochMillis( - | ${qualifyMethod(BuiltInMethods.STRING_TO_TIMESTAMP)}($operandTerm.toString())) - """.stripMargin + | ${qualifyMethod(BuiltInMethods.STRING_TO_TIMESTAMP)}($operandTerm.toString())) + """.stripMargin } case (VARCHAR | CHAR, TIMESTAMP_WITH_LOCAL_TIME_ZONE) => @@ -2220,9 +2220,9 @@ object ScalarOperatorGens { s"${qualifyMethod(BuiltInMethods.STRING_TO_TIME)}($operandTerm.toString())" case TIMESTAMP_WITHOUT_TIME_ZONE => s""" - |${SQL_TIMESTAMP_TERM}.fromEpochMillis( + |$SQL_TIMESTAMP_TERM.fromEpochMillis( | ${qualifyMethod(BuiltInMethods.STRING_TO_TIMESTAMP)}($operandTerm.toString())) - |""".stripMargin + """.stripMargin case _ => throw new UnsupportedOperationException } @@ -2236,8 +2236,20 @@ object ScalarOperatorGens { case TIME_WITHOUT_TIME_ZONE => s"${qualifyMethod(BuiltInMethods.UNIX_TIME_TO_STRING)}($operandTerm)" case TIMESTAMP_WITHOUT_TIME_ZONE => // including rowtime indicator - val longTerm = s"$operandTerm.getMillisecond()" - s"${qualifyMethod(BuiltInMethods.TIMESTAMP_TO_STRING)}($longTerm, 3)" + // casting TimestampType to VARCHAR, if precision <= 3, keep the string representation + // consistent with the original design. Otherwise, use SqlTimestamp.toString(), which + // follows one of the following ISO-8601 formats: + // uuuu-MM-dd'T'HH:mm:ss.SSSSSS + // uuuu-MM-dd'T'HH:mm:ss.SSSSSSSSS + // The format used will be the shortest that outputs the full value of + // the time where the omitted parts are implied to be zero. + val precision = fromType.asInstanceOf[TimestampType].getPrecision + if (precision <= 3) { + val longTerm = s"$operandTerm.getMillisecond()" + s"${qualifyMethod(BuiltInMethods.TIMESTAMP_TO_STRING)}($longTerm, 3)" + } else { + s"$operandTerm.toString()" + } case TIMESTAMP_WITH_LOCAL_TIME_ZONE => val method = qualifyMethod(BuiltInMethods.TIMESTAMP_TO_STRING_TIME_ZONE) val zone = ctx.addReusableTimeZone() diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala index 6ce17a298414f..38d93b0997332 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala @@ -748,12 +748,6 @@ class PlannerExpressionConverter private extends ApiExpressionVisitor[PlannerExp } } - else if (hasRoot(logicalType, TIMESTAMP_WITHOUT_TIME_ZONE)) { - if (getPrecision(logicalType) <= 3) { - return Types.SQL_TIMESTAMP - } - } - fromDataTypeToTypeInfo(literal.getOutputDataType) } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala index 4e3acd74bb96c..d81a1ff7bc33b 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala @@ -23,6 +23,8 @@ import org.apache.flink.table.api.scala._ import org.apache.flink.table.planner.expressions.utils.ArrayTypeTestBase import org.apache.flink.table.planner.utils.DateTimeTestUtil.{localDate, localDateTime, localTime => gLocalTime} +import java.time.{LocalDateTime => JLocalDateTime} + import org.junit.Test class ArrayTypeTest extends ArrayTypeTestBase { @@ -94,6 +96,48 @@ class ArrayTypeTest extends ArrayTypeTestBase { "ARRAY[TIMESTAMP '1985-04-11 14:15:16', TIMESTAMP '2018-07-26 17:18:19']", "[1985-04-11 14:15:16.000, 2018-07-26 17:18:19.000]") + // localDateTime use DateTimeUtils.timestampStringToUnixDate to parse a time string, + // which only support millisecond's precision. + testTableApi( + Array( + JLocalDateTime.of(1985, 4, 11, 14, 15, 16, 123456789), + JLocalDateTime.of(2018, 7, 26, 17, 18, 19, 123456789)), + "[1985-04-11T14:15:16.123456789, 2018-07-26T17:18:19.123456789]") + + testTableApi( + Array( + JLocalDateTime.of(1985, 4, 11, 14, 15, 16, 123456700), + JLocalDateTime.of(2018, 7, 26, 17, 18, 19, 123456700)), + "[1985-04-11T14:15:16.123456700, 2018-07-26T17:18:19.123456700]") + + testTableApi( + Array( + JLocalDateTime.of(1985, 4, 11, 14, 15, 16, 123456000), + JLocalDateTime.of(2018, 7, 26, 17, 18, 19, 123456000)), + "[1985-04-11T14:15:16.123456, 2018-07-26T17:18:19.123456]") + + testTableApi( + Array( + JLocalDateTime.of(1985, 4, 11, 14, 15, 16, 123400000), + JLocalDateTime.of(2018, 7, 26, 17, 18, 19, 123400000)), + "[1985-04-11T14:15:16.123400, 2018-07-26T17:18:19.123400]") + + testSqlApi( + "ARRAY[TIMESTAMP '1985-04-11 14:15:16.123456789', TIMESTAMP '2018-07-26 17:18:19.123456789']", + "[1985-04-11T14:15:16.123456789, 2018-07-26T17:18:19.123456789]") + + testSqlApi( + "ARRAY[TIMESTAMP '1985-04-11 14:15:16.1234567', TIMESTAMP '2018-07-26 17:18:19.1234567']", + "[1985-04-11T14:15:16.123456700, 2018-07-26T17:18:19.123456700]") + + testSqlApi( + "ARRAY[TIMESTAMP '1985-04-11 14:15:16.123456', TIMESTAMP '2018-07-26 17:18:19.123456']", + "[1985-04-11T14:15:16.123456, 2018-07-26T17:18:19.123456]") + + testSqlApi( + "ARRAY[TIMESTAMP '1985-04-11 14:15:16.1234', TIMESTAMP '2018-07-26 17:18:19.1234']", + "[1985-04-11T14:15:16.123400, 2018-07-26T17:18:19.123400]") + testAllApis( Array(BigDecimal(2.0002), BigDecimal(2.0003)), "Array(2.0002p, 2.0003p)", diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MapTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MapTypeTest.scala index 5379e35620966..1ba03a42df4ad 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MapTypeTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MapTypeTest.scala @@ -24,6 +24,8 @@ import org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral import org.apache.flink.table.planner.expressions.utils.MapTypeTestBase import org.apache.flink.table.planner.utils.DateTimeTestUtil.{localDate, localDateTime, localTime => gLocalTime} +import java.time.{LocalDateTime => JLocalTimestamp} + import org.junit.Test class MapTypeTest extends MapTypeTestBase { @@ -92,6 +94,29 @@ class MapTypeTest extends MapTypeTestBase { "TIME '17:18:19', TIMESTAMP '2018-07-26 17:18:19']", "{14:15:16=1985-04-11 14:15:16.000, 17:18:19=2018-07-26 17:18:19.000}") + testAllApis( + map(valueLiteral(gLocalTime("14:15:16")), + valueLiteral(localDateTime("1985-04-11 14:15:16.123")), + valueLiteral(gLocalTime("17:18:19")), + valueLiteral(localDateTime("2018-07-26 17:18:19.123"))), + "map('14:15:16'.toTime, '1985-04-11 14:15:16.123'.toTimestamp, " + + "'17:18:19'.toTime, '2018-07-26 17:18:19.123'.toTimestamp)", + "MAP[TIME '14:15:16', TIMESTAMP '1985-04-11 14:15:16.123', " + + "TIME '17:18:19', TIMESTAMP '2018-07-26 17:18:19.123']", + "{14:15:16=1985-04-11 14:15:16.123, 17:18:19=2018-07-26 17:18:19.123}") + + testTableApi( + map(valueLiteral(gLocalTime("14:15:16")), + valueLiteral(JLocalTimestamp.of(1985, 4, 11, 14, 15, 16, 123456000)), + valueLiteral(gLocalTime("17:18:19")), + valueLiteral(JLocalTimestamp.of(2018, 7, 26, 17, 18, 19, 123456000))), + "{14:15:16=1985-04-11T14:15:16.123456, 17:18:19=2018-07-26T17:18:19.123456}") + + testSqlApi( + "MAP[TIME '14:15:16', TIMESTAMP '1985-04-11 14:15:16.123456', " + + "TIME '17:18:19', TIMESTAMP '2018-07-26 17:18:19.123456']", + "{14:15:16=1985-04-11T14:15:16.123456, 17:18:19=2018-07-26T17:18:19.123456}") + testAllApis( map(BigDecimal(2.0002), BigDecimal(2.0003)), "map(2.0002p, 2.0003p)", diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala index ff7a2a2ae2f20..30730f309d942 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala @@ -58,6 +58,11 @@ class RowTypeTest extends RowTypeTestBase { "CAST(0.1 AS DECIMAL(2, 1)), ARRAY[1, 2, 3], MAP['foo', 'bar'], row(1, true))", "(1985-04-11,14:15:16,1985-04-11 14:15:16.000,0.1,[1, 2, 3],{foo=bar},(1,true))") + testSqlApi( + "ROW(DATE '1985-04-11', TIME '14:15:16', TIMESTAMP '1985-04-11 14:15:16.123456', " + + "CAST(0.1 AS DECIMAL(2, 1)), ARRAY[1, 2, 3], MAP['foo', 'bar'], row(1, true))", + "(1985-04-11,14:15:16,1985-04-11T14:15:16.123456,0.1,[1, 2, 3],{foo=bar},(1,true))") + testAllApis( row(1 + 1, 2 * 3, nullOf(DataTypes.STRING())), "row(1 + 1, 2 * 3, Null(STRING))", diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala index 064e8e0b3d701..a5a68a978b775 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala @@ -37,6 +37,17 @@ import java.util.{Locale, TimeZone} class TemporalTypesTest extends ExpressionTestBase { + @Test + def test(): Unit = { + testSqlApi( + "TIMESTAMP '1500-04-30 12:00:00.123456789'", + "1500-05-10T12:00:00.123456789") + + testSqlApi( + "CAST('1500-04-30 12:00:00.123' AS TIMESTAMP(3))", + "1500-05-10 12:00:00.123") + } + @Test def testTimePointLiterals(): Unit = { testAllApis( @@ -88,7 +99,23 @@ class TemporalTypesTest extends ExpressionTestBase { "1500-04-30 12:00:00".cast(DataTypes.TIMESTAMP(3)), "'1500-04-30 12:00:00'.cast(SQL_TIMESTAMP)", "CAST('1500-04-30 12:00:00' AS TIMESTAMP)", - "1500-04-30 12:00:00.000") + "1500-05-10 12:00:00.000") + + testSqlApi( + "TIMESTAMP '1500-04-30 12:00:00.123456789'", + "1500-05-10T12:00:00.123456789") + + testSqlApi( + "TIMESTAMP '1500-04-30 12:00:00.12345678'", + "1500-05-10T12:00:00.123456780") + + testSqlApi( + "TIMESTAMP '1500-04-30 12:00:00.123456'", + "1500-05-10T12:00:00.123456") + + testSqlApi( + "TIMESTAMP '1500-04-30 12:00:00.1234'", + "1500-05-10T12:00:00.123400") } @Test diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala index 36a29ae9b7e2c..15eb2904a8be0 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala @@ -226,6 +226,12 @@ abstract class ExpressionTestBase { addTableApiTestExpr(exprString, expected) } + def testTableApi( + expr: Expression, + expected: String): Unit = { + addTableApiTestExpr(expr, expected) + } + private def addTableApiTestExpr(tableApiString: String, expected: String): Unit = { addTableApiTestExpr(ExpressionParser.parseExpression(tableApiString), expected) } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractorTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractorTest.scala index 509f2b2373f2c..85148af594e02 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractorTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractorTest.scala @@ -439,14 +439,14 @@ class RexNodeExtractorTest extends RexNodeTestBase { relBuilder, functionCatalog) - val timestamp = DateTimeTestUtil.localDateTime("2017-09-10 14:23:01") + val datetime = DateTimeTestUtil.localDateTime("2017-09-10 14:23:01") val date = DateTimeTestUtil.localDate("2017-09-12") val time = DateTimeTestUtil.localTime("14:23:01") { val expected = Array[Expression]( // timestamp_col = '2017-09-10 14:23:01' - unresolvedCall(EQUALS, unresolvedRef("timestamp_col"), valueLiteral(timestamp)), + unresolvedCall(EQUALS, unresolvedRef("timestamp_col"), valueLiteral(datetime)), // date_col = '2017-09-12' unresolvedCall(EQUALS, unresolvedRef("date_col"), valueLiteral(date)), // time_col = '14:23:01' @@ -460,7 +460,7 @@ class RexNodeExtractorTest extends RexNodeTestBase { val expected = Array[Expression]( EqualTo( UnresolvedFieldReference("timestamp_col"), - Literal(Timestamp.valueOf("2017-09-10 14:23:01")) + Literal(datetime) ), EqualTo( UnresolvedFieldReference("date_col"), diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala index c37fe8297e282..3c99192bc0d1f 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala @@ -702,14 +702,14 @@ class CalcITCase extends BatchTestBase { @Test def testValueConstructor(): Unit = { - val data = Seq(row("foo", 12, localDateTime("1984-07-12 14:34:24"))) + val data = Seq(row("foo", 12, localDateTime("1984-07-12 14:34:24.001"))) BatchTableEnvUtil.registerCollection( tEnv, "MyTable", data, new RowTypeInfo(Types.STRING, Types.INT, Types.LOCAL_DATE_TIME), Some(parseFieldNames("a, b, c")), None, None) val table = parseQuery("SELECT ROW(a, b, c), ARRAY[12, b], MAP[a, c] FROM MyTable " + - "WHERE (a, b, c) = ('foo', 12, TIMESTAMP '1984-07-12 14:34:24')") + "WHERE (a, b, c) = ('foo', 12, TIMESTAMP '1984-07-12 14:34:24.001')") val result = executeQuery(table) val baseRow = result.head.getField(0).asInstanceOf[Row] diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java index e7f1422a94ebb..e95709aae08d8 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java @@ -1011,7 +1011,7 @@ private static int ymdToUnixDate(int year, int month, int day) { return julian - EPOCH_JULIAN; } - private static int ymdToJulian(int year, int month, int day) { + public static int ymdToJulian(int year, int month, int day) { int a = (14 - month) / 12; int y = year + 4800 - a; int m = month + 12 * a - 3; From 88605974d496c641ccdc1d2d65c71a5e82304688 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Fri, 8 Nov 2019 18:59:38 +0800 Subject: [PATCH 22/29] fixup: address comments --- .../table/planner/codegen/CodeGenUtils.scala | 2 +- .../codegen/CodeGeneratorContext.scala | 8 +++---- .../planner/codegen/ExprCodeGenerator.scala | 2 +- .../table/planner/codegen/GenerateUtils.scala | 12 +++++----- .../planner/codegen/MatchCodeGenerator.scala | 6 ++--- .../agg/AggsHandlerCodeGenerator.scala | 8 +++---- .../agg/batch/WindowCodeGenerator.scala | 8 +++---- .../codegen/calls/FloorCeilCallGen.scala | 6 ++--- .../planner/codegen/calls/MethodCallGen.scala | 4 ++-- .../codegen/calls/ScalarFunctionCallGen.scala | 2 +- .../codegen/calls/ScalarOperatorGens.scala | 24 +++++++++---------- 11 files changed, 41 insertions(+), 41 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala index f4e3d6fb763e2..505f31b61d0c4 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala @@ -97,7 +97,7 @@ object CodeGenUtils { val STRING_UTIL: String = className[BinaryStringUtil] - val SQL_TIMESTAMP_TERM: String = className[SqlTimestamp] + val SQL_TIMESTAMP: String = className[SqlTimestamp] // ---------------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala index c52e43073a7a7..c63bef5bf485f 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala @@ -410,8 +410,8 @@ class CodeGeneratorContext(val tableConfig: TableConfig) { val fieldTerm = s"timestamp" val field = s""" - |final $SQL_TIMESTAMP_TERM $fieldTerm = - | $SQL_TIMESTAMP_TERM.fromEpochMillis(java.lang.System.currentTimeMillis()); + |final $SQL_TIMESTAMP $fieldTerm = + | $SQL_TIMESTAMP.fromEpochMillis(java.lang.System.currentTimeMillis()); |""".stripMargin reusablePerRecordStatements.add(field) fieldTerm @@ -450,12 +450,12 @@ class CodeGeneratorContext(val tableConfig: TableConfig) { val timestamp = addReusableTimestamp() // declaration - reusableMemberStatements.add(s"private $SQL_TIMESTAMP_TERM $fieldTerm;") + reusableMemberStatements.add(s"private $SQL_TIMESTAMP $fieldTerm;") // assignment val field = s""" - |$fieldTerm = $SQL_TIMESTAMP_TERM.fromEpochMillis( + |$fieldTerm = $SQL_TIMESTAMP.fromEpochMillis( | $timestamp.getMillisecond() + | java.util.TimeZone.getDefault().getOffset($timestamp.getMillisecond())); |""".stripMargin diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala index 3b3169b33487e..3a499b9ddfc01 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala @@ -735,7 +735,7 @@ class ExprCodeGenerator(ctx: CodeGeneratorContext, nullableInput: Boolean) generateProctimeTimestamp(ctx, contextTerm) case STREAMRECORD_TIMESTAMP => - generateRowtimeAccess(ctx, contextTerm) + generateTimestampAccess(ctx, contextTerm) case _: SqlThrowExceptionFunction => val nullValue = generateNullLiteral(resultType, nullCheck = true) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala index fbb8beb7732c0..9072aad6763a7 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala @@ -418,8 +418,8 @@ object GenerateUtils { literalValue.asInstanceOf[TimestampString]) val fieldTimestamp = s""" - |$SQL_TIMESTAMP_TERM $fieldTerm = - | $SQL_TIMESTAMP_TERM.fromEpochMillis(${millis}L, $nanoOfMillis); + |$SQL_TIMESTAMP $fieldTerm = + | $SQL_TIMESTAMP.fromEpochMillis(${millis}L, $nanoOfMillis); """.stripMargin ctx.addReusableMember(fieldTimestamp) generateNonNullLiteral(literalType, fieldTerm, literalType) @@ -494,11 +494,11 @@ object GenerateUtils { val resultTerm = ctx.addReusableLocalVariable(resultTypeTerm, "result") val resultCode = s""" - |$resultTerm = $SQL_TIMESTAMP_TERM.fromEpochMillis( + |$resultTerm = $SQL_TIMESTAMP.fromEpochMillis( | $contextTerm.timerService().currentProcessingTime()); |""".stripMargin.trim // the proctime has been materialized, so it's TIMESTAMP now, not PROCTIME_INDICATOR - GeneratedExpression(resultTerm, NEVER_NULL, resultCode, new TimestampType(3)) + GeneratedExpression(resultTerm, NEVER_NULL, resultCode, resultType) } def generateCurrentTimestamp( @@ -506,7 +506,7 @@ object GenerateUtils { new CurrentTimePointCallGen(false).generate(ctx, Seq(), new TimestampType(3)) } - def generateRowtimeAccess( + def generateTimestampAccess( ctx: CodeGeneratorContext, contextTerm: String): GeneratedExpression = { val resultType = new TimestampType(true, TimestampKind.ROWTIME, 3) @@ -517,7 +517,7 @@ object GenerateUtils { val accessCode = s""" - |$resultTerm = $SQL_TIMESTAMP_TERM.fromEpochMillis($contextTerm.timestamp()); + |$resultTerm = $SQL_TIMESTAMP.fromEpochMillis($contextTerm.timestamp()); |if ($resultTerm == null) { | throw new RuntimeException("Rowtime timestamp is null. Please make sure that a " + | "proper TimestampAssigner is defined and the stream environment uses the EventTime " + diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala index 31c0c0d01de36..e8c12f4b743c4 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala @@ -26,7 +26,7 @@ import org.apache.flink.table.api.{TableConfig, TableException} import org.apache.flink.table.dataformat.{BaseRow, GenericRow} import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.codegen.CodeGenUtils._ -import org.apache.flink.table.planner.codegen.GenerateUtils.{generateNullLiteral, generateRowtimeAccess} +import org.apache.flink.table.planner.codegen.GenerateUtils.{generateNullLiteral, generateTimestampAccess} import org.apache.flink.table.planner.codegen.GeneratedExpression.{NEVER_NULL, NO_CODE} import org.apache.flink.table.planner.codegen.Indenter.toISC import org.apache.flink.table.planner.codegen.MatchCodeGenerator._ @@ -376,7 +376,7 @@ class MatchCodeGenerator( ctx.nullCheck) case MATCH_ROWTIME => - generateRowtimeAccess(ctx, contextTerm) + generateTimestampAccess(ctx, contextTerm) case PROCTIME_MATERIALIZE => // override proctime materialize code generation @@ -393,7 +393,7 @@ class MatchCodeGenerator( val resultTerm = ctx.addReusableLocalVariable(resultTypeTerm, "result") val resultCode = s""" - |$resultTerm = $SQL_TIMESTAMP_TERM.fromEpochMillis($contextTerm.currentProcessingTime()); + |$resultTerm = $SQL_TIMESTAMP.fromEpochMillis($contextTerm.currentProcessingTime()); |""".stripMargin.trim // the proctime has been materialized, so it's TIMESTAMP now, not PROCTIME_INDICATOR GeneratedExpression(resultTerm, NEVER_NULL, resultCode, resultType) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/AggsHandlerCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/AggsHandlerCodeGenerator.scala index c6111af6dbe66..ba81b145cdcbb 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/AggsHandlerCodeGenerator.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/AggsHandlerCodeGenerator.scala @@ -943,27 +943,27 @@ class AggsHandlerCodeGenerator( case w: PlannerWindowStart => // return a Timestamp(Internal is SqlTimestamp) GeneratedExpression( - s"$SQL_TIMESTAMP_TERM.fromEpochMillis($NAMESPACE_TERM.getStart())", + s"$SQL_TIMESTAMP.fromEpochMillis($NAMESPACE_TERM.getStart())", "false", "", w.resultType) case w: PlannerWindowEnd => // return a Timestamp(Internal is SqlTimestamp) GeneratedExpression( - s"$SQL_TIMESTAMP_TERM.fromEpochMillis($NAMESPACE_TERM.getEnd())", + s"$SQL_TIMESTAMP.fromEpochMillis($NAMESPACE_TERM.getEnd())", "false", "", w.resultType) case r: PlannerRowtimeAttribute => // return a rowtime, use SqlTimestamp as internal type GeneratedExpression( - s"$SQL_TIMESTAMP_TERM.fromEpochMillis($NAMESPACE_TERM.getEnd() - 1)", + s"$SQL_TIMESTAMP.fromEpochMillis($NAMESPACE_TERM.getEnd() - 1)", "false", "", r.resultType) case p: PlannerProctimeAttribute => // ignore this property, it will be null at the position later - GeneratedExpression(s"$SQL_TIMESTAMP_TERM.fromEpochMillis(-1L)", "true", "", p.resultType) + GeneratedExpression(s"$SQL_TIMESTAMP.fromEpochMillis(-1L)", "true", "", p.resultType) } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/WindowCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/WindowCodeGenerator.scala index 84f9fb04de818..63225efddcd66 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/WindowCodeGenerator.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/WindowCodeGenerator.scala @@ -26,7 +26,7 @@ import org.apache.flink.table.functions.{AggregateFunction, UserDefinedFunction} import org.apache.flink.table.planner.JLong import org.apache.flink.table.planner.calcite.FlinkRelBuilder.PlannerNamedWindowProperty import org.apache.flink.table.planner.calcite.FlinkTypeFactory -import org.apache.flink.table.planner.codegen.CodeGenUtils.{BINARY_ROW, SQL_TIMESTAMP_TERM, boxedTypeTermForType, newName} +import org.apache.flink.table.planner.codegen.CodeGenUtils.{BINARY_ROW, SQL_TIMESTAMP, boxedTypeTermForType, newName} import org.apache.flink.table.planner.codegen.GenerateUtils.generateFieldAccess import org.apache.flink.table.planner.codegen.GeneratedExpression.{NEVER_NULL, NO_CODE} import org.apache.flink.table.planner.codegen.OperatorCodeGenerator.generateCollect @@ -631,9 +631,9 @@ abstract class WindowCodeGenerator( // get assigned window start timestamp def windowProps(size: Expression) = { val (startWValue, endWValue, rowTimeValue) = ( - s"$SQL_TIMESTAMP_TERM.fromEpochMillis($currentWindowTerm.getStart())", - s"$SQL_TIMESTAMP_TERM.fromEpochMillis($currentWindowTerm.getEnd())", - s"$SQL_TIMESTAMP_TERM.fromEpochMillis($currentWindowTerm.maxTimestamp())") + s"$SQL_TIMESTAMP.fromEpochMillis($currentWindowTerm.getStart())", + s"$SQL_TIMESTAMP.fromEpochMillis($currentWindowTerm.getEnd())", + s"$SQL_TIMESTAMP.fromEpochMillis($currentWindowTerm.maxTimestamp())") val start = if (startPos.isDefined) { s"$propTerm.setTimestamp($lastPos + ${startPos.get}, $startWValue, 3);" } else "" diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/FloorCeilCallGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/FloorCeilCallGen.scala index 38e5b66e8cfb8..6c6a23aad7534 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/FloorCeilCallGen.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/FloorCeilCallGen.scala @@ -18,7 +18,7 @@ package org.apache.flink.table.planner.codegen.calls -import org.apache.flink.table.planner.codegen.CodeGenUtils.{getEnum, primitiveTypeTermForType, qualifyMethod, SQL_TIMESTAMP_TERM} +import org.apache.flink.table.planner.codegen.CodeGenUtils.{getEnum, primitiveTypeTermForType, qualifyMethod, SQL_TIMESTAMP} import org.apache.flink.table.planner.codegen.GenerateUtils.generateCallIfArgsNotNull import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, GeneratedExpression} import org.apache.flink.table.types.logical.{LogicalType, LogicalTypeRoot} @@ -82,7 +82,7 @@ class FloorCeilCallGen( case LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE => val longTerm = s"${terms.head}.getMillisecond()" s""" - |$SQL_TIMESTAMP_TERM.fromEpochMillis( + |$SQL_TIMESTAMP.fromEpochMillis( | ${qualifyMethod(temporalMethod.get)}(${terms(1)}, $longTerm)) """.stripMargin case _ => @@ -96,7 +96,7 @@ class FloorCeilCallGen( case LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE => val longTerm = s"${terms.head}.getMillisecond()" s""" - |$SQL_TIMESTAMP_TERM.fromEpochMillis(${qualifyMethod(arithmeticMethod)}( + |$SQL_TIMESTAMP.fromEpochMillis(${qualifyMethod(arithmeticMethod)}( | $longTerm, | (long) ${unit.startUnit.multiplier.intValue()})) """.stripMargin diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/MethodCallGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/MethodCallGen.scala index 6c9fb11607af6..023c9c11e08d6 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/MethodCallGen.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/MethodCallGen.scala @@ -18,7 +18,7 @@ package org.apache.flink.table.planner.codegen.calls -import org.apache.flink.table.planner.codegen.CodeGenUtils.{BINARY_STRING, qualifyMethod, SQL_TIMESTAMP_TERM} +import org.apache.flink.table.planner.codegen.CodeGenUtils.{BINARY_STRING, qualifyMethod, SQL_TIMESTAMP} import org.apache.flink.table.planner.codegen.GenerateUtils.generateCallIfArgsNotNull import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, GeneratedExpression} import org.apache.flink.table.types.logical.{LogicalType, LogicalTypeRoot} @@ -66,7 +66,7 @@ class MethodCallGen(method: Method) extends CallGenerator { } else if ((method.getReturnType == classOf[Long] || method.getReturnType == classOf[java.lang.Long]) && returnType.getTypeRoot == LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE) { - s"$SQL_TIMESTAMP_TERM.fromEpochMillis($call)" + s"$SQL_TIMESTAMP.fromEpochMillis($call)" } else { call } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarFunctionCallGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarFunctionCallGen.scala index 5aef2fedf307a..be5e17f981ea1 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarFunctionCallGen.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarFunctionCallGen.scala @@ -72,7 +72,7 @@ class ScalarFunctionCallGen(scalarFunction: ScalarFunction) extends CallGenerato val evalResult = if (returnType.getTypeRoot == LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE) { s""" - |$SQL_TIMESTAMP_TERM.fromEpochMillis( + |$SQL_TIMESTAMP.fromEpochMillis( | $functionReference.eval(${parameters.map(_.resultTerm).mkString(", ")})) """.stripMargin } else { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala index c1ac5f877911d..3937aab4937c9 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala @@ -175,7 +175,7 @@ object ScalarOperatorGens { } case TIMESTAMP_WITHOUT_TIME_ZONE => generateOperatorIfNotNull(ctx, new TimestampType(3), left, right) { - (l, r) => s"$SQL_TIMESTAMP_TERM.fromEpochMillis(($l * ${MILLIS_PER_DAY}L) $op $r)" + (l, r) => s"$SQL_TIMESTAMP.fromEpochMillis(($l * ${MILLIS_PER_DAY}L) $op $r)" } } @@ -193,7 +193,7 @@ object ScalarOperatorGens { generateOperatorIfNotNull(ctx, left.resultType, left, right) { (l, r) => { val leftTerm = s"$l.getMillisecond()" - s"$SQL_TIMESTAMP_TERM.fromEpochMillis($leftTerm $op $r)" + s"$SQL_TIMESTAMP.fromEpochMillis($leftTerm $op $r)" } } @@ -202,7 +202,7 @@ object ScalarOperatorGens { (l, r) => { val leftTerm = s"$l.getMillisecond()" s""" - |$SQL_TIMESTAMP_TERM.fromEpochMillis( + |$SQL_TIMESTAMP.fromEpochMillis( | ${qualifyMethod(BuiltInMethod.ADD_MONTHS.method)}($leftTerm, $op($r))) """.stripMargin } @@ -804,7 +804,7 @@ object ScalarOperatorGens { case (BIGINT, TIMESTAMP_WITHOUT_TIME_ZONE) => generateUnaryOperatorIfNotNull(ctx, targetType, operand) { - operandTerm => s"$SQL_TIMESTAMP_TERM.fromEpochMillis($operandTerm)" + operandTerm => s"$SQL_TIMESTAMP.fromEpochMillis($operandTerm)" } case (from, to) => @@ -838,7 +838,7 @@ object ScalarOperatorGens { generateUnaryOperatorIfNotNull(ctx, targetType, operand) { operandTerm => val zone = ctx.addReusableTimeZone() - s"$SQL_TIMESTAMP_TERM.fromEpochMillis($method($operandTerm, $zone))" + s"$SQL_TIMESTAMP.fromEpochMillis($method($operandTerm, $zone))" } // identity casting @@ -973,7 +973,7 @@ object ScalarOperatorGens { resultNullable = true) { operandTerm => s""" - |$SQL_TIMESTAMP_TERM.fromEpochMillis( + |$SQL_TIMESTAMP.fromEpochMillis( | ${qualifyMethod(BuiltInMethods.STRING_TO_TIMESTAMP)}($operandTerm.toString())) """.stripMargin } @@ -1019,7 +1019,7 @@ object ScalarOperatorGens { case (DECIMAL, TIMESTAMP_WITHOUT_TIME_ZONE) => generateUnaryOperatorIfNotNull(ctx, targetType, operand) { operandTerm => - s"$SQL_TIMESTAMP_TERM.fromEpochMillis($DECIMAL_TERM.castToTimestamp($operandTerm))" + s"$SQL_TIMESTAMP.fromEpochMillis($DECIMAL_TERM.castToTimestamp($operandTerm))" } // NUMERIC TYPE -> Boolean @@ -1040,7 +1040,7 @@ object ScalarOperatorGens { generateUnaryOperatorIfNotNull(ctx, targetType, operand) { operandTerm => s""" - |$SQL_TIMESTAMP_TERM.fromEpochMillis( + |$SQL_TIMESTAMP.fromEpochMillis( | $operandTerm * ${classOf[DateTimeUtils].getCanonicalName}.MILLIS_PER_DAY) """.stripMargin } @@ -1059,7 +1059,7 @@ object ScalarOperatorGens { // Time -> Timestamp case (TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE) => generateUnaryOperatorIfNotNull(ctx, targetType, operand) { - operandTerm => s"$SQL_TIMESTAMP_TERM.fromEpochMillis($operandTerm)" + operandTerm => s"$SQL_TIMESTAMP.fromEpochMillis($operandTerm)" } // Timestamp -> Time @@ -1124,7 +1124,7 @@ object ScalarOperatorGens { (INTEGER, TIMESTAMP_WITHOUT_TIME_ZONE) | (BIGINT, TIMESTAMP_WITHOUT_TIME_ZONE) => generateUnaryOperatorIfNotNull(ctx, targetType, operand) { - operandTerm => s"$SQL_TIMESTAMP_TERM.fromEpochMillis(((long) $operandTerm) * 1000)" + operandTerm => s"$SQL_TIMESTAMP.fromEpochMillis(((long) $operandTerm) * 1000)" } // Float -> Timestamp @@ -1132,7 +1132,7 @@ object ScalarOperatorGens { case (FLOAT, TIMESTAMP_WITHOUT_TIME_ZONE) | (DOUBLE, TIMESTAMP_WITHOUT_TIME_ZONE) => generateUnaryOperatorIfNotNull(ctx, targetType, operand) { - operandTerm => s"$SQL_TIMESTAMP_TERM.fromEpochMillis((long) ($operandTerm * 1000))" + operandTerm => s"$SQL_TIMESTAMP.fromEpochMillis((long) ($operandTerm * 1000))" } // Timestamp -> Tinyint @@ -2220,7 +2220,7 @@ object ScalarOperatorGens { s"${qualifyMethod(BuiltInMethods.STRING_TO_TIME)}($operandTerm.toString())" case TIMESTAMP_WITHOUT_TIME_ZONE => s""" - |$SQL_TIMESTAMP_TERM.fromEpochMillis( + |$SQL_TIMESTAMP.fromEpochMillis( | ${qualifyMethod(BuiltInMethods.STRING_TO_TIMESTAMP)}($operandTerm.toString())) """.stripMargin case _ => throw new UnsupportedOperationException From dfeca0570602086c473b78ef3bfd0df9057c10e0 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Sat, 9 Nov 2019 11:57:32 +0800 Subject: [PATCH 23/29] fixup: fix checkstyle --- .../planner/expressions/converter/ExpressionConverter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java index e2d99f56921e7..e889431fb89d1 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java @@ -37,6 +37,7 @@ import org.apache.flink.table.planner.expressions.converter.CallExpressionConvertRule.ConvertContext; import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.TimestampType; import org.apache.calcite.avatica.util.DateTimeUtils; import org.apache.calcite.avatica.util.TimeUnit; @@ -52,7 +53,6 @@ import org.apache.calcite.util.TimeString; import org.apache.calcite.util.TimestampString; import org.apache.calcite.util.TimestampWithTimeZoneString; -import org.apache.flink.table.types.logical.TimestampType; import java.math.BigDecimal; import java.sql.Timestamp; From d17efb5242bc04241da2a1a82212c623c66ac8d3 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Mon, 11 Nov 2019 19:50:34 +0800 Subject: [PATCH 24/29] [FLINK-14599][table-planner-blink] Make the default precision of Timestamp type as 6 which defined in SQL standard --- .../planner/functions/sql/FlinkSqlOperatorTable.java | 2 +- .../functions/sql/ProctimeMaterializeSqlFunction.java | 2 +- .../flink/table/planner/calcite/FlinkTypeSystem.scala | 6 +++++- .../logical/StreamLogicalWindowAggregateRule.scala | 3 ++- .../operations/SqlToOperationConverterTest.java | 4 ++-- .../plan/stream/sql/RelTimeIndicatorConverterTest.xml | 2 +- .../table/planner/expressions/TemporalTypesTest.scala | 10 +++++----- .../stream/sql/RelTimeIndicatorConverterTest.scala | 2 +- 8 files changed, 18 insertions(+), 13 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java index b303da49fb219..a37860185e3c4 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java @@ -647,7 +647,7 @@ public SqlMonotonicity getMonotonicity(SqlOperatorBinding call) { public static final SqlFunction TO_TIMESTAMP = new SqlFunction( "TO_TIMESTAMP", SqlKind.OTHER_FUNCTION, - ReturnTypes.cascade(ReturnTypes.explicit(SqlTypeName.TIMESTAMP), SqlTypeTransforms.FORCE_NULLABLE), + ReturnTypes.cascade(ReturnTypes.explicit(SqlTypeName.TIMESTAMP, 3), SqlTypeTransforms.FORCE_NULLABLE), null, OperandTypes.or( OperandTypes.family(SqlTypeFamily.CHARACTER), diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/ProctimeMaterializeSqlFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/ProctimeMaterializeSqlFunction.java index 8d34f84eb49ea..97e79c00845b9 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/ProctimeMaterializeSqlFunction.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/ProctimeMaterializeSqlFunction.java @@ -42,7 +42,7 @@ public ProctimeMaterializeSqlFunction() { "PROCTIME_MATERIALIZE", SqlKind.OTHER_FUNCTION, ReturnTypes.cascade( - ReturnTypes.explicit(SqlTypeName.TIMESTAMP), + ReturnTypes.explicit(SqlTypeName.TIMESTAMP, 3), SqlTypeTransforms.TO_NULLABLE), InferTypes.RETURN_TYPE, OperandTypes.family(SqlTypeFamily.TIMESTAMP), diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeSystem.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeSystem.scala index 01dc86162b61b..bbc190abfecd0 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeSystem.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeSystem.scala @@ -41,8 +41,12 @@ class FlinkTypeSystem extends RelDataTypeSystemImpl { case SqlTypeName.VARCHAR | SqlTypeName.VARBINARY => Int.MaxValue + // by default we support timestamp with microseconds precision + case SqlTypeName.TIMESTAMP => + 6 + // we currently support only timestamps with milliseconds precision - case SqlTypeName.TIMESTAMP | SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE => + case SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE => 3 case _ => diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/StreamLogicalWindowAggregateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/StreamLogicalWindowAggregateRule.scala index c94e4de07b646..fbaabc38f606f 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/StreamLogicalWindowAggregateRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/StreamLogicalWindowAggregateRule.scala @@ -58,7 +58,8 @@ class StreamLogicalWindowAggregateRule rexBuilder.makeLiteral( 0L, - rexBuilder.getTypeFactory.createSqlType(SqlTypeName.TIMESTAMP), + rexBuilder.getTypeFactory.createSqlType( + SqlTypeName.TIMESTAMP, windowExpression.getType.getPrecision), true) } diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/operations/SqlToOperationConverterTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/operations/SqlToOperationConverterTest.java index 7fa149e22684c..a635982eac130 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/operations/SqlToOperationConverterTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/operations/SqlToOperationConverterTest.java @@ -286,8 +286,8 @@ public void testCreateTableWithFullDataTypes() { createTestItem("TIME(3)", DataTypes.TIME()), // Expect to be TIME(3). createTestItem("TIME(3) WITHOUT TIME ZONE", DataTypes.TIME()), - createTestItem("TIMESTAMP", DataTypes.TIMESTAMP(3)), - createTestItem("TIMESTAMP WITHOUT TIME ZONE", DataTypes.TIMESTAMP(3)), + createTestItem("TIMESTAMP", DataTypes.TIMESTAMP(6)), + createTestItem("TIMESTAMP WITHOUT TIME ZONE", DataTypes.TIMESTAMP(6)), createTestItem("TIMESTAMP(3)", DataTypes.TIMESTAMP(3)), createTestItem("TIMESTAMP(3) WITHOUT TIME ZONE", DataTypes.TIMESTAMP(3)), createTestItem("TIMESTAMP WITH LOCAL TIME ZONE", diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.xml index 62f238d9437f5..6c0fafe0ccaee 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.xml @@ -62,7 +62,7 @@ Calc(select=[EXPR$0]) - CAST('1990-12-02 12:11:11' AS TIMESTAMP)]]> + CAST('1990-12-02 12:11:11' AS TIMESTAMP(3))]]> f2", + "CAST(f0 AS TIMESTAMP(3)) <> f2", "true") testAllApis( 'f0.cast(DataTypes.TIMESTAMP(3)) === 'f6, "f0.cast(SQL_TIMESTAMP) === f6", - "CAST(f0 AS TIMESTAMP) = f6", + "CAST(f0 AS TIMESTAMP(3)) = f6", "true") } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.scala index ee34a5d20e491..26e1f68a083ad 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.scala @@ -57,7 +57,7 @@ class RelTimeIndicatorConverterTest extends TableTestBase { @Test def testFilteringOnRowtime(): Unit = { val sqlQuery = - "SELECT rowtime FROM MyTable1 WHERE rowtime > CAST('1990-12-02 12:11:11' AS TIMESTAMP)" + "SELECT rowtime FROM MyTable1 WHERE rowtime > CAST('1990-12-02 12:11:11' AS TIMESTAMP(3))" util.verifyPlan(sqlQuery) } From c3412c42b47874d302019e1c3781d4da0c2d38d2 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Tue, 12 Nov 2019 14:51:37 +0800 Subject: [PATCH 25/29] [FLINK-14599][table-planner-blink] Make the interpreted string of Timestamp type conforms to the definition of timestamp literal which defined in SQL standard --- .../planner/codegen/ExpressionReducer.scala | 15 ++--- .../codegen/calls/BuiltInMethods.scala | 3 +- .../codegen/calls/ScalarOperatorGens.scala | 17 +----- .../planner/expressions/ArrayTypeTest.scala | 18 +++--- .../planner/expressions/MapTypeTest.scala | 6 +- .../planner/expressions/RowTypeTest.scala | 6 +- .../expressions/ScalarFunctionsTest.scala | 36 +++++------ .../expressions/TemporalTypesTest.scala | 61 ++++++++----------- .../UserDefinedScalarFunctionTest.scala | 2 +- flink-table/flink-table-runtime-blink/pom.xml | 61 +++++++++++++++++++ .../runtime/functions/SqlDateTimeUtils.java | 15 +++++ 11 files changed, 145 insertions(+), 95 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala index a1186094bc747..c50e3026bc1a9 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala @@ -175,16 +175,11 @@ class ExpressionReducer( case SqlTypeName.TIMESTAMP => val reducedValue = reduced.getField(reducedIdx) val value = if (reducedValue != null) { - val dt = reducedValue.asInstanceOf[SqlTimestamp].toLocalDateTime - val timestampString = - new TimestampString( - dt.getYear, - dt.getMonthValue, - dt.getDayOfMonth, - dt.getHour, - dt.getMinute, - dt.getSecond) - timestampString.withNanos(dt.getNano) + val ts = reducedValue.asInstanceOf[SqlTimestamp] + val milliseconds = ts.getMillisecond + val nanoseconds = ts.toLocalDateTime.getNano + val timestampString = TimestampString.fromMillisSinceEpoch(milliseconds) + timestampString.withNanos(nanoseconds) } else { reducedValue } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/BuiltInMethods.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/BuiltInMethods.scala index 3307fc15a1f6f..04f1434c3966b 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/BuiltInMethods.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/BuiltInMethods.scala @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.codegen.calls import org.apache.flink.table.dataformat.Decimal +import org.apache.flink.table.dataformat.SqlTimestamp import org.apache.flink.table.runtime.functions._ import org.apache.calcite.avatica.util.TimeUnitRange @@ -220,7 +221,7 @@ object BuiltInMethods { val TIMESTAMP_TO_STRING = Types.lookupMethod( classOf[SqlDateTimeUtils], "timestampToString", - classOf[Long], classOf[Int]) + classOf[SqlTimestamp]) val TIMESTAMP_TO_STRING_TIME_ZONE = Types.lookupMethod( classOf[SqlDateTimeUtils], diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala index 3937aab4937c9..c0f0d117b9ef7 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala @@ -2236,20 +2236,9 @@ object ScalarOperatorGens { case TIME_WITHOUT_TIME_ZONE => s"${qualifyMethod(BuiltInMethods.UNIX_TIME_TO_STRING)}($operandTerm)" case TIMESTAMP_WITHOUT_TIME_ZONE => // including rowtime indicator - // casting TimestampType to VARCHAR, if precision <= 3, keep the string representation - // consistent with the original design. Otherwise, use SqlTimestamp.toString(), which - // follows one of the following ISO-8601 formats: - // uuuu-MM-dd'T'HH:mm:ss.SSSSSS - // uuuu-MM-dd'T'HH:mm:ss.SSSSSSSSS - // The format used will be the shortest that outputs the full value of - // the time where the omitted parts are implied to be zero. - val precision = fromType.asInstanceOf[TimestampType].getPrecision - if (precision <= 3) { - val longTerm = s"$operandTerm.getMillisecond()" - s"${qualifyMethod(BuiltInMethods.TIMESTAMP_TO_STRING)}($longTerm, 3)" - } else { - s"$operandTerm.toString()" - } + // The interpreted string conforms to the definition of timestamp literal + // SQL 2011 Part 2 Section 6.13 General Rules 11) d) + s"${qualifyMethod(BuiltInMethods.TIMESTAMP_TO_STRING)}($operandTerm)" case TIMESTAMP_WITH_LOCAL_TIME_ZONE => val method = qualifyMethod(BuiltInMethods.TIMESTAMP_TO_STRING_TIME_ZONE) val zone = ctx.addReusableTimeZone() diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala index d81a1ff7bc33b..614f782574e37 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala @@ -94,7 +94,7 @@ class ArrayTypeTest extends ArrayTypeTestBase { Array(localDateTime("1985-04-11 14:15:16"), localDateTime("2018-07-26 17:18:19")), "array('1985-04-11 14:15:16'.toTimestamp, '2018-07-26 17:18:19'.toTimestamp)", "ARRAY[TIMESTAMP '1985-04-11 14:15:16', TIMESTAMP '2018-07-26 17:18:19']", - "[1985-04-11 14:15:16.000, 2018-07-26 17:18:19.000]") + "[1985-04-11 14:15:16, 2018-07-26 17:18:19]") // localDateTime use DateTimeUtils.timestampStringToUnixDate to parse a time string, // which only support millisecond's precision. @@ -102,41 +102,41 @@ class ArrayTypeTest extends ArrayTypeTestBase { Array( JLocalDateTime.of(1985, 4, 11, 14, 15, 16, 123456789), JLocalDateTime.of(2018, 7, 26, 17, 18, 19, 123456789)), - "[1985-04-11T14:15:16.123456789, 2018-07-26T17:18:19.123456789]") + "[1985-04-11 14:15:16.123456789, 2018-07-26 17:18:19.123456789]") testTableApi( Array( JLocalDateTime.of(1985, 4, 11, 14, 15, 16, 123456700), JLocalDateTime.of(2018, 7, 26, 17, 18, 19, 123456700)), - "[1985-04-11T14:15:16.123456700, 2018-07-26T17:18:19.123456700]") + "[1985-04-11 14:15:16.1234567, 2018-07-26 17:18:19.1234567]") testTableApi( Array( JLocalDateTime.of(1985, 4, 11, 14, 15, 16, 123456000), JLocalDateTime.of(2018, 7, 26, 17, 18, 19, 123456000)), - "[1985-04-11T14:15:16.123456, 2018-07-26T17:18:19.123456]") + "[1985-04-11 14:15:16.123456, 2018-07-26 17:18:19.123456]") testTableApi( Array( JLocalDateTime.of(1985, 4, 11, 14, 15, 16, 123400000), JLocalDateTime.of(2018, 7, 26, 17, 18, 19, 123400000)), - "[1985-04-11T14:15:16.123400, 2018-07-26T17:18:19.123400]") + "[1985-04-11 14:15:16.1234, 2018-07-26 17:18:19.1234]") testSqlApi( "ARRAY[TIMESTAMP '1985-04-11 14:15:16.123456789', TIMESTAMP '2018-07-26 17:18:19.123456789']", - "[1985-04-11T14:15:16.123456789, 2018-07-26T17:18:19.123456789]") + "[1985-04-11 14:15:16.123456789, 2018-07-26 17:18:19.123456789]") testSqlApi( "ARRAY[TIMESTAMP '1985-04-11 14:15:16.1234567', TIMESTAMP '2018-07-26 17:18:19.1234567']", - "[1985-04-11T14:15:16.123456700, 2018-07-26T17:18:19.123456700]") + "[1985-04-11 14:15:16.1234567, 2018-07-26 17:18:19.1234567]") testSqlApi( "ARRAY[TIMESTAMP '1985-04-11 14:15:16.123456', TIMESTAMP '2018-07-26 17:18:19.123456']", - "[1985-04-11T14:15:16.123456, 2018-07-26T17:18:19.123456]") + "[1985-04-11 14:15:16.123456, 2018-07-26 17:18:19.123456]") testSqlApi( "ARRAY[TIMESTAMP '1985-04-11 14:15:16.1234', TIMESTAMP '2018-07-26 17:18:19.1234']", - "[1985-04-11T14:15:16.123400, 2018-07-26T17:18:19.123400]") + "[1985-04-11 14:15:16.1234, 2018-07-26 17:18:19.1234]") testAllApis( Array(BigDecimal(2.0002), BigDecimal(2.0003)), diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MapTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MapTypeTest.scala index 1ba03a42df4ad..aad557243a658 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MapTypeTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MapTypeTest.scala @@ -92,7 +92,7 @@ class MapTypeTest extends MapTypeTestBase { "'17:18:19'.toTime, '2018-07-26 17:18:19'.toTimestamp)", "MAP[TIME '14:15:16', TIMESTAMP '1985-04-11 14:15:16', " + "TIME '17:18:19', TIMESTAMP '2018-07-26 17:18:19']", - "{14:15:16=1985-04-11 14:15:16.000, 17:18:19=2018-07-26 17:18:19.000}") + "{14:15:16=1985-04-11 14:15:16, 17:18:19=2018-07-26 17:18:19}") testAllApis( map(valueLiteral(gLocalTime("14:15:16")), @@ -110,12 +110,12 @@ class MapTypeTest extends MapTypeTestBase { valueLiteral(JLocalTimestamp.of(1985, 4, 11, 14, 15, 16, 123456000)), valueLiteral(gLocalTime("17:18:19")), valueLiteral(JLocalTimestamp.of(2018, 7, 26, 17, 18, 19, 123456000))), - "{14:15:16=1985-04-11T14:15:16.123456, 17:18:19=2018-07-26T17:18:19.123456}") + "{14:15:16=1985-04-11 14:15:16.123456, 17:18:19=2018-07-26 17:18:19.123456}") testSqlApi( "MAP[TIME '14:15:16', TIMESTAMP '1985-04-11 14:15:16.123456', " + "TIME '17:18:19', TIMESTAMP '2018-07-26 17:18:19.123456']", - "{14:15:16=1985-04-11T14:15:16.123456, 17:18:19=2018-07-26T17:18:19.123456}") + "{14:15:16=1985-04-11 14:15:16.123456, 17:18:19=2018-07-26 17:18:19.123456}") testAllApis( map(BigDecimal(2.0002), BigDecimal(2.0003)), diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala index 30730f309d942..35d38d576413f 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala @@ -51,17 +51,17 @@ class RowTypeTest extends RowTypeTestBase { "0.1p, Array(1, 2, 3), Map('foo', 'bar'), row(1, true))", "ROW(DATE '1985-04-11', TIME '14:15:16', TIMESTAMP '1985-04-11 14:15:16', " + "CAST(0.1 AS DECIMAL(2, 1)), ARRAY[1, 2, 3], MAP['foo', 'bar'], row(1, true))", - "(1985-04-11,14:15:16,1985-04-11 14:15:16.000,0.1,[1, 2, 3],{foo=bar},(1,true))") + "(1985-04-11,14:15:16,1985-04-11 14:15:16,0.1,[1, 2, 3],{foo=bar},(1,true))") testSqlApi( "ROW(DATE '1985-04-11', TIME '14:15:16', TIMESTAMP '1985-04-11 14:15:16', " + "CAST(0.1 AS DECIMAL(2, 1)), ARRAY[1, 2, 3], MAP['foo', 'bar'], row(1, true))", - "(1985-04-11,14:15:16,1985-04-11 14:15:16.000,0.1,[1, 2, 3],{foo=bar},(1,true))") + "(1985-04-11,14:15:16,1985-04-11 14:15:16,0.1,[1, 2, 3],{foo=bar},(1,true))") testSqlApi( "ROW(DATE '1985-04-11', TIME '14:15:16', TIMESTAMP '1985-04-11 14:15:16.123456', " + "CAST(0.1 AS DECIMAL(2, 1)), ARRAY[1, 2, 3], MAP['foo', 'bar'], row(1, true))", - "(1985-04-11,14:15:16,1985-04-11T14:15:16.123456,0.1,[1, 2, 3],{foo=bar},(1,true))") + "(1985-04-11,14:15:16,1985-04-11 14:15:16.123456,0.1,[1, 2, 3],{foo=bar},(1,true))") testAllApis( row(1 + 1, 2 * 3, nullOf(DataTypes.STRING())), diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala index 01b0f3eae0ea0..b823fe37e709a 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala @@ -3014,31 +3014,31 @@ class ScalarFunctionsTest extends ScalarTypesTestBase { 'f18.floor(TimeIntervalUnit.YEAR), "f18.floor(YEAR)", "FLOOR(f18 TO YEAR)", - "1996-01-01 00:00:00.000") + "1996-01-01 00:00:00") testAllApis( 'f18.floor(TimeIntervalUnit.MONTH), "f18.floor(MONTH)", "FLOOR(f18 TO MONTH)", - "1996-11-01 00:00:00.000") + "1996-11-01 00:00:00") testAllApis( 'f18.floor(TimeIntervalUnit.DAY), "f18.floor(DAY)", "FLOOR(f18 TO DAY)", - "1996-11-10 00:00:00.000") + "1996-11-10 00:00:00") testAllApis( 'f18.floor(TimeIntervalUnit.MINUTE), "f18.floor(MINUTE)", "FLOOR(f18 TO MINUTE)", - "1996-11-10 06:55:00.000") + "1996-11-10 06:55:00") testAllApis( 'f18.floor(TimeIntervalUnit.SECOND), "f18.floor(SECOND)", "FLOOR(f18 TO SECOND)", - "1996-11-10 06:55:44.000") + "1996-11-10 06:55:44") testAllApis( 'f17.floor(TimeIntervalUnit.HOUR), @@ -3074,31 +3074,31 @@ class ScalarFunctionsTest extends ScalarTypesTestBase { 'f18.ceil(TimeIntervalUnit.YEAR), "f18.ceil(YEAR)", "CEIL(f18 TO YEAR)", - "1997-01-01 00:00:00.000") + "1997-01-01 00:00:00") testAllApis( 'f18.ceil(TimeIntervalUnit.MONTH), "f18.ceil(MONTH)", "CEIL(f18 TO MONTH)", - "1996-12-01 00:00:00.000") + "1996-12-01 00:00:00") testAllApis( 'f18.ceil(TimeIntervalUnit.DAY), "f18.ceil(DAY)", "CEIL(f18 TO DAY)", - "1996-11-11 00:00:00.000") + "1996-11-11 00:00:00") testAllApis( 'f18.ceil(TimeIntervalUnit.MINUTE), "f18.ceil(MINUTE)", "CEIL(f18 TO MINUTE)", - "1996-11-10 06:56:00.000") + "1996-11-10 06:56:00") testAllApis( 'f18.ceil(TimeIntervalUnit.SECOND), "f18.ceil(SECOND)", "CEIL(f18 TO SECOND)", - "1996-11-10 06:55:45.000") + "1996-11-10 06:55:45") testAllApis( 'f17.ceil(TimeIntervalUnit.HOUR), @@ -3367,7 +3367,7 @@ class ScalarFunctionsTest extends ScalarTypesTestBase { "2020-11-29 22:58:58.998", "2016-11-29 22:58:58.998", "1956-11-29 22:58:58.998", - "1017-11-29 22:58:58.998") + "1017-12-05 22:58:58.998") val QUARTER = Seq( "2018-03-01 22:58:58.998", @@ -3525,22 +3525,22 @@ class ScalarFunctionsTest extends ScalarTypesTestBase { testAllApis("2016-06-15".toTimestamp - 1.hour, "'2016-06-15'.toTimestamp - 1.hour", "timestampadd(HOUR, -1, date '2016-06-15')", - "2016-06-14 23:00:00.000") + "2016-06-14 23:00:00") testAllApis("2016-06-15".toTimestamp + 1.minute, "'2016-06-15'.toTimestamp + 1.minute", "timestampadd(MINUTE, 1, date '2016-06-15')", - "2016-06-15 00:01:00.000") + "2016-06-15 00:01:00") testAllApis("2016-06-15".toTimestamp - 1.second, "'2016-06-15'.toTimestamp - 1.second", "timestampadd(SQL_TSI_SECOND, -1, date '2016-06-15')", - "2016-06-14 23:59:59.000") + "2016-06-14 23:59:59") testAllApis("2016-06-15".toTimestamp + 1.second, "'2016-06-15'.toTimestamp + 1.second", "timestampadd(SECOND, 1, date '2016-06-15')", - "2016-06-15 00:00:01.000") + "2016-06-15 00:00:01") testAllApis(nullOf(Types.SQL_TIMESTAMP) + 1.second, "nullOf(SQL_TIMESTAMP) + 1.second", @@ -3577,9 +3577,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase { @Test def testToTimestamp(): Unit = { testSqlApi("to_timestamp('abc')", "null") - testSqlApi("to_timestamp('2017-09-15 00:00:00')", "2017-09-15 00:00:00.000") - testSqlApi("to_timestamp('20170915000000', 'yyyyMMddHHmmss')", "2017-09-15 00:00:00.000") - testSqlApi("to_timestamp('2017-09-15', 'yyyy-MM-dd')", "2017-09-15 00:00:00.000") + testSqlApi("to_timestamp('2017-09-15 00:00:00')", "2017-09-15 00:00:00") + testSqlApi("to_timestamp('20170915000000', 'yyyyMMddHHmmss')", "2017-09-15 00:00:00") + testSqlApi("to_timestamp('2017-09-15', 'yyyy-MM-dd')", "2017-09-15 00:00:00") // test with null input testSqlApi("to_timestamp(cast(null as varchar))", "null") } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala index e1e91f6e6bd26..aad05f9655716 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala @@ -37,17 +37,6 @@ import java.util.{Locale, TimeZone} class TemporalTypesTest extends ExpressionTestBase { - @Test - def test(): Unit = { - testSqlApi( - "TIMESTAMP '1500-04-30 12:00:00.123456789'", - "1500-05-10T12:00:00.123456789") - - testSqlApi( - "CAST('1500-04-30 12:00:00.123' AS TIMESTAMP(3))", - "1500-05-10 12:00:00.123") - } - @Test def testTimePointLiterals(): Unit = { testAllApis( @@ -93,29 +82,29 @@ class TemporalTypesTest extends ExpressionTestBase { testTableApi( localDateTime2Literal(localDateTime("2040-09-11 00:00:00.000")), "'2040-09-11 00:00:00.000'.toTimestamp", - "2040-09-11 00:00:00.000") + "2040-09-11 00:00:00") testAllApis( "1500-04-30 12:00:00".cast(DataTypes.TIMESTAMP(3)), "'1500-04-30 12:00:00'.cast(SQL_TIMESTAMP)", "CAST('1500-04-30 12:00:00' AS TIMESTAMP(3))", - "1500-05-10 12:00:00.000") + "1500-05-10 12:00:00") testSqlApi( "TIMESTAMP '1500-04-30 12:00:00.123456789'", - "1500-05-10T12:00:00.123456789") + "1500-05-10 12:00:00.123456789") testSqlApi( "TIMESTAMP '1500-04-30 12:00:00.12345678'", - "1500-05-10T12:00:00.123456780") + "1500-05-10 12:00:00.12345678") testSqlApi( "TIMESTAMP '1500-04-30 12:00:00.123456'", - "1500-05-10T12:00:00.123456") + "1500-05-10 12:00:00.123456") testSqlApi( "TIMESTAMP '1500-04-30 12:00:00.1234'", - "1500-05-10T12:00:00.123400") + "1500-05-10 12:00:00.1234") } @Test @@ -205,13 +194,13 @@ class TemporalTypesTest extends ExpressionTestBase { 'f0.cast(DataTypes.TIMESTAMP(3)), "f0.cast(SQL_TIMESTAMP)", "CAST(f0 AS TIMESTAMP(3))", - "1990-10-14 00:00:00.000") + "1990-10-14 00:00:00") testAllApis( 'f1.cast(DataTypes.TIMESTAMP(3)), "f1.cast(SQL_TIMESTAMP)", "CAST(f1 AS TIMESTAMP(3))", - "1970-01-01 10:20:45.000") + "1970-01-01 10:20:45") testAllApis( 'f2.cast(DataTypes.DATE), @@ -254,12 +243,12 @@ class TemporalTypesTest extends ExpressionTestBase { testTableApi( 'f15.cast(DataTypes.TIMESTAMP(3)), "f15.cast(SQL_TIMESTAMP)", - "2016-06-27 07:23:33.000") + "2016-06-27 07:23:33") testTableApi( 'f15.toTimestamp, "f15.toTimestamp", - "2016-06-27 07:23:33.000") + "2016-06-27 07:23:33") testTableApi( 'f8.cast(DataTypes.TIMESTAMP(3)).cast(DataTypes.BIGINT()), @@ -707,21 +696,21 @@ class TemporalTypesTest extends ExpressionTestBase { testSqlApi("CEIL(TIME '12:44:31' TO MINUTE)", "12:45:00") testSqlApi("CEIL(TIME '12:44:31' TO HOUR)", "13:00:00") - testSqlApi("FLOOR(TIMESTAMP '2018-03-20 06:44:31' TO HOUR)", "2018-03-20 06:00:00.000") - testSqlApi("FLOOR(TIMESTAMP '2018-03-20 06:44:31' TO DAY)", "2018-03-20 00:00:00.000") - testSqlApi("FLOOR(TIMESTAMP '2018-03-20 00:00:00' TO DAY)", "2018-03-20 00:00:00.000") - testSqlApi("FLOOR(TIMESTAMP '2018-04-01 06:44:31' TO MONTH)", "2018-04-01 00:00:00.000") - testSqlApi("FLOOR(TIMESTAMP '2018-01-01 06:44:31' TO MONTH)", "2018-01-01 00:00:00.000") - testSqlApi("CEIL(TIMESTAMP '2018-03-20 06:44:31' TO HOUR)", "2018-03-20 07:00:00.000") - testSqlApi("CEIL(TIMESTAMP '2018-03-20 06:00:00' TO HOUR)", "2018-03-20 06:00:00.000") - testSqlApi("CEIL(TIMESTAMP '2018-03-20 06:44:31' TO DAY)", "2018-03-21 00:00:00.000") - testSqlApi("CEIL(TIMESTAMP '2018-03-01 00:00:00' TO DAY)", "2018-03-01 00:00:00.000") - testSqlApi("CEIL(TIMESTAMP '2018-03-31 00:00:01' TO DAY)", "2018-04-01 00:00:00.000") - testSqlApi("CEIL(TIMESTAMP '2018-03-01 21:00:01' TO MONTH)", "2018-03-01 00:00:00.000") - testSqlApi("CEIL(TIMESTAMP '2018-03-01 00:00:00' TO MONTH)", "2018-03-01 00:00:00.000") - testSqlApi("CEIL(TIMESTAMP '2018-12-02 00:00:00' TO MONTH)", "2019-01-01 00:00:00.000") - testSqlApi("CEIL(TIMESTAMP '2018-01-01 21:00:01' TO YEAR)", "2018-01-01 00:00:00.000") - testSqlApi("CEIL(TIMESTAMP '2018-01-02 21:00:01' TO YEAR)", "2019-01-01 00:00:00.000") + testSqlApi("FLOOR(TIMESTAMP '2018-03-20 06:44:31' TO HOUR)", "2018-03-20 06:00:00") + testSqlApi("FLOOR(TIMESTAMP '2018-03-20 06:44:31' TO DAY)", "2018-03-20 00:00:00") + testSqlApi("FLOOR(TIMESTAMP '2018-03-20 00:00:00' TO DAY)", "2018-03-20 00:00:00") + testSqlApi("FLOOR(TIMESTAMP '2018-04-01 06:44:31' TO MONTH)", "2018-04-01 00:00:00") + testSqlApi("FLOOR(TIMESTAMP '2018-01-01 06:44:31' TO MONTH)", "2018-01-01 00:00:00") + testSqlApi("CEIL(TIMESTAMP '2018-03-20 06:44:31' TO HOUR)", "2018-03-20 07:00:00") + testSqlApi("CEIL(TIMESTAMP '2018-03-20 06:00:00' TO HOUR)", "2018-03-20 06:00:00") + testSqlApi("CEIL(TIMESTAMP '2018-03-20 06:44:31' TO DAY)", "2018-03-21 00:00:00") + testSqlApi("CEIL(TIMESTAMP '2018-03-01 00:00:00' TO DAY)", "2018-03-01 00:00:00") + testSqlApi("CEIL(TIMESTAMP '2018-03-31 00:00:01' TO DAY)", "2018-04-01 00:00:00") + testSqlApi("CEIL(TIMESTAMP '2018-03-01 21:00:01' TO MONTH)", "2018-03-01 00:00:00") + testSqlApi("CEIL(TIMESTAMP '2018-03-01 00:00:00' TO MONTH)", "2018-03-01 00:00:00") + testSqlApi("CEIL(TIMESTAMP '2018-12-02 00:00:00' TO MONTH)", "2019-01-01 00:00:00") + testSqlApi("CEIL(TIMESTAMP '2018-01-01 21:00:01' TO YEAR)", "2018-01-01 00:00:00") + testSqlApi("CEIL(TIMESTAMP '2018-01-02 21:00:01' TO YEAR)", "2019-01-01 00:00:00") testSqlApi(s"FLOOR(${timestampTz("2018-03-20 06:44:31")} TO HOUR)", "2018-03-20 06:00:00.000") testSqlApi(s"FLOOR(${timestampTz("2018-03-20 06:44:31")} TO DAY)", "2018-03-20 00:00:00.000") diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/UserDefinedScalarFunctionTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/UserDefinedScalarFunctionTest.scala index 01585efa5ebc5..687417105cc1a 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/UserDefinedScalarFunctionTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/UserDefinedScalarFunctionTest.scala @@ -238,7 +238,7 @@ class UserDefinedScalarFunctionTest extends ExpressionTestBase { Func10('f6), "Func10(f6)", "Func10(f6)", - "1990-10-14 12:10:10.000") + "1990-10-14 12:10:10") } @Test diff --git a/flink-table/flink-table-runtime-blink/pom.xml b/flink-table/flink-table-runtime-blink/pom.xml index 52891a92ac3f2..9ad5c1077c61b 100644 --- a/flink-table/flink-table-runtime-blink/pom.xml +++ b/flink-table/flink-table-runtime-blink/pom.xml @@ -74,6 +74,67 @@ under the License. ${janino.version} + + + org.apache.calcite + calcite-core + + ${calcite.version} + + + + org.apache.calcite.avatica + avatica-metrics + + + com.google.protobuf + protobuf-java + + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + + + org.apache.commons + commons-dbcp2 + + + com.esri.geometry + esri-geometry-api + + + com.fasterxml.jackson.dataformat + jackson-dataformat-yaml + + + com.yahoo.datasketches + sketches-core + + + net.hydromatic + aggdesigner-algorithm + + + + org.apache.calcite.avatica avatica-core diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java index e95709aae08d8..2b67aee1c73a8 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java @@ -17,8 +17,10 @@ package org.apache.flink.table.runtime.functions; +import org.apache.calcite.util.TimestampString; import org.apache.flink.table.api.TableException; import org.apache.flink.table.dataformat.Decimal; +import org.apache.flink.table.dataformat.SqlTimestamp; import org.apache.flink.table.types.logical.DateType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.TimestampType; @@ -377,6 +379,19 @@ public static String convertTz(String dateStr, String tzFrom, String tzTo) { return convertTz(dateStr, TIMESTAMP_FORMAT_STRING, tzFrom, tzTo); } + public static String timestampToString(SqlTimestamp ts) { + LocalDateTime ldt = ts.toLocalDateTime(); + TimestampString timestampString = + new TimestampString( + ldt.getYear(), + ldt.getMonthValue(), + ldt.getDayOfMonth(), + ldt.getHour(), + ldt.getMinute(), + ldt.getSecond()).withNanos(ldt.getNano()); + return timestampString.toString(); + } + public static String timestampToString(long ts, int precision) { int p = (precision <= 3 && precision >= 0) ? precision : 3; String format = DEFAULT_DATETIME_FORMATS[p]; From 6b820480e47e4b2e6e4d3f1212e31f75ea9e0ed3 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Tue, 12 Nov 2019 18:41:31 +0800 Subject: [PATCH 26/29] fixup: ignore Gregorian shift --- .../flink/table/planner/codegen/GenerateUtils.scala | 4 ++-- .../table/planner/expressions/TemporalTypesTest.scala | 10 +++++----- .../table/runtime/functions/SqlDateTimeUtils.java | 4 ++-- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala index 9072aad6763a7..374ba9309fa1a 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala @@ -408,12 +408,12 @@ object GenerateUtils { val m = JInteger.valueOf(v.substring(14, 16)) val s = JInteger.valueOf(v.substring(17, 19)) val ms = getMillisInSecond(timestampString) - val d = SqlDateTimeUtils.ymdToJulian(year, month, day) + val d = SqlDateTimeUtils.ymdToUnixDate(year, month, day) d * 86400000L + h * 3600000L + m * 60000L + s * 1000L + ms.toLong } val fieldTerm = newName("timestamp") - val millis = literalValue.asInstanceOf[TimestampString].getMillisSinceEpoch + val millis = getMillisSinceEpoch(literalValue.asInstanceOf[TimestampString]) val nanoOfMillis = getNanoOfMillisSinceEpoch( literalValue.asInstanceOf[TimestampString]) val fieldTimestamp = diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala index aad05f9655716..48fda176c5083 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala @@ -88,23 +88,23 @@ class TemporalTypesTest extends ExpressionTestBase { "1500-04-30 12:00:00".cast(DataTypes.TIMESTAMP(3)), "'1500-04-30 12:00:00'.cast(SQL_TIMESTAMP)", "CAST('1500-04-30 12:00:00' AS TIMESTAMP(3))", - "1500-05-10 12:00:00") + "1500-04-30 12:00:00") testSqlApi( "TIMESTAMP '1500-04-30 12:00:00.123456789'", - "1500-05-10 12:00:00.123456789") + "1500-04-30 12:00:00.123456789") testSqlApi( "TIMESTAMP '1500-04-30 12:00:00.12345678'", - "1500-05-10 12:00:00.12345678") + "1500-04-30 12:00:00.12345678") testSqlApi( "TIMESTAMP '1500-04-30 12:00:00.123456'", - "1500-05-10 12:00:00.123456") + "1500-04-30 12:00:00.123456") testSqlApi( "TIMESTAMP '1500-04-30 12:00:00.1234'", - "1500-05-10 12:00:00.1234") + "1500-04-30 12:00:00.1234") } @Test diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java index 2b67aee1c73a8..284a4c29f6de1 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java @@ -1021,12 +1021,12 @@ public static int localDateToUnixDate(LocalDate date) { return ymdToUnixDate(date.getYear(), date.getMonthValue(), date.getDayOfMonth()); } - private static int ymdToUnixDate(int year, int month, int day) { + public static int ymdToUnixDate(int year, int month, int day) { final int julian = ymdToJulian(year, month, day); return julian - EPOCH_JULIAN; } - public static int ymdToJulian(int year, int month, int day) { + private static int ymdToJulian(int year, int month, int day) { int a = (14 - month) / 12; int y = year + 4800 - a; int m = month + 12 * a - 3; From aab21602fe3098c313ecb5e29faa8e175803e36a Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 13 Nov 2019 11:00:27 +0800 Subject: [PATCH 27/29] fixup: address comments --- .../converter/ExpressionConverter.java | 2 +- .../planner/calcite/FlinkTypeFactory.scala | 4 +- .../planner/calcite/FlinkTypeSystem.scala | 8 ++-- .../table/planner/codegen/GenerateUtils.scala | 45 ++---------------- .../planner/codegen/calls/MethodCallGen.scala | 3 ++ .../planner/plan/utils/RexNodeExtractor.scala | 12 +++-- ...ypesTest.scala => DateTimeTypesTest.scala} | 2 +- .../flink/table/dataformat/BinaryArray.java | 3 +- .../runtime/functions/SqlDateTimeUtils.java | 47 ++++++++++++++++++- 9 files changed, 69 insertions(+), 57 deletions(-) rename flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/{TemporalTypesTest.scala => DateTimeTypesTest.scala} (99%) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java index e889431fb89d1..1207b78b971cc 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java @@ -148,7 +148,7 @@ public RexNode visit(ValueLiteralExpression valueLiteral) { datetime = valueLiteral.getValueAs(Timestamp.class) .orElseThrow(() -> new TableException("Invalid literal.")).toLocalDateTime(); } else { - throw new TableException("Invalid literal."); + throw new TableException(String.format("Invalid literal of %s.", clazz.getCanonicalName())); } return relBuilder.getRexBuilder().makeTimestampLiteral( new TimestampString( diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala index 9d43161cdead2..b917a9f44b707 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala @@ -425,9 +425,9 @@ object FlinkTypeFactory { new TimeType() case TIMESTAMP => val precision = relDataType.getPrecision - if (precision > 9 || precision < 0) { + if (precision > TimestampType.MAX_PRECISION || precision < TimestampType.MIN_PRECISION) { throw new TableException( - s"TIMESTAMP precision is not supported: ${precision}") + s"TIMESTAMP precision is not supported: $precision") } new TimestampType(precision) case TIMESTAMP_WITH_LOCAL_TIME_ZONE => diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeSystem.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeSystem.scala index bbc190abfecd0..376d53c894f0d 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeSystem.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeSystem.scala @@ -19,8 +19,7 @@ package org.apache.flink.table.planner.calcite import org.apache.flink.table.runtime.typeutils.TypeCheckUtils -import org.apache.flink.table.types.logical.{DecimalType, LogicalType} - +import org.apache.flink.table.types.logical.{DecimalType, LogicalType, TimestampType} import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory, RelDataTypeSystemImpl} import org.apache.calcite.sql.`type`.{SqlTypeName, SqlTypeUtil} @@ -43,7 +42,7 @@ class FlinkTypeSystem extends RelDataTypeSystemImpl { // by default we support timestamp with microseconds precision case SqlTypeName.TIMESTAMP => - 6 + TimestampType.DEFAULT_PRECISION // we currently support only timestamps with milliseconds precision case SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE => @@ -58,7 +57,8 @@ class FlinkTypeSystem extends RelDataTypeSystemImpl { Int.MaxValue // The maximal precision of TIMESTAMP is 3, change it to 9 to support nanoseconds precision - case SqlTypeName.TIMESTAMP => 9 + case SqlTypeName.TIMESTAMP => + TimestampType.MAX_PRECISION case _ => super.getMaxPrecision(typeName) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala index 374ba9309fa1a..631fc457e4f0c 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala @@ -373,49 +373,10 @@ object GenerateUtils { generateNonNullLiteral(literalType, literalValue.toString, literalValue) case TIMESTAMP_WITHOUT_TIME_ZONE => - def getNanoOfMillisSinceEpoch(timestampString: TimestampString): Int = { - val v = timestampString.toString() - val length = v.length - val nanoOfSeconds = length match { - case 19 | 20 => 0 - case _ => - JInteger.valueOf(v.substring(20)) * pow(10, 9 - (length - 20)).intValue() - } - nanoOfSeconds % 1000000 - } - - // TODO: we copied the logical of TimestampString::getMillisSinceEpoch since the copied - // DateTimeUtils.ymdToJulian is wrong. - // SEE CALCITE-1884 - def getMillisInSecond(timestampString: TimestampString): Int = { - val v = timestampString.toString() - val length = v.length - val milliOfSeconds = length match { - case 19 => 0 - case 21 => JInteger.valueOf(v.substring(20)).intValue() * 100 - case 22 => JInteger.valueOf(v.substring(20)).intValue() * 10 - case 20 | 23 | _ => JInteger.valueOf(v.substring(20, 23)).intValue() - } - milliOfSeconds - } - - def getMillisSinceEpoch(timestampString: TimestampString): Long = { - val v = timestampString.toString() - val year = JInteger.valueOf(v.substring(0, 4)) - val month = JInteger.valueOf(v.substring(5, 7)) - val day = JInteger.valueOf(v.substring(8, 10)) - val h = JInteger.valueOf(v.substring(11, 13)) - val m = JInteger.valueOf(v.substring(14, 16)) - val s = JInteger.valueOf(v.substring(17, 19)) - val ms = getMillisInSecond(timestampString) - val d = SqlDateTimeUtils.ymdToUnixDate(year, month, day) - d * 86400000L + h * 3600000L + m * 60000L + s * 1000L + ms.toLong - } - val fieldTerm = newName("timestamp") - val millis = getMillisSinceEpoch(literalValue.asInstanceOf[TimestampString]) - val nanoOfMillis = getNanoOfMillisSinceEpoch( - literalValue.asInstanceOf[TimestampString]) + val timestampString = literalValue.asInstanceOf[TimestampString] + val millis = SqlDateTimeUtils.getMillisSinceEpoch(timestampString.toString) + val nanoOfMillis = SqlDateTimeUtils.getNanoOfMillisSinceEpoch(timestampString.toString) val fieldTimestamp = s""" |$SQL_TIMESTAMP $fieldTerm = diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/MethodCallGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/MethodCallGen.scala index 023c9c11e08d6..b340eb882672a 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/MethodCallGen.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/MethodCallGen.scala @@ -40,6 +40,7 @@ class MethodCallGen(method: Method) extends CallGenerator { s"$term.toString()" } else if ((clazz == classOf[Long] || clazz == classOf[java.lang.Long]) && operands(i).resultType.getTypeRoot == LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE) { + // convert the SqlTimestamp parameter to long if the method parameter accept long s"$term.getMillisecond()" } else { term @@ -66,6 +67,8 @@ class MethodCallGen(method: Method) extends CallGenerator { } else if ((method.getReturnType == classOf[Long] || method.getReturnType == classOf[java.lang.Long]) && returnType.getTypeRoot == LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE) { + // convert long to SqlTimestamp if the return type is SqlTimestamp and the method + // returns long s"$SQL_TIMESTAMP.fromEpochMillis($call)" } else { call diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala index b7a81515dc41a..86eb5eee10e21 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala @@ -27,7 +27,7 @@ import org.apache.flink.table.expressions.utils.ApiExpressionUtils._ import org.apache.flink.table.functions.BuiltInFunctionDefinitions.{AND, CAST, OR} import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.utils.Logging -import org.apache.flink.table.runtime.functions.SqlDateTimeUtils.unixTimestampToLocalDateTime +import org.apache.flink.table.runtime.functions.SqlDateTimeUtils.{getMillisSinceEpoch, getNanoOfMillisSinceEpoch, unixTimestampToLocalDateTime} import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType import org.apache.flink.table.types.DataType import org.apache.flink.table.types.logical.LogicalTypeRoot._ @@ -36,7 +36,7 @@ import org.apache.calcite.plan.RelOptUtil import org.apache.calcite.rex._ import org.apache.calcite.sql.fun.{SqlStdOperatorTable, SqlTrimFunction} import org.apache.calcite.sql.{SqlFunction, SqlPostfixOperator} -import org.apache.calcite.util.Util +import org.apache.calcite.util.{TimestampString, Util} import java.util import java.util.{TimeZone, List => JList} @@ -338,8 +338,12 @@ class RexNodeToExpressionConverter( LocalTimeConverter.INSTANCE.toExternal(v) case TIMESTAMP_WITHOUT_TIME_ZONE => - val v = literal.getValueAs(classOf[java.lang.Long]) - LocalDateTimeConverter.INSTANCE.toExternal(SqlTimestamp.fromEpochMillis(v)) + val timestampString = literal.getValueAs(classOf[TimestampString]) + val millis = getMillisSinceEpoch(timestampString.toString) + val nanoOfMillis = getNanoOfMillisSinceEpoch(timestampString.toString) + + LocalDateTimeConverter.INSTANCE.toExternal( + SqlTimestamp.fromEpochMillis(millis, nanoOfMillis)) case TIMESTAMP_WITH_LOCAL_TIME_ZONE => val v = literal.getValueAs(classOf[java.lang.Long]) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/DateTimeTypesTest.scala similarity index 99% rename from flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala rename to flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/DateTimeTypesTest.scala index 48fda176c5083..dd0ec49b3b32e 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/DateTimeTypesTest.scala @@ -35,7 +35,7 @@ import java.text.SimpleDateFormat import java.time.{Instant, ZoneId} import java.util.{Locale, TimeZone} -class TemporalTypesTest extends ExpressionTestBase { +class DateTimeTypesTest extends ExpressionTestBase { @Test def testTimePointLiterals(): Unit = { diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java index 8f32827257659..57e1938c34836 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java @@ -196,7 +196,8 @@ public SqlTimestamp getTimestamp(int pos, int precision) { assertIndexIsValid(pos); if (SqlTimestamp.isCompact(precision)) { - return SqlTimestamp.fromEpochMillis(segments[0].getLong(getElementOffset(pos, 8))); + return SqlTimestamp.fromEpochMillis( + SegmentsUtil.getLong(segments, getElementOffset(pos, 8))); } int fieldOffset = getElementOffset(pos, 8); diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java index 284a4c29f6de1..c3283167ec6a6 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java @@ -17,7 +17,6 @@ package org.apache.flink.table.runtime.functions; -import org.apache.calcite.util.TimestampString; import org.apache.flink.table.api.TableException; import org.apache.flink.table.dataformat.Decimal; import org.apache.flink.table.dataformat.SqlTimestamp; @@ -29,6 +28,7 @@ import org.apache.calcite.avatica.util.DateTimeUtils; import org.apache.calcite.avatica.util.TimeUnit; import org.apache.calcite.avatica.util.TimeUnitRange; +import org.apache.calcite.util.TimestampString; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1021,7 +1021,7 @@ public static int localDateToUnixDate(LocalDate date) { return ymdToUnixDate(date.getYear(), date.getMonthValue(), date.getDayOfMonth()); } - public static int ymdToUnixDate(int year, int month, int day) { + private static int ymdToUnixDate(int year, int month, int day) { final int julian = ymdToJulian(year, month, day); return julian - EPOCH_JULIAN; } @@ -1312,4 +1312,47 @@ private static int parseFraction(String v, int multiplier) { return r; } + // TODO: we copied the logical of TimestampString::getMillisSinceEpoch since the copied + // DateTimeUtils.ymdToJulian is wrong. + // SEE CALCITE-1884 + public static long getMillisSinceEpoch(final String v) { + final int year = Integer.valueOf(v.substring(0, 4)); + final int month = Integer.valueOf(v.substring(5, 7)); + final int day = Integer.valueOf(v.substring(8, 10)); + final int h = Integer.valueOf(v.substring(11, 13)); + final int m = Integer.valueOf(v.substring(14, 16)); + final int s = Integer.valueOf(v.substring(17, 19)); + final int ms = getMillisInSecond(v); + final int d = ymdToUnixDate(year, month, day); + return d * DateTimeUtils.MILLIS_PER_DAY + + h * DateTimeUtils.MILLIS_PER_HOUR + + m * DateTimeUtils.MILLIS_PER_MINUTE + + s * DateTimeUtils.MILLIS_PER_SECOND + + ms; + } + + private static int getMillisInSecond(String v) { + switch (v.length()) { + case 19: // "1999-12-31 12:34:56" + return 0; + case 21: // "1999-12-31 12:34:56.7" + return Integer.valueOf(v.substring(20)) * 100; + case 22: // "1999-12-31 12:34:56.78" + return Integer.valueOf(v.substring(20)) * 10; + case 23: // "1999-12-31 12:34:56.789" + default: // "1999-12-31 12:34:56.789123456" + return Integer.valueOf(v.substring(20, 23)); + } + } + + public static int getNanoOfMillisSinceEpoch(final String v) { + switch (v.length()) { + case 19: + case 20: + return 0; + default: + return (Integer.valueOf(v.substring(20)) * + (int) Math.pow(10, 9 - (v.length() - 20))) % 1000000; + } + } } From ee2854a14e47390258a3108a2e17e9bdcd64302c Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 13 Nov 2019 11:17:52 +0800 Subject: [PATCH 28/29] fixup: store nanoOfMillisecond in fixed-length part to save 4 bytes per high precision timestamp --- .../table/dataformat/AbstractBinaryWriter.java | 7 +++---- .../apache/flink/table/dataformat/BinaryArray.java | 13 ++++++------- .../apache/flink/table/dataformat/BinaryRow.java | 13 ++++++------- .../apache/flink/table/dataformat/NestedRow.java | 12 ++++++------ 4 files changed, 21 insertions(+), 24 deletions(-) diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/AbstractBinaryWriter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/AbstractBinaryWriter.java index 52a0c69f8f56f..65319cce5d8ea 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/AbstractBinaryWriter.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/AbstractBinaryWriter.java @@ -187,19 +187,18 @@ public void writeTimestamp(int pos, SqlTimestamp value, int precision) { assert 0 == value.getNanoOfMillisecond(); writeLong(pos, value.getMillisecond()); } else { - ensureCapacity(12); + // store the nanoOfMillisecond in fixed-length part as offset and nanoOfMillisecond + ensureCapacity(8); // zero-out the bytes segment.putLong(cursor, 0L); - segment.putInt(cursor + 8, 0); if (value == null) { setNullBit(pos); setOffsetAndSize(pos, cursor, 0); } else { segment.putLong(cursor, value.getMillisecond()); - segment.putInt(cursor + 8, value.getNanoOfMillisecond()); - setOffsetAndSize(pos, cursor, 12); + setOffsetAndSize(pos, cursor, value.getNanoOfMillisecond()); } cursor += 12; diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java index 57e1938c34836..ef90049a1ecb5 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java @@ -201,10 +201,10 @@ public SqlTimestamp getTimestamp(int pos, int precision) { } int fieldOffset = getElementOffset(pos, 8); - final long offsetAndSize = SegmentsUtil.getLong(segments, fieldOffset); - final int subOffset = (int) (offsetAndSize >> 32); + final long offsetAndNanoOfMilli = SegmentsUtil.getLong(segments, fieldOffset); + final int nanoOfMillisecond = (int) offsetAndNanoOfMilli; + final int subOffset = (int) (offsetAndNanoOfMilli >> 32); final long millisecond = SegmentsUtil.getLong(segments, offset + subOffset); - final int nanoOfMillisecond = SegmentsUtil.getInt(segments, offset + subOffset + 8); return SqlTimestamp.fromEpochMillis(millisecond, nanoOfMillisecond); } @@ -391,16 +391,15 @@ public void setTimestamp(int pos, SqlTimestamp value, int precision) { // zero-out the bytes SegmentsUtil.setLong(segments, offset + cursor, 0L); - SegmentsUtil.setInt(segments, offset + cursor + 8, 0); if (value == null) { setNullAt(pos); SegmentsUtil.setLong(segments, fieldOffset, ((long) cursor) << 32); } else { - // write millisecond and nanoOfMillisecond to the variable length portion. + // write millisecond to the variable length portion. SegmentsUtil.setLong(segments, offset + cursor, value.getMillisecond()); - SegmentsUtil.setInt(segments, offset + cursor + 8, value.getNanoOfMillisecond()); - setLong(pos, ((long) cursor << 32) | 12L); + // write nanoOfMillisecond to the fixed-length portion. + setLong(pos, ((long) cursor << 32) | (long) value.getNanoOfMillisecond()); } } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRow.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRow.java index e5b22de424e50..979f556a7cae4 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRow.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRow.java @@ -225,16 +225,15 @@ public void setTimestamp(int pos, SqlTimestamp value, int precision) { // zero-out the bytes SegmentsUtil.setLong(segments, offset + cursor, 0L); - SegmentsUtil.setInt(segments, offset + cursor + 8, 0); if (value == null) { setNullAt(pos); segments[0].putLong(fieldOffset, ((long) cursor) << 32); } else { - // write millisecond and nanoOfMillisecond to the variable length portion. + // write millisecond to the variable length portion. SegmentsUtil.setLong(segments, offset + cursor, value.getMillisecond()); - SegmentsUtil.setInt(segments, offset + cursor + 8, value.getNanoOfMillisecond()); - setLong(pos, ((long) cursor << 32) | 12L); + // write nanoOfMillisecond to the fixed-length portion. + setLong(pos, ((long) cursor << 32) | (long) value.getNanoOfMillisecond()); } } } @@ -340,10 +339,10 @@ public SqlTimestamp getTimestamp(int pos, int precision) { } int fieldOffset = getFieldOffset(pos); - final long offsetAndSize = segments[0].getLong(fieldOffset); - final int subOffset = (int) (offsetAndSize >> 32); + final long offsetAndNanoOfMilli = segments[0].getLong(fieldOffset); + final int nanoOfMillisecond = (int) offsetAndNanoOfMilli; + final int subOffset = (int) (offsetAndNanoOfMilli >> 32); final long millisecond = SegmentsUtil.getLong(segments, offset + subOffset); - final int nanoOfMillisecond = SegmentsUtil.getInt(segments, offset + subOffset + 8); return SqlTimestamp.fromEpochMillis(millisecond, nanoOfMillisecond); } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/NestedRow.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/NestedRow.java index 1a8f1ccc38fd4..e305bf2246218 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/NestedRow.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/NestedRow.java @@ -157,15 +157,15 @@ public void setTimestamp(int pos, SqlTimestamp value, int precision) { // zero-out the bytes SegmentsUtil.setLong(segments, offset + cursor, 0L); - SegmentsUtil.setInt(segments, offset + cursor + 8, 0); if (value == null) { setNullAt(pos); SegmentsUtil.setLong(segments, fieldOffset, ((long) cursor) << 32); } else { + // write millisecond to variable length portion. SegmentsUtil.setLong(segments, offset + cursor, value.getMillisecond()); - SegmentsUtil.setInt(segments, offset + cursor + 8, value.getNanoOfMillisecond()); - setLong(pos, ((long) cursor << 32) | 12L); + // write nanoOfMillisecond to fixed-length portion. + setLong(pos, ((long) cursor << 32) | (long) value.getNanoOfMillisecond()); } } } @@ -277,10 +277,10 @@ public SqlTimestamp getTimestamp(int pos, int precision) { } int fieldOffset = getFieldOffset(pos); - final long offsetAndSize = SegmentsUtil.getLong(segments, fieldOffset); - final int subOffset = (int) (offsetAndSize >> 32); + final long offsetAndNanoOfMilli = SegmentsUtil.getLong(segments, fieldOffset); + final int nanoOfMillisecond = (int) offsetAndNanoOfMilli; + final int subOffset = (int) (offsetAndNanoOfMilli >> 32); final long millisecond = SegmentsUtil.getLong(segments, offset + subOffset); - final int nanoOfMillisecond = SegmentsUtil.getInt(segments, offset + subOffset + 8); return SqlTimestamp.fromEpochMillis(millisecond, nanoOfMillisecond); } From 212e234603bdc8df8e0a64429437e774b5dd7b1b Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 13 Nov 2019 16:40:56 +0800 Subject: [PATCH 29/29] fixup: address comments --- .../table/planner/codegen/ExpressionReducer.scala | 2 +- .../flink/table/planner/codegen/GenerateUtils.scala | 12 ++++-------- 2 files changed, 5 insertions(+), 9 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala index c50e3026bc1a9..bfd62ebec7456 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala @@ -33,11 +33,11 @@ import org.apache.flink.table.types.logical.RowType import org.apache.calcite.avatica.util.ByteString import org.apache.calcite.rex.{RexBuilder, RexExecutor, RexNode} import org.apache.calcite.sql.`type`.SqlTypeName +import org.apache.calcite.util.TimestampString import org.apache.commons.lang3.StringEscapeUtils import java.io.File import java.util.TimeZone -import org.apache.calcite.util.TimestampString import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala index 631fc457e4f0c..1cb11b3fb58d7 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala @@ -26,7 +26,7 @@ import org.apache.flink.table.planner.codegen.CodeGenUtils._ import org.apache.flink.table.planner.codegen.GeneratedExpression.{ALWAYS_NULL, NEVER_NULL, NO_CODE} import org.apache.flink.table.planner.codegen.calls.CurrentTimePointCallGen import org.apache.flink.table.planner.plan.utils.SortUtil -import org.apache.flink.table.runtime.functions.SqlDateTimeUtils.unixTimestampToLocalDateTime +import org.apache.flink.table.runtime.functions.SqlDateTimeUtils.{getMillisSinceEpoch, getNanoOfMillisSinceEpoch, unixTimestampToLocalDateTime} import org.apache.flink.table.runtime.types.PlannerTypeUtils import org.apache.flink.table.runtime.typeutils.TypeCheckUtils.{isCharacterString, isReference, isTemporal} import org.apache.flink.table.types.logical.LogicalTypeRoot._ @@ -35,12 +35,8 @@ import org.apache.calcite.avatica.util.ByteString import org.apache.calcite.util.TimestampString import org.apache.commons.lang3.StringEscapeUtils import java.math.{BigDecimal => JBigDecimal} -import java.lang.{Integer => JInteger} - -import org.apache.flink.table.runtime.functions.SqlDateTimeUtils import scala.collection.mutable -import scala.math.pow /** * Utilities to generate code for general purpose. @@ -374,9 +370,9 @@ object GenerateUtils { case TIMESTAMP_WITHOUT_TIME_ZONE => val fieldTerm = newName("timestamp") - val timestampString = literalValue.asInstanceOf[TimestampString] - val millis = SqlDateTimeUtils.getMillisSinceEpoch(timestampString.toString) - val nanoOfMillis = SqlDateTimeUtils.getNanoOfMillisSinceEpoch(timestampString.toString) + val timestampString = literalValue.asInstanceOf[TimestampString].toString + val millis = getMillisSinceEpoch(timestampString) + val nanoOfMillis = getNanoOfMillisSinceEpoch(timestampString) val fieldTimestamp = s""" |$SQL_TIMESTAMP $fieldTerm =