Skip to content

Commit

Permalink
Add support for delta timestamp_ntz datatype
Browse files Browse the repository at this point in the history
Add support for TIMESTAMP_NTZ datatype columns in v3 delta tables.
  • Loading branch information
infvg committed Jan 29, 2025
1 parent 0d22e37 commit bd1117e
Show file tree
Hide file tree
Showing 21 changed files with 32 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@
import io.delta.kernel.types.ShortType;
import io.delta.kernel.types.StringType;
import io.delta.kernel.types.StructType;
import io.delta.kernel.types.TimestampNTZType;
import io.delta.kernel.types.TimestampType;

import java.math.BigDecimal;
Expand Down Expand Up @@ -230,7 +231,7 @@ else if (deltaType instanceof ShortType) {
else if (deltaType instanceof StringType) {
return createUnboundedVarcharType();
}
else if (deltaType instanceof TimestampType) {
else if (deltaType instanceof TimestampType || deltaType instanceof TimestampNTZType) {
return TIMESTAMP;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,8 @@ public abstract class AbstractDeltaDistributedQueryTestBase
"test-lowercase",
"test-partitions-lowercase",
"test-uppercase",
"test-partitions-uppercase"
"test-partitions-uppercase",
"test-typing"
};

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
*/
package com.facebook.presto.delta;

import com.facebook.presto.Session;
import com.google.common.base.Joiner;
import org.testng.annotations.Test;

Expand All @@ -24,6 +25,7 @@
import java.util.List;
import java.util.concurrent.TimeUnit;

import static com.facebook.presto.common.type.TimeZoneKey.UTC_KEY;
import static java.lang.String.format;

/**
Expand Down Expand Up @@ -247,6 +249,17 @@ public void readPartitionedTableAllDataTypes(String version)
assertQuery(testQuery, expResultsQuery);
}

@Test(dataProvider = "deltaReaderVersions")
public void testDeltaTimezoneTypeSupport(String version)
{
Session session = Session.builder(getSession())
.setTimeZoneKey(UTC_KEY)
.build();
String testQuery = format("SELECT tpep_dropoff_datetime FROM \"%s\".\"%s\"",
PATH_SCHEMA, goldenTablePathWithPrefix(version, "test-typing"));

assertQuery(session, testQuery, "SELECT CAST('2021-12-31 16:53:29' AS TIMESTAMP)");
}
/**
* Expected results for table "data-reader-primitives"
*/
Expand Down
Binary file not shown.
Binary file not shown.
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
{"txnId":"1dc8f519-a27a-4e10-8cba-b676268f8757","tableSizeBytes":5526,"numFiles":1,"numMetadata":1,"numProtocol":1,"setTransactions":[],"domainMetadata":[],"metadata":{"id":"3b76f6bb-def8-40b9-acfc-f0c1fd74ff91","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"VendorID\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"tpep_pickup_datetime\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"tpep_dropoff_datetime\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"passenger_count\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"trip_distance\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"RatecodeID\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"store_and_fwd_flag\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"PULocationID\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"DOLocationID\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"payment_type\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"fare_amount\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"extra\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"mta_tax\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"tip_amount\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"tolls_amount\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"improvement_surcharge\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"total_amount\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"congestion_surcharge\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"airport_fee\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1738058626356},"protocol":{"minReaderVersion":1,"minWriterVersion":2},"allFiles":[{"path":"part-00000-16ab4bfc-37b2-4e87-961c-e9e18a355eeb-c000.snappy.parquet","partitionValues":{},"size":5526,"modificationTime":1738058627336,"dataChange":false,"stats":"{\"numRecords\":1,\"minValues\":{\"VendorID\":1,\"tpep_pickup_datetime\":\"2022-01-01T00:35:40.000+03:00\",\"tpep_dropoff_datetime\":\"2022-01-01T00:53:29.000+03:00\",\"passenger_count\":2.0,\"trip_distance\":3.8,\"RatecodeID\":1.0,\"store_and_fwd_flag\":\"N\",\"PULocationID\":142,\"DOLocationID\":236,\"payment_type\":1,\"fare_amount\":14.5,\"extra\":3.0,\"mta_tax\":0.5,\"tip_amount\":3.65,\"tolls_amount\":0.0,\"improvement_surcharge\":0.3,\"total_amount\":21.95,\"congestion_surcharge\":2.5,\"airport_fee\":0.0},\"maxValues\":{\"VendorID\":1,\"tpep_pickup_datetime\":\"2022-01-01T00:35:40.000+03:00\",\"tpep_dropoff_datetime\":\"2022-01-01T00:53:29.000+03:00\",\"passenger_count\":2.0,\"trip_distance\":3.8,\"RatecodeID\":1.0,\"store_and_fwd_flag\":\"N\",\"PULocationID\":142,\"DOLocationID\":236,\"payment_type\":1,\"fare_amount\":14.5,\"extra\":3.0,\"mta_tax\":0.5,\"tip_amount\":3.65,\"tolls_amount\":0.0,\"improvement_surcharge\":0.3,\"total_amount\":21.95,\"congestion_surcharge\":2.5,\"airport_fee\":0.0},\"nullCount\":{\"VendorID\":0,\"tpep_pickup_datetime\":0,\"tpep_dropoff_datetime\":0,\"passenger_count\":0,\"trip_distance\":0,\"RatecodeID\":0,\"store_and_fwd_flag\":0,\"PULocationID\":0,\"DOLocationID\":0,\"payment_type\":0,\"fare_amount\":0,\"extra\":0,\"mta_tax\":0,\"tip_amount\":0,\"tolls_amount\":0,\"improvement_surcharge\":0,\"total_amount\":0,\"congestion_surcharge\":0,\"airport_fee\":0}}"}]}
Original file line number Diff line number Diff line change
@@ -0,0 +1,4 @@
{"commitInfo":{"timestamp":1738058627380,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputRows":"1","numOutputBytes":"5526"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"1dc8f519-a27a-4e10-8cba-b676268f8757"}}
{"metaData":{"id":"3b76f6bb-def8-40b9-acfc-f0c1fd74ff91","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"VendorID\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"tpep_pickup_datetime\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"tpep_dropoff_datetime\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"passenger_count\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"trip_distance\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"RatecodeID\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"store_and_fwd_flag\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"PULocationID\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"DOLocationID\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"payment_type\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"fare_amount\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"extra\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"mta_tax\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"tip_amount\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"tolls_amount\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"improvement_surcharge\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"total_amount\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"congestion_surcharge\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"airport_fee\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1738058626356}}
{"protocol":{"minReaderVersion":1,"minWriterVersion":2}}
{"add":{"path":"part-00000-16ab4bfc-37b2-4e87-961c-e9e18a355eeb-c000.snappy.parquet","partitionValues":{},"size":5526,"modificationTime":1738058627336,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"VendorID\":1,\"tpep_pickup_datetime\":\"2022-01-01T00:35:40.000+03:00\",\"tpep_dropoff_datetime\":\"2022-01-01T00:53:29.000+03:00\",\"passenger_count\":2.0,\"trip_distance\":3.8,\"RatecodeID\":1.0,\"store_and_fwd_flag\":\"N\",\"PULocationID\":142,\"DOLocationID\":236,\"payment_type\":1,\"fare_amount\":14.5,\"extra\":3.0,\"mta_tax\":0.5,\"tip_amount\":3.65,\"tolls_amount\":0.0,\"improvement_surcharge\":0.3,\"total_amount\":21.95,\"congestion_surcharge\":2.5,\"airport_fee\":0.0},\"maxValues\":{\"VendorID\":1,\"tpep_pickup_datetime\":\"2022-01-01T00:35:40.000+03:00\",\"tpep_dropoff_datetime\":\"2022-01-01T00:53:29.000+03:00\",\"passenger_count\":2.0,\"trip_distance\":3.8,\"RatecodeID\":1.0,\"store_and_fwd_flag\":\"N\",\"PULocationID\":142,\"DOLocationID\":236,\"payment_type\":1,\"fare_amount\":14.5,\"extra\":3.0,\"mta_tax\":0.5,\"tip_amount\":3.65,\"tolls_amount\":0.0,\"improvement_surcharge\":0.3,\"total_amount\":21.95,\"congestion_surcharge\":2.5,\"airport_fee\":0.0},\"nullCount\":{\"VendorID\":0,\"tpep_pickup_datetime\":0,\"tpep_dropoff_datetime\":0,\"passenger_count\":0,\"trip_distance\":0,\"RatecodeID\":0,\"store_and_fwd_flag\":0,\"PULocationID\":0,\"DOLocationID\":0,\"payment_type\":0,\"fare_amount\":0,\"extra\":0,\"mta_tax\":0,\"tip_amount\":0,\"tolls_amount\":0,\"improvement_surcharge\":0,\"total_amount\":0,\"congestion_surcharge\":0,\"airport_fee\":0}}"}}
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
{"txnId":"f768aae4-300a-4cab-819f-949d69bf126f","tableSizeBytes":5526,"numFiles":1,"numMetadata":1,"numProtocol":1,"setTransactions":[],"domainMetadata":[],"metadata":{"id":"bf378016-c2bd-406a-b90c-59d260f8ff52","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"VendorID\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"tpep_pickup_datetime\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"tpep_dropoff_datetime\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"passenger_count\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"trip_distance\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"RatecodeID\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"store_and_fwd_flag\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"PULocationID\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"DOLocationID\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"payment_type\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"fare_amount\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"extra\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"mta_tax\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"tip_amount\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"tolls_amount\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"improvement_surcharge\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"total_amount\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"congestion_surcharge\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"airport_fee\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1738136311793},"protocol":{"minReaderVersion":1,"minWriterVersion":2},"allFiles":[{"path":"part-00000-4705e2d5-430d-42da-a737-4921ed7fc950-c000.snappy.parquet","partitionValues":{},"size":5526,"modificationTime":1738136312747,"dataChange":false,"stats":"{\"numRecords\":1,\"minValues\":{\"VendorID\":1,\"tpep_pickup_datetime\":\"2021-12-31T19:35:40.000+03:00\",\"tpep_dropoff_datetime\":\"2021-12-31T19:53:29.000+03:00\",\"passenger_count\":2.0,\"trip_distance\":3.8,\"RatecodeID\":1.0,\"store_and_fwd_flag\":\"N\",\"PULocationID\":142,\"DOLocationID\":236,\"payment_type\":1,\"fare_amount\":14.5,\"extra\":3.0,\"mta_tax\":0.5,\"tip_amount\":3.65,\"tolls_amount\":0.0,\"improvement_surcharge\":0.3,\"total_amount\":21.95,\"congestion_surcharge\":2.5,\"airport_fee\":0.0},\"maxValues\":{\"VendorID\":1,\"tpep_pickup_datetime\":\"2021-12-31T19:35:40.000+03:00\",\"tpep_dropoff_datetime\":\"2021-12-31T19:53:29.000+03:00\",\"passenger_count\":2.0,\"trip_distance\":3.8,\"RatecodeID\":1.0,\"store_and_fwd_flag\":\"N\",\"PULocationID\":142,\"DOLocationID\":236,\"payment_type\":1,\"fare_amount\":14.5,\"extra\":3.0,\"mta_tax\":0.5,\"tip_amount\":3.65,\"tolls_amount\":0.0,\"improvement_surcharge\":0.3,\"total_amount\":21.95,\"congestion_surcharge\":2.5,\"airport_fee\":0.0},\"nullCount\":{\"VendorID\":0,\"tpep_pickup_datetime\":0,\"tpep_dropoff_datetime\":0,\"passenger_count\":0,\"trip_distance\":0,\"RatecodeID\":0,\"store_and_fwd_flag\":0,\"PULocationID\":0,\"DOLocationID\":0,\"payment_type\":0,\"fare_amount\":0,\"extra\":0,\"mta_tax\":0,\"tip_amount\":0,\"tolls_amount\":0,\"improvement_surcharge\":0,\"total_amount\":0,\"congestion_surcharge\":0,\"airport_fee\":0}}"}]}
Original file line number Diff line number Diff line change
@@ -0,0 +1,4 @@
{"commitInfo":{"timestamp":1738136312822,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"1","numOutputRows":"1","numOutputBytes":"5526"},"engineInfo":"Apache-Spark/3.5.4 Delta-Lake/3.3.0","txnId":"f768aae4-300a-4cab-819f-949d69bf126f"}}
{"metaData":{"id":"bf378016-c2bd-406a-b90c-59d260f8ff52","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"VendorID\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"tpep_pickup_datetime\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"tpep_dropoff_datetime\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}},{\"name\":\"passenger_count\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"trip_distance\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"RatecodeID\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"store_and_fwd_flag\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"PULocationID\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"DOLocationID\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"payment_type\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"fare_amount\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"extra\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"mta_tax\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"tip_amount\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"tolls_amount\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"improvement_surcharge\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"total_amount\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"congestion_surcharge\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"airport_fee\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1738136311793}}
{"protocol":{"minReaderVersion":1,"minWriterVersion":2}}
{"add":{"path":"part-00000-4705e2d5-430d-42da-a737-4921ed7fc950-c000.snappy.parquet","partitionValues":{},"size":5526,"modificationTime":1738136312747,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"VendorID\":1,\"tpep_pickup_datetime\":\"2021-12-31T19:35:40.000+03:00\",\"tpep_dropoff_datetime\":\"2021-12-31T19:53:29.000+03:00\",\"passenger_count\":2.0,\"trip_distance\":3.8,\"RatecodeID\":1.0,\"store_and_fwd_flag\":\"N\",\"PULocationID\":142,\"DOLocationID\":236,\"payment_type\":1,\"fare_amount\":14.5,\"extra\":3.0,\"mta_tax\":0.5,\"tip_amount\":3.65,\"tolls_amount\":0.0,\"improvement_surcharge\":0.3,\"total_amount\":21.95,\"congestion_surcharge\":2.5,\"airport_fee\":0.0},\"maxValues\":{\"VendorID\":1,\"tpep_pickup_datetime\":\"2021-12-31T19:35:40.000+03:00\",\"tpep_dropoff_datetime\":\"2021-12-31T19:53:29.000+03:00\",\"passenger_count\":2.0,\"trip_distance\":3.8,\"RatecodeID\":1.0,\"store_and_fwd_flag\":\"N\",\"PULocationID\":142,\"DOLocationID\":236,\"payment_type\":1,\"fare_amount\":14.5,\"extra\":3.0,\"mta_tax\":0.5,\"tip_amount\":3.65,\"tolls_amount\":0.0,\"improvement_surcharge\":0.3,\"total_amount\":21.95,\"congestion_surcharge\":2.5,\"airport_fee\":0.0},\"nullCount\":{\"VendorID\":0,\"tpep_pickup_datetime\":0,\"tpep_dropoff_datetime\":0,\"passenger_count\":0,\"trip_distance\":0,\"RatecodeID\":0,\"store_and_fwd_flag\":0,\"PULocationID\":0,\"DOLocationID\":0,\"payment_type\":0,\"fare_amount\":0,\"extra\":0,\"mta_tax\":0,\"tip_amount\":0,\"tolls_amount\":0,\"improvement_surcharge\":0,\"total_amount\":0,\"congestion_surcharge\":0,\"airport_fee\":0}}"}}
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Loading

0 comments on commit bd1117e

Please sign in to comment.