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

Fix Delta Lake insertion issue by setting UTC timezone for timestamp with timezone partition column #16878

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,11 @@
import io.trino.parquet.writer.ParquetSchemaConverter;
ebyhr marked this conversation as resolved.
Show resolved Hide resolved
import io.trino.parquet.writer.ParquetWriterOptions;
import io.trino.plugin.deltalake.DataFileInfo.DataFileType;
import io.trino.plugin.deltalake.util.DeltaLakeWriteUtils;
import io.trino.plugin.hive.FileWriter;
import io.trino.plugin.hive.HivePartitionKey;
import io.trino.plugin.hive.parquet.ParquetFileWriter;
import io.trino.plugin.hive.util.HiveUtil;
import io.trino.plugin.hive.util.HiveWriteUtils;
import io.trino.spi.Page;
import io.trino.spi.PageIndexer;
import io.trino.spi.PageIndexerFactory;
Expand Down Expand Up @@ -440,7 +440,7 @@ private static String makePartName(List<String> partitionColumns, List<String> p

public static List<String> createPartitionValues(List<Type> partitionColumnTypes, Page partitionColumns, int position)
{
return HiveWriteUtils.createPartitionValues(partitionColumnTypes, partitionColumns, position).stream()
return DeltaLakeWriteUtils.createPartitionValues(partitionColumnTypes, partitionColumns, position).stream()
.map(value -> value.equals(HivePartitionKey.HIVE_DEFAULT_DYNAMIC_PARTITION) ? null : value)
.collect(toList());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import java.io.InputStream;
import java.io.UncheckedIOException;
import java.math.BigDecimal;
import java.text.ParsePosition;
import java.time.Duration;
import java.time.LocalDate;
import java.time.LocalDateTime;
Expand All @@ -48,6 +49,7 @@
import java.time.format.ResolverStyle;
import java.time.format.SignStyle;
import java.time.temporal.ChronoField;
import java.time.temporal.TemporalAccessor;
import java.util.Locale;
import java.util.Optional;
import java.util.function.Function;
Expand Down Expand Up @@ -78,7 +80,11 @@
import static java.time.ZoneOffset.UTC;
import static java.time.format.DateTimeFormatter.ISO_LOCAL_TIME;
import static java.time.temporal.ChronoField.DAY_OF_MONTH;
import static java.time.temporal.ChronoField.HOUR_OF_DAY;
import static java.time.temporal.ChronoField.MINUTE_OF_HOUR;
import static java.time.temporal.ChronoField.MONTH_OF_YEAR;
import static java.time.temporal.ChronoField.NANO_OF_SECOND;
import static java.time.temporal.ChronoField.SECOND_OF_MINUTE;
import static java.time.temporal.ChronoField.YEAR;

public final class TransactionLogParser
Expand All @@ -104,13 +110,30 @@ private TransactionLogParser() {}
.appendLiteral('-')
.appendValue(DAY_OF_MONTH, 2)
.appendLiteral(' ')
.appendValue(ChronoField.HOUR_OF_DAY, 2).appendLiteral(':')
.appendValue(ChronoField.MINUTE_OF_HOUR, 2).appendLiteral(':')
.appendValue(HOUR_OF_DAY, 2).appendLiteral(':')
.appendValue(MINUTE_OF_HOUR, 2).appendLiteral(':')
.appendValue(ChronoField.SECOND_OF_MINUTE, 2)
.appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true)
.toFormatter(Locale.ENGLISH)
.withChronology(IsoChronology.INSTANCE)
.withResolverStyle(ResolverStyle.STRICT);
public static final DateTimeFormatter PARTITION_TIMESTAMP_ZONE_OR_OFFSET_FORMATTER = new DateTimeFormatterBuilder()
.parseCaseInsensitive()
.appendValue(YEAR, 4, 10, SignStyle.NORMAL)
.appendLiteral('-')
.appendValue(MONTH_OF_YEAR, 2)
.appendLiteral('-')
.appendValue(DAY_OF_MONTH, 2)
.appendLiteral(' ')
.appendValue(HOUR_OF_DAY, 2).appendLiteral(':')
.appendValue(MINUTE_OF_HOUR, 2).appendLiteral(':')
.appendValue(ChronoField.SECOND_OF_MINUTE, 2)
.appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true)
.appendLiteral(' ')
.appendZoneOrOffsetId()
.toFormatter(Locale.ENGLISH)
.withChronology(IsoChronology.INSTANCE)
.withResolverStyle(ResolverStyle.STRICT);
public static final DateTimeFormatter JSON_STATISTICS_TIMESTAMP_FORMATTER = new DateTimeFormatterBuilder()
.parseCaseInsensitive()
.appendValue(YEAR, 4, 10, SignStyle.NORMAL)
Expand Down Expand Up @@ -157,7 +180,35 @@ public static Object deserializePartitionValue(DeltaLakeColumnHandle column, Opt

private static Long readPartitionTimestamp(String timestamp)
{
ZonedDateTime zonedDateTime = LocalDateTime.parse(timestamp, PARTITION_TIMESTAMP_FORMATTER).atZone(UTC);
ParsePosition position = new ParsePosition(0);
TemporalAccessor accessor = PARTITION_TIMESTAMP_FORMATTER.parseUnresolved(timestamp, position);
ZonedDateTime zonedDateTime;
if (position.getErrorIndex() == -1 && timestamp.length() == position.getIndex()) {
if (accessor.isSupported(NANO_OF_SECOND)) {
Comment on lines +186 to +187
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These conditions look hard to understand for me. Could you leave a code comment with the example?

zonedDateTime = LocalDateTime.of(
accessor.get(YEAR),
accessor.get(MONTH_OF_YEAR),
accessor.get(DAY_OF_MONTH),
accessor.get(HOUR_OF_DAY),
accessor.get(MINUTE_OF_HOUR),
accessor.get(SECOND_OF_MINUTE),
accessor.get(NANO_OF_SECOND))
.atZone(UTC);
}
else {
zonedDateTime = LocalDateTime.of(
accessor.get(YEAR),
accessor.get(MONTH_OF_YEAR),
accessor.get(DAY_OF_MONTH),
accessor.get(HOUR_OF_DAY),
accessor.get(MINUTE_OF_HOUR),
accessor.get(SECOND_OF_MINUTE))
.atZone(UTC);
}
}
else {
zonedDateTime = ZonedDateTime.parse(timestamp, PARTITION_TIMESTAMP_ZONE_OR_OFFSET_FORMATTER);
}
return packDateTimeWithZone(zonedDateTime.toInstant().toEpochMilli(), UTC_KEY);
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,195 @@
/*
* Licensed 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 io.trino.plugin.deltalake.util;

import com.google.common.base.CharMatcher;
import com.google.common.collect.ImmutableList;
import com.google.common.primitives.Shorts;
import com.google.common.primitives.SignedBytes;
import io.trino.spi.Page;
import io.trino.spi.StandardErrorCode;
import io.trino.spi.TrinoException;
import io.trino.spi.block.Block;
import io.trino.spi.type.ArrayType;
import io.trino.spi.type.CharType;
import io.trino.spi.type.DecimalType;
import io.trino.spi.type.MapType;
import io.trino.spi.type.RowType;
import io.trino.spi.type.TimestampType;
import io.trino.spi.type.TimestampWithTimeZoneType;
import io.trino.spi.type.Type;
import io.trino.spi.type.VarcharType;
import org.apache.hadoop.hive.common.type.Date;
import org.apache.hadoop.hive.common.type.TimestampTZ;
import org.apache.hadoop.io.Text;
import org.joda.time.DateTimeZone;

import java.time.Instant;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;

import static com.google.common.io.BaseEncoding.base16;
import static io.airlift.slice.SizeOf.SIZE_OF_LONG;
import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_PARTITION_VALUE;
import static io.trino.plugin.hive.HivePartitionKey.HIVE_DEFAULT_DYNAMIC_PARTITION;
import static io.trino.plugin.hive.util.HiveUtil.checkCondition;
import static io.trino.plugin.hive.util.HiveWriteUtils.getHiveDecimal;
import static io.trino.plugin.hive.util.HiveWriteUtils.getHiveTimestamp;
import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED;
import static io.trino.spi.type.BigintType.BIGINT;
import static io.trino.spi.type.BooleanType.BOOLEAN;
import static io.trino.spi.type.Chars.padSpaces;
import static io.trino.spi.type.DateTimeEncoding.unpackMillisUtc;
import static io.trino.spi.type.DateType.DATE;
import static io.trino.spi.type.DoubleType.DOUBLE;
import static io.trino.spi.type.IntegerType.INTEGER;
import static io.trino.spi.type.RealType.REAL;
import static io.trino.spi.type.SmallintType.SMALLINT;
import static io.trino.spi.type.TimeZoneKey.UTC_KEY;
import static io.trino.spi.type.TinyintType.TINYINT;
import static io.trino.spi.type.VarbinaryType.VARBINARY;
import static java.lang.Float.intBitsToFloat;
import static java.lang.Math.toIntExact;
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.Collections.unmodifiableList;
import static java.util.Collections.unmodifiableMap;

public final class DeltaLakeWriteUtils
{
private DeltaLakeWriteUtils() {}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What's the difference from HiveWriteUtils? Please leave a code comment.


public static List<String> createPartitionValues(List<Type> partitionColumnTypes, Page partitionColumns, int position)
{
ImmutableList.Builder<String> partitionValues = ImmutableList.builder();
for (int field = 0; field < partitionColumns.getChannelCount(); field++) {
Object value = getField(DateTimeZone.UTC, partitionColumnTypes.get(field), partitionColumns.getBlock(field), position);
if (value == null) {
partitionValues.add(HIVE_DEFAULT_DYNAMIC_PARTITION);
}
else {
String valueString = value.toString();
if (!CharMatcher.inRange((char) 0x20, (char) 0x7E).matchesAllOf(valueString)) {
throw new TrinoException(HIVE_INVALID_PARTITION_VALUE,
"Hive partition keys can only contain printable ASCII characters (0x20 - 0x7E). Invalid value: " +
base16().withSeparator(" ", 2).encode(valueString.getBytes(UTF_8)));
}
partitionValues.add(valueString);
}
}
return partitionValues.build();
}

public static Object getField(DateTimeZone localZone, Type type, Block block, int position)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

All types in this method are required in Delta Lake connector?

{
if (block.isNull(position)) {
return null;
}
if (BOOLEAN.equals(type)) {
return type.getBoolean(block, position);
}
if (BIGINT.equals(type)) {
return type.getLong(block, position);
}
if (INTEGER.equals(type)) {
return toIntExact(type.getLong(block, position));
}
if (SMALLINT.equals(type)) {
return Shorts.checkedCast(type.getLong(block, position));
}
if (TINYINT.equals(type)) {
return SignedBytes.checkedCast(type.getLong(block, position));
}
if (REAL.equals(type)) {
return intBitsToFloat((int) type.getLong(block, position));
}
if (DOUBLE.equals(type)) {
return type.getDouble(block, position);
}
if (type instanceof VarcharType) {
return new Text(type.getSlice(block, position).getBytes());
}
if (type instanceof CharType charType) {
return new Text(padSpaces(type.getSlice(block, position), charType).toStringUtf8());
}
if (VARBINARY.equals(type)) {
return type.getSlice(block, position).getBytes();
}
if (DATE.equals(type)) {
return Date.ofEpochDay(toIntExact(type.getLong(block, position)));
}
if (type instanceof TimestampType) {
return getHiveTimestamp(localZone, (TimestampType) type, block, position);
}
if (type instanceof TimestampWithTimeZoneType timestampWithTimeZoneType && timestampWithTimeZoneType.getPrecision() == 3) {
return getTimestampTZ(timestampWithTimeZoneType, block, position);
}
if (type instanceof DecimalType decimalType) {
return getHiveDecimal(decimalType, block, position);
}
if (type instanceof ArrayType) {
Type elementType = ((ArrayType) type).getElementType();
Block arrayBlock = block.getObject(position, Block.class);

List<Object> list = new ArrayList<>(arrayBlock.getPositionCount());
for (int i = 0; i < arrayBlock.getPositionCount(); i++) {
list.add(getField(localZone, elementType, arrayBlock, i));
}
return unmodifiableList(list);
}
if (type instanceof MapType) {
Type keyType = ((MapType) type).getKeyType();
Type valueType = ((MapType) type).getValueType();
Block mapBlock = block.getObject(position, Block.class);

Map<Object, Object> map = new HashMap<>();
for (int i = 0; i < mapBlock.getPositionCount(); i += 2) {
map.put(
getField(localZone, keyType, mapBlock, i),
getField(localZone, valueType, mapBlock, i + 1));
}
return unmodifiableMap(map);
}
if (type instanceof RowType) {
List<Type> fieldTypes = type.getTypeParameters();
Block rowBlock = block.getObject(position, Block.class);
checkCondition(
fieldTypes.size() == rowBlock.getPositionCount(),
StandardErrorCode.GENERIC_INTERNAL_ERROR,
"Expected row value field count does not match type field count");
List<Object> row = new ArrayList<>(rowBlock.getPositionCount());
for (int i = 0; i < rowBlock.getPositionCount(); i++) {
row.add(getField(localZone, fieldTypes.get(i), rowBlock, i));
}
return unmodifiableList(row);
}
throw new TrinoException(NOT_SUPPORTED, "unsupported type: " + type);
}

private static TimestampTZ getTimestampTZ(TimestampWithTimeZoneType type, Block block, int position)
{
long value = block.getLong(position, 0);
long millisUtc = unpackMillisUtc(value);
if (type.isShort()) {
Instant instant = Instant.ofEpochMilli(millisUtc);
return new TimestampTZ((long) Math.floor(millisUtc / 1000), instant.getNano(), UTC_KEY.getZoneId());
}
else {
int picosOfMilli = block.getInt(position, SIZE_OF_LONG);
Instant instant = Instant.ofEpochMilli(millisUtc).plusNanos(picosOfMilli * 1000);
return new TimestampTZ(instant.getEpochSecond(), instant.getNano(), UTC_KEY.getZoneId());
}
Comment on lines +189 to +193
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this condition used in Delta Lake connector?

}
}
Original file line number Diff line number Diff line change
Expand Up @@ -751,7 +751,7 @@ public static HiveDecimal getHiveDecimal(DecimalType decimalType, Block block, i
return HiveDecimal.create(unscaledValue, decimalType.getScale());
}

private static Timestamp getHiveTimestamp(DateTimeZone localZone, TimestampType type, Block block, int position)
public static Timestamp getHiveTimestamp(DateTimeZone localZone, TimestampType type, Block block, int position)
{
verify(type.getPrecision() <= HiveTimestampPrecision.MAX.getPrecision(), "Timestamp precision too high for Hive");

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;

import java.sql.Timestamp;
import java.util.List;
import java.util.Optional;
import java.util.stream.Stream;
Expand Down Expand Up @@ -130,6 +131,37 @@ public void testPartitionedInsertCompatibility()
}
}

@Test(groups = {DELTA_LAKE_DATABRICKS, DELTA_LAKE_OSS, PROFILE_SPECIFIC_TESTS})
public void testPartitionedInsertTimestampTZCompatibility()
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
public void testPartitionedInsertTimestampTZCompatibility()
public void testPartitionedInsertTimestampWithTimeZoneCompatibility()

{
String tableName = "test_dl_partitioned_insert_timestampTZ" + randomNameSuffix();
onTrino().executeQuery("" +
"CREATE TABLE delta.default." + tableName +
ebyhr marked this conversation as resolved.
Show resolved Hide resolved
" (c1 INT, c2 TIMESTAMP WITH TIME ZONE)" +
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please add a test for partitioned by nested type with timestamp with time zone.

" WITH (" +
" partitioned_by = ARRAY['c2']," +
" location = 's3://" + bucketName + "/databricks-compatibility-test-" + tableName + "')");

try {
onTrino().executeQuery("INSERT INTO delta.default." + tableName + " VALUES (1, TIMESTAMP '2023-04-05 10:00:00.666+01:00')");
onTrino().executeQuery("INSERT INTO delta.default." + tableName + " VALUES (2, TIMESTAMP '2023-04-06 10:00:00+01:00')");
Comment on lines +146 to +147
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: No need to split into two INSERT statements.

List<Row> expectedRows = ImmutableList.of(
row(
1,
Timestamp.valueOf("2023-04-05 09:00:00.666")),
row(
2,
Timestamp.valueOf("2023-04-06 09:00:00")));
assertThat(onTrino().executeQuery("SELECT c1, CAST(c2 AS TIMESTAMP) FROM delta.default." + tableName + " ORDER BY c1 ASC"))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Remove redundant ORDER BY

.containsOnly(expectedRows);
assertThat(onDelta().executeQuery("SELECT c1, c2 FROM default." + tableName + " ORDER BY c1 ASC"))
.containsOnly(expectedRows);
}
finally {
onTrino().executeQuery("DROP TABLE delta.default." + tableName);
}
}

@Test(groups = {DELTA_LAKE_DATABRICKS, DELTA_LAKE_OSS, PROFILE_SPECIFIC_TESTS})
@Flaky(issue = DATABRICKS_COMMUNICATION_FAILURE_ISSUE, match = DATABRICKS_COMMUNICATION_FAILURE_MATCH)
public void testTrinoPartitionedDifferentOrderInsertCompatibility()
Expand Down