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

Support for variable precision timestamps in Hive connector (read path) #4953

Closed
wants to merge 5 commits into from
Closed
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -13,7 +13,6 @@
*/
package io.prestosql.plugin.hive;

import com.google.common.collect.ImmutableMap;
import io.airlift.slice.Slice;
import io.airlift.slice.Slices;
import io.prestosql.hadoop.TextLineLengthLimitExceededException;
Expand Down Expand Up @@ -87,6 +86,7 @@
import static java.lang.Math.min;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;
import static org.joda.time.DateTimeZone.UTC;

public class GenericHiveRecordCursor<K, V extends Writable>
implements RecordCursor
Expand All @@ -112,11 +112,10 @@ public class GenericHiveRecordCursor<K, V extends Writable>
private final Slice[] slices;
private final Object[] objects;
private final boolean[] nulls;
private final PrestoTimestampEncoder<?>[] timestampEncoders;

private final long totalBytes;

private final Map<Type, PrestoTimestampEncoder<?>> timestampEncoders;

private long completedBytes;
private Object rowData;
private boolean closed;
Expand Down Expand Up @@ -159,6 +158,7 @@ public GenericHiveRecordCursor(
this.slices = new Slice[size];
this.objects = new Object[size];
this.nulls = new boolean[size];
this.timestampEncoders = new PrestoTimestampEncoder[size];

Map<Type, PrestoTimestampEncoder<?>> timestampEncodersBuilder = new HashMap<>();
// initialize data columns
Expand All @@ -169,15 +169,14 @@ public GenericHiveRecordCursor(
Type columnType = column.getType();
types[i] = columnType;
if (columnType instanceof TimestampType) {
timestampEncodersBuilder.put(columnType, createTimestampEncoder((TimestampType) columnType));
timestampEncoders[i] = createTimestampEncoder((TimestampType) columnType, UTC);
}
hiveTypes[i] = column.getHiveType();

StructField field = rowInspector.getStructFieldRef(column.getName());
structFields[i] = field;
fieldInspectors[i] = field.getFieldObjectInspector();
}
timestampEncoders = ImmutableMap.copyOf(timestampEncodersBuilder);
}

@Override
Expand Down Expand Up @@ -291,18 +290,18 @@ private void parseLongColumn(int column)
else {
Object fieldValue = ((PrimitiveObjectInspector) fieldInspectors[column]).getPrimitiveJavaObject(fieldData);
checkState(fieldValue != null, "fieldValue should not be null");
longs[column] = getLongExpressedValue(fieldValue, types[column]);
longs[column] = getLongExpressedValue(fieldValue, column);
nulls[column] = false;
}
}

private long getLongExpressedValue(Object value, Type type)
private long getLongExpressedValue(Object value, int column)
{
if (value instanceof Date) {
return ((Date) value).toEpochDay();
}
if (value instanceof Timestamp) {
return shortTimestamp((Timestamp) value, type);
return shortTimestamp((Timestamp) value, column);
}
if (value instanceof Float) {
return floatToRawIntBits(((Float) value));
Expand Down Expand Up @@ -491,7 +490,7 @@ private void parseObjectColumn(int column)
}
else if (type instanceof TimestampType) {
Timestamp timestamp = (Timestamp) ((PrimitiveObjectInspector) fieldInspectors[column]).getPrimitiveJavaObject(fieldData);
objects[column] = longTimestamp(timestamp, type);
objects[column] = longTimestamp(timestamp, column);
}
else {
throw new IllegalStateException("Unsupported type: " + type);
Expand Down Expand Up @@ -590,17 +589,17 @@ public void close()
}
}

private long shortTimestamp(Timestamp value, Type type)
private long shortTimestamp(Timestamp value, int column)
{
@SuppressWarnings("unchecked")
PrestoTimestampEncoder<Long> encoder = (PrestoTimestampEncoder<Long>) timestampEncoders.get(type);
PrestoTimestampEncoder<Long> encoder = (PrestoTimestampEncoder<Long>) timestampEncoders[column];
return encoder.getTimestamp(new DecodedTimestamp(value.toEpochSecond(), value.getNanos()));
}

private LongTimestamp longTimestamp(Timestamp value, Type type)
private LongTimestamp longTimestamp(Timestamp value, int column)
{
@SuppressWarnings("unchecked")
PrestoTimestampEncoder<LongTimestamp> encoder = (PrestoTimestampEncoder<LongTimestamp>) timestampEncoders.get(type);
PrestoTimestampEncoder<LongTimestamp> encoder = (PrestoTimestampEncoder<LongTimestamp>) timestampEncoders[column];
return encoder.getTimestamp(new DecodedTimestamp(value.toEpochSecond(), value.getNanos()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,7 @@ public class HiveConfig

private Duration dynamicFilteringProbeBlockingTimeout = new Duration(0, MINUTES);

private int timestampPrecision = 3;
private HiveTimestampPrecision timestampPrecision = HiveTimestampPrecision.MILLIS;

public int getMaxInitialSplits()
{
Expand Down Expand Up @@ -983,15 +983,14 @@ public HiveConfig setDynamicFilteringProbeBlockingTimeout(Duration dynamicFilter
return this;
}

@Min(3)
public int getTimestampPrecision()
public HiveTimestampPrecision getTimestampPrecision()
{
return timestampPrecision;
}

@Config("hive.timestamp-precision")
@ConfigDescription("Precision used to represent timestamps")
public HiveConfig setTimestampPrecision(int timestampPrecision)
public HiveConfig setTimestampPrecision(HiveTimestampPrecision timestampPrecision)
{
this.timestampPrecision = timestampPrecision;
return this;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -372,7 +372,7 @@ public HiveTableHandle getTableHandle(ConnectorSession session, SchemaTableName
tableName.getTableName(),
table.get().getParameters(),
getPartitionKeyColumnHandles(table.get(), typeManager),
getHiveBucketHandle(table.get(), typeManager, getTimestampPrecision(session)));
getHiveBucketHandle(table.get(), typeManager));
}

@Override
Expand Down Expand Up @@ -547,7 +547,7 @@ private ConnectorTableMetadata doGetTableMetadata(ConnectorSession session, Sche

Function<HiveColumnHandle, ColumnMetadata> metadataGetter = columnMetadataGetter(table);
ImmutableList.Builder<ColumnMetadata> columns = ImmutableList.builder();
for (HiveColumnHandle columnHandle : hiveColumnHandles(table, typeManager, getTimestampPrecision(session))) {
for (HiveColumnHandle columnHandle : hiveColumnHandles(table, typeManager, getTimestampPrecision(session).getPrecision())) {
columns.add(metadataGetter.apply(columnHandle));
}

Expand Down Expand Up @@ -691,7 +691,7 @@ public Map<String, ColumnHandle> getColumnHandles(ConnectorSession session, Conn
SchemaTableName tableName = ((HiveTableHandle) tableHandle).getSchemaTableName();
Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName())
.orElseThrow(() -> new TableNotFoundException(tableName));
return hiveColumnHandles(table, typeManager, getTimestampPrecision(session)).stream()
return hiveColumnHandles(table, typeManager, getTimestampPrecision(session).getPrecision()).stream()
.collect(toImmutableMap(HiveColumnHandle::getName, identity()));
}

Expand Down Expand Up @@ -1530,7 +1530,7 @@ public HiveInsertTableHandle beginInsert(ConnectorSession session, ConnectorTabl
}
}

List<HiveColumnHandle> handles = hiveColumnHandles(table, typeManager, getTimestampPrecision(session)).stream()
List<HiveColumnHandle> handles = hiveColumnHandles(table, typeManager, getTimestampPrecision(session).getPrecision()).stream()
.filter(columnHandle -> !columnHandle.isHidden())
.collect(toList());

Expand Down Expand Up @@ -2285,7 +2285,7 @@ public Optional<ConnectorNewTableLayout> getInsertLayout(ConnectorSession sessio
}
}

Optional<HiveBucketHandle> hiveBucketHandle = getHiveBucketHandle(table, typeManager, getTimestampPrecision(session));
Optional<HiveBucketHandle> hiveBucketHandle = getHiveBucketHandle(table, typeManager);
if (hiveBucketHandle.isEmpty()) {
// return preferred layout which is partitioned by partition columns
List<Column> partitionColumns = table.getPartitionColumns();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -369,9 +369,10 @@ public HiveSessionProperties(
"Projection push down enabled for hive",
hiveConfig.isProjectionPushdownEnabled(),
false),
integerProperty(
enumProperty(
TIMESTAMP_PRECISION,
"Precision for timestamp columns in Hive tables",
HiveTimestampPrecision.class,
hiveConfig.getTimestampPrecision(),
false),
aalbu marked this conversation as resolved.
Show resolved Hide resolved
booleanProperty(
Expand Down Expand Up @@ -645,9 +646,9 @@ public static boolean isProjectionPushdownEnabled(ConnectorSession session)
return session.getProperty(PROJECTION_PUSHDOWN_ENABLED, Boolean.class);
}

public static int getTimestampPrecision(ConnectorSession session)
public static HiveTimestampPrecision getTimestampPrecision(ConnectorSession session)
{
return session.getProperty(TIMESTAMP_PRECISION, Integer.class);
return session.getProperty(TIMESTAMP_PRECISION, HiveTimestampPrecision.class);
}

public static boolean isParquetOptimizedWriterEnabled(ConnectorSession session)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/*
* 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.prestosql.plugin.hive;

public enum HiveTimestampPrecision
{
MILLIS(3), MICROS(6), NANOS(9);
aalbu marked this conversation as resolved.
Show resolved Hide resolved

private final int precision;

HiveTimestampPrecision(int precision)
{
this.precision = precision;
}

public int getPrecision()
{
return precision;
}
}
14 changes: 14 additions & 0 deletions presto-hive/src/main/java/io/prestosql/plugin/hive/HiveType.java
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import io.prestosql.spi.type.NamedTypeSignature;
import io.prestosql.spi.type.RowFieldName;
import io.prestosql.spi.type.StandardTypes;
import io.prestosql.spi.type.TimestampType;
import io.prestosql.spi.type.Type;
import io.prestosql.spi.type.TypeManager;
import io.prestosql.spi.type.TypeSignature;
Expand Down Expand Up @@ -120,11 +121,24 @@ public TypeSignature getTypeSignature()
return getTypeSignature(typeInfo);
}

@Deprecated
public Type getType(TypeManager typeManager)
{
return typeManager.getType(getTypeSignature());
}

public Type getType(TypeManager typeManager, int timestampPrecision)
{
Type tentativeType = typeManager.getType(getTypeSignature());
// TODO: handle timestamps in structural types (https://github.com/prestosql/presto/issues/5195)
if (tentativeType instanceof TimestampType) {
if (((TimestampType) tentativeType).getPrecision() != timestampPrecision) {
aalbu marked this conversation as resolved.
Show resolved Hide resolved
return TimestampType.createTimestampType(timestampPrecision);
}
}
return tentativeType;
}

@Override
public boolean equals(Object o)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,6 @@
import static io.prestosql.plugin.hive.HiveSessionProperties.getTimestampPrecision;
import static io.prestosql.plugin.hive.ReaderProjections.projectBaseColumns;
import static io.prestosql.plugin.hive.util.HiveUtil.getDeserializerClassName;
import static io.prestosql.plugin.hive.util.HiveUtil.getType;
import static io.prestosql.rcfile.text.TextRcFileEncoding.DEFAULT_NULL_SEQUENCE;
import static io.prestosql.rcfile.text.TextRcFileEncoding.DEFAULT_SEPARATORS;
import static java.lang.Math.min;
Expand Down Expand Up @@ -181,9 +180,9 @@ else if (deserializerClassName.equals(ColumnarSerDe.class.getName())) {

try {
ImmutableMap.Builder<Integer, Type> readColumns = ImmutableMap.builder();
int timestampPrecision = getTimestampPrecision(session);
int timestampPrecision = getTimestampPrecision(session).getPrecision();
for (HiveColumnHandle column : projectedReaderColumns) {
readColumns.put(column.getBaseHiveColumnIndex(), getType(column.getHiveType(), typeManager, timestampPrecision));
readColumns.put(column.getBaseHiveColumnIndex(), column.getHiveType().getType(typeManager, timestampPrecision));
}

RcFileReader rcFileReader = new RcFileReader(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -174,14 +174,16 @@ static int getBucketNumber(int hashCode, int bucketCount)
return (hashCode & Integer.MAX_VALUE) % bucketCount;
}

public static Optional<HiveBucketHandle> getHiveBucketHandle(Table table, TypeManager typeManager, int precision)
public static Optional<HiveBucketHandle> getHiveBucketHandle(Table table, TypeManager typeManager)
{
Optional<HiveBucketProperty> hiveBucketProperty = table.getStorage().getBucketProperty();
if (hiveBucketProperty.isEmpty()) {
return Optional.empty();
}

Map<String, HiveColumnHandle> map = getRegularColumnHandles(table, typeManager, precision).stream()
// Bucketing on timestamp is not allowed, so we do not have to know session's selected timestamp precision
int dummyTimestampPrecision = -42;
Map<String, HiveColumnHandle> map = getRegularColumnHandles(table, typeManager, dummyTimestampPrecision).stream()
.collect(Collectors.toMap(HiveColumnHandle::getName, identity()));

ImmutableList.Builder<HiveColumnHandle> bucketColumns = ImmutableList.builder();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,6 @@
import io.prestosql.spi.type.Decimals;
import io.prestosql.spi.type.MapType;
import io.prestosql.spi.type.RowType;
import io.prestosql.spi.type.TimestampType;
import io.prestosql.spi.type.Type;
import io.prestosql.spi.type.TypeId;
import io.prestosql.spi.type.TypeManager;
Expand Down Expand Up @@ -913,25 +912,14 @@ public static List<HiveColumnHandle> getRegularColumnHandles(Table table, TypeMa
// ignore unsupported types rather than failing
HiveType hiveType = field.getType();
if (hiveType.isSupportedType(table.getStorage().getStorageFormat())) {
columns.add(createBaseColumn(field.getName(), hiveColumnIndex, hiveType, getType(hiveType, typeManager, timestampPrecision), REGULAR, field.getComment()));
columns.add(createBaseColumn(field.getName(), hiveColumnIndex, hiveType, hiveType.getType(typeManager, timestampPrecision), REGULAR, field.getComment()));
}
hiveColumnIndex++;
}

return columns.build();
}

public static Type getType(HiveType hiveType, TypeManager typeManager, int precision)
{
Type tentativeType = hiveType.getType(typeManager);
if (tentativeType instanceof TimestampType) {
if (((TimestampType) tentativeType).getPrecision() != precision) {
return TimestampType.createTimestampType(precision);
}
}
return tentativeType;
}

public static List<HiveColumnHandle> getPartitionKeyColumnHandles(Table table, TypeManager typeManager)
{
ImmutableList.Builder<HiveColumnHandle> columns = ImmutableList.builder();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ public void testDefaults()
.setPartitionUseColumnNames(false)
.setProjectionPushdownEnabled(true)
.setDynamicFilteringProbeBlockingTimeout(new Duration(0, TimeUnit.MINUTES))
.setTimestampPrecision(3));
.setTimestampPrecision(HiveTimestampPrecision.MILLIS));
}

@Test
Expand Down Expand Up @@ -167,7 +167,7 @@ public void testExplicitPropertyMappings()
.put("hive.partition-use-column-names", "true")
.put("hive.projection-pushdown-enabled", "false")
.put("hive.dynamic-filtering-probe-blocking-timeout", "10s")
.put("hive.timestamp-precision", "12")
.put("hive.timestamp-precision", "NANOS")
.build();

HiveConfig expected = new HiveConfig()
Expand Down Expand Up @@ -234,7 +234,7 @@ public void testExplicitPropertyMappings()
.setPartitionUseColumnNames(true)
.setProjectionPushdownEnabled(false)
.setDynamicFilteringProbeBlockingTimeout(new Duration(10, TimeUnit.SECONDS))
.setTimestampPrecision(12);
.setTimestampPrecision(HiveTimestampPrecision.NANOS);

assertFullMapping(properties, expected);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,9 @@
import static io.prestosql.plugin.hive.HiveTestUtils.createGenericHiveRecordCursorProvider;
import static io.prestosql.plugin.hive.HiveTestUtils.getHiveSession;
import static io.prestosql.plugin.hive.HiveTestUtils.getTypes;
import static io.prestosql.plugin.hive.HiveTimestampPrecision.MICROS;
import static io.prestosql.plugin.hive.HiveTimestampPrecision.MILLIS;
import static io.prestosql.plugin.hive.HiveTimestampPrecision.NANOS;
import static io.prestosql.testing.StructuralTestUtil.rowBlockOf;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;
Expand Down Expand Up @@ -477,11 +480,13 @@ public void testParquetPageSourceSchemaEvolution(int rowCount)

// test name-based access
readColumns = Lists.reverse(writeColumns);
assertThatFileFormat(PARQUET)
.withWriteColumns(writeColumns)
.withReadColumns(readColumns)
.withSession(PARQUET_SESSION_USE_NAME)
.isReadableByPageSource(new ParquetPageSourceFactory(HDFS_ENVIRONMENT, STATS, new ParquetReaderConfig(), new HiveConfig().setTimestampPrecision(9)));
for (HiveTimestampPrecision timestampPrecision : List.of(MILLIS, MICROS, NANOS)) {
aalbu marked this conversation as resolved.
Show resolved Hide resolved
assertThatFileFormat(PARQUET)
.withWriteColumns(writeColumns)
.withReadColumns(readColumns)
.withSession(PARQUET_SESSION_USE_NAME)
.isReadableByPageSource(new ParquetPageSourceFactory(HDFS_ENVIRONMENT, STATS, new ParquetReaderConfig(), new HiveConfig().setTimestampPrecision(timestampPrecision)));
}
}

private static List<TestColumn> getTestColumnsSupportedByParquet()
Expand Down
Loading