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

Respect timestamp coercion when coercing complex column #17900

Merged
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 @@ -210,8 +210,10 @@ public static Optional<ConnectorPageSource> createHivePageSource(
Optional<BucketAdaptation> bucketAdaptation = createBucketAdaptation(bucketConversion, tableBucketNumber, regularAndInterimColumnMappings);
Optional<BucketValidator> bucketValidator = createBucketValidator(path, bucketValidation, tableBucketNumber, regularAndInterimColumnMappings);

HiveTimestampPrecision timestampPrecision = getTimestampPrecision(session);

for (HivePageSourceFactory pageSourceFactory : pageSourceFactories) {
List<HiveColumnHandle> desiredColumns = toColumnHandles(regularAndInterimColumnMappings, true, typeManager);
List<HiveColumnHandle> desiredColumns = toColumnHandles(regularAndInterimColumnMappings, true, typeManager, timestampPrecision);

Optional<ReaderPageSource> readerWithProjections = pageSourceFactory.createPageSource(
configuration,
Expand Down Expand Up @@ -243,13 +245,13 @@ public static Optional<ConnectorPageSource> createHivePageSource(
bucketValidator,
adapter,
typeManager,
getTimestampPrecision(session),
timestampPrecision,
pageSource));
}
}

for (HiveRecordCursorProvider provider : cursorProviders) {
List<HiveColumnHandle> desiredColumns = toColumnHandles(regularAndInterimColumnMappings, false, typeManager);
List<HiveColumnHandle> desiredColumns = toColumnHandles(regularAndInterimColumnMappings, false, typeManager, timestampPrecision);
Optional<ReaderRecordCursorWithProjections> readerWithProjections = provider.createRecordCursor(
configuration,
session,
Expand Down Expand Up @@ -544,7 +546,7 @@ public static List<ColumnMapping> extractRegularAndInterimColumnMappings(List<Co
.collect(toImmutableList());
}

public static List<HiveColumnHandle> toColumnHandles(List<ColumnMapping> regularColumnMappings, boolean doCoercion, TypeManager typeManager)
public static List<HiveColumnHandle> toColumnHandles(List<ColumnMapping> regularColumnMappings, boolean doCoercion, TypeManager typeManager, HiveTimestampPrecision timestampPrecision)
{
return regularColumnMappings.stream()
.map(columnMapping -> {
Expand All @@ -560,14 +562,14 @@ public static List<HiveColumnHandle> toColumnHandles(List<ColumnMapping> regular
projectedColumn.getDereferenceIndices(),
projectedColumn.getDereferenceNames(),
fromHiveType,
fromHiveType.getType(typeManager));
fromHiveType.getType(typeManager, timestampPrecision));
Praveen2112 marked this conversation as resolved.
Show resolved Hide resolved
Praveen2112 marked this conversation as resolved.
Show resolved Hide resolved
});

return new HiveColumnHandle(
columnHandle.getBaseColumnName(),
columnHandle.getBaseHiveColumnIndex(),
fromHiveTypeBase,
fromHiveTypeBase.getType(typeManager),
fromHiveTypeBase.getType(typeManager, timestampPrecision),
newColumnProjectionInfo,
columnHandle.getColumnType(),
columnHandle.getComment());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@

import java.util.Optional;

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

public final class OrcTypeTranslator
Expand All @@ -32,7 +33,7 @@ private OrcTypeTranslator() {}

public static Optional<TypeCoercer<? extends Type, ? extends Type>> createCoercer(OrcTypeKind fromOrcType, Type toTrinoType, HiveTimestampPrecision timestampPrecision)
{
if (fromOrcType.equals(OrcTypeKind.TIMESTAMP) && toTrinoType instanceof VarcharType varcharType) {
if (fromOrcType == TIMESTAMP && toTrinoType instanceof VarcharType varcharType) {
TimestampType timestampType = createTimestampType(timestampPrecision.getPrecision());
if (timestampType.isShort()) {
return Optional.of(new ShortTimestampToVarcharCoercer(timestampType, varcharType));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@
import io.trino.plugin.hive.HiveColumnHandle;
import io.trino.plugin.hive.HiveConfig;
import io.trino.plugin.hive.HivePageSourceFactory;
import io.trino.plugin.hive.HiveTimestampPrecision;
import io.trino.plugin.hive.ReaderColumns;
import io.trino.plugin.hive.ReaderPageSource;
import io.trino.plugin.hive.acid.AcidTransaction;
Expand All @@ -45,7 +44,6 @@
import io.trino.spi.connector.EmptyPageSource;
import io.trino.spi.predicate.TupleDomain;
import io.trino.spi.type.Type;
import io.trino.spi.type.TypeManager;
import org.apache.hadoop.conf.Configuration;
import org.joda.time.DateTimeZone;

Expand All @@ -62,7 +60,6 @@
import static io.trino.plugin.hive.HiveErrorCode.HIVE_BAD_DATA;
import static io.trino.plugin.hive.HiveErrorCode.HIVE_CANNOT_OPEN_SPLIT;
import static io.trino.plugin.hive.HivePageSourceProvider.projectBaseColumns;
import static io.trino.plugin.hive.HiveSessionProperties.getTimestampPrecision;
import static io.trino.plugin.hive.ReaderPageSource.noProjectionAdaptation;
import static io.trino.plugin.hive.util.HiveClassNames.COLUMNAR_SERDE_CLASS;
import static io.trino.plugin.hive.util.HiveClassNames.LAZY_BINARY_COLUMNAR_SERDE_CLASS;
Expand All @@ -77,15 +74,13 @@ public class RcFilePageSourceFactory
{
private static final DataSize BUFFER_SIZE = DataSize.of(8, Unit.MEGABYTE);

private final TypeManager typeManager;
private final TrinoFileSystemFactory fileSystemFactory;
private final FileFormatDataSourceStats stats;
private final DateTimeZone timeZone;

@Inject
public RcFilePageSourceFactory(TypeManager typeManager, TrinoFileSystemFactory fileSystemFactory, FileFormatDataSourceStats stats, HiveConfig hiveConfig)
public RcFilePageSourceFactory(TrinoFileSystemFactory fileSystemFactory, FileFormatDataSourceStats stats, HiveConfig hiveConfig)
{
this.typeManager = requireNonNull(typeManager, "typeManager is null");
this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null");
this.stats = requireNonNull(stats, "stats is null");
this.timeZone = hiveConfig.getRcfileDateTimeZone();
Expand Down Expand Up @@ -168,9 +163,8 @@ else if (deserializerClassName.equals(COLUMNAR_SERDE_CLASS)) {

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

RcFileReader rcFileReader = new RcFileReader(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -206,7 +206,7 @@ public static Set<HivePageSourceFactory> getDefaultHivePageSourceFactories(HdfsE
.add(new RegexPageSourceFactory(fileSystemFactory, stats, hiveConfig))
.add(new SimpleTextFilePageSourceFactory(fileSystemFactory, stats, hiveConfig))
.add(new SimpleSequenceFilePageSourceFactory(fileSystemFactory, stats, hiveConfig))
.add(new RcFilePageSourceFactory(TESTING_TYPE_MANAGER, fileSystemFactory, stats, hiveConfig))
.add(new RcFilePageSourceFactory(fileSystemFactory, stats, hiveConfig))
.add(new OrcPageSourceFactory(new OrcReaderConfig(), fileSystemFactory, stats, hiveConfig))
.add(new ParquetPageSourceFactory(fileSystemFactory, stats, new ParquetReaderConfig(), hiveConfig))
.build();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -282,7 +282,7 @@ public void testRcTextPageSource(int rowCount, long fileSizePadding)
.withColumns(TEST_COLUMNS)
.withRowsCount(rowCount)
.withFileSizePadding(fileSizePadding)
.isReadableByPageSource(new RcFilePageSourceFactory(TESTING_TYPE_MANAGER, FILE_SYSTEM_FACTORY, STATS, new HiveConfig()));
.isReadableByPageSource(new RcFilePageSourceFactory(FILE_SYSTEM_FACTORY, STATS, new HiveConfig()));
}

@Test(dataProvider = "rowCount")
Expand All @@ -299,7 +299,7 @@ public void testRcTextOptimizedWriter(int rowCount)
.withRowsCount(rowCount)
.withFileWriterFactory(new RcFileFileWriterFactory(FILE_SYSTEM_FACTORY, TESTING_TYPE_MANAGER, new NodeVersion("test"), HIVE_STORAGE_TIME_ZONE))
.isReadableByRecordCursor(createGenericHiveRecordCursorProvider(HDFS_ENVIRONMENT))
.isReadableByPageSource(new RcFilePageSourceFactory(TESTING_TYPE_MANAGER, FILE_SYSTEM_FACTORY, STATS, new HiveConfig()));
.isReadableByPageSource(new RcFilePageSourceFactory(FILE_SYSTEM_FACTORY, STATS, new HiveConfig()));
}

@Test(dataProvider = "rowCount")
Expand All @@ -316,7 +316,7 @@ public void testRcBinaryPageSource(int rowCount)
assertThatFileFormat(RCBINARY)
.withColumns(testColumns)
.withRowsCount(rowCount)
.isReadableByPageSource(new RcFilePageSourceFactory(TESTING_TYPE_MANAGER, FILE_SYSTEM_FACTORY, STATS, new HiveConfig()));
.isReadableByPageSource(new RcFilePageSourceFactory(FILE_SYSTEM_FACTORY, STATS, new HiveConfig()));
}

@Test(dataProvider = "rowCount")
Expand All @@ -341,7 +341,7 @@ public void testRcBinaryOptimizedWriter(int rowCount)
// generic Hive writer corrupts timestamps
.withSkipGenericWriterTest()
.withFileWriterFactory(new RcFileFileWriterFactory(FILE_SYSTEM_FACTORY, TESTING_TYPE_MANAGER, new NodeVersion("test"), HIVE_STORAGE_TIME_ZONE))
.isReadableByPageSource(new RcFilePageSourceFactory(TESTING_TYPE_MANAGER, FILE_SYSTEM_FACTORY, STATS, new HiveConfig()))
.isReadableByPageSource(new RcFilePageSourceFactory(FILE_SYSTEM_FACTORY, STATS, new HiveConfig()))
.withColumns(testColumnsNoTimestamps)
.isReadableByRecordCursor(createGenericHiveRecordCursorProvider(HDFS_ENVIRONMENT));
}
Expand Down Expand Up @@ -571,13 +571,13 @@ public void testTruncateVarcharColumn()
assertThatFileFormat(RCTEXT)
.withWriteColumns(ImmutableList.of(writeColumn))
.withReadColumns(ImmutableList.of(readColumn))
.isReadableByPageSource(new RcFilePageSourceFactory(TESTING_TYPE_MANAGER, FILE_SYSTEM_FACTORY, STATS, new HiveConfig()))
.isReadableByPageSource(new RcFilePageSourceFactory(FILE_SYSTEM_FACTORY, STATS, new HiveConfig()))
.isReadableByRecordCursor(createGenericHiveRecordCursorProvider(HDFS_ENVIRONMENT));

assertThatFileFormat(RCBINARY)
.withWriteColumns(ImmutableList.of(writeColumn))
.withReadColumns(ImmutableList.of(readColumn))
.isReadableByPageSource(new RcFilePageSourceFactory(TESTING_TYPE_MANAGER, FILE_SYSTEM_FACTORY, STATS, new HiveConfig()))
.isReadableByPageSource(new RcFilePageSourceFactory(FILE_SYSTEM_FACTORY, STATS, new HiveConfig()))
.isReadableByRecordCursor(createGenericHiveRecordCursorProvider(HDFS_ENVIRONMENT));

assertThatFileFormat(ORC)
Expand Down Expand Up @@ -804,7 +804,7 @@ public void testRCTextProjectedColumnsPageSource(int rowCount)
.withWriteColumns(writeColumns)
.withReadColumns(readColumns)
.withRowsCount(rowCount)
.isReadableByPageSource(new RcFilePageSourceFactory(TESTING_TYPE_MANAGER, FILE_SYSTEM_FACTORY, STATS, new HiveConfig()));
.isReadableByPageSource(new RcFilePageSourceFactory(FILE_SYSTEM_FACTORY, STATS, new HiveConfig()));
}

@Test(dataProvider = "rowCount")
Expand Down Expand Up @@ -837,7 +837,7 @@ public void testRCBinaryProjectedColumns(int rowCount)
// generic Hive writer corrupts timestamps
.withSkipGenericWriterTest()
.withFileWriterFactory(new RcFileFileWriterFactory(FILE_SYSTEM_FACTORY, TESTING_TYPE_MANAGER, new NodeVersion("test"), HIVE_STORAGE_TIME_ZONE))
.isReadableByPageSource(new RcFilePageSourceFactory(TESTING_TYPE_MANAGER, FILE_SYSTEM_FACTORY, STATS, new HiveConfig()));
.isReadableByPageSource(new RcFilePageSourceFactory(FILE_SYSTEM_FACTORY, STATS, new HiveConfig()));
}

@Test(dataProvider = "rowCount")
Expand Down Expand Up @@ -867,7 +867,7 @@ public void testRCBinaryProjectedColumnsPageSource(int rowCount)
// generic Hive writer corrupts timestamps
.withSkipGenericWriterTest()
.withFileWriterFactory(new RcFileFileWriterFactory(FILE_SYSTEM_FACTORY, TESTING_TYPE_MANAGER, new NodeVersion("test"), HIVE_STORAGE_TIME_ZONE))
.isReadableByPageSource(new RcFilePageSourceFactory(TESTING_TYPE_MANAGER, FILE_SYSTEM_FACTORY, STATS, new HiveConfig()));
.isReadableByPageSource(new RcFilePageSourceFactory(FILE_SYSTEM_FACTORY, STATS, new HiveConfig()));
}

@Test
Expand All @@ -884,12 +884,12 @@ public void testFailForLongVarcharPartitionColumn()

assertThatFileFormat(RCTEXT)
.withColumns(columns)
.isFailingForPageSource(new RcFilePageSourceFactory(TESTING_TYPE_MANAGER, FILE_SYSTEM_FACTORY, STATS, new HiveConfig()), expectedErrorCode, expectedMessage)
.isFailingForPageSource(new RcFilePageSourceFactory(FILE_SYSTEM_FACTORY, STATS, new HiveConfig()), expectedErrorCode, expectedMessage)
.isFailingForRecordCursor(createGenericHiveRecordCursorProvider(HDFS_ENVIRONMENT), expectedErrorCode, expectedMessage);

assertThatFileFormat(RCBINARY)
.withColumns(columns)
.isFailingForPageSource(new RcFilePageSourceFactory(TESTING_TYPE_MANAGER, FILE_SYSTEM_FACTORY, STATS, new HiveConfig()), expectedErrorCode, expectedMessage)
.isFailingForPageSource(new RcFilePageSourceFactory(FILE_SYSTEM_FACTORY, STATS, new HiveConfig()), expectedErrorCode, expectedMessage)
.isFailingForRecordCursor(createGenericHiveRecordCursorProvider(HDFS_ENVIRONMENT), expectedErrorCode, expectedMessage);

assertThatFileFormat(ORC)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,6 @@
import static io.trino.parquet.writer.ParquetSchemaConverter.HIVE_PARQUET_USE_LEGACY_DECIMAL_ENCODING;
import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY;
import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_STATS;
import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER;
import static org.joda.time.DateTimeZone.UTC;

public final class StandardFileFormats
Expand All @@ -73,7 +72,7 @@ public HiveStorageFormat getFormat()
@Override
public Optional<HivePageSourceFactory> getHivePageSourceFactory(HdfsEnvironment hdfsEnvironment)
{
return Optional.of(new RcFilePageSourceFactory(TESTING_TYPE_MANAGER, HDFS_FILE_SYSTEM_FACTORY, new FileFormatDataSourceStats(), new HiveConfig().setRcfileTimeZone("UTC")));
return Optional.of(new RcFilePageSourceFactory(HDFS_FILE_SYSTEM_FACTORY, new FileFormatDataSourceStats(), new HiveConfig().setRcfileTimeZone("UTC")));
}

@Override
Expand Down Expand Up @@ -104,7 +103,7 @@ public HiveStorageFormat getFormat()
@Override
public Optional<HivePageSourceFactory> getHivePageSourceFactory(HdfsEnvironment hdfsEnvironment)
{
return Optional.of(new RcFilePageSourceFactory(TESTING_TYPE_MANAGER, HDFS_FILE_SYSTEM_FACTORY, new FileFormatDataSourceStats(), new HiveConfig().setRcfileTimeZone("UTC")));
return Optional.of(new RcFilePageSourceFactory(HDFS_FILE_SYSTEM_FACTORY, new FileFormatDataSourceStats(), new HiveConfig().setRcfileTimeZone("UTC")));
}

@Override
Expand Down
Loading