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

Simplify schema field in HiveSplit for ORC, Parquet and RCFile tables #23561

Merged
merged 1 commit into from
Sep 30, 2024
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 @@ -115,11 +115,11 @@
import static io.trino.plugin.hive.util.HiveBucketing.getBucketingVersion;
import static io.trino.plugin.hive.util.HiveTypeUtil.typeSupported;
import static io.trino.plugin.hive.util.HiveUtil.checkCondition;
import static io.trino.plugin.hive.util.HiveUtil.getDeserializerClassName;
import static io.trino.plugin.hive.util.HiveUtil.getFooterCount;
import static io.trino.plugin.hive.util.HiveUtil.getHeaderCount;
import static io.trino.plugin.hive.util.HiveUtil.getInputFormatName;
import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeyColumnHandles;
import static io.trino.plugin.hive.util.HiveUtil.getSerializationLibraryName;
import static io.trino.plugin.hive.util.PartitionMatchSupplier.createPartitionMatchSupplier;
import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED;
import static java.lang.Integer.parseInt;
Expand Down Expand Up @@ -414,7 +414,7 @@ private ListenableFuture<Void> loadPartition(HivePartitionMetadata partition)
if (tableBucketInfo.isPresent()) {
throw new TrinoException(NOT_SUPPORTED, "Bucketed table in SymlinkTextInputFormat is not yet supported");
}
HiveStorageFormat targetStorageFormat = getSymlinkStorageFormat(getDeserializerClassName(schema));
HiveStorageFormat targetStorageFormat = getSymlinkStorageFormat(getSerializationLibraryName(schema));
ListMultimap<Location, Location> targets = getTargetLocationsByParentFromSymlink(location);

InternalHiveSplitFactory splitFactory = new InternalHiveSplitFactory(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
import io.trino.spi.predicate.TupleDomain;

import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.OptionalInt;

Expand All @@ -32,7 +31,7 @@ Optional<ReaderPageSource> createPageSource(
long length,
long estimatedFileSize,
long fileModifiedTime,
Map<String, String> schema,
Schema schema,
List<HiveColumnHandle> columns,
TupleDomain<HiveColumnHandle> effectivePredicate,
Optional<AcidInfo> acidInfo,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,6 @@
import static io.trino.plugin.hive.util.HiveBucketing.HiveBucketFilter;
import static io.trino.plugin.hive.util.HiveBucketing.getHiveBucketFilter;
import static io.trino.plugin.hive.util.HiveTypeUtil.getHiveTypeForDereferences;
import static io.trino.plugin.hive.util.HiveUtil.getDeserializerClassName;
import static io.trino.plugin.hive.util.HiveUtil.getInputFormatName;
import static io.trino.plugin.hive.util.HiveUtil.getPrefilledColumnValue;
import static java.util.Objects.requireNonNull;
Expand Down Expand Up @@ -162,8 +161,8 @@ public ConnectorPageSource createPageSource(
}

throw new TrinoException(HIVE_UNSUPPORTED_FORMAT, "Unsupported input format: serde=%s, format=%s, partition=%s, path=%s".formatted(
getDeserializerClassName(hiveSplit.getSchema()),
getInputFormatName(hiveSplit.getSchema()).orElse(null),
hiveSplit.getSchema().serializationLibraryName(),
getInputFormatName(hiveSplit.getSchema().serdeProperties()).orElse(null),
hiveSplit.getPartitionName(),
hiveSplit.getPath()));
}
Expand All @@ -177,7 +176,7 @@ public static Optional<ConnectorPageSource> createHivePageSource(
long length,
long estimatedFileSize,
long fileModifiedTime,
Map<String, String> schema,
Schema schema,
TupleDomain<HiveColumnHandle> effectivePredicate,
TypeManager typeManager,
Optional<BucketConversion> bucketConversion,
Expand All @@ -196,7 +195,7 @@ public static Optional<ConnectorPageSource> createHivePageSource(
Optional<BucketAdaptation> bucketAdaptation = createBucketAdaptation(bucketConversion, tableBucketNumber, regularAndInterimColumnMappings);
Optional<BucketValidator> bucketValidator = createBucketValidator(path, bucketValidation, tableBucketNumber, regularAndInterimColumnMappings);

CoercionContext coercionContext = new CoercionContext(getTimestampPrecision(session), extractHiveStorageFormat(getDeserializerClassName(schema)));
CoercionContext coercionContext = new CoercionContext(getTimestampPrecision(session), extractHiveStorageFormat(schema.serializationLibraryName()));

for (HivePageSourceFactory pageSourceFactory : pageSourceFactories) {
List<HiveColumnHandle> desiredColumns = toColumnHandles(regularAndInterimColumnMappings, typeManager, coercionContext);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@
import static io.airlift.slice.SizeOf.estimatedSizeOf;
import static io.airlift.slice.SizeOf.instanceSize;
import static io.airlift.slice.SizeOf.sizeOf;
import static io.trino.plugin.hive.util.HiveUtil.getDeserializerClassName;
import static java.util.Objects.requireNonNull;
import static java.util.stream.Collectors.joining;

Expand All @@ -49,7 +48,7 @@ public class HiveSplit
private final long length;
private final long estimatedFileSize;
private final long fileModifiedTime;
private final Map<String, String> schema;
private final Schema schema;
private final List<HivePartitionKey> partitionKeys;
private final List<HostAddress> addresses;
private final String partitionName;
Expand All @@ -70,7 +69,7 @@ public HiveSplit(
@JsonProperty("length") long length,
@JsonProperty("estimatedFileSize") long estimatedFileSize,
@JsonProperty("fileModifiedTime") long fileModifiedTime,
@JsonProperty("schema") Map<String, String> schema,
@JsonProperty("schema") Schema schema,
@JsonProperty("partitionKeys") List<HivePartitionKey> partitionKeys,
@JsonProperty("readBucketNumber") OptionalInt readBucketNumber,
@JsonProperty("tableBucketNumber") OptionalInt tableBucketNumber,
Expand Down Expand Up @@ -108,7 +107,7 @@ public HiveSplit(
long length,
long estimatedFileSize,
long fileModifiedTime,
Map<String, String> schema,
Schema schema,
List<HivePartitionKey> partitionKeys,
List<HostAddress> addresses,
OptionalInt readBucketNumber,
Expand Down Expand Up @@ -191,7 +190,7 @@ public long getFileModifiedTime()
}

@JsonProperty
public Map<String, String> getSchema()
public Schema getSchema()
{
return schema;
}
Expand Down Expand Up @@ -270,7 +269,7 @@ public long getRetainedSizeInBytes()
{
return INSTANCE_SIZE
+ estimatedSizeOf(path)
+ estimatedSizeOf(schema, key -> estimatedSizeOf((String) key), value -> estimatedSizeOf((String) value))
+ schema.getRetainedSizeInBytes()
+ estimatedSizeOf(partitionKeys, HivePartitionKey::estimatedSizeInBytes)
+ estimatedSizeOf(addresses, HostAddress::getRetainedSizeInBytes)
+ estimatedSizeOf(partitionName)
Expand All @@ -294,7 +293,7 @@ public Map<String, String> getSplitInfo()
.put("hosts", addresses.stream().map(HostAddress::toString).collect(joining(",")))
.put("forceLocalScheduling", String.valueOf(forceLocalScheduling))
.put("partitionName", partitionName)
.put("deserializerClassName", getDeserializerClassName(schema))
.put("serializationLibraryName", schema.serializationLibraryName())
.buildOrThrow();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ public class InternalHiveSplit
private final long end;
private final long estimatedFileSize;
private final long fileModifiedTime;
private final Map<String, String> schema;
private final Schema schema;
private final List<HivePartitionKey> partitionKeys;
private final List<InternalHiveBlock> blocks;
private final String partitionName;
Expand All @@ -70,7 +70,7 @@ public InternalHiveSplit(
long end,
long estimatedFileSize,
long fileModifiedTime,
Map<String, String> schema,
Schema schema,
List<HivePartitionKey> partitionKeys,
List<InternalHiveBlock> blocks,
OptionalInt readBucketNumber,
Expand Down Expand Up @@ -144,7 +144,7 @@ public long getFileModifiedTime()
return fileModifiedTime;
}

public Map<String, String> getSchema()
public Schema getSchema()
{
return schema;
}
Expand Down
43 changes: 43 additions & 0 deletions plugin/trino-hive/src/main/java/io/trino/plugin/hive/Schema.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/*
* 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.hive;

import io.airlift.slice.SizeOf;

import java.util.Map;

import static io.airlift.slice.SizeOf.estimatedSizeOf;
import static io.airlift.slice.SizeOf.instanceSize;
import static java.util.Objects.requireNonNull;

public record Schema(
String serializationLibraryName,
boolean isFullAcidTable,
Map<String, String> serdeProperties)
{
private static final int INSTANCE_SIZE = instanceSize(Schema.class);

public Schema
{
requireNonNull(serializationLibraryName, "serializationLibraryName is null");
requireNonNull(serdeProperties, "serdeProperties is null");
}

public long getRetainedSizeInBytes()
{
return INSTANCE_SIZE
+ estimatedSizeOf(serializationLibraryName)
+ estimatedSizeOf(serdeProperties, SizeOf::estimatedSizeOf, SizeOf::estimatedSizeOf);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,6 @@
import static io.trino.plugin.hive.ReaderPageSource.noProjectionAdaptation;
import static io.trino.plugin.hive.avro.AvroHiveFileUtils.getCanonicalToGivenFieldName;
import static io.trino.plugin.hive.avro.AvroHiveFileUtils.wrapInUnionWithNull;
import static io.trino.plugin.hive.util.HiveUtil.getDeserializerClassName;
import static io.trino.plugin.hive.util.HiveUtil.splitError;
import static io.trino.spi.type.TimestampType.createTimestampType;
import static java.lang.Math.min;
Expand All @@ -88,15 +87,15 @@ public Optional<ReaderPageSource> createPageSource(
long length,
long estimatedFileSize,
long fileModifiedTime,
Map<String, String> schema,
io.trino.plugin.hive.Schema schema,
List<HiveColumnHandle> columns,
TupleDomain<HiveColumnHandle> effectivePredicate,
Optional<AcidInfo> acidInfo,
OptionalInt bucketNumber,
boolean originalFile,
AcidTransaction transaction)
{
if (!AVRO_SERDE_CLASS.equals(getDeserializerClassName(schema))) {
if (!AVRO_SERDE_CLASS.equals(schema.serializationLibraryName())) {
return Optional.empty();
}
checkArgument(acidInfo.isEmpty(), "Acid is not supported");
Expand All @@ -116,7 +115,7 @@ public Optional<ReaderPageSource> createPageSource(

Schema tableSchema;
try {
tableSchema = AvroHiveFileUtils.determineSchemaOrThrowException(trinoFileSystem, schema);
tableSchema = AvroHiveFileUtils.determineSchemaOrThrowException(trinoFileSystem, schema.serdeProperties());
}
catch (IOException | org.apache.avro.AvroTypeException e) {
throw new TrinoException(HIVE_CANNOT_OPEN_SPLIT, "Unable to load or parse schema", e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import io.trino.plugin.hive.HivePageSourceFactory;
import io.trino.plugin.hive.ReaderColumns;
import io.trino.plugin.hive.ReaderPageSource;
import io.trino.plugin.hive.Schema;
import io.trino.plugin.hive.acid.AcidTransaction;
import io.trino.spi.TrinoException;
import io.trino.spi.connector.ConnectorSession;
Expand All @@ -39,7 +40,6 @@

import java.io.InputStream;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.OptionalInt;

Expand All @@ -50,7 +50,6 @@
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.ReaderPageSource.noProjectionAdaptation;
import static io.trino.plugin.hive.util.HiveUtil.getDeserializerClassName;
import static io.trino.plugin.hive.util.HiveUtil.getFooterCount;
import static io.trino.plugin.hive.util.HiveUtil.getHeaderCount;
import static io.trino.plugin.hive.util.HiveUtil.splitError;
Expand Down Expand Up @@ -83,27 +82,27 @@ public Optional<ReaderPageSource> createPageSource(
long length,
long estimatedFileSize,
long fileModifiedTime,
Map<String, String> schema,
Schema schema,
List<HiveColumnHandle> columns,
TupleDomain<HiveColumnHandle> effectivePredicate,
Optional<AcidInfo> acidInfo,
OptionalInt bucketNumber,
boolean originalFile,
AcidTransaction transaction)
{
if (!lineReaderFactory.getHiveInputFormatClassNames().contains(schema.get(FILE_INPUT_FORMAT)) ||
!lineDeserializerFactory.getHiveSerDeClassNames().contains(getDeserializerClassName(schema))) {
if (!lineReaderFactory.getHiveInputFormatClassNames().contains(schema.serdeProperties().get(FILE_INPUT_FORMAT)) ||
!lineDeserializerFactory.getHiveSerDeClassNames().contains(schema.serializationLibraryName())) {
return Optional.empty();
}

checkArgument(acidInfo.isEmpty(), "Acid is not supported");

// get header and footer count
int headerCount = getHeaderCount(schema);
int headerCount = getHeaderCount(schema.serdeProperties());
if (headerCount > 1) {
checkArgument(start == 0, "Multiple header rows are not supported for a split file");
}
int footerCount = getFooterCount(schema);
int footerCount = getFooterCount(schema.serdeProperties());
if (footerCount > 0) {
checkArgument(start == 0, "Footer not supported for a split file");
}
Expand All @@ -124,7 +123,7 @@ public Optional<ReaderPageSource> createPageSource(
projectedReaderColumns.stream()
.map(column -> new Column(column.getName(), column.getType(), column.getBaseHiveColumnIndex()))
.collect(toImmutableList()),
schema);
schema.serdeProperties());
}

// Skip empty inputs
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@
import io.trino.plugin.hive.HivePageSourceFactory;
import io.trino.plugin.hive.ReaderColumns;
import io.trino.plugin.hive.ReaderPageSource;
import io.trino.plugin.hive.Schema;
import io.trino.plugin.hive.acid.AcidSchema;
import io.trino.plugin.hive.acid.AcidTransaction;
import io.trino.plugin.hive.coercions.TypeCoercer;
Expand Down Expand Up @@ -95,10 +96,7 @@
import static io.trino.plugin.hive.orc.OrcPageSource.ColumnAdaptation.mergedRowColumns;
import static io.trino.plugin.hive.orc.OrcPageSource.handleException;
import static io.trino.plugin.hive.orc.OrcTypeTranslator.createCoercer;
import static io.trino.plugin.hive.util.AcidTables.isFullAcidTable;
import static io.trino.plugin.hive.util.HiveUtil.getDeserializerClassName;
import static io.trino.plugin.hive.util.HiveUtil.splitError;
import static io.trino.plugin.hive.util.SerdeConstants.SERIALIZATION_LIB;
import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED;
import static io.trino.spi.type.BigintType.BIGINT;
import static io.trino.spi.type.IntegerType.INTEGER;
Expand Down Expand Up @@ -159,12 +157,9 @@ public OrcPageSourceFactory(
this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null");
}

public static Map<String, String> stripUnnecessaryProperties(Map<String, String> schema)
public static boolean stripUnnecessaryProperties(String serializationLibraryName)
{
if (ORC_SERDE_CLASS.equals(getDeserializerClassName(schema)) && !isFullAcidTable(schema)) {
return ImmutableMap.of(SERIALIZATION_LIB, schema.get(SERIALIZATION_LIB));
}
return schema;
return ORC_SERDE_CLASS.equals(serializationLibraryName);
}

@Override
Expand All @@ -175,15 +170,15 @@ public Optional<ReaderPageSource> createPageSource(
long length,
long estimatedFileSize,
long fileModifiedTime,
Map<String, String> schema,
Schema schema,
List<HiveColumnHandle> columns,
TupleDomain<HiveColumnHandle> effectivePredicate,
Optional<AcidInfo> acidInfo,
OptionalInt bucketNumber,
boolean originalFile,
AcidTransaction transaction)
{
if (!ORC_SERDE_CLASS.equals(getDeserializerClassName(schema))) {
if (!ORC_SERDE_CLASS.equals(schema.serializationLibraryName())) {
return Optional.empty();
}

Expand All @@ -206,7 +201,7 @@ public Optional<ReaderPageSource> createPageSource(
readerColumnHandles,
columns,
isUseOrcColumnNames(session),
isFullAcidTable(schema),
schema.isFullAcidTable(),
effectivePredicate,
legacyTimeZone,
orcReaderOptions
Expand Down
Loading