From f75f9db63bc83348cc66a330ff9eb4d80ef642cd Mon Sep 17 00:00:00 2001 From: "Jalpreet Singh Nanda (:imjalpreet)" Date: Wed, 17 Jan 2024 13:57:00 +0530 Subject: [PATCH] Convert BaseHiveColumnHandle interface into a class --- .../presto/hive/BaseHiveColumnHandle.java | 62 +++++++++++++++- presto-hive-hadoop2/pom.xml | 5 ++ ...TestHiveFileSystemS3SelectCsvPushdown.java | 2 +- ...leSystemS3SelectCsvPushdownWithSplits.java | 2 +- ...estHiveFileSystemS3SelectJsonPushdown.java | 2 +- ...eSystemS3SelectJsonPushdownWithSplits.java | 2 +- .../presto/hive/GenericHiveRecordCursor.java | 2 +- .../presto/hive/HiveColumnHandle.java | 74 +++++-------------- .../facebook/presto/hive/HiveMetadata.java | 8 +- .../presto/hive/HivePageSourceProvider.java | 8 +- .../presto/hive/HivePartitionManager.java | 8 +- .../presto/hive/HivePartitionResult.java | 6 +- .../presto/hive/HiveSplitManager.java | 2 +- .../com/facebook/presto/hive/HiveUtil.java | 4 +- .../presto/hive/orc/OrcBatchPageSource.java | 2 +- .../hive/orc/OrcBatchPageSourceFactory.java | 4 +- .../orc/OrcSelectivePageSourceFactory.java | 4 +- .../parquet/ParquetPageSourceFactory.java | 6 +- .../hive/rcfile/RcFilePageSourceFactory.java | 2 +- .../presto/hive/rule/HiveFilterPushdown.java | 3 +- .../rule/HiveParquetDereferencePushDown.java | 2 +- .../presto/hive/AbstractTestHiveClient.java | 4 +- .../hive/AbstractTestHiveFileFormats.java | 4 +- ...stractDwrfEncryptionInformationSource.java | 2 +- .../hive/TestBackgroundHiveSplitLoader.java | 2 +- .../presto/hive/TestDynamicPruning.java | 4 +- .../TestGenericHiveRecordCursorProvider.java | 2 +- .../presto/hive/TestHiveColumnHandle.java | 4 +- .../presto/hive/TestHiveLogicalPlanner.java | 2 +- .../presto/hive/TestHivePageSink.java | 2 +- .../presto/hive/TestHivePartitionManager.java | 4 +- .../facebook/presto/hive/TestHiveSplit.java | 2 +- .../presto/hive/TestHiveSplitManager.java | 4 +- .../hive/TestHiveTableLayoutHandle.java | 4 +- .../presto/hive/TestIonSqlQueryBuilder.java | 2 +- .../presto/hive/TestJsonHiveHandles.java | 2 +- ...TestMergeRequestedAndPredicateColumns.java | 2 +- .../TestOrcBatchPageSourceMemoryTracking.java | 4 +- .../presto/hive/benchmark/FileFormat.java | 2 +- .../parquet/TestParquetPageSourceFactory.java | 4 +- .../predicate/TestParquetPredicateUtils.java | 2 +- .../s3select/TestS3SelectRecordCursor.java | 4 +- .../TestMetastoreHiveStatisticsProvider.java | 4 +- .../presto/iceberg/IcebergColumnHandle.java | 59 +++------------ .../presto/iceberg/IcebergOrcColumn.java | 2 +- .../iceberg/IcebergPageSourceProvider.java | 4 +- .../facebook/presto/iceberg/IcebergUtil.java | 4 +- .../iceberg/TestIcebergColumnHandle.java | 2 +- .../iceberg/TestIcebergLogicalPlanner.java | 2 +- .../presto/iceberg/TestStatisticsUtil.java | 2 +- .../hive/TestIcebergHiveStatistics.java | 6 +- 51 files changed, 172 insertions(+), 185 deletions(-) diff --git a/presto-hive-common/src/main/java/com/facebook/presto/hive/BaseHiveColumnHandle.java b/presto-hive-common/src/main/java/com/facebook/presto/hive/BaseHiveColumnHandle.java index 8d0c2302e5bb..956a213a847c 100644 --- a/presto-hive-common/src/main/java/com/facebook/presto/hive/BaseHiveColumnHandle.java +++ b/presto-hive-common/src/main/java/com/facebook/presto/hive/BaseHiveColumnHandle.java @@ -13,10 +13,66 @@ */ package com.facebook.presto.hive; +import com.facebook.presto.common.Subfield; import com.facebook.presto.spi.ColumnHandle; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; -public interface BaseHiveColumnHandle - extends ColumnHandle +import java.util.List; +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +public class BaseHiveColumnHandle + implements ColumnHandle { - String getName(); + public enum ColumnType + { + PARTITION_KEY, + REGULAR, + SYNTHESIZED, + AGGREGATED, + } + + private final String name; + private final Optional comment; + private final ColumnType columnType; + private final List requiredSubfields; + + @JsonCreator + public BaseHiveColumnHandle( + @JsonProperty("name") String name, + @JsonProperty("comment") Optional comment, + @JsonProperty("columnType") ColumnType columnType, + @JsonProperty("requiredSubfields") List requiredSubfields) + { + this.name = requireNonNull(name, "name is null"); + this.comment = requireNonNull(comment, "comment is null"); + this.columnType = requireNonNull(columnType, "columnType is null"); + this.requiredSubfields = requireNonNull(requiredSubfields, "requiredSubfields is null"); + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public Optional getComment() + { + return comment; + } + + @JsonProperty + public ColumnType getColumnType() + { + return columnType; + } + + @JsonProperty + public List getRequiredSubfields() + { + return requiredSubfields; + } } diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index b46d8c95ae51..2688c5056e96 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -22,6 +22,11 @@ presto-hive + + com.facebook.presto + presto-hive-common + + com.facebook.presto presto-hive-metastore diff --git a/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/s3select/TestHiveFileSystemS3SelectCsvPushdown.java b/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/s3select/TestHiveFileSystemS3SelectCsvPushdown.java index e0bac6fa3c77..cb8fb26ae7cd 100644 --- a/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/s3select/TestHiveFileSystemS3SelectCsvPushdown.java +++ b/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/s3select/TestHiveFileSystemS3SelectCsvPushdown.java @@ -28,7 +28,7 @@ import static com.facebook.airlift.testing.Assertions.assertEqualsIgnoreOrder; import static com.facebook.presto.common.type.BigintType.BIGINT; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveType.HIVE_LONG; public class TestHiveFileSystemS3SelectCsvPushdown diff --git a/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/s3select/TestHiveFileSystemS3SelectCsvPushdownWithSplits.java b/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/s3select/TestHiveFileSystemS3SelectCsvPushdownWithSplits.java index f1442a45d3a1..d5b5e5b3e7e8 100644 --- a/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/s3select/TestHiveFileSystemS3SelectCsvPushdownWithSplits.java +++ b/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/s3select/TestHiveFileSystemS3SelectCsvPushdownWithSplits.java @@ -26,7 +26,7 @@ import java.util.Optional; import static com.facebook.airlift.testing.Assertions.assertEqualsIgnoreOrder; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveFileSystemTestUtils.newSession; import static com.facebook.presto.hive.HiveType.HIVE_LONG; import static com.facebook.presto.hive.s3select.S3SelectTestHelper.expectedResult; diff --git a/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/s3select/TestHiveFileSystemS3SelectJsonPushdown.java b/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/s3select/TestHiveFileSystemS3SelectJsonPushdown.java index ac6d37291870..22d2cc78bc46 100644 --- a/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/s3select/TestHiveFileSystemS3SelectJsonPushdown.java +++ b/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/s3select/TestHiveFileSystemS3SelectJsonPushdown.java @@ -28,7 +28,7 @@ import static com.facebook.airlift.testing.Assertions.assertEqualsIgnoreOrder; import static com.facebook.presto.common.type.BigintType.BIGINT; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveFileSystemTestUtils.filterTable; import static com.facebook.presto.hive.HiveFileSystemTestUtils.newSession; import static com.facebook.presto.hive.HiveFileSystemTestUtils.readTable; diff --git a/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/s3select/TestHiveFileSystemS3SelectJsonPushdownWithSplits.java b/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/s3select/TestHiveFileSystemS3SelectJsonPushdownWithSplits.java index b73bcffd32f5..7be723cdac20 100644 --- a/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/s3select/TestHiveFileSystemS3SelectJsonPushdownWithSplits.java +++ b/presto-hive-hadoop2/src/test/java/com/facebook/presto/hive/s3select/TestHiveFileSystemS3SelectJsonPushdownWithSplits.java @@ -26,7 +26,7 @@ import java.util.Optional; import static com.facebook.airlift.testing.Assertions.assertEqualsIgnoreOrder; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveFileSystemTestUtils.newSession; import static com.facebook.presto.hive.HiveType.HIVE_LONG; import static com.facebook.presto.hive.s3select.S3SelectTestHelper.expectedResult; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/GenericHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/hive/GenericHiveRecordCursor.java index c051efd885b3..eddc776ed3c7 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/GenericHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/GenericHiveRecordCursor.java @@ -69,7 +69,7 @@ import static com.facebook.presto.common.type.VarbinaryType.VARBINARY; import static com.facebook.presto.common.type.Varchars.isVarcharType; import static com.facebook.presto.common.type.Varchars.truncateToLength; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveErrorCode.HIVE_BAD_DATA; import static com.facebook.presto.hive.HiveErrorCode.HIVE_CURSOR_ERROR; import static com.facebook.presto.hive.HiveUtil.closeWithSuppression; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveColumnHandle.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveColumnHandle.java index 81e47dc9c42b..a9250eb40f18 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveColumnHandle.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveColumnHandle.java @@ -27,9 +27,9 @@ import java.util.Optional; import static com.facebook.presto.common.type.BigintType.BIGINT; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.AGGREGATED; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.SYNTHESIZED; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.AGGREGATED; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.SYNTHESIZED; import static com.facebook.presto.hive.HiveType.HIVE_INT; import static com.facebook.presto.hive.HiveType.HIVE_LONG; import static com.facebook.presto.hive.HiveType.HIVE_STRING; @@ -40,7 +40,7 @@ import static java.util.Objects.requireNonNull; public class HiveColumnHandle - implements BaseHiveColumnHandle + extends BaseHiveColumnHandle { public static final int PATH_COLUMN_INDEX = -11; public static final String PATH_COLUMN_NAME = "$path"; @@ -67,21 +67,9 @@ public class HiveColumnHandle // Ids <= this can be used for distinguishing between different prefilled columns. public static final int MAX_PARTITION_KEY_COLUMN_INDEX = -13; - public enum ColumnType - { - PARTITION_KEY, - REGULAR, - SYNTHESIZED, - AGGREGATED, - } - - private final String name; private final HiveType hiveType; private final TypeSignature typeName; private final int hiveColumnIndex; - private final ColumnType columnType; - private final Optional comment; - private final List requiredSubfields; private final Optional partialAggregation; @JsonCreator @@ -95,14 +83,12 @@ public HiveColumnHandle( @JsonProperty("requiredSubfields") List requiredSubfields, @JsonProperty("partialAggregation") Optional partialAggregation) { - this.name = requireNonNull(name, "name is null"); + super(name, comment, columnType, requiredSubfields); + checkArgument(hiveColumnIndex >= 0 || columnType == PARTITION_KEY || columnType == SYNTHESIZED || columnType == AGGREGATED, format("hiveColumnIndex:%d is negative, columnType:%s", hiveColumnIndex, columnType.toString())); this.hiveColumnIndex = hiveColumnIndex; this.hiveType = requireNonNull(hiveType, "hiveType is null"); this.typeName = requireNonNull(typeSignature, "type is null"); - this.columnType = requireNonNull(columnType, "columnType is null"); - this.comment = requireNonNull(comment, "comment is null"); - this.requiredSubfields = requireNonNull(requiredSubfields, "requiredSubfields is null"); this.partialAggregation = requireNonNull(partialAggregation, "partialAggregation is null"); checkArgument(columnType != AGGREGATED || partialAggregation.isPresent(), "Aggregated column does not have aggregate function"); } @@ -119,12 +105,6 @@ public HiveColumnHandle( this(name, hiveType, typeSignature, hiveColumnIndex, columnType, comment, ImmutableList.of(), partialAggregation); } - @JsonProperty - public String getName() - { - return name; - } - @JsonProperty public HiveType getHiveType() { @@ -139,23 +119,17 @@ public int getHiveColumnIndex() public boolean isPartitionKey() { - return columnType == PARTITION_KEY; + return getColumnType() == PARTITION_KEY; } public boolean isHidden() { - return columnType == SYNTHESIZED; + return getColumnType() == SYNTHESIZED; } public ColumnMetadata getColumnMetadata(TypeManager typeManager) { - return new ColumnMetadata(name, typeManager.getType(typeName), null, isHidden()); - } - - @JsonProperty - public Optional getComment() - { - return comment; + return new ColumnMetadata(getName(), typeManager.getType(typeName), null, isHidden()); } @JsonProperty @@ -170,18 +144,6 @@ public TypeSignature getTypeSignature() return typeName; } - @JsonProperty - public ColumnType getColumnType() - { - return columnType; - } - - @JsonProperty - public List getRequiredSubfields() - { - return requiredSubfields; - } - @Override public ColumnHandle withRequiredSubfields(List subfields) { @@ -190,13 +152,13 @@ public ColumnHandle withRequiredSubfields(List subfields) return this; } - return new HiveColumnHandle(name, hiveType, typeName, hiveColumnIndex, columnType, comment, subfields, partialAggregation); + return new HiveColumnHandle(getName(), hiveType, typeName, hiveColumnIndex, getColumnType(), getComment(), subfields, partialAggregation); } @Override public int hashCode() { - return Objects.hash(name, hiveColumnIndex, hiveType, columnType, comment); + return Objects.hash(getName(), hiveColumnIndex, hiveType, getColumnType(), getComment()); } @Override @@ -209,22 +171,22 @@ public boolean equals(Object obj) return false; } HiveColumnHandle other = (HiveColumnHandle) obj; - return Objects.equals(this.name, other.name) && + return Objects.equals(this.getName(), other.getName()) && Objects.equals(this.hiveColumnIndex, other.hiveColumnIndex) && Objects.equals(this.hiveType, other.hiveType) && - Objects.equals(this.columnType, other.columnType) && - Objects.equals(this.comment, other.comment) && - Objects.equals(this.requiredSubfields, other.requiredSubfields); + Objects.equals(this.getColumnType(), other.getColumnType()) && + Objects.equals(this.getComment(), other.getComment()) && + Objects.equals(this.getRequiredSubfields(), other.getRequiredSubfields()); } @Override public String toString() { - if (requiredSubfields.isEmpty()) { - return name + ":" + hiveType + ":" + hiveColumnIndex + ":" + columnType; + if (getRequiredSubfields().isEmpty()) { + return getName() + ":" + hiveType + ":" + hiveColumnIndex + ":" + getColumnType(); } - return name + ":" + hiveType + ":" + hiveColumnIndex + ":" + columnType + ":" + requiredSubfields; + return getName() + ":" + hiveType + ":" + hiveColumnIndex + ":" + getColumnType() + ":" + getRequiredSubfields(); } public static HiveColumnHandle updateRowIdHandle() diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java index 80700ceba6f2..b0f9ad034241 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java @@ -159,6 +159,9 @@ import static com.facebook.presto.common.type.Varchars.isVarcharType; import static com.facebook.presto.expressions.LogicalRowExpressions.TRUE_CONSTANT; import static com.facebook.presto.expressions.LogicalRowExpressions.binaryExpression; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.SYNTHESIZED; import static com.facebook.presto.hive.BucketFunctionType.HIVE_COMPATIBLE; import static com.facebook.presto.hive.BucketFunctionType.PRESTO_NATIVE; import static com.facebook.presto.hive.ColumnEncryptionInformation.ColumnWithStructSubfield; @@ -173,9 +176,6 @@ import static com.facebook.presto.hive.HiveBucketing.getHiveBucketHandle; import static com.facebook.presto.hive.HiveClientConfig.InsertExistingPartitionsBehavior; import static com.facebook.presto.hive.HiveColumnHandle.BUCKET_COLUMN_NAME; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.SYNTHESIZED; import static com.facebook.presto.hive.HiveColumnHandle.FILE_MODIFIED_TIME_COLUMN_NAME; import static com.facebook.presto.hive.HiveColumnHandle.FILE_SIZE_COLUMN_NAME; import static com.facebook.presto.hive.HiveColumnHandle.PATH_COLUMN_NAME; @@ -2633,7 +2633,7 @@ public List getTableLayouts(ConnectorSession session new HiveTableLayoutHandle.Builder() .setSchemaTableName(handle.getSchemaTableName()) .setTablePath(table.getStorage().getLocation()) - .setPartitionColumns(ImmutableList.copyOf(hivePartitionResult.getPartitionColumns())) + .setPartitionColumns(hivePartitionResult.getPartitionColumns()) .setDataColumns(pruneColumnComments(hivePartitionResult.getDataColumns())) .setTableParameters(hivePartitionResult.getTableParameters()) .setDomainPredicate(domainPredicate) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSourceProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSourceProvider.java index f7778e2f415f..292bb240df5f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSourceProvider.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSourceProvider.java @@ -61,12 +61,12 @@ import java.util.Set; import java.util.function.Function; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.AGGREGATED; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.SYNTHESIZED; import static com.facebook.presto.hive.HiveBucketing.getHiveBucketFilter; import static com.facebook.presto.hive.HiveCoercer.createCoercer; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.AGGREGATED; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.SYNTHESIZED; import static com.facebook.presto.hive.HiveColumnHandle.isPushedDownSubfield; import static com.facebook.presto.hive.HiveErrorCode.HIVE_UNKNOWN_ERROR; import static com.facebook.presto.hive.HivePageSourceProvider.ColumnMapping.toColumnHandles; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java index 8d1cf936d5a8..a20c019357f0 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java @@ -277,7 +277,7 @@ && queryAccessesTooManyBuckets(hiveBucketHandle.get(), bucketFilter, partitions, if (effectivePredicate.isNone()) { return new HivePartitionResult( - partitionColumns, + ImmutableList.copyOf(partitionColumns), table.getDataColumns(), table.getParameters(), partitions, @@ -292,7 +292,7 @@ && queryAccessesTooManyBuckets(hiveBucketHandle.get(), bucketFilter, partitions, if (partitionColumns.isEmpty()) { return new HivePartitionResult( - partitionColumns, + ImmutableList.copyOf(partitionColumns), table.getDataColumns(), table.getParameters(), partitions, @@ -307,7 +307,7 @@ && queryAccessesTooManyBuckets(hiveBucketHandle.get(), bucketFilter, partitions, TupleDomain remainingTupleDomain = TupleDomain.withColumnDomains(Maps.filterKeys(effectivePredicate.getDomains().get(), not(Predicates.in(partitionColumns)))); TupleDomain enforcedTupleDomain = TupleDomain.withColumnDomains(Maps.filterKeys(effectivePredicate.getDomains().get(), Predicates.in(partitionColumns))); return new HivePartitionResult( - partitionColumns, + ImmutableList.copyOf(partitionColumns), table.getDataColumns(), table.getParameters(), partitions, @@ -398,7 +398,7 @@ public HivePartitionResult getPartitions(SemiTransactionalHiveMetastore metastor Optional bucketHandle = shouldIgnoreTableBucketing(session) ? Optional.empty() : getHiveBucketHandle(table); return new HivePartitionResult( - partitionColumns, + ImmutableList.copyOf(partitionColumns), table.getDataColumns(), table.getParameters(), partitionList, diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionResult.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionResult.java index bb6213899177..905461f74048 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionResult.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionResult.java @@ -36,7 +36,7 @@ */ public class HivePartitionResult { - private final List partitionColumns; + private final List partitionColumns; private final List dataColumns; private final Map tableParameters; private final List partitions; @@ -47,7 +47,7 @@ public class HivePartitionResult private final Optional bucketFilter; public HivePartitionResult( - List partitionColumns, + List partitionColumns, List dataColumns, Map tableParameters, List partitions, @@ -68,7 +68,7 @@ public HivePartitionResult( this.bucketFilter = requireNonNull(bucketFilter, "bucketFilter is null"); } - public List getPartitionColumns() + public List getPartitionColumns() { return partitionColumns; } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplitManager.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplitManager.java index 72ae6bfaf46d..8c309c642c84 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplitManager.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveSplitManager.java @@ -79,7 +79,7 @@ import static com.facebook.presto.common.type.Decimals.encodeScaledValue; import static com.facebook.presto.common.type.Decimals.isShortDecimal; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveColumnHandle.isPathColumnHandle; import static com.facebook.presto.hive.HiveCommonSessionProperties.isUseParquetColumnNames; import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java index 7f4882a0f60f..d359da729a34 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java @@ -129,8 +129,8 @@ import static com.facebook.presto.common.type.TypeUtils.isDistinctType; import static com.facebook.presto.common.type.TypeUtils.isEnumType; import static com.facebook.presto.common.type.Varchars.isVarcharType; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveColumnHandle.MAX_PARTITION_KEY_COLUMN_INDEX; import static com.facebook.presto.hive.HiveColumnHandle.bucketColumnHandle; import static com.facebook.presto.hive.HiveColumnHandle.fileModifiedTimeColumnHandle; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcBatchPageSource.java b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcBatchPageSource.java index 46477b7b2cc4..71b2e9bbdf42 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcBatchPageSource.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcBatchPageSource.java @@ -37,7 +37,7 @@ import java.util.List; import java.util.Optional; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveErrorCode.HIVE_BAD_DATA; import static com.facebook.presto.hive.HiveErrorCode.HIVE_CURSOR_ERROR; import static com.facebook.presto.orc.OrcReader.MAX_BATCH_SIZE; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcBatchPageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcBatchPageSourceFactory.java index 76adaea46336..0083c96e04ce 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcBatchPageSourceFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcBatchPageSourceFactory.java @@ -63,8 +63,8 @@ import java.util.Map; import java.util.Optional; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.AGGREGATED; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.AGGREGATED; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveCommonSessionProperties.getOrcLazyReadSmallRanges; import static com.facebook.presto.hive.HiveCommonSessionProperties.getOrcMaxBufferSize; import static com.facebook.presto.hive.HiveCommonSessionProperties.getOrcMaxMergeDistance; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcSelectivePageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcSelectivePageSourceFactory.java index e07461816489..9f7406624620 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcSelectivePageSourceFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcSelectivePageSourceFactory.java @@ -103,9 +103,9 @@ import static com.facebook.presto.expressions.LogicalRowExpressions.binaryExpression; import static com.facebook.presto.expressions.LogicalRowExpressions.extractConjuncts; import static com.facebook.presto.expressions.RowExpressionNodeInliner.replaceExpression; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.AGGREGATED; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveBucketing.getHiveBucket; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.AGGREGATED; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveCommonSessionProperties.getOrcLazyReadSmallRanges; import static com.facebook.presto.hive.HiveCommonSessionProperties.getOrcMaxBufferSize; import static com.facebook.presto.hive.HiveCommonSessionProperties.getOrcMaxMergeDistance; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java index 46aa1c5d0945..40471d325a0a 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java @@ -96,9 +96,9 @@ import static com.facebook.presto.common.type.StandardTypes.TINYINT; import static com.facebook.presto.common.type.StandardTypes.VARBINARY; import static com.facebook.presto.common.type.StandardTypes.VARCHAR; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.AGGREGATED; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.SYNTHESIZED; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.AGGREGATED; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.SYNTHESIZED; import static com.facebook.presto.hive.HiveColumnHandle.getPushedDownSubfield; import static com.facebook.presto.hive.HiveColumnHandle.isPushedDownSubfield; import static com.facebook.presto.hive.HiveCommonSessionProperties.getParquetMaxReadBlockSize; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/rcfile/RcFilePageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/rcfile/RcFilePageSourceFactory.java index ad4df04b6e80..b01e780efdfd 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/rcfile/RcFilePageSourceFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/rcfile/RcFilePageSourceFactory.java @@ -57,7 +57,7 @@ import java.util.Optional; import java.util.Properties; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.AGGREGATED; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.AGGREGATED; import static com.facebook.presto.hive.HiveErrorCode.HIVE_BAD_DATA; import static com.facebook.presto.hive.HiveErrorCode.HIVE_CANNOT_OPEN_SPLIT; import static com.facebook.presto.hive.HiveErrorCode.HIVE_MISSING_DATA; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveFilterPushdown.java b/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveFilterPushdown.java index b491b948c1e3..c76492a49045 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveFilterPushdown.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveFilterPushdown.java @@ -48,7 +48,6 @@ import com.facebook.presto.spi.relation.RowExpression; import com.facebook.presto.spi.relation.RowExpressionService; import com.google.common.base.Functions; -import com.google.common.collect.ImmutableList; import java.util.List; import java.util.Map; @@ -180,7 +179,7 @@ public ConnectorPushdownFilterResult getConnectorPushdownFilterResult( new HiveTableLayoutHandle.Builder() .setSchemaTableName(tableName) .setTablePath(table.getStorage().getLocation()) - .setPartitionColumns(ImmutableList.copyOf(hivePartitionResult.getPartitionColumns())) + .setPartitionColumns(hivePartitionResult.getPartitionColumns()) .setDataColumns(pruneColumnComments(hivePartitionResult.getDataColumns())) .setTableParameters(hivePartitionResult.getTableParameters()) .setDomainPredicate(domainPredicate) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveParquetDereferencePushDown.java b/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveParquetDereferencePushDown.java index 3b07475e975e..010db98a74e7 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveParquetDereferencePushDown.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/rule/HiveParquetDereferencePushDown.java @@ -31,7 +31,7 @@ import java.util.Optional; import java.util.stream.Collectors; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.SYNTHESIZED; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.SYNTHESIZED; import static com.facebook.presto.hive.HiveSessionProperties.isParquetDereferencePushdownEnabled; import static com.facebook.presto.hive.HiveStorageFormat.PARQUET; import static com.facebook.presto.hive.HiveTableProperties.getHiveStorageFormat; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java index 2f830bb6e14d..5b3bfd30071c 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java @@ -212,14 +212,14 @@ import static com.facebook.presto.hive.AbstractTestHiveClient.TransactionDeleteInsertTestTag.ROLLBACK_AFTER_FINISH_INSERT; import static com.facebook.presto.hive.AbstractTestHiveClient.TransactionDeleteInsertTestTag.ROLLBACK_AFTER_SINK_FINISH; import static com.facebook.presto.hive.AbstractTestHiveClient.TransactionDeleteInsertTestTag.ROLLBACK_RIGHT_AWAY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.BucketFunctionType.HIVE_COMPATIBLE; import static com.facebook.presto.hive.CacheQuotaScope.TABLE; import static com.facebook.presto.hive.HiveBasicStatistics.createEmptyStatistics; import static com.facebook.presto.hive.HiveBasicStatistics.createZeroStatistics; import static com.facebook.presto.hive.HiveColumnConverterProvider.DEFAULT_COLUMN_CONVERTER_PROVIDER; import static com.facebook.presto.hive.HiveColumnHandle.BUCKET_COLUMN_NAME; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveColumnHandle.MAX_PARTITION_KEY_COLUMN_INDEX; import static com.facebook.presto.hive.HiveColumnHandle.bucketColumnHandle; import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_PARTITION_VALUE; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveFileFormats.java b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveFileFormats.java index d40049346031..fbcb5be6a538 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveFileFormats.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveFileFormats.java @@ -98,8 +98,8 @@ import static com.facebook.presto.common.type.VarcharType.createUnboundedVarcharType; import static com.facebook.presto.common.type.VarcharType.createVarcharType; import static com.facebook.presto.common.type.Varchars.isVarcharType; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveManifestUtils.getFileSize; import static com.facebook.presto.hive.HiveStorageFormat.DWRF; import static com.facebook.presto.hive.HiveStorageFormat.ORC; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestAbstractDwrfEncryptionInformationSource.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestAbstractDwrfEncryptionInformationSource.java index 54d6f9e49f1c..9afbde21d582 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestAbstractDwrfEncryptionInformationSource.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestAbstractDwrfEncryptionInformationSource.java @@ -26,10 +26,10 @@ import java.util.Map; import java.util.Optional; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.ColumnEncryptionInformation.fromHiveProperty; import static com.facebook.presto.hive.DwrfTableEncryptionProperties.forPerColumn; import static com.facebook.presto.hive.DwrfTableEncryptionProperties.forTable; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveStorageFormat.DWRF; import static com.facebook.presto.hive.HiveStorageFormat.ORC; import static com.facebook.presto.hive.HiveTestUtils.SESSION; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestBackgroundHiveSplitLoader.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestBackgroundHiveSplitLoader.java index acc8aa1b23e7..4ccd4794d307 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestBackgroundHiveSplitLoader.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestBackgroundHiveSplitLoader.java @@ -15,8 +15,8 @@ import com.facebook.airlift.stats.CounterStat; import com.facebook.presto.common.predicate.Domain; +import com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType; import com.facebook.presto.hive.HiveBucketing.HiveBucketFilter; -import com.facebook.presto.hive.HiveColumnHandle.ColumnType; import com.facebook.presto.hive.authentication.NoHdfsAuthentication; import com.facebook.presto.hive.filesystem.ExtendedFileSystem; import com.facebook.presto.hive.metastore.Column; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestDynamicPruning.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestDynamicPruning.java index ede2adb0a327..1c92f8a2d5a6 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestDynamicPruning.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestDynamicPruning.java @@ -47,10 +47,10 @@ import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature; import static com.facebook.presto.common.type.VarcharType.createVarcharType; import static com.facebook.presto.expressions.LogicalRowExpressions.TRUE_CONSTANT; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.BucketFunctionType.HIVE_COMPATIBLE; import static com.facebook.presto.hive.CacheQuotaRequirement.NO_CACHE_REQUIREMENT; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveQueryRunner.HIVE_CATALOG; import static com.facebook.presto.hive.HiveTestUtils.FUNCTION_AND_TYPE_MANAGER; import static com.facebook.presto.hive.HiveTestUtils.ROW_EXPRESSION_SERVICE; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestGenericHiveRecordCursorProvider.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestGenericHiveRecordCursorProvider.java index 808ac88f29d9..70cb117aa295 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestGenericHiveRecordCursorProvider.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestGenericHiveRecordCursorProvider.java @@ -42,7 +42,7 @@ import java.util.stream.Collectors; import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveTestUtils.FUNCTION_AND_TYPE_MANAGER; import static com.facebook.presto.hive.HiveTestUtils.SESSION; import static com.facebook.presto.hive.HiveType.HIVE_STRING; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveColumnHandle.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveColumnHandle.java index 05f08434b5d8..9475dd548d1e 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveColumnHandle.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveColumnHandle.java @@ -20,8 +20,8 @@ import java.util.Optional; import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static org.testng.Assert.assertEquals; public class TestHiveColumnHandle diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveLogicalPlanner.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveLogicalPlanner.java index fa6cae81bbfe..098371a81f4c 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveLogicalPlanner.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveLogicalPlanner.java @@ -97,7 +97,7 @@ import static com.facebook.presto.common.type.VarcharType.VARCHAR; import static com.facebook.presto.common.type.VarcharType.createVarcharType; import static com.facebook.presto.expressions.LogicalRowExpressions.TRUE_CONSTANT; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.SYNTHESIZED; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.SYNTHESIZED; import static com.facebook.presto.hive.HiveColumnHandle.isPushedDownSubfield; import static com.facebook.presto.hive.HiveCommonSessionProperties.RANGE_FILTERS_ON_SUBSCRIPTS_ENABLED; import static com.facebook.presto.hive.HiveQueryRunner.HIVE_CATALOG; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java index 5b967024e008..9ff28729e625 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java @@ -68,8 +68,8 @@ import static com.facebook.presto.common.type.VarcharType.createUnboundedVarcharType; import static com.facebook.presto.expressions.LogicalRowExpressions.TRUE_CONSTANT; import static com.facebook.presto.hive.AbstractTestHiveClient.TEST_HIVE_PAGE_SINK_CONTEXT; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.CacheQuotaRequirement.NO_CACHE_REQUIREMENT; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveCompressionCodec.NONE; import static com.facebook.presto.hive.HiveQueryRunner.HIVE_CATALOG; import static com.facebook.presto.hive.HiveQueryRunner.METASTORE_CONTEXT; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePartitionManager.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePartitionManager.java index 9a0d1f754343..db0986ab5531 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePartitionManager.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePartitionManager.java @@ -37,9 +37,9 @@ import static com.facebook.presto.common.type.IntegerType.INTEGER; import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature; import static com.facebook.presto.common.type.VarcharType.VARCHAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.BucketFunctionType.HIVE_COMPATIBLE; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveColumnHandle.MAX_PARTITION_KEY_COLUMN_INDEX; import static com.facebook.presto.hive.HiveColumnHandle.bucketColumnHandle; import static com.facebook.presto.hive.HiveStorageFormat.ORC; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveSplit.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveSplit.java index 0a731674274c..ce8c666e8cb1 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveSplit.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveSplit.java @@ -53,8 +53,8 @@ import static com.facebook.airlift.json.JsonBinder.jsonBinder; import static com.facebook.airlift.json.JsonCodecBinder.jsonCodecBinder; import static com.facebook.presto.common.type.BigintType.BIGINT; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.CacheQuotaRequirement.NO_CACHE_REQUIREMENT; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveType.HIVE_LONG; import static com.facebook.presto.hive.HiveType.HIVE_STRING; import static com.facebook.presto.metadata.FunctionAndTypeManager.createTestFunctionAndTypeManager; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveSplitManager.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveSplitManager.java index df79b44228a7..3aa460a05cc5 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveSplitManager.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveSplitManager.java @@ -82,8 +82,8 @@ import static com.facebook.presto.common.type.VarcharType.createUnboundedVarcharType; import static com.facebook.presto.expressions.LogicalRowExpressions.TRUE_CONSTANT; import static com.facebook.presto.hive.AbstractTestHiveClient.TEST_SERVER_VERSION; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveFileInfo.createHiveFileInfo; import static com.facebook.presto.hive.HiveStorageFormat.ORC; import static com.facebook.presto.hive.HiveTestUtils.FILTER_STATS_CALCULATOR_SERVICE; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveTableLayoutHandle.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveTableLayoutHandle.java index ce00711f3185..86061711f52d 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveTableLayoutHandle.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveTableLayoutHandle.java @@ -24,8 +24,8 @@ import static com.facebook.presto.common.plan.PlanCanonicalizationStrategy.CONNECTOR; import static com.facebook.presto.common.predicate.Domain.singleValue; import static com.facebook.presto.common.type.VarcharType.VARCHAR; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveTableLayoutHandle.canonicalizeDomainPredicate; import static com.facebook.presto.hive.HiveType.HIVE_STRING; import static io.airlift.slice.Slices.utf8Slice; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestIonSqlQueryBuilder.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestIonSqlQueryBuilder.java index 94c03cf84f0a..2690e5a430b8 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestIonSqlQueryBuilder.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestIonSqlQueryBuilder.java @@ -38,7 +38,7 @@ import static com.facebook.presto.common.type.StandardTypes.TIMESTAMP; import static com.facebook.presto.common.type.StandardTypes.VARCHAR; import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveTestUtils.longDecimal; import static com.facebook.presto.hive.HiveTestUtils.shortDecimal; import static com.facebook.presto.hive.HiveType.HIVE_DATE; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestJsonHiveHandles.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestJsonHiveHandles.java index acaf1e9421c9..7538783a47e5 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestJsonHiveHandles.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestJsonHiveHandles.java @@ -28,7 +28,7 @@ import static com.facebook.airlift.testing.Assertions.assertEqualsIgnoreOrder; import static com.facebook.presto.common.type.DoubleType.DOUBLE; import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestMergeRequestedAndPredicateColumns.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestMergeRequestedAndPredicateColumns.java index bb19cfe236f1..f87da4fb8ed6 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestMergeRequestedAndPredicateColumns.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestMergeRequestedAndPredicateColumns.java @@ -24,7 +24,7 @@ import static com.facebook.presto.common.type.StandardTypes.BIGINT; import static com.facebook.presto.common.type.StandardTypes.VARCHAR; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveSplitManager.mergeRequestedAndPredicateColumns; import static com.facebook.presto.hive.HiveType.HIVE_LONG; import static com.facebook.presto.hive.HiveType.HIVE_STRING; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestOrcBatchPageSourceMemoryTracking.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestOrcBatchPageSourceMemoryTracking.java index e22fe6668295..7cfbcf738062 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestOrcBatchPageSourceMemoryTracking.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestOrcBatchPageSourceMemoryTracking.java @@ -103,8 +103,8 @@ import static com.facebook.airlift.concurrent.Threads.daemonThreadsNamed; import static com.facebook.airlift.testing.Assertions.assertBetweenInclusive; import static com.facebook.presto.common.type.VarcharType.createUnboundedVarcharType; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveFileContext.DEFAULT_HIVE_FILE_CONTEXT; import static com.facebook.presto.hive.HiveTestUtils.FUNCTION_AND_TYPE_MANAGER; import static com.facebook.presto.hive.HiveTestUtils.FUNCTION_RESOLUTION; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java b/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java index c662f20e955c..53ed9ef76587 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java @@ -81,8 +81,8 @@ import java.util.OptionalLong; import java.util.Properties; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.CacheQuota.NO_CACHE_CONSTRAINTS; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveCompressionCodec.NONE; import static com.facebook.presto.hive.HiveStorageFormat.PAGEFILE; import static com.facebook.presto.hive.HiveTestUtils.FUNCTION_AND_TYPE_MANAGER; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/TestParquetPageSourceFactory.java b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/TestParquetPageSourceFactory.java index 98431919c6c4..8aa6e74ddab0 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/TestParquetPageSourceFactory.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/TestParquetPageSourceFactory.java @@ -27,8 +27,8 @@ import java.util.Optional; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.SYNTHESIZED; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.SYNTHESIZED; import static com.facebook.presto.hive.parquet.ParquetPageSourceFactory.getColumnType; import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/predicate/TestParquetPredicateUtils.java b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/predicate/TestParquetPredicateUtils.java index cf49eac45981..4a14d1d7d61b 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/predicate/TestParquetPredicateUtils.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/predicate/TestParquetPredicateUtils.java @@ -40,7 +40,7 @@ import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.common.type.IntegerType.INTEGER; import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.parquet.ParquetPageSourceFactory.getParquetTupleDomain; import static com.facebook.presto.parquet.ParquetTypeUtils.getDescriptors; import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/s3select/TestS3SelectRecordCursor.java b/presto-hive/src/test/java/com/facebook/presto/hive/s3select/TestS3SelectRecordCursor.java index bdbdef0cb8bb..cf38952ac69c 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/s3select/TestS3SelectRecordCursor.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/s3select/TestS3SelectRecordCursor.java @@ -32,8 +32,8 @@ import java.util.stream.Stream; import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveType.HIVE_INT; import static com.facebook.presto.hive.HiveType.HIVE_STRING; import static com.facebook.presto.hive.s3select.S3SelectRecordCursor.updateSplitSchema; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/statistics/TestMetastoreHiveStatisticsProvider.java b/presto-hive/src/test/java/com/facebook/presto/hive/statistics/TestMetastoreHiveStatisticsProvider.java index 44944c82d02d..31f0189df7b5 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/statistics/TestMetastoreHiveStatisticsProvider.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/statistics/TestMetastoreHiveStatisticsProvider.java @@ -58,8 +58,8 @@ import static com.facebook.presto.common.type.SmallintType.SMALLINT; import static com.facebook.presto.common.type.TinyintType.TINYINT; import static com.facebook.presto.common.type.VarcharType.VARCHAR; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveErrorCode.HIVE_CORRUPTED_COLUMN_STATISTICS; import static com.facebook.presto.hive.HivePartition.UNPARTITIONED_ID; import static com.facebook.presto.hive.HivePartitionManager.parsePartition; diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergColumnHandle.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergColumnHandle.java index 8bb4cf4e8f7a..14a410db75e3 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergColumnHandle.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergColumnHandle.java @@ -28,10 +28,10 @@ import java.util.Objects; import java.util.Optional; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.SYNTHESIZED; import static com.facebook.presto.iceberg.ColumnIdentity.createColumnIdentity; import static com.facebook.presto.iceberg.ColumnIdentity.primitiveColumnIdentity; -import static com.facebook.presto.iceberg.IcebergColumnHandle.ColumnType.REGULAR; -import static com.facebook.presto.iceberg.IcebergColumnHandle.ColumnType.SYNTHESIZED; import static com.facebook.presto.iceberg.TypeConverter.toPrestoType; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.Iterables.getOnlyElement; @@ -40,13 +40,10 @@ import static org.apache.iceberg.MetadataColumns.ROW_POSITION; public class IcebergColumnHandle - implements BaseHiveColumnHandle + extends BaseHiveColumnHandle { private final ColumnIdentity columnIdentity; private final Type type; - private final Optional comment; - private final ColumnType columnType; - private final List requiredSubfields; @JsonCreator public IcebergColumnHandle( @@ -56,11 +53,10 @@ public IcebergColumnHandle( @JsonProperty("columnType") ColumnType columnType, @JsonProperty("requiredSubfields") List requiredSubfields) { + super(columnIdentity.getName(), comment, columnType, requiredSubfields); + this.columnIdentity = requireNonNull(columnIdentity, "columnIdentity is null"); this.type = requireNonNull(type, "type is null"); - this.comment = requireNonNull(comment, "comment is null"); - this.columnType = requireNonNull(columnType, "columnType is null"); - this.requiredSubfields = requireNonNull(requiredSubfields, "requiredSubfields is null"); } public IcebergColumnHandle(ColumnIdentity columnIdentity, Type type, Optional comment, ColumnType columnType) @@ -80,36 +76,12 @@ public int getId() return columnIdentity.getId(); } - @JsonProperty - public String getName() - { - return columnIdentity.getName(); - } - @JsonProperty public Type getType() { return type; } - @JsonProperty - public Optional getComment() - { - return comment; - } - - @JsonProperty - public ColumnType getColumnType() - { - return columnType; - } - - @JsonProperty - public List getRequiredSubfields() - { - return requiredSubfields; - } - @JsonIgnore public boolean isRowPositionColumn() { @@ -124,13 +96,13 @@ public ColumnHandle withRequiredSubfields(List subfields) return this; } - return new IcebergColumnHandle(columnIdentity, type, comment, columnType, subfields); + return new IcebergColumnHandle(columnIdentity, type, getComment(), getColumnType(), subfields); } @Override public int hashCode() { - return Objects.hash(columnIdentity, type, comment, columnType, requiredSubfields); + return Objects.hash(columnIdentity, type, getComment(), getColumnType(), getRequiredSubfields()); } @Override @@ -145,19 +117,19 @@ public boolean equals(Object obj) IcebergColumnHandle other = (IcebergColumnHandle) obj; return Objects.equals(this.columnIdentity, other.columnIdentity) && Objects.equals(this.type, other.type) && - Objects.equals(this.comment, other.comment) && - Objects.equals(this.columnType, other.columnType) && - Objects.equals(this.requiredSubfields, other.requiredSubfields); + Objects.equals(this.getComment(), other.getComment()) && + Objects.equals(this.getColumnType(), other.getColumnType()) && + Objects.equals(this.getRequiredSubfields(), other.getRequiredSubfields()); } @Override public String toString() { - if (requiredSubfields.isEmpty()) { + if (getRequiredSubfields().isEmpty()) { return getId() + ":" + getName() + ":" + type.getDisplayName(); } - return getId() + ":" + getName() + ":" + type.getDisplayName() + ":" + columnType + ":" + requiredSubfields; + return getId() + ":" + getName() + ":" + type.getDisplayName() + ":" + getColumnType() + ":" + getRequiredSubfields(); } public static IcebergColumnHandle primitiveIcebergColumnHandle(int id, String name, Type type, Optional comment) @@ -174,13 +146,6 @@ public static IcebergColumnHandle create(Types.NestedField column, TypeManager t columnType); } - public enum ColumnType - { - PARTITION_KEY, - REGULAR, - SYNTHESIZED - } - public static Subfield getPushedDownSubfield(IcebergColumnHandle column) { checkArgument(isPushedDownSubfield(column), format("not a valid pushed down subfield: type=%s, subfields=%s", column.getColumnType(), column.getRequiredSubfields())); diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergOrcColumn.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergOrcColumn.java index 5ce39b6b9042..88aa52c7891c 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergOrcColumn.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergOrcColumn.java @@ -13,7 +13,7 @@ */ package com.facebook.presto.iceberg; -import com.facebook.presto.hive.HiveColumnHandle.ColumnType; +import com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType; import com.facebook.presto.orc.metadata.OrcType.OrcTypeKind; import com.google.common.collect.ImmutableMap; diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergPageSourceProvider.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergPageSourceProvider.java index a70d7352cfe6..84c46b5c2d18 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergPageSourceProvider.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergPageSourceProvider.java @@ -118,9 +118,9 @@ import java.util.stream.IntStream; import static com.facebook.presto.common.type.VarcharType.VARCHAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.SYNTHESIZED; import static com.facebook.presto.hive.CacheQuota.NO_CACHE_CONSTRAINTS; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; -import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.SYNTHESIZED; import static com.facebook.presto.hive.HiveCommonSessionProperties.getOrcLazyReadSmallRanges; import static com.facebook.presto.hive.HiveCommonSessionProperties.getOrcMaxBufferSize; import static com.facebook.presto.hive.HiveCommonSessionProperties.getOrcMaxMergeDistance; diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java index 670070f25e41..54e31adf06ea 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java @@ -103,6 +103,8 @@ import static com.facebook.presto.common.type.TinyintType.TINYINT; import static com.facebook.presto.common.type.VarbinaryType.VARBINARY; import static com.facebook.presto.common.type.Varchars.isVarcharType; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveUtil.bigintPartitionKey; import static com.facebook.presto.hive.HiveUtil.booleanPartitionKey; import static com.facebook.presto.hive.HiveUtil.charPartitionKey; @@ -122,8 +124,6 @@ import static com.facebook.presto.hive.metastore.MetastoreUtil.PRESTO_VIEW_FLAG; import static com.facebook.presto.hive.metastore.MetastoreUtil.TABLE_COMMENT; import static com.facebook.presto.iceberg.ExpressionConverter.toIcebergExpression; -import static com.facebook.presto.iceberg.IcebergColumnHandle.ColumnType.PARTITION_KEY; -import static com.facebook.presto.iceberg.IcebergColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.iceberg.IcebergErrorCode.ICEBERG_INVALID_PARTITION_VALUE; import static com.facebook.presto.iceberg.IcebergErrorCode.ICEBERG_INVALID_SNAPSHOT_ID; import static com.facebook.presto.iceberg.IcebergErrorCode.ICEBERG_INVALID_TABLE_TIMESTAMP; diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergColumnHandle.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergColumnHandle.java index 3266da510d18..bb4599f44b01 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergColumnHandle.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergColumnHandle.java @@ -28,10 +28,10 @@ import java.util.Optional; import static com.facebook.presto.common.type.BigintType.BIGINT; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.iceberg.ColumnIdentity.TypeCategory.ARRAY; import static com.facebook.presto.iceberg.ColumnIdentity.TypeCategory.PRIMITIVE; import static com.facebook.presto.iceberg.ColumnIdentity.TypeCategory.STRUCT; -import static com.facebook.presto.iceberg.IcebergColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.iceberg.IcebergColumnHandle.primitiveIcebergColumnHandle; import static com.facebook.presto.metadata.FunctionAndTypeManager.createTestFunctionAndTypeManager; import static org.testng.Assert.assertEquals; diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergLogicalPlanner.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergLogicalPlanner.java index ed0e0f8a0ef6..3075621afee3 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergLogicalPlanner.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergLogicalPlanner.java @@ -69,8 +69,8 @@ import static com.facebook.presto.common.type.IntegerType.INTEGER; import static com.facebook.presto.common.type.VarcharType.VARCHAR; import static com.facebook.presto.expressions.LogicalRowExpressions.TRUE_CONSTANT; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.SYNTHESIZED; import static com.facebook.presto.iceberg.IcebergAbstractMetadata.isEntireColumn; -import static com.facebook.presto.iceberg.IcebergColumnHandle.ColumnType.SYNTHESIZED; import static com.facebook.presto.iceberg.IcebergColumnHandle.getSynthesizedIcebergColumnHandle; import static com.facebook.presto.iceberg.IcebergColumnHandle.isPushedDownSubfield; import static com.facebook.presto.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestStatisticsUtil.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestStatisticsUtil.java index b197b72f65aa..6d412d20bcda 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestStatisticsUtil.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestStatisticsUtil.java @@ -32,7 +32,7 @@ import java.util.OptionalLong; import static com.facebook.presto.common.type.IntegerType.INTEGER; -import static com.facebook.presto.iceberg.IcebergColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.iceberg.util.HiveStatisticsMergeStrategy.NONE; import static com.facebook.presto.iceberg.util.HiveStatisticsMergeStrategy.USE_NDV; import static com.facebook.presto.iceberg.util.HiveStatisticsMergeStrategy.USE_NULLS_FRACTIONS; diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/hive/TestIcebergHiveStatistics.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/hive/TestIcebergHiveStatistics.java index 8e70bc9ff952..09bca1ef506a 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/hive/TestIcebergHiveStatistics.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/hive/TestIcebergHiveStatistics.java @@ -46,8 +46,8 @@ import java.util.stream.Collectors; import static com.facebook.presto.common.type.DoubleType.DOUBLE; -import static com.facebook.presto.iceberg.IcebergColumnHandle.ColumnType.PARTITION_KEY; -import static com.facebook.presto.iceberg.IcebergColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.PARTITION_KEY; +import static com.facebook.presto.hive.BaseHiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.iceberg.IcebergQueryRunner.createIcebergQueryRunner; import static com.facebook.presto.testing.assertions.Assert.assertEquals; import static org.testng.Assert.assertFalse; @@ -286,7 +286,7 @@ private void columnStatsEqual(Map actualStats, M ((IcebergColumnHandle) handle).getType(), ((IcebergColumnHandle) handle).getComment(), REGULAR, - ((IcebergColumnHandle) handle).getRequiredSubfields()); + handle.getRequiredSubfields()); } ColumnStatistics actual = actualStats.get(handle); assertEquals(actual.getRange(), expected.getRange(), "range for col: " + handle);