Skip to content

Commit

Permalink
Treat precision as NANOSECONDS for timestamp to be coerced
Browse files Browse the repository at this point in the history
This will be irrespective of the precision configured or specified as
session property.
  • Loading branch information
Praveen2112 committed Jul 14, 2023
1 parent 88b065d commit dbd3e0f
Show file tree
Hide file tree
Showing 8 changed files with 84 additions and 117 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@
import static io.trino.spi.block.ColumnarRow.toColumnarRow;
import static io.trino.spi.type.DoubleType.DOUBLE;
import static io.trino.spi.type.RealType.REAL;
import static io.trino.spi.type.TimestampType.TIMESTAMP_NANOS;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;

Expand Down Expand Up @@ -135,11 +136,8 @@ public static Type createTypeFromCoercer(TypeManager typeManager, HiveType fromH
if (fromType == REAL && toType instanceof DecimalType toDecimalType) {
return Optional.of(createRealToDecimalCoercer(toDecimalType));
}
if (fromType instanceof TimestampType timestampType && toType instanceof VarcharType varcharType) {
if (timestampType.isShort()) {
return Optional.of(new TimestampCoercer.ShortTimestampToVarcharCoercer(timestampType, varcharType));
}
return Optional.of(new TimestampCoercer.LongTimestampToVarcharCoercer(timestampType, varcharType));
if (fromType instanceof TimestampType && toType instanceof VarcharType varcharType) {
return Optional.of(new TimestampCoercer.LongTimestampToVarcharCoercer(TIMESTAMP_NANOS, varcharType));
}
if ((fromType instanceof ArrayType) && (toType instanceof ArrayType)) {
return createCoercerForList(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,29 +48,6 @@ public final class TimestampCoercer

private TimestampCoercer() {}

public static class ShortTimestampToVarcharCoercer
extends TypeCoercer<TimestampType, VarcharType>
{
public ShortTimestampToVarcharCoercer(TimestampType fromType, VarcharType toType)
{
super(fromType, toType);
}

@Override
protected void applyCoercedValue(BlockBuilder blockBuilder, Block block, int position)
{
long epochMicros = fromType.getLong(block, position);
long epochSecond = floorDiv(epochMicros, MICROSECONDS_PER_SECOND);
int nanoFraction = floorMod(epochMicros, MICROSECONDS_PER_SECOND) * NANOSECONDS_PER_MICROSECOND;
toType.writeSlice(
blockBuilder,
truncateToLength(
Slices.utf8Slice(
LOCAL_DATE_TIME.format(LocalDateTime.ofEpochSecond(epochSecond, nanoFraction, UTC))),
toType));
}
}

public static class LongTimestampToVarcharCoercer
extends TypeCoercer<TimestampType, VarcharType>
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,6 @@
import static io.trino.plugin.hive.HiveSessionProperties.getOrcMaxReadBlockSize;
import static io.trino.plugin.hive.HiveSessionProperties.getOrcStreamBufferSize;
import static io.trino.plugin.hive.HiveSessionProperties.getOrcTinyStripeThreshold;
import static io.trino.plugin.hive.HiveSessionProperties.getTimestampPrecision;
import static io.trino.plugin.hive.HiveSessionProperties.isOrcBloomFiltersEnabled;
import static io.trino.plugin.hive.HiveSessionProperties.isOrcNestedLazy;
import static io.trino.plugin.hive.HiveSessionProperties.isUseOrcColumnNames;
Expand Down Expand Up @@ -364,7 +363,7 @@ else if (column.getBaseHiveColumnIndex() < fileColumns.size()) {
Type readType = column.getType();
if (orcColumn != null) {
int sourceIndex = fileReadColumns.size();
Optional<TypeCoercer<?, ?>> coercer = createCoercer(orcColumn.getColumnType(), readType, getTimestampPrecision(session));
Optional<TypeCoercer<?, ?>> coercer = createCoercer(orcColumn.getColumnType(), readType);
if (coercer.isPresent()) {
fileReadTypes.add(coercer.get().getFromType());
columnAdaptations.add(ColumnAdaptation.coercedColumn(sourceIndex, coercer.get()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,31 +14,24 @@
package io.trino.plugin.hive.orc;

import io.trino.orc.metadata.OrcType.OrcTypeKind;
import io.trino.plugin.hive.HiveTimestampPrecision;
import io.trino.plugin.hive.coercions.TimestampCoercer.LongTimestampToVarcharCoercer;
import io.trino.plugin.hive.coercions.TimestampCoercer.ShortTimestampToVarcharCoercer;
import io.trino.plugin.hive.coercions.TypeCoercer;
import io.trino.spi.type.TimestampType;
import io.trino.spi.type.Type;
import io.trino.spi.type.VarcharType;

import java.util.Optional;

import static io.trino.orc.metadata.OrcType.OrcTypeKind.TIMESTAMP;
import static io.trino.spi.type.TimestampType.createTimestampType;
import static io.trino.spi.type.TimestampType.TIMESTAMP_NANOS;

public final class OrcTypeTranslator
{
private OrcTypeTranslator() {}

public static Optional<TypeCoercer<? extends Type, ? extends Type>> createCoercer(OrcTypeKind fromOrcType, Type toTrinoType, HiveTimestampPrecision timestampPrecision)
public static Optional<TypeCoercer<? extends Type, ? extends Type>> createCoercer(OrcTypeKind fromOrcType, Type toTrinoType)
{
if (fromOrcType == TIMESTAMP && toTrinoType instanceof VarcharType varcharType) {
TimestampType timestampType = createTimestampType(timestampPrecision.getPrecision());
if (timestampType.isShort()) {
return Optional.of(new ShortTimestampToVarcharCoercer(timestampType, varcharType));
}
return Optional.of(new LongTimestampToVarcharCoercer(timestampType, varcharType));
return Optional.of(new LongTimestampToVarcharCoercer(TIMESTAMP_NANOS, varcharType));
}
return Optional.empty();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,13 +26,11 @@

import java.time.LocalDateTime;

import static io.trino.plugin.hive.HiveTimestampPrecision.MICROSECONDS;
import static io.trino.plugin.hive.HiveTimestampPrecision.NANOSECONDS;
import static io.trino.plugin.hive.HiveType.toHiveType;
import static io.trino.plugin.hive.coercions.CoercionUtils.createCoercer;
import static io.trino.spi.predicate.Utils.blockToNativeValue;
import static io.trino.spi.predicate.Utils.nativeValueToBlock;
import static io.trino.spi.type.TimestampType.TIMESTAMP_MICROS;
import static io.trino.spi.type.TimestampType.TIMESTAMP_PICOS;
import static io.trino.spi.type.VarcharType.createUnboundedVarcharType;
import static io.trino.spi.type.VarcharType.createVarcharType;
Expand All @@ -44,63 +42,48 @@
public class TestTimestampCoercer
{
@Test(dataProvider = "timestampValuesProvider")
public void testShortTimestampToVarchar(String timestampValue, String hiveTimestampValue)
{
LocalDateTime localDateTime = LocalDateTime.parse(timestampValue);
SqlTimestamp timestamp = SqlTimestamp.fromSeconds(TIMESTAMP_MICROS.getPrecision(), localDateTime.toEpochSecond(UTC), localDateTime.get(NANO_OF_SECOND));
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createUnboundedVarcharType(), hiveTimestampValue);
}

@Test(dataProvider = "timestampValuesProvider")
public void testLongTimestampToVarchar(String timestampValue, String hiveTimestampValue)
public void testTimestampToVarchar(String timestampValue, String hiveTimestampValue)
{
LocalDateTime localDateTime = LocalDateTime.parse(timestampValue);
SqlTimestamp timestamp = SqlTimestamp.fromSeconds(TIMESTAMP_PICOS.getPrecision(), localDateTime.toEpochSecond(UTC), localDateTime.get(NANO_OF_SECOND));
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, new LongTimestamp(timestamp.getEpochMicros(), timestamp.getPicosOfMicros()), createUnboundedVarcharType(), hiveTimestampValue);
}

@Test
public void testShortTimestampToSmallerVarchar()
{
LocalDateTime localDateTime = LocalDateTime.parse("2023-04-11T05:16:12.345678");
SqlTimestamp timestamp = SqlTimestamp.fromSeconds(TIMESTAMP_MICROS.getPrecision(), localDateTime.toEpochSecond(UTC), localDateTime.get(NANO_OF_SECOND));
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(1), "2");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(2), "20");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(3), "202");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(4), "2023");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(5), "2023-");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(6), "2023-0");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(7), "2023-04");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(8), "2023-04-");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(9), "2023-04-1");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(10), "2023-04-11");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(11), "2023-04-11 ");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(12), "2023-04-11 0");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(13), "2023-04-11 05");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(14), "2023-04-11 05:");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(15), "2023-04-11 05:1");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(16), "2023-04-11 05:16");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(17), "2023-04-11 05:16:");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(18), "2023-04-11 05:16:1");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(19), "2023-04-11 05:16:12");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(20), "2023-04-11 05:16:12.");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(21), "2023-04-11 05:16:12.3");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(22), "2023-04-11 05:16:12.34");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(23), "2023-04-11 05:16:12.345");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(24), "2023-04-11 05:16:12.3456");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(25), "2023-04-11 05:16:12.34567");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(26), "2023-04-11 05:16:12.345678");
assertShortTimestampToVarcharCoercions(TIMESTAMP_MICROS, timestamp.getEpochMicros(), createVarcharType(27), "2023-04-11 05:16:12.345678");
}

@Test
public void testLongTimestampToSmallerVarchar()
public void testTimestampToSmallerVarchar()
{
LocalDateTime localDateTime = LocalDateTime.parse("2023-04-11T05:16:12.345678876");
SqlTimestamp timestamp = SqlTimestamp.fromSeconds(TIMESTAMP_PICOS.getPrecision(), localDateTime.toEpochSecond(UTC), localDateTime.get(NANO_OF_SECOND));
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, new LongTimestamp(timestamp.getEpochMicros(), timestamp.getPicosOfMicros()), createVarcharType(27), "2023-04-11 05:16:12.3456788");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, new LongTimestamp(timestamp.getEpochMicros(), timestamp.getPicosOfMicros()), createVarcharType(28), "2023-04-11 05:16:12.34567887");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, new LongTimestamp(timestamp.getEpochMicros(), timestamp.getPicosOfMicros()), createVarcharType(29), "2023-04-11 05:16:12.345678876");
LongTimestamp longTimestamp = new LongTimestamp(timestamp.getEpochMicros(), timestamp.getPicosOfMicros());
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(1), "2");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(2), "20");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(3), "202");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(4), "2023");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(5), "2023-");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(6), "2023-0");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(7), "2023-04");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(8), "2023-04-");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(9), "2023-04-1");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(10), "2023-04-11");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(11), "2023-04-11 ");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(12), "2023-04-11 0");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(13), "2023-04-11 05");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(14), "2023-04-11 05:");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(15), "2023-04-11 05:1");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(16), "2023-04-11 05:16");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(17), "2023-04-11 05:16:");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(18), "2023-04-11 05:16:1");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(19), "2023-04-11 05:16:12");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(20), "2023-04-11 05:16:12.");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(21), "2023-04-11 05:16:12.3");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(22), "2023-04-11 05:16:12.34");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(23), "2023-04-11 05:16:12.345");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(24), "2023-04-11 05:16:12.3456");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(25), "2023-04-11 05:16:12.34567");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(26), "2023-04-11 05:16:12.345678");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(27), "2023-04-11 05:16:12.3456788");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(28), "2023-04-11 05:16:12.34567887");
assertLongTimestampToVarcharCoercions(TIMESTAMP_PICOS, longTimestamp, createVarcharType(29), "2023-04-11 05:16:12.345678876");
}

@DataProvider
Expand Down Expand Up @@ -129,11 +112,6 @@ public Object[][] timestampValuesProvider()
};
}

public static void assertShortTimestampToVarcharCoercions(TimestampType fromType, Long valueToBeCoerced, VarcharType toType, String expectedValue)
{
assertCoercions(fromType, valueToBeCoerced, toType, Slices.utf8Slice(expectedValue), MICROSECONDS);
}

public static void assertLongTimestampToVarcharCoercions(TimestampType fromType, LongTimestamp valueToBeCoerced, VarcharType toType, String expectedValue)
{
assertCoercions(fromType, valueToBeCoerced, toType, Slices.utf8Slice(expectedValue), NANOSECONDS);
Expand Down
Loading

0 comments on commit dbd3e0f

Please sign in to comment.