Skip to content
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

[core]IcebergConversions addType for Timestamp and Time #3866

Closed
wants to merge 12 commits into from
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,11 @@

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 java.nio.ByteBuffer;
import java.nio.ByteOrder;
Expand Down Expand Up @@ -79,6 +82,19 @@ public static ByteBuffer toByteBuffer(DataType type, Object value) {
case DECIMAL:
Decimal decimal = (Decimal) value;
return ByteBuffer.wrap((decimal.toUnscaledBytes()));
case TIMESTAMP_WITHOUT_TIME_ZONE:
TimestampType timestampType = (TimestampType) type;
return convertTimestampWithPrecisionToBuffer(
(Timestamp) value, timestampType.getPrecision());
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
LocalZonedTimestampType localTimestampType = (LocalZonedTimestampType) type;
return convertTimestampWithPrecisionToBuffer(
(Timestamp) value, localTimestampType.getPrecision());
case TIME_WITHOUT_TIME_ZONE:
long microsecondsFromMillis = (int) value * 1_000;
return ByteBuffer.allocate(8)
.order(ByteOrder.LITTLE_ENDIAN)
.putLong(0, microsecondsFromMillis);
default:
throw new UnsupportedOperationException("Cannot serialize type: " + type);
}
Expand Down Expand Up @@ -112,8 +128,39 @@ 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:
TimestampType timestampType = (TimestampType) type;
return convertBytesToTimestamp(bytes, timestampType.getPrecision());
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
LocalZonedTimestampType localTimestampType = (LocalZonedTimestampType) type;
return convertBytesToTimestamp(bytes, localTimestampType.getPrecision());
case TIME_WITHOUT_TIME_ZONE:
return ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN).getLong() / 1000;
default:
throw new UnsupportedOperationException("Cannot deserialize type: " + type);
}
}

private static ByteBuffer convertTimestampWithPrecisionToBuffer(
Timestamp timestamp, int precision) {
long timestampValue;
if (precision <= 3) {
timestampValue = timestamp.getMillisecond() * 1_000_000;
} else {
timestampValue =
timestamp.getMillisecond() * 1_000_000 + timestamp.getNanoOfMillisecond();
}
return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(timestampValue);
}

private static Timestamp convertBytesToTimestamp(byte[] bytes, int precision) {
long timestampValue = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN).getLong();
long milliseconds = timestampValue / 1_000_000;
int nanosOfMillisecond = (int) (timestampValue % 1_000_000);
if (precision <= 3) {
return Timestamp.fromEpochMillis(milliseconds);
} else {
return Timestamp.fromEpochMillis(milliseconds, nanosOfMillisecond);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -130,6 +130,12 @@ 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:
return "timestamp";
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
return "timestamptz";
case TIME_WITHOUT_TIME_ZONE:
return "time";
default:
throw new UnsupportedOperationException("Unsupported data type: " + dataType);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.paimon.data.BinaryString;
import org.apache.paimon.data.Decimal;
import org.apache.paimon.data.GenericRow;
import org.apache.paimon.data.Timestamp;
import org.apache.paimon.disk.IOManagerImpl;
import org.apache.paimon.fs.Path;
import org.apache.paimon.fs.local.LocalFileIO;
Expand Down Expand Up @@ -372,7 +373,8 @@ public void testAllTypeStatistics() throws Exception {
DataTypes.STRING(),
DataTypes.BINARY(20),
DataTypes.VARBINARY(20),
DataTypes.DATE()
DataTypes.DATE(),
DataTypes.TIMESTAMP(6)
},
new String[] {
"v_int",
Expand All @@ -385,7 +387,8 @@ public void testAllTypeStatistics() throws Exception {
"v_varchar",
"v_binary",
"v_varbinary",
"v_date"
"v_date",
"v_timestamp"
});
FileStoreTable table =
createPaimonTable(rowType, Collections.emptyList(), Collections.emptyList(), -1);
Expand All @@ -406,7 +409,8 @@ public void testAllTypeStatistics() throws Exception {
BinaryString.fromString("cat"),
"B_apple".getBytes(),
"B_cat".getBytes(),
100);
100,
Timestamp.fromEpochMillis(1060328130123L, 256000));
write.write(lowerBounds);
GenericRow upperBounds =
GenericRow.of(
Expand All @@ -420,7 +424,8 @@ public void testAllTypeStatistics() throws Exception {
BinaryString.fromString("dog"),
"B_banana".getBytes(),
"B_dog".getBytes(),
200);
200,
Timestamp.fromEpochMillis(1723486530123L, 456000));
write.write(upperBounds);
commit.commit(1, write.prepareCommit(false, 1));

Expand Down Expand Up @@ -448,6 +453,9 @@ public void testAllTypeStatistics() throws Exception {
} else if (type.getTypeRoot() == DataTypeRoot.DECIMAL) {
lower = new BigDecimal(lowerBounds.getField(i).toString());
upper = new BigDecimal(upperBounds.getField(i).toString());
} else if (type.getTypeRoot() == DataTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE) {
lower = ((Timestamp) lowerBounds.getField(i)).toString();
upper = ((Timestamp) upperBounds.getField(i)).toString();
} else {
lower = lowerBounds.getField(i);
upper = upperBounds.getField(i);
Expand All @@ -459,16 +467,18 @@ public void testAllTypeStatistics() throws Exception {
expectedLower = LocalDate.ofEpochDay((int) lower).toString();
expectedUpper = LocalDate.ofEpochDay((int) upper).toString();
}

assertThat(
getIcebergResult(
icebergTable ->
IcebergGenerics.read(icebergTable)
.select(name)
.where(Expressions.lessThan(name, upper))
.build(),
Record::toString))
.containsExactly("Record(" + expectedLower + ")");
if (type.getTypeRoot() != DataTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE) {
// todo iceberg lessthan has bug
assertThat(
getIcebergResult(
icebergTable ->
IcebergGenerics.read(icebergTable)
.select(name)
.where(Expressions.lessThan(name, upper))
.build(),
Record::toString))
.containsExactly("Record(" + expectedLower + ")");
}
assertThat(
getIcebergResult(
icebergTable ->
Expand Down Expand Up @@ -614,6 +624,76 @@ public void testPartitionedPrimaryKeyTable() throws Exception {
Record::toString);
}

@Test
public void testPartitionedPrimaryKeyTableTimestamp() throws Exception {
dbac marked this conversation as resolved.
Show resolved Hide resolved
RowType rowType =
RowType.of(
new DataType[] {
DataTypes.TIMESTAMP(6),
DataTypes.STRING(),
DataTypes.STRING(),
DataTypes.INT(),
DataTypes.BIGINT()
},
new String[] {"pt1", "pt2", "k", "v1", "v2"});

BiFunction<Timestamp, String, BinaryRow> binaryRow =
(pt1, pt2) -> {
BinaryRow b = new BinaryRow(2);
BinaryRowWriter writer = new BinaryRowWriter(b);
writer.writeTimestamp(0, pt1, 6);
writer.writeString(1, BinaryString.fromString(pt2));
writer.complete();
return b;
};

int numRounds = 20;
int numRecords = 100;
ThreadLocalRandom random = ThreadLocalRandom.current();

List<List<TestRecord>> testRecords = new ArrayList<>();
List<List<String>> expected = new ArrayList<>();
Map<String, String> expectedMap = new LinkedHashMap<>();

for (int r = 0; r < numRounds; r++) {
List<TestRecord> round = new ArrayList<>();
for (int i = 0; i < numRecords; i++) {
Timestamp pt1 = generateRandomTimestamp(random, random.nextBoolean() ? 3 : 6);
String pt2 = String.valueOf(random.nextInt(10, 12));
String k = String.valueOf(random.nextInt(0, 100));
int v1 = random.nextInt();
long v2 = random.nextLong();

round.add(
new TestRecord(
binaryRow.apply(pt1, pt2),
GenericRow.of(
pt1,
BinaryString.fromString(pt2),
BinaryString.fromString(k),
v1,
v2)));

expectedMap.put(
String.format("%s, %s, %s", pt1, pt2, k), String.format("%d, %d", v1, v2));
}

testRecords.add(round);
expected.add(
expectedMap.entrySet().stream()
.map(e -> String.format("Record(%s, %s)", e.getKey(), e.getValue()))
.collect(Collectors.toList()));
}

runCompatibilityTest(
rowType,
Arrays.asList("pt1", "pt2"),
Arrays.asList("pt1", "pt2", "k"),
testRecords,
expected,
Record::toString);
}

private void runCompatibilityTest(
RowType rowType,
List<String> partitionKeys,
Expand Down Expand Up @@ -726,4 +806,21 @@ private List<String> getIcebergResult(
result.close();
return actual;
}

private Timestamp generateRandomTimestamp(ThreadLocalRandom random, int precision) {
long milliseconds = random.nextLong(0, Long.MAX_VALUE / 1000_000 - 1_000 * 1000);
int nanoAdjustment;
switch (precision) {
case 3:
nanoAdjustment = 0;
break;
case 6:
nanoAdjustment = random.nextInt(0, 1_000) * 1000;
break;
default:
throw new IllegalArgumentException("Unsupported precision: " + precision);
}

return Timestamp.fromEpochMillis(milliseconds, nanoAdjustment);
}
}
Loading