-
Notifications
You must be signed in to change notification settings - Fork 1.3k
[core] Support timestamp type in Iceberg compatibility #4318
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,8 +20,12 @@ | |
|
|
||
| import org.apache.paimon.data.BinaryString; | ||
| import org.apache.paimon.data.Decimal; | ||
| import org.apache.paimon.data.Timestamp; | ||
| import org.apache.paimon.types.DataType; | ||
| import org.apache.paimon.types.DecimalType; | ||
| import org.apache.paimon.types.LocalZonedTimestampType; | ||
| import org.apache.paimon.types.TimestampType; | ||
| import org.apache.paimon.utils.Preconditions; | ||
|
|
||
| import java.nio.ByteBuffer; | ||
| import java.nio.ByteOrder; | ||
|
|
@@ -79,11 +83,26 @@ public static ByteBuffer toByteBuffer(DataType type, Object value) { | |
| case DECIMAL: | ||
| Decimal decimal = (Decimal) value; | ||
| return ByteBuffer.wrap((decimal.toUnscaledBytes())); | ||
| case TIMESTAMP_WITHOUT_TIME_ZONE: | ||
| return timestampToByteBuffer( | ||
| (Timestamp) value, ((TimestampType) type).getPrecision()); | ||
| case TIMESTAMP_WITH_LOCAL_TIME_ZONE: | ||
| return timestampToByteBuffer( | ||
| (Timestamp) value, ((LocalZonedTimestampType) type).getPrecision()); | ||
| default: | ||
| throw new UnsupportedOperationException("Cannot serialize type: " + type); | ||
| } | ||
| } | ||
|
|
||
| private static ByteBuffer timestampToByteBuffer(Timestamp timestamp, int precision) { | ||
| Preconditions.checkArgument( | ||
| precision > 3 && precision <= 6, | ||
| "Paimon Iceberg compatibility only support timestamp type with precision from 4 to 6."); | ||
| return ByteBuffer.allocate(8) | ||
| .order(ByteOrder.LITTLE_ENDIAN) | ||
| .putLong(0, timestamp.toMicros()); | ||
| } | ||
|
|
||
| public static Object toPaimonObject(DataType type, byte[] bytes) { | ||
| switch (type.getTypeRoot()) { | ||
| case BOOLEAN: | ||
|
|
@@ -112,6 +131,15 @@ public static Object toPaimonObject(DataType type, byte[] bytes) { | |
| DecimalType decimalType = (DecimalType) type; | ||
| return Decimal.fromUnscaledBytes( | ||
| bytes, decimalType.getPrecision(), decimalType.getScale()); | ||
| case TIMESTAMP_WITHOUT_TIME_ZONE: | ||
| case TIMESTAMP_WITH_LOCAL_TIME_ZONE: | ||
| int timestampPrecision = ((TimestampType) type).getPrecision(); | ||
| long timestampLong = | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Change the order of these two statements, for example: Preconditions.checkArgument( |
||
| ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN).getLong(); | ||
| Preconditions.checkArgument( | ||
| timestampPrecision > 3 && timestampPrecision <= 6, | ||
| "Paimon Iceberg compatibility only support timestamp type with precision from 4 to 6."); | ||
| return Timestamp.fromMicros(timestampLong); | ||
| default: | ||
| throw new UnsupportedOperationException("Cannot deserialize type: " + type); | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,8 +20,10 @@ | |
|
|
||
| import org.apache.paimon.types.DataField; | ||
| import org.apache.paimon.types.DataType; | ||
| import org.apache.paimon.types.DataTypes; | ||
| import org.apache.paimon.types.DecimalType; | ||
| import org.apache.paimon.types.LocalZonedTimestampType; | ||
| import org.apache.paimon.types.TimestampType; | ||
| import org.apache.paimon.utils.Preconditions; | ||
|
|
||
| import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; | ||
| import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; | ||
|
|
@@ -57,6 +59,8 @@ public class IcebergDataField { | |
| @JsonProperty(FIELD_TYPE) | ||
| private final String type; | ||
|
|
||
| @JsonIgnore private final DataType dataType; | ||
|
|
||
| @JsonProperty(FIELD_DOC) | ||
| private final String doc; | ||
|
|
||
|
|
@@ -66,6 +70,7 @@ public IcebergDataField(DataField dataField, int bias) { | |
| dataField.name(), | ||
| !dataField.type().isNullable(), | ||
| toTypeString(dataField.type()), | ||
| dataField.type(), | ||
| dataField.description()); | ||
| } | ||
|
|
||
|
|
@@ -76,10 +81,16 @@ public IcebergDataField( | |
| @JsonProperty(FIELD_REQUIRED) boolean required, | ||
| @JsonProperty(FIELD_TYPE) String type, | ||
| @JsonProperty(FIELD_DOC) String doc) { | ||
| this(id, name, required, type, null, doc); | ||
| } | ||
|
|
||
| public IcebergDataField( | ||
| int id, String name, boolean required, String type, DataType dataType, String doc) { | ||
| this.id = id; | ||
| this.name = name; | ||
| this.required = required; | ||
| this.type = type; | ||
| this.dataType = dataType; | ||
| this.doc = doc; | ||
| } | ||
|
|
||
|
|
@@ -110,7 +121,7 @@ public String doc() { | |
|
|
||
| @JsonIgnore | ||
| public DataType dataType() { | ||
| return fromTypeString(type); | ||
| return Preconditions.checkNotNull(dataType); | ||
| } | ||
|
|
||
| private static String toTypeString(DataType dataType) { | ||
|
|
@@ -137,38 +148,23 @@ private static String toTypeString(DataType dataType) { | |
| DecimalType decimalType = (DecimalType) dataType; | ||
| return String.format( | ||
| "decimal(%d, %d)", decimalType.getPrecision(), decimalType.getScale()); | ||
| case TIMESTAMP_WITHOUT_TIME_ZONE: | ||
| int timestampPrecision = ((TimestampType) dataType).getPrecision(); | ||
| Preconditions.checkArgument( | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There are four place check this precision. |
||
| timestampPrecision > 3 && timestampPrecision <= 6, | ||
| "Paimon Iceberg compatibility only support timestamp type with precision from 4 to 6."); | ||
| return "timestamp"; | ||
| case TIMESTAMP_WITH_LOCAL_TIME_ZONE: | ||
| int timestampLtzPrecision = ((LocalZonedTimestampType) dataType).getPrecision(); | ||
| Preconditions.checkArgument( | ||
| timestampLtzPrecision > 3 && timestampLtzPrecision <= 6, | ||
| "Paimon Iceberg compatibility only support timestamp type with precision from 4 to 6."); | ||
| return "timestamptz"; | ||
| default: | ||
| throw new UnsupportedOperationException("Unsupported data type: " + dataType); | ||
| } | ||
| } | ||
|
|
||
| private static DataType fromTypeString(String type) { | ||
| if ("boolean".equals(type)) { | ||
| return DataTypes.BOOLEAN(); | ||
| } else if ("int".equals(type)) { | ||
| return DataTypes.INT(); | ||
| } else if ("long".equals(type)) { | ||
| return DataTypes.BIGINT(); | ||
| } else if ("float".equals(type)) { | ||
| return DataTypes.FLOAT(); | ||
| } else if ("double".equals(type)) { | ||
| return DataTypes.DOUBLE(); | ||
| } else if ("date".equals(type)) { | ||
| return DataTypes.DATE(); | ||
| } else if ("string".equals(type)) { | ||
| return DataTypes.STRING(); | ||
| } else if ("binary".equals(type)) { | ||
| return DataTypes.BYTES(); | ||
| } else if (type.startsWith("decimal")) { | ||
| String[] precisionAndScale = | ||
| type.substring("decimal(".length(), type.length() - 1).split(", "); | ||
| return DataTypes.DECIMAL( | ||
| Integer.parseInt(precisionAndScale[0]), Integer.parseInt(precisionAndScale[1])); | ||
| } else { | ||
| throw new UnsupportedOperationException("Unsupported data type: " + type); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public int hashCode() { | ||
| return Objects.hash(id, name, required, type, doc); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
from 4 to 6.