From 933ac095c303b85e1b2f452098d57e5e478d626e Mon Sep 17 00:00:00 2001 From: David Phillips Date: Wed, 31 Jul 2024 00:10:50 -0700 Subject: [PATCH 01/12] Remove unused metastore classes --- .../hive/metastore/DatabaseFunctionKey.java | 25 ------ .../DatabaseFunctionSignatureKey.java | 29 ------- .../hive/metastore/UserDatabaseKey.java | 78 ------------------- 3 files changed, 132 deletions(-) delete mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/DatabaseFunctionKey.java delete mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/DatabaseFunctionSignatureKey.java delete mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/UserDatabaseKey.java diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/DatabaseFunctionKey.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/DatabaseFunctionKey.java deleted file mode 100644 index 1237525abb70..000000000000 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/DatabaseFunctionKey.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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.metastore; - -import static java.util.Objects.requireNonNull; - -public record DatabaseFunctionKey(String databaseName, String functionName) -{ - public DatabaseFunctionKey - { - requireNonNull(databaseName, "databaseName is null"); - requireNonNull(functionName, "functionName is null"); - } -} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/DatabaseFunctionSignatureKey.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/DatabaseFunctionSignatureKey.java deleted file mode 100644 index 905d23fcdd50..000000000000 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/DatabaseFunctionSignatureKey.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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.metastore; - -import static java.util.Objects.requireNonNull; - -public record DatabaseFunctionSignatureKey( - String databaseName, - String functionName, - String signatureToken) -{ - public DatabaseFunctionSignatureKey - { - requireNonNull(databaseName, "databaseName is null"); - requireNonNull(functionName, "functionName is null"); - requireNonNull(signatureToken, "signatureToken is null"); - } -} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/UserDatabaseKey.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/UserDatabaseKey.java deleted file mode 100644 index d25645aa5e0f..000000000000 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/UserDatabaseKey.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * 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.metastore; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.errorprone.annotations.Immutable; - -import java.util.Objects; - -import static com.google.common.base.MoreObjects.toStringHelper; -import static java.util.Objects.requireNonNull; - -@Immutable -public class UserDatabaseKey -{ - private final String user; - private final String database; - - @JsonCreator - public UserDatabaseKey(@JsonProperty("user") String user, @JsonProperty("database") String database) - { - this.user = requireNonNull(user, "user is null"); - this.database = requireNonNull(database, "database is null"); - } - - @JsonProperty - public String getUser() - { - return user; - } - - @JsonProperty - public String getDatabase() - { - return database; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - UserDatabaseKey that = (UserDatabaseKey) o; - return Objects.equals(user, that.user) && - Objects.equals(database, that.database); - } - - @Override - public int hashCode() - { - return Objects.hash(user, database); - } - - @Override - public String toString() - { - return toStringHelper(this) - .add("principalName", user) - .add("database", database) - .toString(); - } -} From be7cda982be71d9f78959d81d7d1f7da6ba98d6d Mon Sep 17 00:00:00 2001 From: David Phillips Date: Wed, 31 Jul 2024 00:23:19 -0700 Subject: [PATCH 02/12] Remove unused TestingIcebergHiveMetastoreCatalogModule --- ...tingIcebergHiveMetastoreCatalogModule.java | 60 ------------------- 1 file changed, 60 deletions(-) delete mode 100644 plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestingIcebergHiveMetastoreCatalogModule.java diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestingIcebergHiveMetastoreCatalogModule.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestingIcebergHiveMetastoreCatalogModule.java deleted file mode 100644 index 9bd5e87b1654..000000000000 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestingIcebergHiveMetastoreCatalogModule.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.iceberg.catalog.hms; - -import com.google.inject.Binder; -import com.google.inject.Scopes; -import io.airlift.configuration.AbstractConfigurationAwareModule; -import io.airlift.units.Duration; -import io.trino.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.CachingHiveMetastoreModule; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; -import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastoreConfig; -import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreFactory; -import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; -import io.trino.plugin.iceberg.catalog.TrinoCatalogFactory; - -import java.util.concurrent.TimeUnit; - -import static io.airlift.configuration.ConfigBinder.configBinder; -import static java.util.Objects.requireNonNull; - -public class TestingIcebergHiveMetastoreCatalogModule - extends AbstractConfigurationAwareModule -{ - private final HiveMetastore hiveMetastore; - private final ThriftMetastoreFactory thriftMetastoreFactory; - - public TestingIcebergHiveMetastoreCatalogModule(HiveMetastore hiveMetastore, ThriftMetastoreFactory thriftMetastoreFactory) - { - this.hiveMetastore = requireNonNull(hiveMetastore, "hiveMetastore is null"); - this.thriftMetastoreFactory = requireNonNull(thriftMetastoreFactory, "thriftMetastoreFactory is null"); - } - - @Override - protected void setup(Binder binder) - { - install(new CachingHiveMetastoreModule(false)); - binder.bind(ThriftMetastoreFactory.class).toInstance(this.thriftMetastoreFactory); - binder.bind(HiveMetastoreFactory.class).annotatedWith(RawHiveMetastoreFactory.class).toInstance(HiveMetastoreFactory.ofInstance(this.hiveMetastore)); - binder.bind(IcebergTableOperationsProvider.class).to(HiveMetastoreTableOperationsProvider.class).in(Scopes.SINGLETON); - binder.bind(TrinoCatalogFactory.class).to(TrinoHiveCatalogFactory.class).in(Scopes.SINGLETON); - - configBinder(binder).bindConfigDefaults(CachingHiveMetastoreConfig.class, config -> { - // ensure caching metastore wrapper isn't created, as it's not leveraged by Iceberg - config.setStatsCacheTtl(new Duration(0, TimeUnit.SECONDS)); - }); - } -} From 8cddf32165b45adc63ed90dfd6cf09ad8d684db2 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Wed, 31 Jul 2024 10:32:34 -0700 Subject: [PATCH 03/12] Replace usage of RetryDriver in HiveMetadata --- .../main/java/io/trino/plugin/hive/HiveMetadata.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java index 6b958cd2c694..4d0996f6724b 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java @@ -24,6 +24,8 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import dev.failsafe.Failsafe; +import dev.failsafe.RetryPolicy; import io.airlift.json.JsonCodec; import io.airlift.log.Logger; import io.airlift.slice.Slice; @@ -319,7 +321,6 @@ import static io.trino.plugin.hive.util.HiveWriteUtils.createPartitionValues; import static io.trino.plugin.hive.util.HiveWriteUtils.isFileCreatedByQuery; import static io.trino.plugin.hive.util.HiveWriteUtils.isWritableType; -import static io.trino.plugin.hive.util.RetryDriver.retry; import static io.trino.plugin.hive.util.Statistics.createComputedStatisticsToPartitionMap; import static io.trino.plugin.hive.util.Statistics.createEmptyPartitionStatistics; import static io.trino.plugin.hive.util.Statistics.fromComputedStatistics; @@ -388,6 +389,12 @@ public class HiveMetadata public static final String MODIFYING_NON_TRANSACTIONAL_TABLE_MESSAGE = "Modifying Hive table rows is only supported for transactional tables"; + private static final RetryPolicy DELETE_RETRY_POLICY = RetryPolicy.builder() + .withDelay(java.time.Duration.ofSeconds(1)) + .withMaxDuration(java.time.Duration.ofSeconds(30)) + .withMaxAttempts(10) + .build(); + private final CatalogName catalogName; private final SemiTransactionalHiveMetastore metastore; private final boolean autoCommit; @@ -2687,14 +2694,13 @@ private void finishOptimize(ConnectorSession session, ConnectorTableExecuteHandl if (firstScannedPath.isEmpty()) { firstScannedPath = Optional.of(scannedPath); } - retry().run("delete " + scannedPath, () -> { + Failsafe.with(DELETE_RETRY_POLICY).run(() -> { try { fileSystem.deleteFile(scannedPath); } catch (FileNotFoundException e) { // ignore missing files } - return null; }); someDeleted = true; remainingFilesToDelete.remove(scannedPath); From c8efb1b6d9a754d38aabaf0b9308d650e43e9562 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Tue, 30 Jul 2024 23:40:24 -0700 Subject: [PATCH 04/12] Move partition utility methods to Partitions class --- .../java/io/trino/metastore/Partition.java | 64 -------- .../metastore/PartitionWithStatistics.java | 2 +- .../java/io/trino/metastore/Partitions.java | 154 ++++++++++++++++++ .../io/trino/metastore/TestPartitions.java | 116 +++++++++++++ .../deltalake/AbstractDeltaLakePageSink.java | 10 +- .../deltalake/util/DeltaLakeWriteUtils.java | 2 +- .../plugin/hive/HiveAnalyzeProperties.java | 2 +- .../io/trino/plugin/hive/HiveMetadata.java | 5 +- .../trino/plugin/hive/HivePartitionKey.java | 3 +- .../plugin/hive/HivePartitionManager.java | 2 +- .../trino/plugin/hive/HiveWriterFactory.java | 2 +- .../hive/metastore/HivePartitionName.java | 2 +- .../plugin/hive/metastore/MetastoreUtil.java | 2 +- .../SemiTransactionalHiveMetastore.java | 4 +- .../metastore/cache/CachingHiveMetastore.java | 2 +- .../metastore/file/FileHiveMetastore.java | 6 +- .../DefaultGlueColumnStatisticsProvider.java | 2 +- .../metastore/glue/v1/GlueHiveMetastore.java | 5 +- .../thrift/BridgingHiveMetastore.java | 3 +- .../CreateEmptyPartitionProcedure.java | 2 +- .../hive/procedure/DropStatsProcedure.java | 2 +- .../FlushMetadataCacheProcedure.java | 2 +- .../procedure/RegisterPartitionProcedure.java | 2 +- .../UnregisterPartitionProcedure.java | 2 +- .../hive/projection/PartitionProjection.java | 4 +- .../io/trino/plugin/hive/util/HiveUtil.java | 65 +------- .../plugin/hive/util/HiveWriteUtils.java | 2 +- .../plugin/hive/TestHiveFileFormats.java | 2 +- .../TestMetastoreHiveStatisticsProvider.java | 2 +- .../trino/plugin/hive/util/TestHiveUtil.java | 96 ----------- .../plugin/hudi/HudiPageSourceProvider.java | 2 +- .../hudi/partition/HiveHudiPartitionInfo.java | 5 +- 32 files changed, 314 insertions(+), 262 deletions(-) create mode 100644 lib/trino-metastore/src/main/java/io/trino/metastore/Partitions.java create mode 100644 lib/trino-metastore/src/test/java/io/trino/metastore/TestPartitions.java diff --git a/lib/trino-metastore/src/main/java/io/trino/metastore/Partition.java b/lib/trino-metastore/src/main/java/io/trino/metastore/Partition.java index ed3f5dddac88..f24395ee56ff 100644 --- a/lib/trino-metastore/src/main/java/io/trino/metastore/Partition.java +++ b/lib/trino-metastore/src/main/java/io/trino/metastore/Partition.java @@ -21,7 +21,6 @@ import com.google.errorprone.annotations.Immutable; import io.trino.spi.connector.SchemaTableName; -import java.util.HexFormat; import java.util.List; import java.util.Map; import java.util.Objects; @@ -221,67 +220,4 @@ public Partition build() return new Partition(databaseName, tableName, values, storageBuilder.build(), columns, parameters); } } - - public static List toPartitionValues(String partitionName) - { - // mimics Warehouse.makeValsFromName - ImmutableList.Builder resultBuilder = ImmutableList.builder(); - int start = 0; - while (true) { - while (start < partitionName.length() && partitionName.charAt(start) != '=') { - start++; - } - start++; - int end = start; - while (end < partitionName.length() && partitionName.charAt(end) != '/') { - end++; - } - if (start > partitionName.length()) { - break; - } - resultBuilder.add(unescapePathName(partitionName.substring(start, end))); - start = end + 1; - } - return resultBuilder.build(); - } - - // copy of org.apache.hadoop.hive.common.FileUtils#unescapePathName - @SuppressWarnings("NumericCastThatLosesPrecision") - public static String unescapePathName(String path) - { - // fast path, no escaped characters and therefore no copying necessary - int escapedAtIndex = path.indexOf('%'); - if (escapedAtIndex < 0 || escapedAtIndex + 2 >= path.length()) { - return path; - } - - // slow path, unescape into a new string copy - StringBuilder sb = new StringBuilder(); - int fromIndex = 0; - while (escapedAtIndex >= 0 && escapedAtIndex + 2 < path.length()) { - // preceding sequence without escaped characters - if (escapedAtIndex > fromIndex) { - sb.append(path, fromIndex, escapedAtIndex); - } - // try to parse the to digits after the percent sign as hex - try { - int code = HexFormat.fromHexDigits(path, escapedAtIndex + 1, escapedAtIndex + 3); - sb.append((char) code); - // advance past the percent sign and both hex digits - fromIndex = escapedAtIndex + 3; - } - catch (NumberFormatException e) { - // invalid escape sequence, only advance past the percent sign - sb.append('%'); - fromIndex = escapedAtIndex + 1; - } - // find next escaped character - escapedAtIndex = path.indexOf('%', fromIndex); - } - // trailing sequence without escaped characters - if (fromIndex < path.length()) { - sb.append(path, fromIndex, path.length()); - } - return sb.toString(); - } } diff --git a/lib/trino-metastore/src/main/java/io/trino/metastore/PartitionWithStatistics.java b/lib/trino-metastore/src/main/java/io/trino/metastore/PartitionWithStatistics.java index 77c136c0e6f1..fa695bb5d410 100644 --- a/lib/trino-metastore/src/main/java/io/trino/metastore/PartitionWithStatistics.java +++ b/lib/trino-metastore/src/main/java/io/trino/metastore/PartitionWithStatistics.java @@ -14,7 +14,7 @@ package io.trino.metastore; import static com.google.common.base.Preconditions.checkArgument; -import static io.trino.metastore.Partition.toPartitionValues; +import static io.trino.metastore.Partitions.toPartitionValues; import static java.util.Objects.requireNonNull; public class PartitionWithStatistics diff --git a/lib/trino-metastore/src/main/java/io/trino/metastore/Partitions.java b/lib/trino-metastore/src/main/java/io/trino/metastore/Partitions.java new file mode 100644 index 000000000000..f6450f32a50a --- /dev/null +++ b/lib/trino-metastore/src/main/java/io/trino/metastore/Partitions.java @@ -0,0 +1,154 @@ +/* + * 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.metastore; + +import com.google.common.base.CharMatcher; +import com.google.common.collect.ImmutableList; + +import java.util.HexFormat; +import java.util.List; + +import static com.google.common.base.Strings.isNullOrEmpty; +import static java.util.Locale.ENGLISH; + +public final class Partitions +{ + public static final String HIVE_DEFAULT_DYNAMIC_PARTITION = "__HIVE_DEFAULT_PARTITION__"; + + private static final HexFormat HEX_UPPER_FORMAT = HexFormat.of().withUpperCase(); + + private static final CharMatcher PATH_CHAR_TO_ESCAPE = CharMatcher.inRange((char) 0, (char) 31) + .or(CharMatcher.anyOf("\"#%'*/:=?\\\u007F{[]^")) + .precomputed(); + + private Partitions() {} + + public static List toPartitionValues(String partitionName) + { + // mimics Warehouse.makeValsFromName + ImmutableList.Builder resultBuilder = ImmutableList.builder(); + int start = 0; + while (true) { + while (start < partitionName.length() && partitionName.charAt(start) != '=') { + start++; + } + start++; + int end = start; + while (end < partitionName.length() && partitionName.charAt(end) != '/') { + end++; + } + if (start > partitionName.length()) { + break; + } + resultBuilder.add(unescapePathName(partitionName.substring(start, end))); + start = end + 1; + } + return resultBuilder.build(); + } + + // copy of org.apache.hadoop.hive.common.FileUtils#unescapePathName + @SuppressWarnings("NumericCastThatLosesPrecision") + public static String unescapePathName(String path) + { + // fast path, no escaped characters and therefore no copying necessary + int escapedAtIndex = path.indexOf('%'); + if (escapedAtIndex < 0 || escapedAtIndex + 2 >= path.length()) { + return path; + } + + // slow path, unescape into a new string copy + StringBuilder sb = new StringBuilder(); + int fromIndex = 0; + while (escapedAtIndex >= 0 && escapedAtIndex + 2 < path.length()) { + // preceding sequence without escaped characters + if (escapedAtIndex > fromIndex) { + sb.append(path, fromIndex, escapedAtIndex); + } + // try to parse the to digits after the percent sign as hex + try { + int code = HexFormat.fromHexDigits(path, escapedAtIndex + 1, escapedAtIndex + 3); + sb.append((char) code); + // advance past the percent sign and both hex digits + fromIndex = escapedAtIndex + 3; + } + catch (NumberFormatException e) { + // invalid escape sequence, only advance past the percent sign + sb.append('%'); + fromIndex = escapedAtIndex + 1; + } + // find next escaped character + escapedAtIndex = path.indexOf('%', fromIndex); + } + // trailing sequence without escaped characters + if (fromIndex < path.length()) { + sb.append(path, fromIndex, path.length()); + } + return sb.toString(); + } + + // copy of org.apache.hadoop.hive.common.FileUtils#escapePathName + public static String escapePathName(String path) + { + if (isNullOrEmpty(path)) { + return HIVE_DEFAULT_DYNAMIC_PARTITION; + } + + // Fast-path detection, no escaping and therefore no copying necessary + int escapeAtIndex = PATH_CHAR_TO_ESCAPE.indexIn(path); + if (escapeAtIndex < 0) { + return path; + } + + // slow path, escape beyond the first required escape character into a new string + StringBuilder sb = new StringBuilder(); + int fromIndex = 0; + while (escapeAtIndex >= 0 && escapeAtIndex < path.length()) { + // preceding characters without escaping needed + if (escapeAtIndex > fromIndex) { + sb.append(path, fromIndex, escapeAtIndex); + } + // escape single character + char c = path.charAt(escapeAtIndex); + sb.append('%').append(HEX_UPPER_FORMAT.toHighHexDigit(c)).append(HEX_UPPER_FORMAT.toLowHexDigit(c)); + // find next character to escape + fromIndex = escapeAtIndex + 1; + if (fromIndex < path.length()) { + escapeAtIndex = PATH_CHAR_TO_ESCAPE.indexIn(path, fromIndex); + } + else { + escapeAtIndex = -1; + } + } + // trailing characters without escaping needed + if (fromIndex < path.length()) { + sb.append(path, fromIndex, path.length()); + } + return sb.toString(); + } + + // copy of org.apache.hadoop.hive.common.FileUtils#makePartName + public static String makePartName(List columns, List values) + { + StringBuilder name = new StringBuilder(); + for (int i = 0; i < columns.size(); i++) { + if (i > 0) { + name.append('/'); + } + name.append(escapePathName(columns.get(i).toLowerCase(ENGLISH))); + name.append('='); + name.append(escapePathName(values.get(i))); + } + return name.toString(); + } +} diff --git a/lib/trino-metastore/src/test/java/io/trino/metastore/TestPartitions.java b/lib/trino-metastore/src/test/java/io/trino/metastore/TestPartitions.java new file mode 100644 index 000000000000..c19a3afc5cd8 --- /dev/null +++ b/lib/trino-metastore/src/test/java/io/trino/metastore/TestPartitions.java @@ -0,0 +1,116 @@ +/* + * 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.metastore; + +import org.apache.hadoop.hive.common.FileUtils; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.junit.jupiter.api.Test; + +import java.util.AbstractList; +import java.util.ArrayList; +import java.util.List; + +import static io.trino.metastore.Partitions.toPartitionValues; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestPartitions +{ + @Test + public void testToPartitionValues() + throws MetaException + { + assertToPartitionValues("ds=2015-12-30/event_type=QueryCompletion"); + assertToPartitionValues("ds=2015-12-30"); + assertToPartitionValues("a=1/b=2/c=3"); + assertToPartitionValues("a=1"); + assertToPartitionValues("pk=!@%23$%25%5E&%2A()%2F%3D"); + assertToPartitionValues("pk=__HIVE_DEFAULT_PARTITION__"); + } + + private static void assertToPartitionValues(String partitionName) + throws MetaException + { + List actual = toPartitionValues(partitionName); + AbstractList expected = new ArrayList<>(); + actual.forEach(s -> expected.add(null)); + Warehouse.makeValsFromName(partitionName, expected); + assertThat(actual).isEqualTo(expected); + } + + @Test + public void testUnescapePathName() + { + assertUnescapePathName("", ""); + assertUnescapePathName("x", "x"); + assertUnescapePathName("abc", "abc"); + assertUnescapePathName("abc%", "abc%"); + assertUnescapePathName("%", "%"); + assertUnescapePathName("%41", "A"); + assertUnescapePathName("%41%x", "A%x"); + assertUnescapePathName("%41%xxZ", "A%xxZ"); + assertUnescapePathName("%41%%Z", "A%%Z"); + assertUnescapePathName("%41%25%25Z", "A%%Z"); + assertUnescapePathName("abc%41%42%43", "abcABC"); + assertUnescapePathName("abc%3Axyz", "abc:xyz"); + assertUnescapePathName("abc%3axyz", "abc:xyz"); + assertUnescapePathName("abc%BBxyz", "abc\u00BBxyz"); + } + + private static void assertUnescapePathName(String value, String expected) + { + assertThat(FileUtils.unescapePathName(value)).isEqualTo(expected); + assertThat(Partitions.unescapePathName(value)).isEqualTo(expected); + } + + @Test + public void testEscapePathName() + { + assertEscapePathName(null, "__HIVE_DEFAULT_PARTITION__"); + assertEscapePathName("", "__HIVE_DEFAULT_PARTITION__"); + assertEscapePathName("x", "x"); + assertEscapePathName("abc", "abc"); + assertEscapePathName("%", "%25"); + assertEscapePathName("A", "A"); + assertEscapePathName("A%x", "A%25x"); + assertEscapePathName("A%xxZ", "A%25xxZ"); + assertEscapePathName("A%%Z", "A%25%25Z"); + assertEscapePathName("abcABC", "abcABC"); + assertEscapePathName("abc:xyz", "abc%3Axyz"); + assertEscapePathName("abc\u00BBxyz", "abc\u00BBxyz"); + assertEscapePathName("\u0000\t\b\r\n\u001F", "%00%09%08%0D%0A%1F"); + assertEscapePathName("#%^&*=[]{\\:'\"/?", "%23%25%5E&%2A%3D%5B%5D%7B%5C%3A%27%22%2F%3F"); + assertEscapePathName("~`!@$()-_+}|;,.<>", "~`!@$()-_+}|;,.<>"); + } + + private static void assertEscapePathName(String value, String expected) + { + assertThat(FileUtils.escapePathName(value)).isEqualTo(expected); + assertThat(Partitions.escapePathName(value)).isEqualTo(expected); + } + + @Test + public void testMakePartName() + { + assertMakePartName(List.of("abc"), List.of("xyz"), "abc=xyz"); + assertMakePartName(List.of("abc:qqq"), List.of("xyz/yyy=zzz"), "abc%3Aqqq=xyz%2Fyyy%3Dzzz"); + assertMakePartName(List.of("abc", "def", "xyz"), List.of("qqq", "rrr", "sss"), "abc=qqq/def=rrr/xyz=sss"); + } + + private static void assertMakePartName(List columns, List values, String expected) + { + assertThat(FileUtils.makePartName(columns, values)).isEqualTo(expected); + assertThat(Partitions.makePartName(columns, values)).isEqualTo(expected); + } +} diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/AbstractDeltaLakePageSink.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/AbstractDeltaLakePageSink.java index 14a0f162b55a..0029fda7970d 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/AbstractDeltaLakePageSink.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/AbstractDeltaLakePageSink.java @@ -24,12 +24,11 @@ import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.metastore.Partitions; import io.trino.parquet.writer.ParquetWriterOptions; import io.trino.plugin.deltalake.DataFileInfo.DataFileType; import io.trino.plugin.deltalake.util.DeltaLakeWriteUtils; -import io.trino.plugin.hive.HivePartitionKey; import io.trino.plugin.hive.parquet.ParquetFileWriter; -import io.trino.plugin.hive.util.HiveUtil; import io.trino.spi.Page; import io.trino.spi.PageIndexer; import io.trino.spi.PageIndexerFactory; @@ -56,13 +55,14 @@ import static com.google.common.base.Verify.verify; import static com.google.common.collect.ImmutableList.toImmutableList; import static io.airlift.slice.Slices.wrappedBuffer; +import static io.trino.metastore.Partitions.HIVE_DEFAULT_DYNAMIC_PARTITION; +import static io.trino.metastore.Partitions.escapePathName; import static io.trino.plugin.deltalake.DeltaLakeErrorCode.DELTA_LAKE_BAD_WRITE; import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.getCompressionCodec; import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.getParquetWriterBlockSize; import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.getParquetWriterPageSize; import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.getParquetWriterPageValueCount; import static io.trino.plugin.deltalake.DeltaLakeTypes.toParquetType; -import static io.trino.plugin.hive.util.HiveUtil.escapePathName; import static java.lang.Math.min; import static java.lang.String.format; import static java.util.Objects.requireNonNull; @@ -442,7 +442,7 @@ protected void closeWriter(int writerIndex) } /** - * Copy of {@link HiveUtil#makePartName} modified to preserve case of partition columns. + * Copy of {@link Partitions#makePartName} modified to preserve case of partition columns. */ private static String makePartName(List partitionColumns, List partitionValues) { @@ -464,7 +464,7 @@ private static String makePartName(List partitionColumns, List p public static List createPartitionValues(List partitionColumnTypes, Page partitionColumns, int position) { return DeltaLakeWriteUtils.createPartitionValues(partitionColumnTypes, partitionColumns, position).stream() - .map(value -> value.equals(HivePartitionKey.HIVE_DEFAULT_DYNAMIC_PARTITION) ? null : value) + .map(value -> value.equals(HIVE_DEFAULT_DYNAMIC_PARTITION) ? null : value) .collect(toList()); } diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/util/DeltaLakeWriteUtils.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/util/DeltaLakeWriteUtils.java index a30846588cf2..33c6f1997d89 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/util/DeltaLakeWriteUtils.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/util/DeltaLakeWriteUtils.java @@ -32,8 +32,8 @@ import java.util.List; import static com.google.common.io.BaseEncoding.base16; +import static io.trino.metastore.Partitions.HIVE_DEFAULT_DYNAMIC_PARTITION; import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_PARTITION_VALUE; -import static io.trino.plugin.hive.HivePartitionKey.HIVE_DEFAULT_DYNAMIC_PARTITION; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.BooleanType.BOOLEAN; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveAnalyzeProperties.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveAnalyzeProperties.java index 549fc0c32995..77f25932cbf2 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveAnalyzeProperties.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveAnalyzeProperties.java @@ -28,7 +28,7 @@ import static com.google.common.base.MoreObjects.firstNonNull; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableSet.toImmutableSet; -import static io.trino.plugin.hive.HivePartitionKey.HIVE_DEFAULT_DYNAMIC_PARTITION; +import static io.trino.metastore.Partitions.HIVE_DEFAULT_DYNAMIC_PARTITION; import static io.trino.spi.StandardErrorCode.INVALID_ANALYZE_PROPERTY; import static io.trino.spi.type.VarcharType.VARCHAR; import static java.lang.String.format; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java index 4d0996f6724b..73ff0d872439 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java @@ -45,6 +45,7 @@ import io.trino.metastore.HiveType; import io.trino.metastore.Partition; import io.trino.metastore.PartitionStatistics; +import io.trino.metastore.Partitions; import io.trino.metastore.PrincipalPrivileges; import io.trino.metastore.SortingColumn; import io.trino.metastore.StorageFormat; @@ -173,7 +174,7 @@ import static io.trino.metastore.HiveBasicStatistics.createEmptyStatistics; import static io.trino.metastore.HiveBasicStatistics.createZeroStatistics; import static io.trino.metastore.HiveType.HIVE_STRING; -import static io.trino.metastore.Partition.toPartitionValues; +import static io.trino.metastore.Partitions.toPartitionValues; import static io.trino.metastore.PrincipalPrivileges.NO_PRIVILEGES; import static io.trino.metastore.PrincipalPrivileges.fromHivePrivilegeInfos; import static io.trino.metastore.StatisticsUpdateMode.MERGE_INCREMENTAL; @@ -1694,7 +1695,7 @@ public void finishStatisticsCollection(ConnectorSession session, ConnectorTableH .orElseThrow(() -> new TableNotFoundException(tableName)); partitionValuesList = partitionNames .stream() - .map(Partition::toPartitionValues) + .map(Partitions::toPartitionValues) .collect(toImmutableList()); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionKey.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionKey.java index cfc0ee267e2f..a6dd1dcc452a 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionKey.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionKey.java @@ -15,14 +15,13 @@ import static io.airlift.slice.SizeOf.estimatedSizeOf; import static io.airlift.slice.SizeOf.instanceSize; +import static io.trino.metastore.Partitions.HIVE_DEFAULT_DYNAMIC_PARTITION; import static java.util.Objects.requireNonNull; public record HivePartitionKey(String name, String value) { private static final int INSTANCE_SIZE = instanceSize(HivePartitionKey.class); - public static final String HIVE_DEFAULT_DYNAMIC_PARTITION = "__HIVE_DEFAULT_PARTITION__"; - public HivePartitionKey { requireNonNull(name, "name is null"); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionManager.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionManager.java index fd2e7dcb873e..1bfd12de825a 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionManager.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionManager.java @@ -38,7 +38,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.metastore.Partition.unescapePathName; +import static io.trino.metastore.Partitions.unescapePathName; import static io.trino.plugin.hive.metastore.MetastoreUtil.computePartitionKeyFilter; import static io.trino.plugin.hive.metastore.MetastoreUtil.toPartitionName; import static io.trino.plugin.hive.util.HiveBucketing.getHiveBucketFilter; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriterFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriterFactory.java index 28d60c873c1a..1cdc3935d84d 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriterFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriterFactory.java @@ -65,6 +65,7 @@ import static com.google.common.collect.MoreCollectors.onlyElement; import static io.trino.hive.formats.HiveClassNames.HIVE_IGNORE_KEY_OUTPUT_FORMAT_CLASS; import static io.trino.metastore.AcidOperation.CREATE_TABLE; +import static io.trino.metastore.Partitions.makePartName; import static io.trino.plugin.hive.HiveCompressionCodecs.selectCompressionCodec; import static io.trino.plugin.hive.HiveErrorCode.HIVE_FILESYSTEM_ERROR; import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_METADATA; @@ -84,7 +85,6 @@ import static io.trino.plugin.hive.util.HiveTypeUtil.getType; import static io.trino.plugin.hive.util.HiveUtil.getColumnNames; import static io.trino.plugin.hive.util.HiveUtil.getColumnTypes; -import static io.trino.plugin.hive.util.HiveUtil.makePartName; import static io.trino.plugin.hive.util.HiveWriteUtils.createPartitionValues; import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMNS; import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMN_TYPES; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HivePartitionName.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HivePartitionName.java index 9c8a9c5ce3cd..9aeba984c4fa 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HivePartitionName.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HivePartitionName.java @@ -23,7 +23,7 @@ import java.util.Optional; import static com.google.common.base.MoreObjects.toStringHelper; -import static io.trino.metastore.Partition.toPartitionValues; +import static io.trino.metastore.Partitions.toPartitionValues; import static java.util.Objects.requireNonNull; @Immutable diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/MetastoreUtil.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/MetastoreUtil.java index 5280dd5bb578..561a4a336ff7 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/MetastoreUtil.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/MetastoreUtil.java @@ -88,12 +88,12 @@ import static io.trino.hive.thrift.metastore.hive_metastoreConstants.META_TABLE_NAME; import static io.trino.hive.thrift.metastore.hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS; import static io.trino.hive.thrift.metastore.hive_metastoreConstants.META_TABLE_PARTITION_COLUMN_TYPES; +import static io.trino.metastore.Partitions.makePartName; import static io.trino.plugin.hive.HiveMetadata.AVRO_SCHEMA_LITERAL_KEY; import static io.trino.plugin.hive.HiveMetadata.AVRO_SCHEMA_URL_KEY; import static io.trino.plugin.hive.HiveSplitManager.PRESTO_OFFLINE; import static io.trino.plugin.hive.HiveStorageFormat.AVRO; import static io.trino.plugin.hive.metastore.SparkMetastoreUtil.getSparkBasicStatistics; -import static io.trino.plugin.hive.util.HiveUtil.makePartName; import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMN_COMMENTS; import static io.trino.plugin.hive.util.SerdeConstants.SERIALIZATION_LIB; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java index 5f4015494c93..070ea9e0b631 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java @@ -111,7 +111,8 @@ import static com.google.common.collect.ImmutableSet.toImmutableSet; import static io.airlift.concurrent.MoreFutures.getFutureValue; import static io.trino.metastore.HivePrivilegeInfo.HivePrivilege.OWNERSHIP; -import static io.trino.metastore.Partition.toPartitionValues; +import static io.trino.metastore.Partitions.makePartName; +import static io.trino.metastore.Partitions.toPartitionValues; import static io.trino.metastore.PrincipalPrivileges.NO_PRIVILEGES; import static io.trino.metastore.StatisticsUpdateMode.MERGE_INCREMENTAL; import static io.trino.metastore.StatisticsUpdateMode.OVERWRITE_ALL; @@ -135,7 +136,6 @@ import static io.trino.plugin.hive.metastore.SparkMetastoreUtil.getSparkTableStatistics; import static io.trino.plugin.hive.projection.PartitionProjectionProperties.getPartitionProjectionFromTable; import static io.trino.plugin.hive.util.AcidTables.isTransactionalTable; -import static io.trino.plugin.hive.util.HiveUtil.makePartName; import static io.trino.plugin.hive.util.HiveWriteUtils.isFileCreatedByQuery; import static io.trino.spi.StandardErrorCode.ALREADY_EXISTS; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java index f3cf6ee5db16..b8c44879c49f 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java @@ -85,13 +85,13 @@ import static com.google.common.util.concurrent.Futures.immediateFuture; import static io.trino.cache.CacheUtils.invalidateAllIf; import static io.trino.cache.CacheUtils.uncheckedCacheGet; +import static io.trino.metastore.Partitions.makePartName; import static io.trino.plugin.hive.metastore.HivePartitionName.hivePartitionName; import static io.trino.plugin.hive.metastore.HiveTableName.hiveTableName; import static io.trino.plugin.hive.metastore.PartitionFilter.partitionFilter; import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.ObjectType.OTHER; import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.ObjectType.PARTITION; import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.ObjectType.STATS; -import static io.trino.plugin.hive.util.HiveUtil.makePartName; import static java.util.Collections.unmodifiableSet; import static java.util.Objects.requireNonNull; import static java.util.concurrent.TimeUnit.MILLISECONDS; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java index 7f6efda69f4b..68b3b0edcc2f 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java @@ -93,8 +93,9 @@ import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.hash.Hashing.sha256; import static io.trino.metastore.HivePrivilegeInfo.HivePrivilege.OWNERSHIP; -import static io.trino.metastore.Partition.toPartitionValues; -import static io.trino.metastore.Partition.unescapePathName; +import static io.trino.metastore.Partitions.escapePathName; +import static io.trino.metastore.Partitions.toPartitionValues; +import static io.trino.metastore.Partitions.unescapePathName; import static io.trino.metastore.Table.TABLE_COMMENT; import static io.trino.plugin.hive.HiveErrorCode.HIVE_CONCURRENT_MODIFICATION_DETECTED; import static io.trino.plugin.hive.HiveErrorCode.HIVE_METASTORE_ERROR; @@ -116,7 +117,6 @@ import static io.trino.plugin.hive.metastore.file.FileHiveMetastoreConfig.VersionCompatibility.UNSAFE_ASSUME_COMPATIBILITY; import static io.trino.plugin.hive.util.HiveUtil.DELTA_LAKE_PROVIDER; import static io.trino.plugin.hive.util.HiveUtil.SPARK_TABLE_PROVIDER_KEY; -import static io.trino.plugin.hive.util.HiveUtil.escapePathName; import static io.trino.plugin.hive.util.HiveUtil.escapeSchemaName; import static io.trino.plugin.hive.util.HiveUtil.escapeTableName; import static io.trino.plugin.hive.util.HiveUtil.isIcebergTable; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/DefaultGlueColumnStatisticsProvider.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/DefaultGlueColumnStatisticsProvider.java index 7dc27115fa8f..fb3e912956a2 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/DefaultGlueColumnStatisticsProvider.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/DefaultGlueColumnStatisticsProvider.java @@ -53,7 +53,7 @@ import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.collect.Sets.difference; import static io.airlift.concurrent.MoreFutures.getFutureValue; -import static io.trino.metastore.Partition.toPartitionValues; +import static io.trino.metastore.Partitions.toPartitionValues; import static io.trino.plugin.hive.HiveErrorCode.HIVE_METASTORE_ERROR; import static io.trino.plugin.hive.HiveErrorCode.HIVE_PARTITION_NOT_FOUND; import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueStatConverter.fromGlueColumnStatistics; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueHiveMetastore.java index 5734df27bd79..10ad2d54c40b 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueHiveMetastore.java @@ -85,6 +85,7 @@ import io.trino.metastore.Partition; import io.trino.metastore.PartitionStatistics; import io.trino.metastore.PartitionWithStatistics; +import io.trino.metastore.Partitions; import io.trino.metastore.PrincipalPrivileges; import io.trino.metastore.StatisticsUpdateMode; import io.trino.metastore.Table; @@ -136,7 +137,7 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.ImmutableSet.toImmutableSet; -import static io.trino.metastore.Partition.toPartitionValues; +import static io.trino.metastore.Partitions.toPartitionValues; import static io.trino.metastore.Table.TABLE_COMMENT; import static io.trino.plugin.hive.HiveErrorCode.HIVE_FILESYSTEM_ERROR; import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_METADATA; @@ -857,7 +858,7 @@ private Map> getPartitionsByNamesInternal(Table tabl List partitions = batchGetPartition(table, partitionNames); Map> partitionNameToPartitionValuesMap = partitionNames.stream() - .collect(toMap(identity(), Partition::toPartitionValues)); + .collect(toMap(identity(), Partitions::toPartitionValues)); Map, Partition> partitionValuesToPartitionMap = partitions.stream() .collect(toMap(Partition::getValues, identity())); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java index 7d1efdafe889..ecb79587b170 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java @@ -30,6 +30,7 @@ import io.trino.metastore.Partition; import io.trino.metastore.PartitionStatistics; import io.trino.metastore.PartitionWithStatistics; +import io.trino.metastore.Partitions; import io.trino.metastore.PrincipalPrivileges; import io.trino.metastore.StatisticsUpdateMode; import io.trino.metastore.Table; @@ -388,7 +389,7 @@ public Map> getPartitionsByNames(Table table, List> partitionNameToPartitionValuesMap = partitionNames.stream() - .collect(Collectors.toMap(identity(), Partition::toPartitionValues)); + .collect(Collectors.toMap(identity(), Partitions::toPartitionValues)); Map, Partition> partitionValuesToPartitionMap = delegate.getPartitionsByNames(table.getDatabaseName(), table.getTableName(), partitionNames).stream() .map(partition -> fromMetastoreApiPartition(table, partition)) .collect(Collectors.toMap(Partition::getValues, identity())); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/CreateEmptyPartitionProcedure.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/CreateEmptyPartitionProcedure.java index a8f20e585d7d..97044f95cdc6 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/CreateEmptyPartitionProcedure.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/CreateEmptyPartitionProcedure.java @@ -45,8 +45,8 @@ import java.util.Optional; import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.metastore.Partitions.makePartName; import static io.trino.plugin.base.util.Procedures.checkProcedureArgument; -import static io.trino.plugin.hive.util.HiveUtil.makePartName; import static io.trino.spi.StandardErrorCode.ALREADY_EXISTS; import static io.trino.spi.StandardErrorCode.INVALID_PROCEDURE_ARGUMENT; import static io.trino.spi.connector.RetryMode.NO_RETRIES; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/DropStatsProcedure.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/DropStatsProcedure.java index e084fac9a238..add2f10235e8 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/DropStatsProcedure.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/DropStatsProcedure.java @@ -43,9 +43,9 @@ import java.util.OptionalLong; import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.metastore.Partitions.makePartName; import static io.trino.metastore.StatisticsUpdateMode.CLEAR_ALL; import static io.trino.plugin.base.util.Procedures.checkProcedureArgument; -import static io.trino.plugin.hive.util.HiveUtil.makePartName; import static io.trino.spi.StandardErrorCode.INVALID_PROCEDURE_ARGUMENT; import static io.trino.spi.type.VarcharType.VARCHAR; import static java.lang.String.format; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/FlushMetadataCacheProcedure.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/FlushMetadataCacheProcedure.java index 78a7e63931f1..80b9a544dcc2 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/FlushMetadataCacheProcedure.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/FlushMetadataCacheProcedure.java @@ -41,7 +41,7 @@ import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.plugin.hive.util.HiveUtil.makePartName; +import static io.trino.metastore.Partitions.makePartName; import static io.trino.spi.type.VarcharType.VARCHAR; import static java.lang.String.format; import static java.lang.invoke.MethodHandles.lookup; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/RegisterPartitionProcedure.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/RegisterPartitionProcedure.java index ff0ecee88e85..dbf699288e1f 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/RegisterPartitionProcedure.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/RegisterPartitionProcedure.java @@ -42,12 +42,12 @@ import java.util.List; import java.util.Optional; +import static io.trino.metastore.Partitions.makePartName; import static io.trino.plugin.base.util.Procedures.checkProcedureArgument; import static io.trino.plugin.hive.HiveErrorCode.HIVE_FILESYSTEM_ERROR; import static io.trino.plugin.hive.HiveMetadata.TRINO_QUERY_ID_NAME; import static io.trino.plugin.hive.procedure.Procedures.checkIsPartitionedTable; import static io.trino.plugin.hive.procedure.Procedures.checkPartitionColumns; -import static io.trino.plugin.hive.util.HiveUtil.makePartName; import static io.trino.spi.StandardErrorCode.ALREADY_EXISTS; import static io.trino.spi.StandardErrorCode.INVALID_PROCEDURE_ARGUMENT; import static io.trino.spi.StandardErrorCode.PERMISSION_DENIED; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/UnregisterPartitionProcedure.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/UnregisterPartitionProcedure.java index 111c4e57f0f4..8ef9d3a5de69 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/UnregisterPartitionProcedure.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/UnregisterPartitionProcedure.java @@ -34,10 +34,10 @@ import java.lang.invoke.MethodHandle; import java.util.List; +import static io.trino.metastore.Partitions.makePartName; import static io.trino.plugin.base.util.Procedures.checkProcedureArgument; import static io.trino.plugin.hive.procedure.Procedures.checkIsPartitionedTable; import static io.trino.plugin.hive.procedure.Procedures.checkPartitionColumns; -import static io.trino.plugin.hive.util.HiveUtil.makePartName; import static io.trino.spi.StandardErrorCode.NOT_FOUND; import static io.trino.spi.type.VarcharType.VARCHAR; import static java.lang.String.format; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/projection/PartitionProjection.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/projection/PartitionProjection.java index b6c22d7d8143..4f29ed1d63eb 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/projection/PartitionProjection.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/projection/PartitionProjection.java @@ -33,9 +33,9 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.collect.Sets.cartesianProduct; -import static io.trino.metastore.Partition.toPartitionValues; +import static io.trino.metastore.Partitions.escapePathName; +import static io.trino.metastore.Partitions.toPartitionValues; import static io.trino.plugin.hive.projection.InvalidProjectionException.invalidProjectionMessage; -import static io.trino.plugin.hive.util.HiveUtil.escapePathName; import static java.lang.String.format; import static java.util.Objects.requireNonNull; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveUtil.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveUtil.java index a4cf437aa3b4..cf4339f11bd2 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveUtil.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveUtil.java @@ -60,7 +60,6 @@ import org.joda.time.format.DateTimePrinter; import java.math.BigDecimal; -import java.util.HexFormat; import java.util.List; import java.util.Map; import java.util.Optional; @@ -78,6 +77,8 @@ import static io.trino.hive.formats.HiveClassNames.HUDI_REALTIME_INPUT_FORMAT; import static io.trino.hive.thrift.metastore.hive_metastoreConstants.FILE_INPUT_FORMAT; import static io.trino.metastore.HiveType.toHiveTypes; +import static io.trino.metastore.Partitions.HIVE_DEFAULT_DYNAMIC_PARTITION; +import static io.trino.metastore.Partitions.escapePathName; import static io.trino.metastore.SortingColumn.Order.ASCENDING; import static io.trino.metastore.SortingColumn.Order.DESCENDING; import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; @@ -101,7 +102,6 @@ import static io.trino.plugin.hive.HiveMetadata.PARQUET_BLOOM_FILTER_COLUMNS_KEY; import static io.trino.plugin.hive.HiveMetadata.SKIP_FOOTER_COUNT_KEY; import static io.trino.plugin.hive.HiveMetadata.SKIP_HEADER_COUNT_KEY; -import static io.trino.plugin.hive.HivePartitionKey.HIVE_DEFAULT_DYNAMIC_PARTITION; import static io.trino.plugin.hive.HiveSessionProperties.getTimestampPrecision; import static io.trino.plugin.hive.HiveTableProperties.ORC_BLOOM_FILTER_FPP; import static io.trino.plugin.hive.projection.PartitionProjectionProperties.getPartitionProjectionTrinoColumnProperties; @@ -149,8 +149,6 @@ public final class HiveUtil public static final String ICEBERG_TABLE_TYPE_NAME = "table_type"; public static final String ICEBERG_TABLE_TYPE_VALUE = "iceberg"; - private static final HexFormat HEX_UPPER_FORMAT = HexFormat.of().withUpperCase(); - private static final LocalDateTime EPOCH_DAY = new LocalDateTime(1970, 1, 1, 0, 0); private static final DateTimeFormatter HIVE_DATE_PARSER; private static final DateTimeFormatter HIVE_TIMESTAMP_PARSER; @@ -159,10 +157,6 @@ public final class HiveUtil private static final Splitter COLUMN_NAMES_SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); - private static final CharMatcher PATH_CHAR_TO_ESCAPE = CharMatcher.inRange((char) 0, (char) 31) - .or(CharMatcher.anyOf("\"#%'*/:=?\\\u007F{[]^")) - .precomputed(); - private static final CharMatcher DOT_MATCHER = CharMatcher.is('.'); public static String splitError(Throwable t, Location location, long start, long length) @@ -879,59 +873,4 @@ public static String escapeTableName(String tableName) } return escapePathName(tableName); } - - // copy of org.apache.hadoop.hive.common.FileUtils#escapePathName - public static String escapePathName(String path) - { - if (isNullOrEmpty(path)) { - return HIVE_DEFAULT_DYNAMIC_PARTITION; - } - - // Fast-path detection, no escaping and therefore no copying necessary - int escapeAtIndex = PATH_CHAR_TO_ESCAPE.indexIn(path); - if (escapeAtIndex < 0) { - return path; - } - - // slow path, escape beyond the first required escape character into a new string - StringBuilder sb = new StringBuilder(); - int fromIndex = 0; - while (escapeAtIndex >= 0 && escapeAtIndex < path.length()) { - // preceding characters without escaping needed - if (escapeAtIndex > fromIndex) { - sb.append(path, fromIndex, escapeAtIndex); - } - // escape single character - char c = path.charAt(escapeAtIndex); - sb.append('%').append(HEX_UPPER_FORMAT.toHighHexDigit(c)).append(HEX_UPPER_FORMAT.toLowHexDigit(c)); - // find next character to escape - fromIndex = escapeAtIndex + 1; - if (fromIndex < path.length()) { - escapeAtIndex = PATH_CHAR_TO_ESCAPE.indexIn(path, fromIndex); - } - else { - escapeAtIndex = -1; - } - } - // trailing characters without escaping needed - if (fromIndex < path.length()) { - sb.append(path, fromIndex, path.length()); - } - return sb.toString(); - } - - // copy of org.apache.hadoop.hive.common.FileUtils#makePartName - public static String makePartName(List columns, List values) - { - StringBuilder name = new StringBuilder(); - for (int i = 0; i < columns.size(); i++) { - if (i > 0) { - name.append('/'); - } - name.append(escapePathName(columns.get(i).toLowerCase(ENGLISH))); - name.append('='); - name.append(escapePathName(values.get(i))); - } - return name.toString(); - } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveWriteUtils.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveWriteUtils.java index 8744cc44283c..fc6cf307e54a 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveWriteUtils.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveWriteUtils.java @@ -54,10 +54,10 @@ import java.util.Optional; import static com.google.common.io.BaseEncoding.base16; +import static io.trino.metastore.Partitions.HIVE_DEFAULT_DYNAMIC_PARTITION; import static io.trino.plugin.hive.HiveErrorCode.HIVE_DATABASE_LOCATION_ERROR; import static io.trino.plugin.hive.HiveErrorCode.HIVE_FILESYSTEM_ERROR; import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_PARTITION_VALUE; -import static io.trino.plugin.hive.HivePartitionKey.HIVE_DEFAULT_DYNAMIC_PARTITION; import static io.trino.plugin.hive.TableType.MANAGED_TABLE; import static io.trino.plugin.hive.TableType.MATERIALIZED_VIEW; import static io.trino.plugin.hive.metastore.MetastoreUtil.getProtectMode; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java index 332f2a3ae9bb..d4efe5fe6fc3 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java @@ -133,12 +133,12 @@ import static com.google.common.base.Verify.verify; import static com.google.common.collect.ImmutableList.toImmutableList; import static io.airlift.slice.Slices.utf8Slice; +import static io.trino.metastore.Partitions.HIVE_DEFAULT_DYNAMIC_PARTITION; import static io.trino.plugin.base.type.TrinoTimestampEncoderFactory.createTimestampEncoder; import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.REGULAR; import static io.trino.plugin.hive.HiveColumnHandle.createBaseColumn; import static io.trino.plugin.hive.HivePageSourceProvider.ColumnMapping.buildColumnMappings; -import static io.trino.plugin.hive.HivePartitionKey.HIVE_DEFAULT_DYNAMIC_PARTITION; import static io.trino.plugin.hive.HiveStorageFormat.AVRO; import static io.trino.plugin.hive.HiveStorageFormat.CSV; import static io.trino.plugin.hive.HiveStorageFormat.JSON; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/statistics/TestMetastoreHiveStatisticsProvider.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/statistics/TestMetastoreHiveStatisticsProvider.java index 61845ce0bac8..f74ea30fb771 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/statistics/TestMetastoreHiveStatisticsProvider.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/statistics/TestMetastoreHiveStatisticsProvider.java @@ -53,11 +53,11 @@ import static io.trino.metastore.HivePartition.UNPARTITIONED_ID; import static io.trino.metastore.HiveType.HIVE_LONG; import static io.trino.metastore.HiveType.HIVE_STRING; +import static io.trino.metastore.Partitions.HIVE_DEFAULT_DYNAMIC_PARTITION; import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.REGULAR; import static io.trino.plugin.hive.HiveColumnHandle.createBaseColumn; import static io.trino.plugin.hive.HiveErrorCode.HIVE_CORRUPTED_COLUMN_STATISTICS; -import static io.trino.plugin.hive.HivePartitionKey.HIVE_DEFAULT_DYNAMIC_PARTITION; import static io.trino.plugin.hive.HivePartitionManager.parsePartition; import static io.trino.plugin.hive.HiveTestUtils.SESSION; import static io.trino.plugin.hive.HiveTestUtils.getHiveSession; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/util/TestHiveUtil.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/util/TestHiveUtil.java index 6630f9b96858..b4b80d360c56 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/util/TestHiveUtil.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/util/TestHiveUtil.java @@ -13,20 +13,11 @@ */ package io.trino.plugin.hive.util; -import io.trino.metastore.Partition; -import org.apache.hadoop.hive.common.FileUtils; -import org.apache.hadoop.hive.metastore.Warehouse; -import org.apache.hadoop.hive.metastore.api.MetaException; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormat; import org.junit.jupiter.api.Test; -import java.util.AbstractList; -import java.util.ArrayList; -import java.util.List; - -import static io.trino.metastore.Partition.toPartitionValues; import static io.trino.plugin.hive.util.HiveUtil.escapeSchemaName; import static io.trino.plugin.hive.util.HiveUtil.escapeTableName; import static io.trino.plugin.hive.util.HiveUtil.parseHiveTimestamp; @@ -47,43 +38,6 @@ public void testParseHiveTimestamp() assertThat(parse(time, "yyyy-MM-dd HH:mm:ss.SSSSSSSSS")).isEqualTo(unixTime(time, 7)); } - @Test - public void testToPartitionValues() - throws MetaException - { - assertToPartitionValues("ds=2015-12-30/event_type=QueryCompletion"); - assertToPartitionValues("ds=2015-12-30"); - assertToPartitionValues("a=1/b=2/c=3"); - assertToPartitionValues("a=1"); - assertToPartitionValues("pk=!@%23$%25%5E&%2A()%2F%3D"); - assertToPartitionValues("pk=__HIVE_DEFAULT_PARTITION__"); - } - - @Test - public void testUnescapePathName() - { - assertUnescapePathName("", ""); - assertUnescapePathName("x", "x"); - assertUnescapePathName("abc", "abc"); - assertUnescapePathName("abc%", "abc%"); - assertUnescapePathName("%", "%"); - assertUnescapePathName("%41", "A"); - assertUnescapePathName("%41%x", "A%x"); - assertUnescapePathName("%41%xxZ", "A%xxZ"); - assertUnescapePathName("%41%%Z", "A%%Z"); - assertUnescapePathName("%41%25%25Z", "A%%Z"); - assertUnescapePathName("abc%41%42%43", "abcABC"); - assertUnescapePathName("abc%3Axyz", "abc:xyz"); - assertUnescapePathName("abc%3axyz", "abc:xyz"); - assertUnescapePathName("abc%BBxyz", "abc\u00BBxyz"); - } - - private static void assertUnescapePathName(String value, String expected) - { - assertThat(FileUtils.unescapePathName(value)).isEqualTo(expected); - assertThat(Partition.unescapePathName(value)).isEqualTo(expected); - } - @Test public void testEscapeDatabaseName() { @@ -108,56 +62,6 @@ public void testEscapeTableName() assertThat(escapeTableName("../../table1")).isEqualTo("..%2F..%2Ftable1"); } - @Test - public void testEscapePathName() - { - assertEscapePathName(null, "__HIVE_DEFAULT_PARTITION__"); - assertEscapePathName("", "__HIVE_DEFAULT_PARTITION__"); - assertEscapePathName("x", "x"); - assertEscapePathName("abc", "abc"); - assertEscapePathName("%", "%25"); - assertEscapePathName("A", "A"); - assertEscapePathName("A%x", "A%25x"); - assertEscapePathName("A%xxZ", "A%25xxZ"); - assertEscapePathName("A%%Z", "A%25%25Z"); - assertEscapePathName("abcABC", "abcABC"); - assertEscapePathName("abc:xyz", "abc%3Axyz"); - assertEscapePathName("abc\u00BBxyz", "abc\u00BBxyz"); - assertEscapePathName("\u0000\t\b\r\n\u001F", "%00%09%08%0D%0A%1F"); - assertEscapePathName("#%^&*=[]{\\:'\"/?", "%23%25%5E&%2A%3D%5B%5D%7B%5C%3A%27%22%2F%3F"); - assertEscapePathName("~`!@$()-_+}|;,.<>", "~`!@$()-_+}|;,.<>"); - } - - private static void assertEscapePathName(String value, String expected) - { - assertThat(FileUtils.escapePathName(value)).isEqualTo(expected); - assertThat(HiveUtil.escapePathName(value)).isEqualTo(expected); - } - - @Test - public void testMakePartName() - { - assertMakePartName(List.of("abc"), List.of("xyz"), "abc=xyz"); - assertMakePartName(List.of("abc:qqq"), List.of("xyz/yyy=zzz"), "abc%3Aqqq=xyz%2Fyyy%3Dzzz"); - assertMakePartName(List.of("abc", "def", "xyz"), List.of("qqq", "rrr", "sss"), "abc=qqq/def=rrr/xyz=sss"); - } - - private static void assertMakePartName(List columns, List values, String expected) - { - assertThat(FileUtils.makePartName(columns, values)).isEqualTo(expected); - assertThat(HiveUtil.makePartName(columns, values)).isEqualTo(expected); - } - - private static void assertToPartitionValues(String partitionName) - throws MetaException - { - List actual = toPartitionValues(partitionName); - AbstractList expected = new ArrayList<>(); - actual.forEach(s -> expected.add(null)); - Warehouse.makeValsFromName(partitionName, expected); - assertThat(actual).isEqualTo(expected); - } - private static long parse(DateTime time, String pattern) { return parseHiveTimestamp(DateTimeFormat.forPattern(pattern).print(time)); diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java index bee386c71553..3bb7b1c42a79 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java @@ -70,6 +70,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static io.airlift.slice.Slices.utf8Slice; import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; +import static io.trino.metastore.Partitions.makePartName; import static io.trino.parquet.ParquetTypeUtils.getColumnIO; import static io.trino.parquet.ParquetTypeUtils.getDescriptors; import static io.trino.parquet.predicate.PredicateUtils.buildPredicate; @@ -80,7 +81,6 @@ import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.createParquetPageSource; import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.getParquetMessageType; import static io.trino.plugin.hive.parquet.ParquetPageSourceFactory.getParquetTupleDomain; -import static io.trino.plugin.hive.util.HiveUtil.makePartName; import static io.trino.plugin.hudi.HudiErrorCode.HUDI_BAD_DATA; import static io.trino.plugin.hudi.HudiErrorCode.HUDI_CANNOT_OPEN_SPLIT; import static io.trino.plugin.hudi.HudiErrorCode.HUDI_CURSOR_ERROR; diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/partition/HiveHudiPartitionInfo.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/partition/HiveHudiPartitionInfo.java index 7efa635f4456..906be1436c0f 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/partition/HiveHudiPartitionInfo.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/partition/HiveHudiPartitionInfo.java @@ -29,6 +29,7 @@ import java.util.Optional; import static com.google.common.base.MoreObjects.toStringHelper; +import static io.trino.metastore.Partitions.toPartitionValues; import static io.trino.plugin.hudi.HudiErrorCode.HUDI_PARTITION_NOT_FOUND; import static io.trino.plugin.hudi.HudiUtil.buildPartitionKeys; import static io.trino.plugin.hudi.HudiUtil.partitionMatchesPredicates; @@ -72,7 +73,7 @@ public HiveHudiPartitionInfo( public String getRelativePartitionPath() { if (relativePartitionPath == null) { - loadPartitionInfo(hiveMetastore.getPartition(table, Partition.toPartitionValues(hivePartitionName))); + loadPartitionInfo(hiveMetastore.getPartition(table, toPartitionValues(hivePartitionName))); } return relativePartitionPath; } @@ -81,7 +82,7 @@ public String getRelativePartitionPath() public List getHivePartitionKeys() { if (hivePartitionKeys == null) { - loadPartitionInfo(hiveMetastore.getPartition(table, Partition.toPartitionValues(hivePartitionName))); + loadPartitionInfo(hiveMetastore.getPartition(table, toPartitionValues(hivePartitionName))); } return hivePartitionKeys; } From 4eab3e7d216d123d6761569605917ed428586c53 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Tue, 30 Jul 2024 23:49:14 -0700 Subject: [PATCH 05/12] Move HiveMetastoreFactory to metastore module --- .../main/java/io/trino}/metastore/HiveMetastoreFactory.java | 3 +-- .../io/trino/plugin/deltalake/DeltaLakeMetadataFactory.java | 2 +- .../file/DeltaLakeFileMetastoreTableOperationsProvider.java | 2 +- .../DeltaLakeThriftMetastoreTableOperationsProvider.java | 2 +- .../deltalake/BaseDeltaLakeRegisterTableProcedureTest.java | 2 +- .../plugin/deltalake/BaseDeltaLakeTableWithCustomLocation.java | 2 +- .../io/trino/plugin/deltalake/TestDeltaLakeConnectorTest.java | 2 +- .../io/trino/plugin/deltalake/TestDeltaLakeFileOperations.java | 2 +- .../java/io/trino/plugin/deltalake/TestDeltaLakeMetadata.java | 2 +- .../TestDeltaLakeMinioAndLockBasedSynchronizerSmokeTest.java | 2 +- .../plugin/deltalake/TestDeltaLakeProjectionPushdownPlans.java | 2 +- .../io/trino/plugin/deltalake/TestDeltaLakeSplitManager.java | 2 +- .../metastore/TestDeltaLakeMetastoreAccessOperations.java | 2 +- .../deltalake/metastore/TestingDeltaLakeMetastoreModule.java | 2 +- .../deltalake/metastore/glue/TestDeltaLakeGlueMetastore.java | 2 +- .../main/java/io/trino/plugin/hive/HiveMetadataFactory.java | 2 +- .../main/java/io/trino/plugin/hive/HivePageSinkProvider.java | 2 +- .../plugin/hive/metastore/CachingHiveMetastoreModule.java | 1 + .../io/trino/plugin/hive/metastore/HiveMetastoreModule.java | 1 + .../plugin/hive/metastore/cache/SharedHiveMetastoreCache.java | 2 +- .../plugin/hive/metastore/file/FileHiveMetastoreFactory.java | 2 +- .../trino/plugin/hive/metastore/file/FileMetastoreModule.java | 2 +- .../plugin/hive/metastore/glue/GlueHiveMetastoreFactory.java | 2 +- .../trino/plugin/hive/metastore/glue/GlueMetastoreModule.java | 2 +- .../hive/metastore/glue/v1/GlueHiveMetastoreFactory.java | 2 +- .../plugin/hive/metastore/glue/v1/GlueMetastoreModule.java | 2 +- .../hive/metastore/thrift/BridgingHiveMetastoreFactory.java | 2 +- .../plugin/hive/metastore/thrift/ThriftMetastoreModule.java | 2 +- .../plugin/hive/procedure/FlushMetadataCacheProcedure.java | 2 +- .../test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java | 2 +- .../src/test/java/io/trino/plugin/hive/HiveQueryRunner.java | 2 +- .../plugin/hive/TestHiveCustomCatalogConnectorSmokeTest.java | 2 +- .../src/test/java/io/trino/plugin/hive/TestHivePageSink.java | 2 +- .../trino/plugin/hive/fs/BaseCachingDirectoryListerTest.java | 2 +- .../cache/TestCachingHiveMetastoreWithQueryRunner.java | 2 +- .../hive/metastore/thrift/TestHiveMetastoreCatalogs.java | 2 +- .../TestHiveMetastoreMetadataQueriesAccessOperations.java | 2 +- .../hive/optimizer/TestConnectorPushdownRulesWithHive.java | 2 +- .../java/io/trino/plugin/hive/optimizer/TestHivePlans.java | 2 +- .../optimizer/TestHiveProjectionPushdownIntoTableScan.java | 2 +- .../main/java/io/trino/plugin/hudi/HudiMetadataFactory.java | 2 +- .../src/test/java/io/trino/plugin/hudi/HudiQueryRunner.java | 2 +- .../plugin/hudi/testing/ResourceHudiTablesInitializer.java | 2 +- .../trino/plugin/hudi/testing/TpchHudiTablesInitializer.java | 2 +- .../src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java | 2 +- .../java/io/trino/plugin/iceberg/IcebergMetadataFactory.java | 2 +- .../src/main/java/io/trino/plugin/iceberg/IcebergModule.java | 2 +- .../plugin/iceberg/catalog/hms/TrinoHiveCatalogFactory.java | 2 +- .../io/trino/plugin/iceberg/procedure/MigrateProcedure.java | 2 +- .../java/io/trino/plugin/iceberg/procedure/MigrationUtils.java | 2 +- .../test/java/io/trino/plugin/iceberg/IcebergTestUtils.java | 2 +- .../io/trino/plugin/iceberg/TestIcebergMetadataListing.java | 2 +- .../plugin/iceberg/TestIcebergProjectionPushdownPlans.java | 2 +- .../io/trino/plugin/iceberg/TestMetadataQueryOptimization.java | 2 +- .../catalog/file/TestingIcebergFileMetastoreCatalogModule.java | 2 +- .../optimizer/TestConnectorPushdownRulesWithIceberg.java | 2 +- .../io/trino/sql/planner/IcebergCostBasedPlanTestSetup.java | 2 +- 57 files changed, 57 insertions(+), 56 deletions(-) rename {plugin/trino-hive/src/main/java/io/trino/plugin/hive => lib/trino-metastore/src/main/java/io/trino}/metastore/HiveMetastoreFactory.java (95%) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastoreFactory.java b/lib/trino-metastore/src/main/java/io/trino/metastore/HiveMetastoreFactory.java similarity index 95% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastoreFactory.java rename to lib/trino-metastore/src/main/java/io/trino/metastore/HiveMetastoreFactory.java index cab15145c2df..5b0564c8ffe0 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastoreFactory.java +++ b/lib/trino-metastore/src/main/java/io/trino/metastore/HiveMetastoreFactory.java @@ -11,9 +11,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.metastore; +package io.trino.metastore; -import io.trino.metastore.HiveMetastore; import io.trino.spi.security.ConnectorIdentity; import java.util.Optional; diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadataFactory.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadataFactory.java index 49cfc79e0c52..dc1241ed504d 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadataFactory.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadataFactory.java @@ -17,6 +17,7 @@ import io.airlift.concurrent.BoundedExecutor; import io.airlift.json.JsonCodec; import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.deltalake.metastore.DeltaLakeTableMetadataScheduler; import io.trino.plugin.deltalake.metastore.HiveMetastoreBackedDeltaLakeMetastore; import io.trino.plugin.deltalake.statistics.CachingExtendedStatisticsAccess; @@ -26,7 +27,6 @@ import io.trino.plugin.deltalake.transactionlog.writer.TransactionLogWriterFactory; import io.trino.plugin.hive.NodeVersion; import io.trino.plugin.hive.TrinoViewHiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.security.AccessControlMetadata; import io.trino.spi.NodeManager; diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/file/DeltaLakeFileMetastoreTableOperationsProvider.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/file/DeltaLakeFileMetastoreTableOperationsProvider.java index 898679cb6497..5a7ecbdee9ac 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/file/DeltaLakeFileMetastoreTableOperationsProvider.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/file/DeltaLakeFileMetastoreTableOperationsProvider.java @@ -14,9 +14,9 @@ package io.trino.plugin.deltalake.metastore.file; import com.google.inject.Inject; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.deltalake.metastore.DeltaLakeTableOperations; import io.trino.plugin.deltalake.metastore.DeltaLakeTableOperationsProvider; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.connector.ConnectorSession; import java.util.Optional; diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/thrift/DeltaLakeThriftMetastoreTableOperationsProvider.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/thrift/DeltaLakeThriftMetastoreTableOperationsProvider.java index 9ff99a10cc55..d170376601ed 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/thrift/DeltaLakeThriftMetastoreTableOperationsProvider.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/thrift/DeltaLakeThriftMetastoreTableOperationsProvider.java @@ -14,9 +14,9 @@ package io.trino.plugin.deltalake.metastore.thrift; import com.google.inject.Inject; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.deltalake.metastore.DeltaLakeTableOperations; import io.trino.plugin.deltalake.metastore.DeltaLakeTableOperationsProvider; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreFactory; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.security.ConnectorIdentity; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeRegisterTableProcedureTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeRegisterTableProcedureTest.java index 471b6c212d62..843d708028e1 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeRegisterTableProcedureTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeRegisterTableProcedureTest.java @@ -17,7 +17,7 @@ import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.spi.security.ConnectorIdentity; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeTableWithCustomLocation.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeTableWithCustomLocation.java index ddbdfbac1148..46fa3e58002a 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeTableWithCustomLocation.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeTableWithCustomLocation.java @@ -16,8 +16,8 @@ import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.Table; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.MaterializedRow; import org.junit.jupiter.api.Test; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConnectorTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConnectorTest.java index 11f1544e92bc..c16c2c1c641c 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConnectorTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConnectorTest.java @@ -24,10 +24,10 @@ import io.trino.Session; import io.trino.execution.QueryInfo; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.Table; import io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.ColumnMappingMode; import io.trino.plugin.hive.HiveCompressionCodec; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.tpch.TpchPlugin; import io.trino.spi.connector.ColumnMetadata; import io.trino.sql.planner.plan.FilterNode; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFileOperations.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFileOperations.java index 38e348644f4e..98230e78aa5d 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFileOperations.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFileOperations.java @@ -21,9 +21,9 @@ import io.trino.Session; import io.trino.SystemSessionProperties; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.Table; import io.trino.plugin.deltalake.metastore.DeltaLakeTableMetadataScheduler; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMetadata.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMetadata.java index fbecc7fa09be..8ffa699674f9 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMetadata.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMetadata.java @@ -30,13 +30,13 @@ import io.trino.hdfs.HdfsEnvironment; import io.trino.hdfs.TrinoHdfsFileSystemStats; import io.trino.metastore.Database; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.deltalake.metastore.DeltaLakeMetastore; import io.trino.plugin.deltalake.metastore.DeltaLakeMetastoreModule; import io.trino.plugin.deltalake.metastore.HiveMetastoreBackedDeltaLakeMetastore; import io.trino.plugin.deltalake.transactionlog.MetadataEntry; import io.trino.plugin.deltalake.transactionlog.ProtocolEntry; import io.trino.plugin.hive.NodeVersion; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.spi.NodeManager; import io.trino.spi.PageIndexerFactory; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMinioAndLockBasedSynchronizerSmokeTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMinioAndLockBasedSynchronizerSmokeTest.java index a3e4e6842e16..9bb7240b94a1 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMinioAndLockBasedSynchronizerSmokeTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMinioAndLockBasedSynchronizerSmokeTest.java @@ -19,8 +19,8 @@ import io.airlift.json.ObjectMapperProvider; import io.airlift.units.Duration; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.deltalake.transactionlog.writer.S3LockBasedTransactionLogSynchronizer; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.tpch.TpchPlugin; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeProjectionPushdownPlans.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeProjectionPushdownPlans.java index 93f0741f7ae0..755a065daea2 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeProjectionPushdownPlans.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeProjectionPushdownPlans.java @@ -23,7 +23,7 @@ import io.trino.metadata.TestingFunctionResolution; import io.trino.metastore.Database; import io.trino.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.function.OperatorType; import io.trino.spi.predicate.Domain; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSplitManager.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSplitManager.java index 779318497889..15ca606fb0e6 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSplitManager.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSplitManager.java @@ -22,6 +22,7 @@ import io.trino.filesystem.cache.DefaultCachingHostAddressProvider; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; import io.trino.filesystem.memory.MemoryFileSystemFactory; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.base.metrics.FileFormatDataSourceStats; import io.trino.plugin.deltalake.metastore.DeltaLakeTableMetadataScheduler; import io.trino.plugin.deltalake.metastore.file.DeltaLakeFileMetastoreTableOperationsProvider; @@ -41,7 +42,6 @@ import io.trino.plugin.deltalake.transactionlog.writer.TransactionLogWriterFactory; import io.trino.plugin.hive.HiveTransactionHandle; import io.trino.plugin.hive.NodeVersion; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.parquet.ParquetReaderConfig; import io.trino.plugin.hive.parquet.ParquetWriterConfig; import io.trino.spi.SplitWeight; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java index e2f7348e2477..5211e96c5490 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java @@ -20,10 +20,10 @@ import io.opentelemetry.sdk.trace.data.SpanData; import io.trino.Session; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.Table; import io.trino.plugin.deltalake.DeltaLakeQueryRunner; import io.trino.plugin.deltalake.TestingDeltaLakeUtils; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.MetastoreMethod; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestingDeltaLakeMetastoreModule.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestingDeltaLakeMetastoreModule.java index adea03218e84..98488d0330e4 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestingDeltaLakeMetastoreModule.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestingDeltaLakeMetastoreModule.java @@ -18,12 +18,12 @@ import com.google.inject.Scopes; import io.airlift.configuration.AbstractConfigurationAwareModule; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.deltalake.AllowDeltaLakeManagedTableRename; import io.trino.plugin.deltalake.MaxTableParameterLength; import io.trino.plugin.deltalake.metastore.file.DeltaLakeFileMetastoreTableOperationsProvider; import io.trino.plugin.hive.HideDeltaLakeTables; import io.trino.plugin.hive.metastore.CachingHiveMetastoreModule; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import static java.util.Objects.requireNonNull; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeGlueMetastore.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeGlueMetastore.java index 9ffae4586f88..3922dc250f73 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeGlueMetastore.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeGlueMetastore.java @@ -28,6 +28,7 @@ import io.trino.metastore.Column; import io.trino.metastore.Database; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.PrincipalPrivileges; import io.trino.metastore.Table; import io.trino.plugin.base.session.SessionPropertiesProvider; @@ -36,7 +37,6 @@ import io.trino.plugin.deltalake.DeltaLakeModule; import io.trino.plugin.deltalake.metastore.DeltaLakeMetastoreModule; import io.trino.plugin.hive.NodeVersion; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.NodeManager; import io.trino.spi.PageIndexerFactory; import io.trino.spi.TrinoException; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadataFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadataFactory.java index 136b9ab09267..f63c5dad2b42 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadataFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadataFactory.java @@ -20,10 +20,10 @@ import io.airlift.units.Duration; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.fs.DirectoryLister; import io.trino.plugin.hive.fs.TransactionScopeCachingDirectoryListerFactory; import io.trino.plugin.hive.metastore.HiveMetastoreConfig; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.SemiTransactionalHiveMetastore; import io.trino.plugin.hive.security.AccessControlMetadataFactory; import io.trino.plugin.hive.statistics.MetastoreHiveStatisticsProvider; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java index e8e987b1c728..d1030f73de35 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java @@ -21,8 +21,8 @@ import io.airlift.json.JsonCodec; import io.airlift.units.DataSize; import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.SortingColumn; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.HivePageSinkMetadataProvider; import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.spi.PageIndexerFactory; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/CachingHiveMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/CachingHiveMetastoreModule.java index d9294b4eb916..3d6182349366 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/CachingHiveMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/CachingHiveMetastoreModule.java @@ -18,6 +18,7 @@ import com.google.inject.Scopes; import com.google.inject.Singleton; import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.fs.DirectoryLister; import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.metastore.cache.CachingHiveMetastoreConfig; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastoreModule.java index d3e8b21e4182..9703885f428b 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastoreModule.java @@ -20,6 +20,7 @@ import com.google.inject.Singleton; import io.airlift.configuration.AbstractConfigurationAwareModule; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.AllowHiveTableRename; import io.trino.plugin.hive.HideDeltaLakeTables; import io.trino.plugin.hive.metastore.file.FileMetastoreModule; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/SharedHiveMetastoreCache.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/SharedHiveMetastoreCache.java index 364d25a02047..8e2c731b8d92 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/SharedHiveMetastoreCache.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/SharedHiveMetastoreCache.java @@ -24,7 +24,7 @@ import com.google.inject.Inject; import io.airlift.units.Duration; import io.trino.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.ObjectType; import io.trino.spi.NodeManager; import io.trino.spi.TrinoException; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastoreFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastoreFactory.java index 28141fb1c668..6d1a9101c7a4 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastoreFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastoreFactory.java @@ -17,10 +17,10 @@ import io.opentelemetry.api.trace.Tracer; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.tracing.TracingHiveMetastore; import io.trino.plugin.hive.HideDeltaLakeTables; import io.trino.plugin.hive.NodeVersion; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.security.ConnectorIdentity; import java.util.Optional; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileMetastoreModule.java index 71b0a7450679..e990434bea34 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileMetastoreModule.java @@ -17,8 +17,8 @@ import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.Scopes; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.AllowHiveTableRename; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import static io.airlift.configuration.ConfigBinder.configBinder; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastoreFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastoreFactory.java index 83ca8b6090c7..8b5075e606e6 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastoreFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastoreFactory.java @@ -16,8 +16,8 @@ import com.google.inject.Inject; import io.opentelemetry.api.trace.Tracer; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.tracing.TracingHiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.security.ConnectorIdentity; import java.util.Optional; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreModule.java index 584ad46efe92..d1431bb30de7 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreModule.java @@ -27,9 +27,9 @@ import io.airlift.units.Duration; import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.instrumentation.awssdk.v2_2.AwsSdkTelemetry; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.AllowHiveTableRename; import io.trino.plugin.hive.HideDeltaLakeTables; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.plugin.hive.metastore.cache.CachingHiveMetastoreConfig; import io.trino.spi.NodeManager; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueHiveMetastoreFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueHiveMetastoreFactory.java index e52fdcf75c56..8d6412b08f5f 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueHiveMetastoreFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueHiveMetastoreFactory.java @@ -16,8 +16,8 @@ import com.google.inject.Inject; import io.opentelemetry.api.trace.Tracer; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.tracing.TracingHiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.security.ConnectorIdentity; import java.util.Optional; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueMetastoreModule.java index 5bdd79f6ace3..eddec0839eb6 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueMetastoreModule.java @@ -31,8 +31,8 @@ import io.airlift.configuration.AbstractConfigurationAwareModule; import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.instrumentation.awssdk.v1_11.AwsSdkTelemetry; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.AllowHiveTableRename; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.plugin.hive.metastore.glue.GlueMetastoreStats; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastoreFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastoreFactory.java index b4875165b79f..632045081656 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastoreFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastoreFactory.java @@ -16,8 +16,8 @@ import com.google.inject.Inject; import io.opentelemetry.api.trace.Tracer; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.tracing.TracingHiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.security.ConnectorIdentity; import java.util.Optional; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java index f3b3c488260d..8c5f0b71b37c 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java @@ -21,10 +21,10 @@ import com.google.inject.TypeLiteral; import com.google.inject.multibindings.OptionalBinder; import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.base.security.UserNameProvider; import io.trino.plugin.hive.AllowHiveTableRename; import io.trino.plugin.hive.ForHiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import java.util.concurrent.ExecutorService; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/FlushMetadataCacheProcedure.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/FlushMetadataCacheProcedure.java index 80b9a544dcc2..8cf5ae65c4c7 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/FlushMetadataCacheProcedure.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/FlushMetadataCacheProcedure.java @@ -18,10 +18,10 @@ import com.google.inject.Provider; import io.trino.metastore.Column; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.Table; import io.trino.plugin.hive.HiveErrorCode; import io.trino.plugin.hive.fs.DirectoryLister; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.metastore.glue.GlueCache; import io.trino.plugin.hive.metastore.glue.PartitionName; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java index 2f2e3b53a63b..765e7625827c 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java @@ -32,11 +32,11 @@ import io.trino.metadata.TableMetadata; import io.trino.metastore.Column; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.HiveType; import io.trino.metastore.PrincipalPrivileges; import io.trino.metastore.Storage; import io.trino.metastore.Table; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.connector.CatalogSchemaTableName; import io.trino.spi.connector.ColumnMetadata; import io.trino.spi.connector.Constraint; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java index 786108fc342a..a84652e6320a 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java @@ -22,7 +22,7 @@ import io.trino.metadata.QualifiedObjectName; import io.trino.metastore.Database; import io.trino.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.tpcds.TpcdsPlugin; import io.trino.plugin.tpch.ColumnNaming; import io.trino.plugin.tpch.DecimalTypeMapping; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveCustomCatalogConnectorSmokeTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveCustomCatalogConnectorSmokeTest.java index 6d55f9cd915d..ced91cf9a85d 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveCustomCatalogConnectorSmokeTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveCustomCatalogConnectorSmokeTest.java @@ -15,9 +15,9 @@ import io.trino.metastore.Database; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.plugin.hive.containers.HiveHadoop; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.security.PrincipalType; import io.trino.testing.BaseConnectorSmokeTest; import io.trino.testing.QueryRunner; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java index b0bb0e9854ff..af40a28e0d83 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java @@ -25,9 +25,9 @@ import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.memory.MemoryFileSystemFactory; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.operator.FlatHashStrategyCompiler; import io.trino.operator.GroupByHashPageIndexerFactory; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.HivePageSinkMetadata; import io.trino.spi.Page; import io.trino.spi.PageBuilder; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/fs/BaseCachingDirectoryListerTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/fs/BaseCachingDirectoryListerTest.java index b37bdddc2825..53324069d5f0 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/fs/BaseCachingDirectoryListerTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/fs/BaseCachingDirectoryListerTest.java @@ -16,10 +16,10 @@ import com.google.common.collect.ImmutableList; import io.trino.filesystem.Location; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.PrincipalPrivileges; import io.trino.metastore.Table; import io.trino.plugin.hive.HiveQueryRunner; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.MaterializedRow; import io.trino.testing.QueryRunner; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java index 2bdfd788ff01..6acda90db491 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java @@ -19,8 +19,8 @@ import com.google.inject.Key; import io.trino.Session; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.HiveQueryRunner; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.spi.security.Identity; import io.trino.spi.security.SelectedRole; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreCatalogs.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreCatalogs.java index 23a18de4e7e5..11ab15937fa5 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreCatalogs.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreCatalogs.java @@ -17,10 +17,10 @@ import io.trino.Session; import io.trino.metastore.Database; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.HiveQueryRunner; import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.plugin.hive.containers.HiveHadoop; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.security.PrincipalType; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java index 0c6bbcf0d0d9..aa8fcba701c5 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java @@ -21,11 +21,11 @@ import io.trino.metastore.Column; import io.trino.metastore.Database; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.HiveType; import io.trino.metastore.Table; import io.trino.plugin.hive.HiveQueryRunner; import io.trino.plugin.hive.containers.HiveHadoop; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.MetastoreMethod; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestConnectorPushdownRulesWithHive.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestConnectorPushdownRulesWithHive.java index 844787214493..25b6b48853c3 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestConnectorPushdownRulesWithHive.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestConnectorPushdownRulesWithHive.java @@ -23,12 +23,12 @@ import io.trino.metadata.TestingFunctionResolution; import io.trino.metastore.Database; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.HiveColumnHandle; import io.trino.plugin.hive.HiveColumnProjectionInfo; import io.trino.plugin.hive.HiveTableHandle; import io.trino.plugin.hive.HiveTransactionHandle; import io.trino.plugin.hive.TestingHiveConnectorFactory; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.connector.CatalogHandle; import io.trino.spi.function.OperatorType; import io.trino.spi.predicate.Domain; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHivePlans.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHivePlans.java index 61af8171ee8a..e177d5415f39 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHivePlans.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHivePlans.java @@ -20,8 +20,8 @@ import io.trino.metadata.TestingFunctionResolution; import io.trino.metastore.Database; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.TestingHiveConnectorFactory; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.function.OperatorType; import io.trino.spi.security.PrincipalType; import io.trino.sql.ir.Between; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHiveProjectionPushdownIntoTableScan.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHiveProjectionPushdownIntoTableScan.java index 478cdc257609..fa6499bce512 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHiveProjectionPushdownIntoTableScan.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHiveProjectionPushdownIntoTableScan.java @@ -23,10 +23,10 @@ import io.trino.metadata.TestingFunctionResolution; import io.trino.metastore.Database; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.HiveColumnHandle; import io.trino.plugin.hive.HiveTableHandle; import io.trino.plugin.hive.TestingHiveConnectorFactory; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.function.OperatorType; import io.trino.spi.predicate.Domain; diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java index 4a5fe257a598..1489d191d0a0 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java @@ -15,7 +15,7 @@ import com.google.inject.Inject; import io.trino.filesystem.TrinoFileSystemFactory; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.spi.security.ConnectorIdentity; import io.trino.spi.type.TypeManager; diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/HudiQueryRunner.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/HudiQueryRunner.java index 915ca2b9ead2..e7ebda90a29e 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/HudiQueryRunner.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/HudiQueryRunner.java @@ -19,9 +19,9 @@ import io.airlift.log.Logging; import io.trino.filesystem.Location; import io.trino.metastore.Database; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.base.util.Closables; import io.trino.plugin.hive.containers.Hive3MinioDataLake; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hudi.testing.HudiTablesInitializer; import io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer; import io.trino.plugin.hudi.testing.TpchHudiTablesInitializer; diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/ResourceHudiTablesInitializer.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/ResourceHudiTablesInitializer.java index 47c7b195abeb..55d7c78b22fb 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/ResourceHudiTablesInitializer.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/ResourceHudiTablesInitializer.java @@ -20,6 +20,7 @@ import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.metastore.Column; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.HiveType; import io.trino.metastore.Partition; import io.trino.metastore.PartitionStatistics; @@ -27,7 +28,6 @@ import io.trino.metastore.PrincipalPrivileges; import io.trino.metastore.StorageFormat; import io.trino.metastore.Table; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hudi.HudiConnector; import io.trino.spi.security.ConnectorIdentity; import io.trino.testing.QueryRunner; diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/TpchHudiTablesInitializer.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/TpchHudiTablesInitializer.java index fc9d04ced4a6..9987fdba387e 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/TpchHudiTablesInitializer.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/TpchHudiTablesInitializer.java @@ -23,11 +23,11 @@ import io.trino.hdfs.HdfsEnvironment; import io.trino.metastore.Column; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.HiveType; import io.trino.metastore.PrincipalPrivileges; import io.trino.metastore.StorageFormat; import io.trino.metastore.Table; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hudi.HudiConnector; import io.trino.plugin.tpch.TpchPlugin; import io.trino.spi.connector.CatalogSchemaName; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index 0e3843c56f39..b34240379f8a 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -36,6 +36,7 @@ import io.trino.filesystem.TrinoFileSystem; import io.trino.metastore.Column; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.TableInfo; import io.trino.plugin.base.classloader.ClassLoaderSafeSystemTable; import io.trino.plugin.base.filter.UtcConstraintExtractor; @@ -43,7 +44,6 @@ import io.trino.plugin.base.projection.ApplyProjectionUtil.ProjectedColumnRepresentation; import io.trino.plugin.hive.HiveStorageFormat; import io.trino.plugin.hive.HiveWrittenPartitions; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.iceberg.aggregation.DataSketchStateSerializer; import io.trino.plugin.iceberg.aggregation.IcebergThetaSketchForStats; import io.trino.plugin.iceberg.catalog.TrinoCatalog; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadataFactory.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadataFactory.java index 6b451c241b31..e7fbb29f4a55 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadataFactory.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadataFactory.java @@ -18,7 +18,7 @@ import com.google.inject.Inject; import io.airlift.concurrent.BoundedExecutor; import io.airlift.json.JsonCodec; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.plugin.iceberg.catalog.TrinoCatalogFactory; import io.trino.spi.connector.CatalogHandle; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java index 386f7dd221b9..254a994cb673 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java @@ -22,6 +22,7 @@ import com.google.inject.Singleton; import com.google.inject.multibindings.Multibinder; import io.trino.filesystem.cache.CacheKeyProvider; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorPageSinkProvider; import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorPageSourceProviderFactory; import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitManager; @@ -30,7 +31,6 @@ import io.trino.plugin.base.metrics.FileFormatDataSourceStats; import io.trino.plugin.base.session.SessionPropertiesProvider; import io.trino.plugin.hive.SortingFileWriterConfig; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.plugin.hive.metastore.thrift.TranslateHiveViews; import io.trino.plugin.hive.orc.OrcReaderConfig; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalogFactory.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalogFactory.java index 98476c7ac942..d18785562efa 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalogFactory.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalogFactory.java @@ -16,9 +16,9 @@ import com.google.inject.Inject; import io.airlift.concurrent.BoundedExecutor; import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.NodeVersion; import io.trino.plugin.hive.TrinoViewHiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.iceberg.ForIcebergMetadata; import io.trino.plugin.iceberg.IcebergConfig; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java index c9718c3ceb92..b56f2b8289f7 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java @@ -23,11 +23,11 @@ import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.metastore.Column; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.Partition; import io.trino.metastore.PrincipalPrivileges; import io.trino.metastore.Storage; import io.trino.plugin.hive.HiveStorageFormat; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.plugin.iceberg.IcebergConfig; import io.trino.plugin.iceberg.IcebergFileFormat; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrationUtils.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrationUtils.java index f869928de41d..ab33cb3330ec 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrationUtils.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrationUtils.java @@ -23,6 +23,7 @@ import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoInputFile; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.Partition; import io.trino.metastore.Storage; import io.trino.parquet.ParquetDataSource; @@ -31,7 +32,6 @@ import io.trino.parquet.reader.MetadataReader; import io.trino.plugin.base.metrics.FileFormatDataSourceStats; import io.trino.plugin.hive.HiveStorageFormat; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.parquet.TrinoParquetDataSource; import io.trino.plugin.iceberg.catalog.TrinoCatalog; import io.trino.plugin.iceberg.fileio.ForwardingInputFile; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java index 702cd94ff027..c1d4f0a0b485 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java @@ -22,6 +22,7 @@ import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.orc.OrcDataSource; import io.trino.orc.OrcReader; import io.trino.orc.OrcReaderOptions; @@ -35,7 +36,6 @@ import io.trino.parquet.reader.MetadataReader; import io.trino.plugin.base.metrics.FileFormatDataSourceStats; import io.trino.plugin.hive.TrinoViewHiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.parquet.TrinoParquetDataSource; import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java index d72e9454fb8e..b1d8ae8afce6 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java @@ -16,8 +16,8 @@ import com.google.common.collect.ImmutableMap; import io.trino.Session; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.TestingHivePlugin; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.security.Identity; import io.trino.spi.security.SelectedRole; import io.trino.testing.AbstractTestQueryFramework; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergProjectionPushdownPlans.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergProjectionPushdownPlans.java index 25f7f6269e37..e68182fac3df 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergProjectionPushdownPlans.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergProjectionPushdownPlans.java @@ -23,7 +23,7 @@ import io.trino.metadata.TestingFunctionResolution; import io.trino.metastore.Database; import io.trino.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.function.OperatorType; import io.trino.spi.predicate.Domain; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestMetadataQueryOptimization.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestMetadataQueryOptimization.java index f38d6fafb324..446b0d5b4c70 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestMetadataQueryOptimization.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestMetadataQueryOptimization.java @@ -18,7 +18,7 @@ import io.trino.Session; import io.trino.metastore.Database; import io.trino.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.spi.security.PrincipalType; import io.trino.sql.ir.Constant; import io.trino.sql.planner.assertions.BasePushdownPlanTest; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestingIcebergFileMetastoreCatalogModule.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestingIcebergFileMetastoreCatalogModule.java index 25a343fe398f..fc02c4b7f82c 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestingIcebergFileMetastoreCatalogModule.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestingIcebergFileMetastoreCatalogModule.java @@ -18,8 +18,8 @@ import io.airlift.configuration.AbstractConfigurationAwareModule; import io.airlift.units.Duration; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.CachingHiveMetastoreModule; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.plugin.hive.metastore.cache.CachingHiveMetastoreConfig; import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/optimizer/TestConnectorPushdownRulesWithIceberg.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/optimizer/TestConnectorPushdownRulesWithIceberg.java index 676f4cf1f7d2..37ebe5d04f8d 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/optimizer/TestConnectorPushdownRulesWithIceberg.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/optimizer/TestConnectorPushdownRulesWithIceberg.java @@ -24,8 +24,8 @@ import io.trino.metadata.TestingFunctionResolution; import io.trino.metastore.Database; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.HiveTransactionHandle; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.iceberg.ColumnIdentity; import io.trino.plugin.iceberg.IcebergColumnHandle; import io.trino.plugin.iceberg.IcebergConnector; diff --git a/testing/trino-tests/src/test/java/io/trino/sql/planner/IcebergCostBasedPlanTestSetup.java b/testing/trino-tests/src/test/java/io/trino/sql/planner/IcebergCostBasedPlanTestSetup.java index 574f02faa61b..3d446863f4f0 100644 --- a/testing/trino-tests/src/test/java/io/trino/sql/planner/IcebergCostBasedPlanTestSetup.java +++ b/testing/trino-tests/src/test/java/io/trino/sql/planner/IcebergCostBasedPlanTestSetup.java @@ -19,8 +19,8 @@ import io.airlift.log.Logger; import io.trino.metastore.Database; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.Table; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.iceberg.IcebergConnector; import io.trino.plugin.iceberg.IcebergConnectorFactory; import io.trino.spi.connector.Connector; From 4be392b29c807557d9b5df0bb6126b09dd6abb88 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Wed, 31 Jul 2024 00:22:47 -0700 Subject: [PATCH 06/12] Cleanup binding of FlushMetadataCacheProcedure --- .../metastore/DeltaLakeMetastoreModule.java | 2 +- .../TestingDeltaLakeMetastoreModule.java | 2 +- .../metastore/CachingHiveMetastoreModule.java | 19 ------------------- .../hive/metastore/HiveMetastoreModule.java | 2 +- .../hive/procedure/HiveProcedureModule.java | 7 +++++++ .../IcebergFileMetastoreCatalogModule.java | 2 +- .../IcebergHiveMetastoreCatalogModule.java | 2 +- ...tingIcebergFileMetastoreCatalogModule.java | 2 +- 8 files changed, 13 insertions(+), 25 deletions(-) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/DeltaLakeMetastoreModule.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/DeltaLakeMetastoreModule.java index 4eb4f8fe3ab7..6bfd78808c7a 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/DeltaLakeMetastoreModule.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/DeltaLakeMetastoreModule.java @@ -39,7 +39,7 @@ protected void setup(Binder binder) bindMetastoreModule("glue", new DeltaLakeGlueMetastoreModule()); bindMetastoreModule("glue-v1", new DeltaLakeGlueV1MetastoreModule()); - install(new CachingHiveMetastoreModule(false)); + install(new CachingHiveMetastoreModule()); } private void bindMetastoreModule(String name, Module module) diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestingDeltaLakeMetastoreModule.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestingDeltaLakeMetastoreModule.java index 98488d0330e4..2c772a0aef39 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestingDeltaLakeMetastoreModule.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestingDeltaLakeMetastoreModule.java @@ -42,7 +42,7 @@ public TestingDeltaLakeMetastoreModule(HiveMetastore metastore) public void setup(Binder binder) { binder.bind(HiveMetastoreFactory.class).annotatedWith(RawHiveMetastoreFactory.class).toInstance(HiveMetastoreFactory.ofInstance(metastore)); - install(new CachingHiveMetastoreModule(false)); + install(new CachingHiveMetastoreModule()); binder.bind(DeltaLakeTableOperationsProvider.class).to(DeltaLakeFileMetastoreTableOperationsProvider.class).in(Scopes.SINGLETON); binder.bind(Key.get(boolean.class, HideDeltaLakeTables.class)).toInstance(false); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/CachingHiveMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/CachingHiveMetastoreModule.java index 3d6182349366..b0ba34f6e411 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/CachingHiveMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/CachingHiveMetastoreModule.java @@ -19,33 +19,20 @@ import com.google.inject.Singleton; import io.airlift.configuration.AbstractConfigurationAwareModule; import io.trino.metastore.HiveMetastoreFactory; -import io.trino.plugin.hive.fs.DirectoryLister; import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.metastore.cache.CachingHiveMetastoreConfig; import io.trino.plugin.hive.metastore.cache.ImpersonationCachingConfig; import io.trino.plugin.hive.metastore.cache.SharedHiveMetastoreCache; import io.trino.plugin.hive.metastore.cache.SharedHiveMetastoreCache.CachingHiveMetastoreFactory; -import io.trino.plugin.hive.metastore.glue.GlueCache; -import io.trino.plugin.hive.procedure.FlushMetadataCacheProcedure; -import io.trino.spi.procedure.Procedure; import java.util.Optional; -import static com.google.inject.multibindings.Multibinder.newSetBinder; -import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; import static io.airlift.configuration.ConfigBinder.configBinder; import static org.weakref.jmx.guice.ExportBinder.newExporter; public class CachingHiveMetastoreModule extends AbstractConfigurationAwareModule { - private final boolean installFlushMetadataCacheProcedure; - - public CachingHiveMetastoreModule(boolean installFlushMetadataCacheProcedure) - { - this.installFlushMetadataCacheProcedure = installFlushMetadataCacheProcedure; - } - @Override protected void setup(Binder binder) { @@ -56,12 +43,6 @@ protected void setup(Binder binder) // export under the old name, for backwards compatibility newExporter(binder).export(HiveMetastoreFactory.class) .as(generator -> generator.generatedNameOf(CachingHiveMetastore.class)); - - if (installFlushMetadataCacheProcedure) { - newOptionalBinder(binder, GlueCache.class); - newOptionalBinder(binder, DirectoryLister.class); - newSetBinder(binder, Procedure.class).addBinding().toProvider(FlushMetadataCacheProcedure.class).in(Scopes.SINGLETON); - } } @Provides diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastoreModule.java index 9703885f428b..15f153afe7f7 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastoreModule.java @@ -55,7 +55,7 @@ protected void setup(Binder binder) bindMetastoreModule("glue-v1", new io.trino.plugin.hive.metastore.glue.v1.GlueMetastoreModule()); } - install(new CachingHiveMetastoreModule(true)); + install(new CachingHiveMetastoreModule()); } private void bindMetastoreModule(String name, Module module) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/HiveProcedureModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/HiveProcedureModule.java index 465d4d70ebe0..1f7cb0bd32dc 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/HiveProcedureModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/HiveProcedureModule.java @@ -17,10 +17,13 @@ import com.google.inject.Module; import com.google.inject.Scopes; import com.google.inject.multibindings.Multibinder; +import io.trino.plugin.hive.fs.DirectoryLister; +import io.trino.plugin.hive.metastore.glue.GlueCache; import io.trino.spi.connector.TableProcedureMetadata; import io.trino.spi.procedure.Procedure; import static com.google.inject.multibindings.Multibinder.newSetBinder; +import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; public class HiveProcedureModule implements Module @@ -34,8 +37,12 @@ public void configure(Binder binder) procedures.addBinding().toProvider(UnregisterPartitionProcedure.class).in(Scopes.SINGLETON); procedures.addBinding().toProvider(SyncPartitionMetadataProcedure.class).in(Scopes.SINGLETON); procedures.addBinding().toProvider(DropStatsProcedure.class).in(Scopes.SINGLETON); + procedures.addBinding().toProvider(FlushMetadataCacheProcedure.class).in(Scopes.SINGLETON); Multibinder tableProcedures = newSetBinder(binder, TableProcedureMetadata.class); tableProcedures.addBinding().toProvider(OptimizeTableProcedure.class).in(Scopes.SINGLETON); + + newOptionalBinder(binder, GlueCache.class); + newOptionalBinder(binder, DirectoryLister.class); } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/IcebergFileMetastoreCatalogModule.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/IcebergFileMetastoreCatalogModule.java index cef2b7927262..97a334c34420 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/IcebergFileMetastoreCatalogModule.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/IcebergFileMetastoreCatalogModule.java @@ -47,7 +47,7 @@ protected void setup(Binder binder) binder.bind(TrinoCatalogFactory.class).to(TrinoHiveCatalogFactory.class).in(Scopes.SINGLETON); binder.bind(MetastoreValidator.class).asEagerSingleton(); binder.bind(Key.get(boolean.class, HideDeltaLakeTables.class)).toInstance(HIDE_DELTA_LAKE_TABLES_IN_ICEBERG); - install(new CachingHiveMetastoreModule(false)); + install(new CachingHiveMetastoreModule()); configBinder(binder).bindConfigDefaults(CachingHiveMetastoreConfig.class, config -> { // ensure caching metastore wrapper isn't created, as it's not leveraged by Iceberg diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/IcebergHiveMetastoreCatalogModule.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/IcebergHiveMetastoreCatalogModule.java index b254247f9828..c5ee875fda0e 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/IcebergHiveMetastoreCatalogModule.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/IcebergHiveMetastoreCatalogModule.java @@ -49,7 +49,7 @@ protected void setup(Binder binder) binder.bind(MetastoreValidator.class).asEagerSingleton(); binder.bind(Key.get(boolean.class, TranslateHiveViews.class)).toInstance(false); binder.bind(Key.get(boolean.class, HideDeltaLakeTables.class)).toInstance(HIDE_DELTA_LAKE_TABLES_IN_ICEBERG); - install(new CachingHiveMetastoreModule(false)); + install(new CachingHiveMetastoreModule()); configBinder(binder).bindConfigDefaults(CachingHiveMetastoreConfig.class, config -> { // ensure caching metastore wrapper isn't created, as it's not leveraged by Iceberg diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestingIcebergFileMetastoreCatalogModule.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestingIcebergFileMetastoreCatalogModule.java index fc02c4b7f82c..695ae115c5c5 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestingIcebergFileMetastoreCatalogModule.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestingIcebergFileMetastoreCatalogModule.java @@ -45,7 +45,7 @@ public TestingIcebergFileMetastoreCatalogModule(HiveMetastore metastore) protected void setup(Binder binder) { binder.bind(HiveMetastoreFactory.class).annotatedWith(RawHiveMetastoreFactory.class).toInstance(HiveMetastoreFactory.ofInstance(metastore)); - install(new CachingHiveMetastoreModule(false)); + install(new CachingHiveMetastoreModule()); binder.bind(IcebergTableOperationsProvider.class).to(FileMetastoreTableOperationsProvider.class).in(Scopes.SINGLETON); binder.bind(TrinoCatalogFactory.class).to(TrinoHiveCatalogFactory.class).in(Scopes.SINGLETON); From c3739375edf4ac4bd9cd5cc9b6566f1779dd71d5 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Tue, 30 Jul 2024 23:51:20 -0700 Subject: [PATCH 07/12] Move CachingHiveMetastore to metastore module --- lib/trino-metastore/pom.xml | 40 +++++++++++++++++++ .../metastore/RawHiveMetastoreFactory.java | 2 +- .../metastore/cache/CachingHiveMetastore.java | 17 ++++---- .../cache/CachingHiveMetastoreConfig.java | 2 +- .../metastore/cache}/HivePartitionName.java | 2 +- .../trino/metastore/cache}/HiveTableName.java | 2 +- .../cache/ImpersonationCachingConfig.java | 2 +- .../metastore/cache}/PartitionFilter.java | 4 +- .../cache/ReentrantBoundedExecutor.java | 2 +- .../cache/SharedHiveMetastoreCache.java | 4 +- .../cache/TestCachingHiveMetastoreConfig.java | 4 +- .../cache/TestImpersonationCachingConfig.java | 2 +- .../cache/TestReentrantBoundedExecutor.java | 2 +- .../deltalake/DeltaLakeMetadataFactory.java | 4 +- .../FlushMetadataCacheProcedure.java | 2 +- .../deltalake/TestDeltaLakeMetadata.java | 2 +- .../TestingDeltaLakeMetastoreModule.java | 2 +- .../plugin/hive/HiveMetadataFactory.java | 2 +- .../plugin/hive/HivePageSinkProvider.java | 4 +- .../metastore/CachingHiveMetastoreModule.java | 11 ++--- .../hive/metastore/HiveMetastoreModule.java | 1 + .../metastore/file/FileMetastoreModule.java | 2 +- .../metastore/glue/GlueMetastoreModule.java | 4 +- .../metastore/glue/InMemoryGlueCache.java | 2 +- .../glue/v1/GlueMetastoreModule.java | 2 +- .../thrift/ThriftMetastoreModule.java | 2 +- .../FlushMetadataCacheProcedure.java | 2 +- .../cache/TestCachingHiveMetastore.java | 3 +- ...stCachingHiveMetastoreWithQueryRunner.java | 2 +- .../plugin/hudi/HudiMetadataFactory.java | 4 +- .../iceberg/IcebergMetadataFactory.java | 2 +- .../trino/plugin/iceberg/IcebergModule.java | 2 +- .../iceberg/catalog/MetastoreValidator.java | 2 +- .../file/FileMetastoreTableOperations.java | 2 +- .../IcebergFileMetastoreCatalogModule.java | 2 +- .../hms/AbstractMetastoreTableOperations.java | 2 +- .../hms/HiveMetastoreTableOperations.java | 2 +- .../IcebergHiveMetastoreCatalogModule.java | 2 +- .../iceberg/catalog/hms/TrinoHiveCatalog.java | 2 +- .../catalog/hms/TrinoHiveCatalogFactory.java | 4 +- .../iceberg/procedure/MigrateProcedure.java | 2 +- .../plugin/iceberg/IcebergTestUtils.java | 4 +- .../iceberg/TestIcebergMergeAppend.java | 4 +- .../TestIcebergOrcMetricsCollection.java | 4 +- .../iceberg/TestIcebergSplitSource.java | 8 ++-- .../TestAbstractIcebergTableOperations.java | 2 +- ...TestTrinoHiveCatalogWithFileMetastore.java | 4 +- ...tingIcebergFileMetastoreCatalogModule.java | 4 +- ...TestTrinoHiveCatalogWithHiveMetastore.java | 4 +- .../product/deltalake/TestDeltaLakeJmx.java | 4 +- 50 files changed, 119 insertions(+), 79 deletions(-) rename {plugin/trino-hive/src/main/java/io/trino/plugin/hive => lib/trino-metastore/src/main/java/io/trino}/metastore/RawHiveMetastoreFactory.java (96%) rename {plugin/trino-hive/src/main/java/io/trino/plugin/hive => lib/trino-metastore/src/main/java/io/trino}/metastore/cache/CachingHiveMetastore.java (98%) rename {plugin/trino-hive/src/main/java/io/trino/plugin/hive => lib/trino-metastore/src/main/java/io/trino}/metastore/cache/CachingHiveMetastoreConfig.java (99%) rename {plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore => lib/trino-metastore/src/main/java/io/trino/metastore/cache}/HivePartitionName.java (98%) rename {plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore => lib/trino-metastore/src/main/java/io/trino/metastore/cache}/HiveTableName.java (98%) rename {plugin/trino-hive/src/main/java/io/trino/plugin/hive => lib/trino-metastore/src/main/java/io/trino}/metastore/cache/ImpersonationCachingConfig.java (97%) rename {plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore => lib/trino-metastore/src/main/java/io/trino/metastore/cache}/PartitionFilter.java (96%) rename {plugin/trino-hive/src/main/java/io/trino/plugin/hive => lib/trino-metastore/src/main/java/io/trino}/metastore/cache/ReentrantBoundedExecutor.java (97%) rename {plugin/trino-hive/src/main/java/io/trino/plugin/hive => lib/trino-metastore/src/main/java/io/trino}/metastore/cache/SharedHiveMetastoreCache.java (99%) rename {plugin/trino-hive/src/test/java/io/trino/plugin/hive => lib/trino-metastore/src/test/java/io/trino}/metastore/cache/TestCachingHiveMetastoreConfig.java (96%) rename {plugin/trino-hive/src/test/java/io/trino/plugin/hive => lib/trino-metastore/src/test/java/io/trino}/metastore/cache/TestImpersonationCachingConfig.java (97%) rename {plugin/trino-hive/src/test/java/io/trino/plugin/hive => lib/trino-metastore/src/test/java/io/trino}/metastore/cache/TestReentrantBoundedExecutor.java (97%) diff --git a/lib/trino-metastore/pom.xml b/lib/trino-metastore/pom.xml index c0549ec5c888..a4415f39af65 100644 --- a/lib/trino-metastore/pom.xml +++ b/lib/trino-metastore/pom.xml @@ -33,11 +33,31 @@ guava + + com.google.inject + guice + + + + io.airlift + concurrent + + + + io.airlift + configuration + + io.airlift slice + + io.airlift + units + + io.opentelemetry opentelemetry-api @@ -53,11 +73,31 @@ opentelemetry-semconv + + io.trino + trino-cache + + io.trino trino-spi + + jakarta.annotation + jakarta.annotation-api + + + + jakarta.validation + jakarta.validation-api + + + + org.weakref + jmxutils + + io.airlift junit-extensions diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/RawHiveMetastoreFactory.java b/lib/trino-metastore/src/main/java/io/trino/metastore/RawHiveMetastoreFactory.java similarity index 96% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/RawHiveMetastoreFactory.java rename to lib/trino-metastore/src/main/java/io/trino/metastore/RawHiveMetastoreFactory.java index a9d7b5b793b0..573bb4b23efd 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/RawHiveMetastoreFactory.java +++ b/lib/trino-metastore/src/main/java/io/trino/metastore/RawHiveMetastoreFactory.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.metastore; +package io.trino.metastore; import com.google.inject.BindingAnnotation; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java b/lib/trino-metastore/src/main/java/io/trino/metastore/cache/CachingHiveMetastore.java similarity index 98% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java rename to lib/trino-metastore/src/main/java/io/trino/metastore/cache/CachingHiveMetastore.java index b8c44879c49f..0c22b77d39f9 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java +++ b/lib/trino-metastore/src/main/java/io/trino/metastore/cache/CachingHiveMetastore.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.metastore.cache; +package io.trino.metastore.cache; import com.google.common.cache.Cache; import com.google.common.cache.CacheLoader; @@ -44,9 +44,6 @@ import io.trino.metastore.StatisticsUpdateMode; import io.trino.metastore.Table; import io.trino.metastore.TableInfo; -import io.trino.plugin.hive.metastore.HivePartitionName; -import io.trino.plugin.hive.metastore.HiveTableName; -import io.trino.plugin.hive.metastore.PartitionFilter; import io.trino.spi.TrinoException; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.function.LanguageFunction; @@ -86,12 +83,12 @@ import static io.trino.cache.CacheUtils.invalidateAllIf; import static io.trino.cache.CacheUtils.uncheckedCacheGet; import static io.trino.metastore.Partitions.makePartName; -import static io.trino.plugin.hive.metastore.HivePartitionName.hivePartitionName; -import static io.trino.plugin.hive.metastore.HiveTableName.hiveTableName; -import static io.trino.plugin.hive.metastore.PartitionFilter.partitionFilter; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.ObjectType.OTHER; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.ObjectType.PARTITION; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.ObjectType.STATS; +import static io.trino.metastore.cache.CachingHiveMetastore.ObjectType.OTHER; +import static io.trino.metastore.cache.CachingHiveMetastore.ObjectType.PARTITION; +import static io.trino.metastore.cache.CachingHiveMetastore.ObjectType.STATS; +import static io.trino.metastore.cache.HivePartitionName.hivePartitionName; +import static io.trino.metastore.cache.HiveTableName.hiveTableName; +import static io.trino.metastore.cache.PartitionFilter.partitionFilter; import static java.util.Collections.unmodifiableSet; import static java.util.Objects.requireNonNull; import static java.util.concurrent.TimeUnit.MILLISECONDS; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastoreConfig.java b/lib/trino-metastore/src/main/java/io/trino/metastore/cache/CachingHiveMetastoreConfig.java similarity index 99% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastoreConfig.java rename to lib/trino-metastore/src/main/java/io/trino/metastore/cache/CachingHiveMetastoreConfig.java index 2f97cbc454ed..b69c62d6b131 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastoreConfig.java +++ b/lib/trino-metastore/src/main/java/io/trino/metastore/cache/CachingHiveMetastoreConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.metastore.cache; +package io.trino.metastore.cache; import io.airlift.configuration.Config; import io.airlift.units.Duration; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HivePartitionName.java b/lib/trino-metastore/src/main/java/io/trino/metastore/cache/HivePartitionName.java similarity index 98% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HivePartitionName.java rename to lib/trino-metastore/src/main/java/io/trino/metastore/cache/HivePartitionName.java index 9aeba984c4fa..dab9f21cfa00 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HivePartitionName.java +++ b/lib/trino-metastore/src/main/java/io/trino/metastore/cache/HivePartitionName.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.metastore; +package io.trino.metastore.cache; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveTableName.java b/lib/trino-metastore/src/main/java/io/trino/metastore/cache/HiveTableName.java similarity index 98% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveTableName.java rename to lib/trino-metastore/src/main/java/io/trino/metastore/cache/HiveTableName.java index dde5609a0bac..85281946e1a8 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveTableName.java +++ b/lib/trino-metastore/src/main/java/io/trino/metastore/cache/HiveTableName.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.metastore; +package io.trino.metastore.cache; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/ImpersonationCachingConfig.java b/lib/trino-metastore/src/main/java/io/trino/metastore/cache/ImpersonationCachingConfig.java similarity index 97% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/ImpersonationCachingConfig.java rename to lib/trino-metastore/src/main/java/io/trino/metastore/cache/ImpersonationCachingConfig.java index 15eb31787d94..29268b002c1d 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/ImpersonationCachingConfig.java +++ b/lib/trino-metastore/src/main/java/io/trino/metastore/cache/ImpersonationCachingConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.metastore.cache; +package io.trino.metastore.cache; import io.airlift.configuration.Config; import io.airlift.units.Duration; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/PartitionFilter.java b/lib/trino-metastore/src/main/java/io/trino/metastore/cache/PartitionFilter.java similarity index 96% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/PartitionFilter.java rename to lib/trino-metastore/src/main/java/io/trino/metastore/cache/PartitionFilter.java index 883b9a1641c2..73695573927c 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/PartitionFilter.java +++ b/lib/trino-metastore/src/main/java/io/trino/metastore/cache/PartitionFilter.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.metastore; +package io.trino.metastore.cache; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -23,7 +23,7 @@ import java.util.Objects; import static com.google.common.base.MoreObjects.toStringHelper; -import static io.trino.plugin.hive.metastore.HiveTableName.hiveTableName; +import static io.trino.metastore.cache.HiveTableName.hiveTableName; import static java.util.Objects.requireNonNull; @Immutable diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/ReentrantBoundedExecutor.java b/lib/trino-metastore/src/main/java/io/trino/metastore/cache/ReentrantBoundedExecutor.java similarity index 97% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/ReentrantBoundedExecutor.java rename to lib/trino-metastore/src/main/java/io/trino/metastore/cache/ReentrantBoundedExecutor.java index 6005d3c77d6d..0480695a8f00 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/ReentrantBoundedExecutor.java +++ b/lib/trino-metastore/src/main/java/io/trino/metastore/cache/ReentrantBoundedExecutor.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.metastore.cache; +package io.trino.metastore.cache; import io.airlift.concurrent.BoundedExecutor; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/SharedHiveMetastoreCache.java b/lib/trino-metastore/src/main/java/io/trino/metastore/cache/SharedHiveMetastoreCache.java similarity index 99% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/SharedHiveMetastoreCache.java rename to lib/trino-metastore/src/main/java/io/trino/metastore/cache/SharedHiveMetastoreCache.java index 8e2c731b8d92..68adfd506063 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/SharedHiveMetastoreCache.java +++ b/lib/trino-metastore/src/main/java/io/trino/metastore/cache/SharedHiveMetastoreCache.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.metastore.cache; +package io.trino.metastore.cache; import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; @@ -25,7 +25,7 @@ import io.airlift.units.Duration; import io.trino.metastore.HiveMetastore; import io.trino.metastore.HiveMetastoreFactory; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.ObjectType; +import io.trino.metastore.cache.CachingHiveMetastore.ObjectType; import io.trino.spi.NodeManager; import io.trino.spi.TrinoException; import io.trino.spi.catalog.CatalogName; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreConfig.java b/lib/trino-metastore/src/test/java/io/trino/metastore/cache/TestCachingHiveMetastoreConfig.java similarity index 96% rename from plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreConfig.java rename to lib/trino-metastore/src/test/java/io/trino/metastore/cache/TestCachingHiveMetastoreConfig.java index 38d59830b42b..5047c09a1301 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreConfig.java +++ b/lib/trino-metastore/src/test/java/io/trino/metastore/cache/TestCachingHiveMetastoreConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.metastore.cache; +package io.trino.metastore.cache; import com.google.common.collect.ImmutableMap; import io.airlift.units.Duration; @@ -22,7 +22,7 @@ import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastoreConfig.DEFAULT_STATS_CACHE_TTL; +import static io.trino.metastore.cache.CachingHiveMetastoreConfig.DEFAULT_STATS_CACHE_TTL; import static java.util.concurrent.TimeUnit.DAYS; import static java.util.concurrent.TimeUnit.HOURS; import static java.util.concurrent.TimeUnit.MILLISECONDS; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestImpersonationCachingConfig.java b/lib/trino-metastore/src/test/java/io/trino/metastore/cache/TestImpersonationCachingConfig.java similarity index 97% rename from plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestImpersonationCachingConfig.java rename to lib/trino-metastore/src/test/java/io/trino/metastore/cache/TestImpersonationCachingConfig.java index 63edc9a23e1e..7127c141532d 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestImpersonationCachingConfig.java +++ b/lib/trino-metastore/src/test/java/io/trino/metastore/cache/TestImpersonationCachingConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.metastore.cache; +package io.trino.metastore.cache; import com.google.common.collect.ImmutableMap; import io.airlift.units.Duration; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestReentrantBoundedExecutor.java b/lib/trino-metastore/src/test/java/io/trino/metastore/cache/TestReentrantBoundedExecutor.java similarity index 97% rename from plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestReentrantBoundedExecutor.java rename to lib/trino-metastore/src/test/java/io/trino/metastore/cache/TestReentrantBoundedExecutor.java index 3f4c04ae283c..feadf2e49a18 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestReentrantBoundedExecutor.java +++ b/lib/trino-metastore/src/test/java/io/trino/metastore/cache/TestReentrantBoundedExecutor.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.metastore.cache; +package io.trino.metastore.cache; import com.google.common.util.concurrent.SettableFuture; import org.junit.jupiter.api.Test; diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadataFactory.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadataFactory.java index dc1241ed504d..34b048049ab1 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadataFactory.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadataFactory.java @@ -18,6 +18,7 @@ import io.airlift.json.JsonCodec; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.plugin.deltalake.metastore.DeltaLakeTableMetadataScheduler; import io.trino.plugin.deltalake.metastore.HiveMetastoreBackedDeltaLakeMetastore; import io.trino.plugin.deltalake.statistics.CachingExtendedStatisticsAccess; @@ -27,7 +28,6 @@ import io.trino.plugin.deltalake.transactionlog.writer.TransactionLogWriterFactory; import io.trino.plugin.hive.NodeVersion; import io.trino.plugin.hive.TrinoViewHiveMetastore; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.security.AccessControlMetadata; import io.trino.spi.NodeManager; import io.trino.spi.security.ConnectorIdentity; @@ -38,7 +38,7 @@ import java.util.concurrent.ExecutorService; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; +import static io.trino.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static java.util.Objects.requireNonNull; public class DeltaLakeMetadataFactory diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/procedure/FlushMetadataCacheProcedure.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/procedure/FlushMetadataCacheProcedure.java index 220636d141f1..97fa7e73c306 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/procedure/FlushMetadataCacheProcedure.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/procedure/FlushMetadataCacheProcedure.java @@ -16,9 +16,9 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Inject; import com.google.inject.Provider; +import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.plugin.deltalake.statistics.CachingExtendedStatisticsAccess; import io.trino.plugin.deltalake.transactionlog.TransactionLogAccess; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.spi.TrinoException; import io.trino.spi.classloader.ThreadContextClassLoader; import io.trino.spi.connector.SchemaTableName; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMetadata.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMetadata.java index 8ffa699674f9..f2527040d271 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMetadata.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMetadata.java @@ -31,13 +31,13 @@ import io.trino.hdfs.TrinoHdfsFileSystemStats; import io.trino.metastore.Database; import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.RawHiveMetastoreFactory; import io.trino.plugin.deltalake.metastore.DeltaLakeMetastore; import io.trino.plugin.deltalake.metastore.DeltaLakeMetastoreModule; import io.trino.plugin.deltalake.metastore.HiveMetastoreBackedDeltaLakeMetastore; import io.trino.plugin.deltalake.transactionlog.MetadataEntry; import io.trino.plugin.deltalake.transactionlog.ProtocolEntry; import io.trino.plugin.hive.NodeVersion; -import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.spi.NodeManager; import io.trino.spi.PageIndexerFactory; import io.trino.spi.TrinoException; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestingDeltaLakeMetastoreModule.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestingDeltaLakeMetastoreModule.java index 2c772a0aef39..07bf6000f02d 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestingDeltaLakeMetastoreModule.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestingDeltaLakeMetastoreModule.java @@ -19,12 +19,12 @@ import io.airlift.configuration.AbstractConfigurationAwareModule; import io.trino.metastore.HiveMetastore; import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.RawHiveMetastoreFactory; import io.trino.plugin.deltalake.AllowDeltaLakeManagedTableRename; import io.trino.plugin.deltalake.MaxTableParameterLength; import io.trino.plugin.deltalake.metastore.file.DeltaLakeFileMetastoreTableOperationsProvider; import io.trino.plugin.hive.HideDeltaLakeTables; import io.trino.plugin.hive.metastore.CachingHiveMetastoreModule; -import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import static java.util.Objects.requireNonNull; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadataFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadataFactory.java index f63c5dad2b42..b8fd3fc20f59 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadataFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadataFactory.java @@ -39,7 +39,7 @@ import java.util.concurrent.ScheduledExecutorService; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; +import static io.trino.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static java.util.Objects.requireNonNull; public class HiveMetadataFactory diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java index d1030f73de35..d71d24935d34 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java @@ -23,8 +23,8 @@ import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.SortingColumn; +import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.metastore.HivePageSinkMetadataProvider; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.spi.PageIndexerFactory; import io.trino.spi.PageSorter; import io.trino.spi.connector.ConnectorInsertTableHandle; @@ -48,7 +48,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator; import static io.airlift.concurrent.Threads.daemonThreadsNamed; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; +import static io.trino.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static java.util.Objects.requireNonNull; import static java.util.concurrent.Executors.newFixedThreadPool; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/CachingHiveMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/CachingHiveMetastoreModule.java index b0ba34f6e411..5b2e0cc2c7ae 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/CachingHiveMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/CachingHiveMetastoreModule.java @@ -19,11 +19,12 @@ import com.google.inject.Singleton; import io.airlift.configuration.AbstractConfigurationAwareModule; import io.trino.metastore.HiveMetastoreFactory; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastoreConfig; -import io.trino.plugin.hive.metastore.cache.ImpersonationCachingConfig; -import io.trino.plugin.hive.metastore.cache.SharedHiveMetastoreCache; -import io.trino.plugin.hive.metastore.cache.SharedHiveMetastoreCache.CachingHiveMetastoreFactory; +import io.trino.metastore.RawHiveMetastoreFactory; +import io.trino.metastore.cache.CachingHiveMetastore; +import io.trino.metastore.cache.CachingHiveMetastoreConfig; +import io.trino.metastore.cache.ImpersonationCachingConfig; +import io.trino.metastore.cache.SharedHiveMetastoreCache; +import io.trino.metastore.cache.SharedHiveMetastoreCache.CachingHiveMetastoreFactory; import java.util.Optional; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastoreModule.java index 15f153afe7f7..7e247958ab31 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastoreModule.java @@ -21,6 +21,7 @@ import io.airlift.configuration.AbstractConfigurationAwareModule; import io.trino.metastore.HiveMetastore; import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.RawHiveMetastoreFactory; import io.trino.plugin.hive.AllowHiveTableRename; import io.trino.plugin.hive.HideDeltaLakeTables; import io.trino.plugin.hive.metastore.file.FileMetastoreModule; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileMetastoreModule.java index e990434bea34..2490ec55ea4c 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileMetastoreModule.java @@ -18,8 +18,8 @@ import com.google.inject.Module; import com.google.inject.Scopes; import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.RawHiveMetastoreFactory; import io.trino.plugin.hive.AllowHiveTableRename; -import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import static io.airlift.configuration.ConfigBinder.configBinder; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreModule.java index d1431bb30de7..b4fe87a8ee7e 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreModule.java @@ -28,10 +28,10 @@ import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.instrumentation.awssdk.v2_2.AwsSdkTelemetry; import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.RawHiveMetastoreFactory; +import io.trino.metastore.cache.CachingHiveMetastoreConfig; import io.trino.plugin.hive.AllowHiveTableRename; import io.trino.plugin.hive.HideDeltaLakeTables; -import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastoreConfig; import io.trino.spi.NodeManager; import io.trino.spi.catalog.CatalogName; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/InMemoryGlueCache.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/InMemoryGlueCache.java index fe1980b1f69f..dc627293c35e 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/InMemoryGlueCache.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/InMemoryGlueCache.java @@ -26,7 +26,7 @@ import io.trino.metastore.Partition; import io.trino.metastore.Table; import io.trino.metastore.TableInfo; -import io.trino.plugin.hive.metastore.cache.ReentrantBoundedExecutor; +import io.trino.metastore.cache.ReentrantBoundedExecutor; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.function.LanguageFunction; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueMetastoreModule.java index eddec0839eb6..1d1bbf384cf3 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueMetastoreModule.java @@ -32,8 +32,8 @@ import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.instrumentation.awssdk.v1_11.AwsSdkTelemetry; import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.RawHiveMetastoreFactory; import io.trino.plugin.hive.AllowHiveTableRename; -import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.plugin.hive.metastore.glue.GlueMetastoreStats; import java.lang.annotation.Annotation; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java index 8c5f0b71b37c..c83d47e60aff 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java @@ -22,10 +22,10 @@ import com.google.inject.multibindings.OptionalBinder; import io.airlift.configuration.AbstractConfigurationAwareModule; import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.RawHiveMetastoreFactory; import io.trino.plugin.base.security.UserNameProvider; import io.trino.plugin.hive.AllowHiveTableRename; import io.trino.plugin.hive.ForHiveMetastore; -import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import java.util.concurrent.ExecutorService; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/FlushMetadataCacheProcedure.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/FlushMetadataCacheProcedure.java index 8cf5ae65c4c7..fffea2dcd713 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/FlushMetadataCacheProcedure.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/FlushMetadataCacheProcedure.java @@ -20,9 +20,9 @@ import io.trino.metastore.HiveMetastore; import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.Table; +import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.HiveErrorCode; import io.trino.plugin.hive.fs.DirectoryLister; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.metastore.glue.GlueCache; import io.trino.plugin.hive.metastore.glue.PartitionName; import io.trino.spi.StandardErrorCode; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java index 148ec287cd1f..c9e62ef88fae 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java @@ -31,6 +31,7 @@ import io.trino.metastore.PartitionStatistics; import io.trino.metastore.Table; import io.trino.metastore.TableInfo; +import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.plugin.base.util.AutoCloseableCloser; import io.trino.plugin.hive.HiveColumnHandle; import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; @@ -72,12 +73,12 @@ import static io.trino.metastore.HiveColumnStatistics.createIntegerColumnStatistics; import static io.trino.metastore.HiveType.HIVE_STRING; import static io.trino.metastore.StatisticsUpdateMode.MERGE_INCREMENTAL; +import static io.trino.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; import static io.trino.plugin.hive.HiveColumnHandle.createBaseColumn; import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; import static io.trino.plugin.hive.metastore.MetastoreUtil.computePartitionKeyFilter; import static io.trino.plugin.hive.metastore.MetastoreUtil.makePartitionName; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static io.trino.plugin.hive.metastore.thrift.MockThriftMetastoreClient.BAD_DATABASE; import static io.trino.plugin.hive.metastore.thrift.MockThriftMetastoreClient.BAD_PARTITION; import static io.trino.plugin.hive.metastore.thrift.MockThriftMetastoreClient.PARTITION_COLUMN_NAMES; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java index 6acda90db491..ea53c3d45836 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java @@ -20,8 +20,8 @@ import io.trino.Session; import io.trino.metastore.HiveMetastore; import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.RawHiveMetastoreFactory; import io.trino.plugin.hive.HiveQueryRunner; -import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.spi.security.Identity; import io.trino.spi.security.SelectedRole; import io.trino.testing.AbstractTestQueryFramework; diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java index 1489d191d0a0..444d20e6a833 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiMetadataFactory.java @@ -16,13 +16,13 @@ import com.google.inject.Inject; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.metastore.HiveMetastoreFactory; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; +import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.spi.security.ConnectorIdentity; import io.trino.spi.type.TypeManager; import java.util.Optional; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; +import static io.trino.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static java.util.Objects.requireNonNull; public class HudiMetadataFactory diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadataFactory.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadataFactory.java index e7fbb29f4a55..6437912766b5 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadataFactory.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadataFactory.java @@ -19,7 +19,7 @@ import io.airlift.concurrent.BoundedExecutor; import io.airlift.json.JsonCodec; import io.trino.metastore.HiveMetastoreFactory; -import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; +import io.trino.metastore.RawHiveMetastoreFactory; import io.trino.plugin.iceberg.catalog.TrinoCatalogFactory; import io.trino.spi.connector.CatalogHandle; import io.trino.spi.security.ConnectorIdentity; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java index 254a994cb673..ed5b6b10fb86 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java @@ -23,6 +23,7 @@ import com.google.inject.multibindings.Multibinder; import io.trino.filesystem.cache.CacheKeyProvider; import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.RawHiveMetastoreFactory; import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorPageSinkProvider; import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorPageSourceProviderFactory; import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitManager; @@ -31,7 +32,6 @@ import io.trino.plugin.base.metrics.FileFormatDataSourceStats; import io.trino.plugin.base.session.SessionPropertiesProvider; import io.trino.plugin.hive.SortingFileWriterConfig; -import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.plugin.hive.metastore.thrift.TranslateHiveViews; import io.trino.plugin.hive.orc.OrcReaderConfig; import io.trino.plugin.hive.orc.OrcWriterConfig; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/MetastoreValidator.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/MetastoreValidator.java index 4cde7c705eec..c0a1fe3869b8 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/MetastoreValidator.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/MetastoreValidator.java @@ -14,7 +14,7 @@ package io.trino.plugin.iceberg.catalog; import com.google.inject.Inject; -import io.trino.plugin.hive.metastore.cache.SharedHiveMetastoreCache; +import io.trino.metastore.cache.SharedHiveMetastoreCache; public class MetastoreValidator { diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/FileMetastoreTableOperations.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/FileMetastoreTableOperations.java index eacdccf9b731..751372cbcfc6 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/FileMetastoreTableOperations.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/FileMetastoreTableOperations.java @@ -16,8 +16,8 @@ import io.trino.annotation.NotThreadSafe; import io.trino.metastore.PrincipalPrivileges; import io.trino.metastore.Table; +import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.metastore.MetastoreUtil; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.iceberg.catalog.hms.AbstractMetastoreTableOperations; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/IcebergFileMetastoreCatalogModule.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/IcebergFileMetastoreCatalogModule.java index 97a334c34420..9c4039a7b393 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/IcebergFileMetastoreCatalogModule.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/IcebergFileMetastoreCatalogModule.java @@ -19,9 +19,9 @@ import com.google.inject.multibindings.Multibinder; import io.airlift.configuration.AbstractConfigurationAwareModule; import io.airlift.units.Duration; +import io.trino.metastore.cache.CachingHiveMetastoreConfig; import io.trino.plugin.hive.HideDeltaLakeTables; import io.trino.plugin.hive.metastore.CachingHiveMetastoreModule; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastoreConfig; import io.trino.plugin.hive.metastore.file.FileMetastoreModule; import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.MetastoreValidator; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/AbstractMetastoreTableOperations.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/AbstractMetastoreTableOperations.java index afdc4dbd211f..b0dc44310858 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/AbstractMetastoreTableOperations.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/AbstractMetastoreTableOperations.java @@ -16,8 +16,8 @@ import io.trino.annotation.NotThreadSafe; import io.trino.metastore.PrincipalPrivileges; import io.trino.metastore.Table; +import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.metastore.MetastoreUtil; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.iceberg.CreateTableException; import io.trino.plugin.iceberg.UnknownTableTypeException; import io.trino.plugin.iceberg.catalog.AbstractIcebergTableOperations; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperations.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperations.java index 65d4dac8f7a0..905a1e4c834c 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperations.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperations.java @@ -18,8 +18,8 @@ import io.trino.metastore.AcidTransactionOwner; import io.trino.metastore.PrincipalPrivileges; import io.trino.metastore.Table; +import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.metastore.MetastoreUtil; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.metastore.thrift.ThriftMetastore; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.TableNotFoundException; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/IcebergHiveMetastoreCatalogModule.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/IcebergHiveMetastoreCatalogModule.java index c5ee875fda0e..bcad82df2c53 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/IcebergHiveMetastoreCatalogModule.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/IcebergHiveMetastoreCatalogModule.java @@ -19,9 +19,9 @@ import com.google.inject.multibindings.Multibinder; import io.airlift.configuration.AbstractConfigurationAwareModule; import io.airlift.units.Duration; +import io.trino.metastore.cache.CachingHiveMetastoreConfig; import io.trino.plugin.hive.HideDeltaLakeTables; import io.trino.plugin.hive.metastore.CachingHiveMetastoreModule; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastoreConfig; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreModule; import io.trino.plugin.hive.metastore.thrift.TranslateHiveViews; import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java index 908d7cc6dde9..665a050f69b3 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java @@ -28,10 +28,10 @@ import io.trino.metastore.HivePrincipal; import io.trino.metastore.PrincipalPrivileges; import io.trino.metastore.TableInfo; +import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.HiveSchemaProperties; import io.trino.plugin.hive.TrinoViewHiveMetastore; import io.trino.plugin.hive.metastore.MetastoreUtil; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.util.HiveUtil; import io.trino.plugin.iceberg.IcebergTableName; import io.trino.plugin.iceberg.UnknownTableTypeException; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalogFactory.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalogFactory.java index d18785562efa..2c32fc663f8f 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalogFactory.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalogFactory.java @@ -17,9 +17,9 @@ import io.airlift.concurrent.BoundedExecutor; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.NodeVersion; import io.trino.plugin.hive.TrinoViewHiveMetastore; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.iceberg.ForIcebergMetadata; import io.trino.plugin.iceberg.IcebergConfig; import io.trino.plugin.iceberg.IcebergSecurityConfig; @@ -35,7 +35,7 @@ import java.util.concurrent.ExecutorService; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; +import static io.trino.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static io.trino.plugin.iceberg.IcebergSecurityConfig.IcebergSecurity.SYSTEM; import static io.trino.plugin.iceberg.catalog.AbstractTrinoCatalog.TRINO_CREATED_BY_VALUE; import static java.util.Objects.requireNonNull; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java index b56f2b8289f7..a8eceeddaeed 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java @@ -26,9 +26,9 @@ import io.trino.metastore.HiveMetastoreFactory; import io.trino.metastore.Partition; import io.trino.metastore.PrincipalPrivileges; +import io.trino.metastore.RawHiveMetastoreFactory; import io.trino.metastore.Storage; import io.trino.plugin.hive.HiveStorageFormat; -import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.plugin.iceberg.IcebergConfig; import io.trino.plugin.iceberg.IcebergFileFormat; import io.trino.plugin.iceberg.IcebergSecurityConfig; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java index c1d4f0a0b485..17d8695f61f5 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java @@ -23,6 +23,7 @@ import io.trino.filesystem.TrinoInputFile; import io.trino.metastore.HiveMetastore; import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.orc.OrcDataSource; import io.trino.orc.OrcReader; import io.trino.orc.OrcReaderOptions; @@ -36,7 +37,6 @@ import io.trino.parquet.reader.MetadataReader; import io.trino.plugin.base.metrics.FileFormatDataSourceStats; import io.trino.plugin.hive.TrinoViewHiveMetastore; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.parquet.TrinoParquetDataSource; import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.TrinoCatalog; @@ -64,7 +64,7 @@ import static com.google.common.collect.Iterators.getOnlyElement; import static com.google.common.collect.MoreCollectors.onlyElement; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; +import static io.trino.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static io.trino.plugin.iceberg.IcebergUtil.loadIcebergTable; import static io.trino.plugin.iceberg.util.FileOperationUtils.FileType.METADATA_JSON; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMergeAppend.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMergeAppend.java index cce80e901972..43a255b7c10c 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMergeAppend.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMergeAppend.java @@ -15,8 +15,8 @@ import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.TrinoViewHiveMetastore; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.TrinoCatalog; import io.trino.plugin.iceberg.catalog.file.FileMetastoreTableOperationsProvider; @@ -32,7 +32,7 @@ import org.junit.jupiter.api.Test; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; +import static io.trino.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory; import static io.trino.plugin.iceberg.IcebergTestUtils.getHiveMetastore; import static org.assertj.core.api.Assertions.assertThat; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java index 02efbf39d572..bd47288fec81 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java @@ -17,8 +17,8 @@ import io.trino.Session; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.TrinoViewHiveMetastore; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.TrinoCatalog; import io.trino.plugin.iceberg.catalog.file.FileMetastoreTableOperationsProvider; @@ -47,7 +47,7 @@ import static io.trino.SystemSessionProperties.TASK_CONCURRENCY; import static io.trino.SystemSessionProperties.TASK_MAX_WRITER_COUNT; import static io.trino.SystemSessionProperties.TASK_MIN_WRITER_COUNT; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; +import static io.trino.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static io.trino.plugin.iceberg.DataFileRecord.toDataFileRecord; import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java index 62be3d6672c3..a7fb5fc85859 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java @@ -20,8 +20,8 @@ import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.cache.DefaultCachingHostAddressProvider; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.TrinoViewHiveMetastore; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.orc.OrcReaderConfig; import io.trino.plugin.hive.orc.OrcWriterConfig; import io.trino.plugin.hive.parquet.ParquetReaderConfig; @@ -84,7 +84,7 @@ import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; +import static io.trino.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static io.trino.plugin.iceberg.IcebergSplitSource.createFileStatisticsDomain; import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory; import static io.trino.plugin.iceberg.IcebergTestUtils.getHiveMetastore; @@ -406,14 +406,14 @@ public void testSplitWeight() // Write position delete file FileIO fileIo = new ForwardingFileIo(fileSystemFactory.create(SESSION)); - PositionDeleteWriter writer = Parquet.writeDeletes(fileIo.newOutputFile("local:///delete_file_" + UUID.randomUUID())) + PositionDeleteWriter writer = Parquet.writeDeletes(fileIo.newOutputFile("local:///delete_file_" + UUID.randomUUID())) .createWriterFunc(GenericParquetWriter::buildWriter) .forTable(nationTable) .overwrite() .rowSchema(nationTable.schema()) .withSpec(PartitionSpec.unpartitioned()) .buildPositionWriter(); - PositionDelete positionDelete = PositionDelete.create(); + PositionDelete positionDelete = PositionDelete.create(); PositionDelete record = positionDelete.set(dataFilePath, 0, GenericRecord.create(nationTable.schema())); try (Closeable ignored = writer) { writer.write(record); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestAbstractIcebergTableOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestAbstractIcebergTableOperations.java index 3f12c1b9e42a..ec4beb206993 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestAbstractIcebergTableOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestAbstractIcebergTableOperations.java @@ -28,7 +28,7 @@ import java.nio.file.Path; import java.util.Optional; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; +import static io.trino.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_INVALID_METADATA; import static io.trino.testing.TestingConnectorSession.SESSION; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestTrinoHiveCatalogWithFileMetastore.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestTrinoHiveCatalogWithFileMetastore.java index c8641e0773f0..9e630d451430 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestTrinoHiveCatalogWithFileMetastore.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestTrinoHiveCatalogWithFileMetastore.java @@ -20,8 +20,8 @@ import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.local.LocalFileSystemFactory; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.TrinoViewHiveMetastore; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.iceberg.IcebergConfig; import io.trino.plugin.iceberg.catalog.BaseTrinoCatalogTest; import io.trino.plugin.iceberg.catalog.TrinoCatalog; @@ -48,7 +48,7 @@ import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; +import static io.trino.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.plugin.iceberg.IcebergFileFormat.PARQUET; import static io.trino.plugin.iceberg.IcebergTableProperties.FILE_FORMAT_PROPERTY; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestingIcebergFileMetastoreCatalogModule.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestingIcebergFileMetastoreCatalogModule.java index 695ae115c5c5..d88c0ac316be 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestingIcebergFileMetastoreCatalogModule.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestingIcebergFileMetastoreCatalogModule.java @@ -19,9 +19,9 @@ import io.airlift.units.Duration; import io.trino.metastore.HiveMetastore; import io.trino.metastore.HiveMetastoreFactory; +import io.trino.metastore.RawHiveMetastoreFactory; +import io.trino.metastore.cache.CachingHiveMetastoreConfig; import io.trino.plugin.hive.metastore.CachingHiveMetastoreModule; -import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastoreConfig; import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.TrinoCatalogFactory; import io.trino.plugin.iceberg.catalog.hms.TrinoHiveCatalogFactory; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestTrinoHiveCatalogWithHiveMetastore.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestTrinoHiveCatalogWithHiveMetastore.java index c0f80a947eeb..63d841b90083 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestTrinoHiveCatalogWithHiveMetastore.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestTrinoHiveCatalogWithHiveMetastore.java @@ -32,11 +32,11 @@ import io.trino.hdfs.s3.TrinoS3ConfigurationInitializer; import io.trino.metastore.Table; import io.trino.metastore.TableInfo; +import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.plugin.base.util.AutoCloseableCloser; import io.trino.plugin.hive.TrinoViewHiveMetastore; import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.plugin.hive.containers.HiveMinioDataLake; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; import io.trino.plugin.hive.metastore.thrift.ThriftMetastore; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; @@ -71,9 +71,9 @@ import static com.google.common.base.Verify.verify; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; import static io.trino.metastore.PrincipalPrivileges.NO_PRIVILEGES; +import static io.trino.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; import static io.trino.plugin.hive.containers.HiveHadoop.HIVE3_IMAGE; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static io.trino.plugin.iceberg.IcebergFileFormat.PARQUET; import static io.trino.plugin.iceberg.IcebergTableProperties.FILE_FORMAT_PROPERTY; import static io.trino.plugin.iceberg.IcebergTableProperties.FORMAT_VERSION_PROPERTY; diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeJmx.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeJmx.java index fa7f14eaba86..6e92f6953bf4 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeJmx.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDeltaLakeJmx.java @@ -35,7 +35,7 @@ public void testJmxTablesExposedByDeltaLakeConnectorBackedByGlueMetastore() { assertThat(onTrino().executeQuery("SHOW TABLES IN jmx.current LIKE '%name=delta%'")).containsOnly( row("io.trino.filesystem.s3:name=delta,type=s3filesystemstats"), - row("io.trino.plugin.hive.metastore.cache:name=delta,type=cachinghivemetastore"), + row("io.trino.metastore.cache:name=delta,type=cachinghivemetastore"), row("io.trino.plugin.hive.metastore.glue:name=delta,type=gluehivemetastore"), row("io.trino.plugin.hive.metastore.glue:name=delta,type=gluemetastorestats"), row("io.trino.plugin.base.metrics:catalog=delta,name=delta,type=fileformatdatasourcestats"), @@ -48,7 +48,7 @@ public void testJmxTablesExposedByDeltaLakeConnectorBackedByThriftMetastore() { assertThat(onTrino().executeQuery("SHOW TABLES IN jmx.current LIKE '%name=delta%'")).containsOnly( row("io.trino.filesystem.s3:name=delta,type=s3filesystemstats"), - row("io.trino.plugin.hive.metastore.cache:name=delta,type=cachinghivemetastore"), + row("io.trino.metastore.cache:name=delta,type=cachinghivemetastore"), row("io.trino.plugin.hive.metastore.thrift:name=delta,type=thrifthivemetastore"), row("io.trino.plugin.hive.metastore.thrift:name=delta,type=thriftmetastorestats"), row("io.trino.plugin.base.metrics:catalog=delta,name=delta,type=fileformatdatasourcestats"), From eb199b18b6b30105f04fef5c7367a198b19cb437 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Wed, 31 Jul 2024 00:30:51 -0700 Subject: [PATCH 08/12] Move Glue v1 converters into Glue package --- ...eltaLakeGlueMetastoreTableFilterProvider.java | 2 +- .../DeltaLakeGlueV1MetastoreTableOperations.java | 4 ++-- .../v1/DefaultGlueColumnStatisticsProvider.java | 4 ++-- .../DefaultGlueMetastoreTableFilterProvider.java | 2 +- .../metastore/glue/v1/GlueHiveMetastore.java | 16 +++++++--------- .../v1/{converter => }/GlueInputConverter.java | 8 ++++---- .../v1/{converter => }/GlueStatConverter.java | 2 +- .../v1/{converter => }/GlueToTrinoConverter.java | 4 ++-- .../glue/v1/{converter => }/Memoizers.java | 2 +- .../glue/v1/TestGlueInputConverter.java | 2 -- .../glue/v1/TestGlueToTrinoConverter.java | 11 +++++------ .../iceberg/catalog/AbstractTrinoCatalog.java | 2 +- .../catalog/glue/GlueIcebergTableOperations.java | 6 +++--- .../iceberg/catalog/glue/TrinoGlueCatalog.java | 10 +++++----- ...TestIcebergGlueCatalogConnectorSmokeTest.java | 6 +++--- .../TestIcebergGlueCatalogMaterializedView.java | 2 +- .../glue/TestIcebergGlueCatalogSkipArchive.java | 2 +- .../TestDatabricksWithGlueMetastoreCleanUp.java | 2 +- 18 files changed, 41 insertions(+), 46 deletions(-) rename plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/{converter => }/GlueInputConverter.java (95%) rename plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/{converter => }/GlueStatConverter.java (99%) rename plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/{converter => }/GlueToTrinoConverter.java (99%) rename plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/{converter => }/Memoizers.java (97%) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/glue/v1/DeltaLakeGlueMetastoreTableFilterProvider.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/glue/v1/DeltaLakeGlueMetastoreTableFilterProvider.java index f44eb3f1c3be..654b251bbb8f 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/glue/v1/DeltaLakeGlueMetastoreTableFilterProvider.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/glue/v1/DeltaLakeGlueMetastoreTableFilterProvider.java @@ -21,7 +21,7 @@ import java.util.function.Predicate; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableParameters; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableParameters; public class DeltaLakeGlueMetastoreTableFilterProvider implements Provider> diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/glue/v1/DeltaLakeGlueV1MetastoreTableOperations.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/glue/v1/DeltaLakeGlueV1MetastoreTableOperations.java index 1e1df8668529..d3e0eed81a98 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/glue/v1/DeltaLakeGlueV1MetastoreTableOperations.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/glue/v1/DeltaLakeGlueV1MetastoreTableOperations.java @@ -28,8 +28,8 @@ import java.util.Optional; import static io.trino.plugin.deltalake.metastore.DeltaLakeTableMetadataScheduler.tableMetadataParameters; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueInputConverter.convertGlueTableToTableInput; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableParameters; +import static io.trino.plugin.hive.metastore.glue.v1.GlueInputConverter.convertGlueTableToTableInput; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableParameters; import static java.util.Objects.requireNonNull; public class DeltaLakeGlueV1MetastoreTableOperations diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/DefaultGlueColumnStatisticsProvider.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/DefaultGlueColumnStatisticsProvider.java index fb3e912956a2..7fe6b886bf62 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/DefaultGlueColumnStatisticsProvider.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/DefaultGlueColumnStatisticsProvider.java @@ -56,8 +56,8 @@ import static io.trino.metastore.Partitions.toPartitionValues; import static io.trino.plugin.hive.HiveErrorCode.HIVE_METASTORE_ERROR; import static io.trino.plugin.hive.HiveErrorCode.HIVE_PARTITION_NOT_FOUND; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueStatConverter.fromGlueColumnStatistics; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueStatConverter.toGlueColumnStatistics; +import static io.trino.plugin.hive.metastore.glue.v1.GlueStatConverter.fromGlueColumnStatistics; +import static io.trino.plugin.hive.metastore.glue.v1.GlueStatConverter.toGlueColumnStatistics; import static java.util.concurrent.CompletableFuture.allOf; import static java.util.concurrent.CompletableFuture.runAsync; import static java.util.concurrent.CompletableFuture.supplyAsync; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/DefaultGlueMetastoreTableFilterProvider.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/DefaultGlueMetastoreTableFilterProvider.java index a072f2682239..54f81a1eb184 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/DefaultGlueMetastoreTableFilterProvider.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/DefaultGlueMetastoreTableFilterProvider.java @@ -20,7 +20,7 @@ import java.util.function.Predicate; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableParameters; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableParameters; import static io.trino.plugin.hive.util.HiveUtil.isDeltaLakeTable; import static java.util.function.Predicate.not; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueHiveMetastore.java index 10ad2d54c40b..79d5ec45ce24 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueHiveMetastore.java @@ -96,9 +96,7 @@ import io.trino.plugin.hive.metastore.glue.AwsApiCallStats; import io.trino.plugin.hive.metastore.glue.GlueExpressionUtil; import io.trino.plugin.hive.metastore.glue.GlueMetastoreStats; -import io.trino.plugin.hive.metastore.glue.v1.converter.GlueInputConverter; -import io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter; -import io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.GluePartitionConverter; +import io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.GluePartitionConverter; import io.trino.spi.TrinoException; import io.trino.spi.connector.ColumnNotFoundException; import io.trino.spi.connector.SchemaNotFoundException; @@ -150,12 +148,12 @@ import static io.trino.plugin.hive.metastore.MetastoreUtil.updateStatisticsParameters; import static io.trino.plugin.hive.metastore.MetastoreUtil.verifyCanDropColumn; import static io.trino.plugin.hive.metastore.glue.v1.AwsSdkUtil.getPaginatedResults; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueInputConverter.convertFunction; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueInputConverter.convertGlueTableToTableInput; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueInputConverter.convertPartition; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableParameters; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableType; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.mappedCopy; +import static io.trino.plugin.hive.metastore.glue.v1.GlueInputConverter.convertFunction; +import static io.trino.plugin.hive.metastore.glue.v1.GlueInputConverter.convertGlueTableToTableInput; +import static io.trino.plugin.hive.metastore.glue.v1.GlueInputConverter.convertPartition; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableParameters; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableType; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.mappedCopy; import static io.trino.plugin.hive.util.HiveUtil.escapeSchemaName; import static io.trino.spi.StandardErrorCode.ALREADY_EXISTS; import static io.trino.spi.StandardErrorCode.FUNCTION_NOT_FOUND; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/converter/GlueInputConverter.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueInputConverter.java similarity index 95% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/converter/GlueInputConverter.java rename to plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueInputConverter.java index 1d12277e87d4..9d16be3760ce 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/converter/GlueInputConverter.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueInputConverter.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.metastore.glue.v1.converter; +package io.trino.plugin.hive.metastore.glue.v1; import com.amazonaws.services.glue.model.DatabaseInput; import com.amazonaws.services.glue.model.Order; @@ -48,9 +48,9 @@ import static io.trino.plugin.hive.metastore.MetastoreUtil.metastoreFunctionName; import static io.trino.plugin.hive.metastore.MetastoreUtil.toResourceUris; import static io.trino.plugin.hive.metastore.MetastoreUtil.updateStatisticsParameters; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getStorageDescriptor; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableParameters; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableTypeNullable; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getStorageDescriptor; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableParameters; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableTypeNullable; public final class GlueInputConverter { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/converter/GlueStatConverter.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueStatConverter.java similarity index 99% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/converter/GlueStatConverter.java rename to plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueStatConverter.java index 13ccbb855996..fc71676c5a36 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/converter/GlueStatConverter.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueStatConverter.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.metastore.glue.v1.converter; +package io.trino.plugin.hive.metastore.glue.v1; import com.amazonaws.services.glue.model.BinaryColumnStatisticsData; import com.amazonaws.services.glue.model.BooleanColumnStatisticsData; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/converter/GlueToTrinoConverter.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueToTrinoConverter.java similarity index 99% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/converter/GlueToTrinoConverter.java rename to plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueToTrinoConverter.java index 53bed07ae493..3ab9906c170c 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/converter/GlueToTrinoConverter.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueToTrinoConverter.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.metastore.glue.v1.converter; +package io.trino.plugin.hive.metastore.glue.v1; import com.amazonaws.services.glue.model.SerDeInfo; import com.amazonaws.services.glue.model.StorageDescriptor; @@ -55,7 +55,7 @@ import static io.trino.plugin.hive.HiveErrorCode.HIVE_UNSUPPORTED_FORMAT; import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE; import static io.trino.plugin.hive.ViewReaderUtil.isTrinoMaterializedView; -import static io.trino.plugin.hive.metastore.glue.v1.converter.Memoizers.memoizeLast; +import static io.trino.plugin.hive.metastore.glue.v1.Memoizers.memoizeLast; import static io.trino.plugin.hive.metastore.thrift.ThriftMetastoreUtil.decodeFunction; import static io.trino.plugin.hive.util.HiveUtil.isDeltaLakeTable; import static io.trino.plugin.hive.util.HiveUtil.isIcebergTable; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/converter/Memoizers.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/Memoizers.java similarity index 97% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/converter/Memoizers.java rename to plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/Memoizers.java index 3ae763a83f4e..43d4af02a23b 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/converter/Memoizers.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/Memoizers.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.metastore.glue.v1.converter; +package io.trino.plugin.hive.metastore.glue.v1; import java.util.Objects; import java.util.function.BiFunction; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/v1/TestGlueInputConverter.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/v1/TestGlueInputConverter.java index e3d713056449..b2c3c42da904 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/v1/TestGlueInputConverter.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/v1/TestGlueInputConverter.java @@ -27,8 +27,6 @@ import io.trino.metastore.Partition; import io.trino.metastore.Storage; import io.trino.metastore.Table; -import io.trino.plugin.hive.metastore.glue.v1.converter.GlueInputConverter; -import io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter; import io.trino.spi.function.LanguageFunction; import org.junit.jupiter.api.Test; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/v1/TestGlueToTrinoConverter.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/v1/TestGlueToTrinoConverter.java index e7231d6ee928..49356c25abe9 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/v1/TestGlueToTrinoConverter.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/v1/TestGlueToTrinoConverter.java @@ -22,8 +22,7 @@ import io.trino.metastore.Column; import io.trino.metastore.HiveBucketProperty; import io.trino.metastore.Storage; -import io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter; -import io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.GluePartitionConverter; +import io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.GluePartitionConverter; import io.trino.spi.security.PrincipalType; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -37,16 +36,16 @@ import static com.amazonaws.util.CollectionUtils.isNullOrEmpty; import static io.trino.metastore.HiveType.HIVE_STRING; import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getPartitionParameters; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getStorageDescriptor; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableParameters; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableTypeNullable; import static io.trino.plugin.hive.metastore.glue.v1.TestingMetastoreObjects.getGlueTestColumn; import static io.trino.plugin.hive.metastore.glue.v1.TestingMetastoreObjects.getGlueTestDatabase; import static io.trino.plugin.hive.metastore.glue.v1.TestingMetastoreObjects.getGlueTestPartition; import static io.trino.plugin.hive.metastore.glue.v1.TestingMetastoreObjects.getGlueTestStorageDescriptor; import static io.trino.plugin.hive.metastore.glue.v1.TestingMetastoreObjects.getGlueTestTable; import static io.trino.plugin.hive.metastore.glue.v1.TestingMetastoreObjects.getGlueTestTrinoMaterializedView; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getPartitionParameters; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getStorageDescriptor; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableParameters; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableTypeNullable; import static io.trino.plugin.hive.util.HiveUtil.DELTA_LAKE_PROVIDER; import static io.trino.plugin.hive.util.HiveUtil.ICEBERG_TABLE_TYPE_NAME; import static io.trino.plugin.hive.util.HiveUtil.ICEBERG_TABLE_TYPE_VALUE; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java index 4daa639b8ce5..55406ef7c86d 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java @@ -71,7 +71,7 @@ import static io.trino.metastore.TableInfo.ICEBERG_MATERIALIZED_VIEW_COMMENT; import static io.trino.plugin.hive.HiveMetadata.STORAGE_TABLE; import static io.trino.plugin.hive.ViewReaderUtil.PRESTO_VIEW_FLAG; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.mappedCopy; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.mappedCopy; import static io.trino.plugin.hive.util.HiveUtil.escapeTableName; import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_FILESYSTEM_ERROR; import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_INVALID_METADATA; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperations.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperations.java index 60aef66c65b8..ec88e7f0618f 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperations.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperations.java @@ -48,9 +48,9 @@ import static com.google.common.base.Verify.verify; import static io.trino.plugin.hive.ViewReaderUtil.isTrinoMaterializedView; import static io.trino.plugin.hive.ViewReaderUtil.isTrinoView; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getStorageDescriptor; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableParameters; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableType; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getStorageDescriptor; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableParameters; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableType; import static io.trino.plugin.hive.util.HiveUtil.isIcebergTable; import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_COMMIT_ERROR; import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_INVALID_METADATA; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalog.java index 6c8a3dfa44bb..0fe8f2fcf278 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalog.java @@ -130,11 +130,11 @@ import static io.trino.plugin.hive.ViewReaderUtil.isTrinoMaterializedView; import static io.trino.plugin.hive.ViewReaderUtil.isTrinoView; import static io.trino.plugin.hive.metastore.glue.v1.AwsSdkUtil.getPaginatedResults; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getColumnParameters; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getStorageDescriptor; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableParameters; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableType; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableTypeNullable; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getColumnParameters; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getStorageDescriptor; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableParameters; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableType; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableTypeNullable; import static io.trino.plugin.hive.util.HiveUtil.isHiveSystemSchema; import static io.trino.plugin.hive.util.HiveUtil.isIcebergTable; import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_BAD_DATA; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogConnectorSmokeTest.java index d88407b09309..24efa6b0e4db 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogConnectorSmokeTest.java @@ -53,9 +53,9 @@ import java.util.List; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getStorageDescriptor; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableParameters; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableType; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getStorageDescriptor; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableParameters; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableType; import static io.trino.plugin.iceberg.IcebergTestUtils.checkParquetFileSorting; import static io.trino.testing.SystemEnvironmentUtils.requireEnv; import static io.trino.testing.TestingConnectorSession.SESSION; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogMaterializedView.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogMaterializedView.java index 9f3eeceab96d..299f7720af5e 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogMaterializedView.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogMaterializedView.java @@ -40,7 +40,7 @@ import static com.google.common.collect.ImmutableSet.toImmutableSet; import static io.trino.plugin.base.util.Closables.closeAllSuppress; import static io.trino.plugin.hive.metastore.glue.v1.AwsSdkUtil.getPaginatedResults; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableParameters; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableParameters; import static io.trino.testing.TestingNames.randomNameSuffix; import static org.apache.iceberg.BaseMetastoreTableOperations.METADATA_LOCATION_PROP; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java index 69be06111502..81e86808e7db 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java @@ -48,7 +48,7 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.Iterables.getOnlyElement; import static io.trino.plugin.hive.metastore.glue.v1.AwsSdkUtil.getPaginatedResults; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableParameters; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableParameters; import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory; import static io.trino.plugin.iceberg.catalog.glue.GlueIcebergUtil.getTableInput; import static io.trino.testing.TestingConnectorSession.SESSION; diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDatabricksWithGlueMetastoreCleanUp.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDatabricksWithGlueMetastoreCleanUp.java index bb600aebc6ef..70fea807582c 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDatabricksWithGlueMetastoreCleanUp.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/deltalake/TestDatabricksWithGlueMetastoreCleanUp.java @@ -34,7 +34,7 @@ import java.util.stream.Collectors; import static com.google.common.collect.ImmutableSet.toImmutableSet; -import static io.trino.plugin.hive.metastore.glue.v1.converter.GlueToTrinoConverter.getTableType; +import static io.trino.plugin.hive.metastore.glue.v1.GlueToTrinoConverter.getTableType; import static io.trino.tests.product.TestGroups.DELTA_LAKE_DATABRICKS; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.product.deltalake.util.DeltaLakeTestUtils.DATABRICKS_COMMUNICATION_FAILURE_ISSUE; From c52b4866fab7a3719124ad632745a0408099d2e1 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Wed, 31 Jul 2024 10:26:02 -0700 Subject: [PATCH 09/12] Move already exists exceptions to metastore module --- .../io/trino/metastore}/SchemaAlreadyExistsException.java | 2 +- .../java/io/trino/metastore}/TableAlreadyExistsException.java | 2 +- .../src/main/java/io/trino/plugin/hive/HiveMetadata.java | 1 + .../java/io/trino/plugin/hive/TrinoViewHiveMetastore.java | 1 + .../plugin/hive/metastore/SemiTransactionalHiveMetastore.java | 2 +- .../trino/plugin/hive/metastore/file/FileHiveMetastore.java | 4 ++-- .../trino/plugin/hive/metastore/glue/GlueHiveMetastore.java | 4 ++-- .../plugin/hive/metastore/glue/v1/GlueHiveMetastore.java | 4 ++-- .../plugin/hive/metastore/thrift/BridgingHiveMetastore.java | 4 ++-- .../plugin/hive/metastore/thrift/ThriftHiveMetastore.java | 4 ++-- .../plugin/hive/metastore/AbstractTestHiveMetastore.java | 4 ++-- .../hive/metastore/thrift/TestBridgingHiveMetastore.java | 4 ++-- .../trino/plugin/iceberg/catalog/glue/TrinoGlueCatalog.java | 2 +- 13 files changed, 20 insertions(+), 18 deletions(-) rename {plugin/trino-hive/src/main/java/io/trino/plugin/hive => lib/trino-metastore/src/main/java/io/trino/metastore}/SchemaAlreadyExistsException.java (97%) rename {plugin/trino-hive/src/main/java/io/trino/plugin/hive => lib/trino-metastore/src/main/java/io/trino/metastore}/TableAlreadyExistsException.java (97%) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/SchemaAlreadyExistsException.java b/lib/trino-metastore/src/main/java/io/trino/metastore/SchemaAlreadyExistsException.java similarity index 97% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/SchemaAlreadyExistsException.java rename to lib/trino-metastore/src/main/java/io/trino/metastore/SchemaAlreadyExistsException.java index 8bff8da39a12..df46c9e8705a 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/SchemaAlreadyExistsException.java +++ b/lib/trino-metastore/src/main/java/io/trino/metastore/SchemaAlreadyExistsException.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive; +package io.trino.metastore; import io.trino.spi.TrinoException; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/TableAlreadyExistsException.java b/lib/trino-metastore/src/main/java/io/trino/metastore/TableAlreadyExistsException.java similarity index 97% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/TableAlreadyExistsException.java rename to lib/trino-metastore/src/main/java/io/trino/metastore/TableAlreadyExistsException.java index ff0e9fdae2bb..83f293809b76 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/TableAlreadyExistsException.java +++ b/lib/trino-metastore/src/main/java/io/trino/metastore/TableAlreadyExistsException.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive; +package io.trino.metastore; import io.trino.spi.TrinoException; import io.trino.spi.connector.SchemaTableName; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java index 73ff0d872439..17d4542a0a0f 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java @@ -50,6 +50,7 @@ import io.trino.metastore.SortingColumn; import io.trino.metastore.StorageFormat; import io.trino.metastore.Table; +import io.trino.metastore.TableAlreadyExistsException; import io.trino.metastore.TableInfo; import io.trino.plugin.base.projection.ApplyProjectionUtil; import io.trino.plugin.base.projection.ApplyProjectionUtil.ProjectedColumnRepresentation; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/TrinoViewHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/TrinoViewHiveMetastore.java index df99de97138c..1e485066bdba 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/TrinoViewHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/TrinoViewHiveMetastore.java @@ -19,6 +19,7 @@ import io.trino.metastore.HiveMetastore; import io.trino.metastore.PrincipalPrivileges; import io.trino.metastore.Table; +import io.trino.metastore.TableAlreadyExistsException; import io.trino.metastore.TableInfo; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java index 070ea9e0b631..e055c767de1c 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java @@ -51,12 +51,12 @@ import io.trino.metastore.PrincipalPrivileges; import io.trino.metastore.StatisticsUpdateMode; import io.trino.metastore.Table; +import io.trino.metastore.TableAlreadyExistsException; import io.trino.metastore.TableInfo; import io.trino.plugin.hive.HiveTableHandle; import io.trino.plugin.hive.LocationHandle.WriteMode; import io.trino.plugin.hive.PartitionNotFoundException; import io.trino.plugin.hive.PartitionUpdateAndMergeResults; -import io.trino.plugin.hive.TableAlreadyExistsException; import io.trino.plugin.hive.TableInvalidationCallback; import io.trino.plugin.hive.acid.AcidTransaction; import io.trino.plugin.hive.projection.PartitionProjection; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java index 68b3b0edcc2f..9ba3811bea5f 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java @@ -44,13 +44,13 @@ import io.trino.metastore.PartitionStatistics; import io.trino.metastore.PartitionWithStatistics; import io.trino.metastore.PrincipalPrivileges; +import io.trino.metastore.SchemaAlreadyExistsException; import io.trino.metastore.StatisticsUpdateMode; import io.trino.metastore.Table; +import io.trino.metastore.TableAlreadyExistsException; import io.trino.metastore.TableInfo; import io.trino.plugin.hive.NodeVersion; import io.trino.plugin.hive.PartitionNotFoundException; -import io.trino.plugin.hive.SchemaAlreadyExistsException; -import io.trino.plugin.hive.TableAlreadyExistsException; import io.trino.plugin.hive.TableType; import io.trino.plugin.hive.metastore.file.FileHiveMetastoreConfig.VersionCompatibility; import io.trino.spi.TrinoException; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java index eee2a234a016..a7103e4326cc 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java @@ -35,13 +35,13 @@ import io.trino.metastore.PartitionStatistics; import io.trino.metastore.PartitionWithStatistics; import io.trino.metastore.PrincipalPrivileges; +import io.trino.metastore.SchemaAlreadyExistsException; import io.trino.metastore.StatisticsUpdateMode; import io.trino.metastore.Table; +import io.trino.metastore.TableAlreadyExistsException; import io.trino.metastore.TableInfo; import io.trino.plugin.hive.HivePartitionManager; import io.trino.plugin.hive.PartitionNotFoundException; -import io.trino.plugin.hive.SchemaAlreadyExistsException; -import io.trino.plugin.hive.TableAlreadyExistsException; import io.trino.spi.ErrorCode; import io.trino.spi.TrinoException; import io.trino.spi.connector.SchemaNotFoundException; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueHiveMetastore.java index 79d5ec45ce24..8e24cdfcf873 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/v1/GlueHiveMetastore.java @@ -87,12 +87,12 @@ import io.trino.metastore.PartitionWithStatistics; import io.trino.metastore.Partitions; import io.trino.metastore.PrincipalPrivileges; +import io.trino.metastore.SchemaAlreadyExistsException; import io.trino.metastore.StatisticsUpdateMode; import io.trino.metastore.Table; +import io.trino.metastore.TableAlreadyExistsException; import io.trino.metastore.TableInfo; import io.trino.plugin.hive.PartitionNotFoundException; -import io.trino.plugin.hive.SchemaAlreadyExistsException; -import io.trino.plugin.hive.TableAlreadyExistsException; import io.trino.plugin.hive.metastore.glue.AwsApiCallStats; import io.trino.plugin.hive.metastore.glue.GlueExpressionUtil; import io.trino.plugin.hive.metastore.glue.GlueMetastoreStats; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java index ecb79587b170..807b39fac17e 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java @@ -32,11 +32,11 @@ import io.trino.metastore.PartitionWithStatistics; import io.trino.metastore.Partitions; import io.trino.metastore.PrincipalPrivileges; +import io.trino.metastore.SchemaAlreadyExistsException; import io.trino.metastore.StatisticsUpdateMode; import io.trino.metastore.Table; +import io.trino.metastore.TableAlreadyExistsException; import io.trino.metastore.TableInfo; -import io.trino.plugin.hive.SchemaAlreadyExistsException; -import io.trino.plugin.hive.TableAlreadyExistsException; import io.trino.spi.TrinoException; import io.trino.spi.connector.SchemaNotFoundException; import io.trino.spi.connector.SchemaTableName; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java index 7b07889050cd..ef91a6a263f6 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java @@ -69,10 +69,10 @@ import io.trino.metastore.HiveType; import io.trino.metastore.PartitionStatistics; import io.trino.metastore.PartitionWithStatistics; +import io.trino.metastore.SchemaAlreadyExistsException; import io.trino.metastore.StatisticsUpdateMode; +import io.trino.metastore.TableAlreadyExistsException; import io.trino.plugin.hive.PartitionNotFoundException; -import io.trino.plugin.hive.SchemaAlreadyExistsException; -import io.trino.plugin.hive.TableAlreadyExistsException; import io.trino.plugin.hive.util.RetryDriver; import io.trino.spi.TrinoException; import io.trino.spi.connector.SchemaNotFoundException; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/AbstractTestHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/AbstractTestHiveMetastore.java index 0f7e63dcbae6..ffb7e8a2451a 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/AbstractTestHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/AbstractTestHiveMetastore.java @@ -16,9 +16,9 @@ import io.trino.metastore.Column; import io.trino.metastore.Database; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.SchemaAlreadyExistsException; import io.trino.metastore.Table; -import io.trino.plugin.hive.SchemaAlreadyExistsException; -import io.trino.plugin.hive.TableAlreadyExistsException; +import io.trino.metastore.TableAlreadyExistsException; import io.trino.spi.connector.TableNotFoundException; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestBridgingHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestBridgingHiveMetastore.java index 3c92e0f43aaa..7184f5732c51 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestBridgingHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestBridgingHiveMetastore.java @@ -15,10 +15,10 @@ import io.trino.metastore.Database; import io.trino.metastore.HiveMetastore; +import io.trino.metastore.SchemaAlreadyExistsException; import io.trino.metastore.Table; +import io.trino.metastore.TableAlreadyExistsException; import io.trino.plugin.base.util.AutoCloseableCloser; -import io.trino.plugin.hive.SchemaAlreadyExistsException; -import io.trino.plugin.hive.TableAlreadyExistsException; import io.trino.plugin.hive.containers.HiveHadoop; import io.trino.plugin.hive.metastore.AbstractTestHiveMetastore; import org.junit.jupiter.api.AfterAll; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalog.java index 0fe8f2fcf278..12782f9957ea 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalog.java @@ -46,8 +46,8 @@ import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.metastore.SchemaAlreadyExistsException; import io.trino.metastore.TableInfo; -import io.trino.plugin.hive.SchemaAlreadyExistsException; import io.trino.plugin.hive.TrinoViewUtil; import io.trino.plugin.hive.ViewAlreadyExistsException; import io.trino.plugin.hive.ViewReaderUtil; From 16883cb0ae409a211eaf7b6c692657e596d2e50d Mon Sep 17 00:00:00 2001 From: David Phillips Date: Wed, 31 Jul 2024 10:12:44 -0700 Subject: [PATCH 10/12] Move ForHiveMetastore annotation to thrift package --- .../plugin/hive/{ => metastore/thrift}/ForHiveMetastore.java | 2 +- .../metastore/thrift/KerberosHiveMetastoreAuthentication.java | 1 - .../metastore/thrift/ThriftMetastoreAuthenticationModule.java | 1 - .../plugin/hive/metastore/thrift/ThriftMetastoreModule.java | 1 - .../metastore/thrift/TokenFetchingMetastoreClientFactory.java | 1 - .../hive/metastore/thrift/UgiBasedMetastoreClientFactory.java | 1 - 6 files changed, 1 insertion(+), 6 deletions(-) rename plugin/trino-hive/src/main/java/io/trino/plugin/hive/{ => metastore/thrift}/ForHiveMetastore.java (95%) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ForHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ForHiveMetastore.java similarity index 95% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/ForHiveMetastore.java rename to plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ForHiveMetastore.java index 1b5ca893bff4..f54ed16dc51d 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ForHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ForHiveMetastore.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive; +package io.trino.plugin.hive.metastore.thrift; import com.google.inject.BindingAnnotation; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/KerberosHiveMetastoreAuthentication.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/KerberosHiveMetastoreAuthentication.java index 834928ea07d0..5f0da82d63f0 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/KerberosHiveMetastoreAuthentication.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/KerberosHiveMetastoreAuthentication.java @@ -19,7 +19,6 @@ import io.airlift.slice.SliceInput; import io.airlift.slice.Slices; import io.trino.plugin.base.authentication.CachingKerberosAuthentication; -import io.trino.plugin.hive.ForHiveMetastore; import org.apache.thrift.transport.TSaslClientTransport; import org.apache.thrift.transport.TTransport; import org.apache.thrift.transport.TTransportException; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreAuthenticationModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreAuthenticationModule.java index 066c6628e01f..b244f7f0cdcb 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreAuthenticationModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreAuthenticationModule.java @@ -21,7 +21,6 @@ import io.trino.plugin.base.authentication.CachingKerberosAuthentication; import io.trino.plugin.base.authentication.KerberosAuthentication; import io.trino.plugin.base.authentication.KerberosConfiguration; -import io.trino.plugin.hive.ForHiveMetastore; import static com.google.inject.Scopes.SINGLETON; import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java index c83d47e60aff..4bef109c2f4d 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreModule.java @@ -25,7 +25,6 @@ import io.trino.metastore.RawHiveMetastoreFactory; import io.trino.plugin.base.security.UserNameProvider; import io.trino.plugin.hive.AllowHiveTableRename; -import io.trino.plugin.hive.ForHiveMetastore; import java.util.concurrent.ExecutorService; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/TokenFetchingMetastoreClientFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/TokenFetchingMetastoreClientFactory.java index ab95396c19e1..3f68f3499633 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/TokenFetchingMetastoreClientFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/TokenFetchingMetastoreClientFactory.java @@ -23,7 +23,6 @@ import dev.failsafe.function.CheckedSupplier; import io.trino.cache.NonEvictableLoadingCache; import io.trino.plugin.base.security.UserNameProvider; -import io.trino.plugin.hive.ForHiveMetastore; import io.trino.spi.TrinoException; import io.trino.spi.security.ConnectorIdentity; import org.apache.thrift.TException; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/UgiBasedMetastoreClientFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/UgiBasedMetastoreClientFactory.java index f69bd4575a94..5408300b951d 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/UgiBasedMetastoreClientFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/UgiBasedMetastoreClientFactory.java @@ -15,7 +15,6 @@ import com.google.inject.Inject; import io.trino.plugin.base.security.UserNameProvider; -import io.trino.plugin.hive.ForHiveMetastore; import io.trino.spi.security.ConnectorIdentity; import org.apache.thrift.TException; From b7249b32a297ba6d846139047ee41ae26c6fa62f Mon Sep 17 00:00:00 2001 From: David Phillips Date: Wed, 31 Jul 2024 10:38:17 -0700 Subject: [PATCH 11/12] Move RetryDriver to thrift package --- .../plugin/hive/{util => metastore/thrift}/RetryDriver.java | 2 +- .../trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java | 1 - .../hive/metastore/thrift/ThriftHttpMetastoreFactory.java | 1 - .../plugin/hive/metastore/thrift/ThriftMetastoreConfig.java | 1 - 4 files changed, 1 insertion(+), 4 deletions(-) rename plugin/trino-hive/src/main/java/io/trino/plugin/hive/{util => metastore/thrift}/RetryDriver.java (99%) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/RetryDriver.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/RetryDriver.java similarity index 99% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/RetryDriver.java rename to plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/RetryDriver.java index 10f31de8482d..7d1aecdd9924 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/RetryDriver.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/RetryDriver.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive.util; +package io.trino.plugin.hive.metastore.thrift; import com.google.common.collect.ImmutableList; import io.airlift.log.Logger; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java index ef91a6a263f6..41fdbffd42ad 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java @@ -73,7 +73,6 @@ import io.trino.metastore.StatisticsUpdateMode; import io.trino.metastore.TableAlreadyExistsException; import io.trino.plugin.hive.PartitionNotFoundException; -import io.trino.plugin.hive.util.RetryDriver; import io.trino.spi.TrinoException; import io.trino.spi.connector.SchemaNotFoundException; import io.trino.spi.connector.SchemaTableName; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHttpMetastoreFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHttpMetastoreFactory.java index bfc8e8c7820f..26e13e5abae0 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHttpMetastoreFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHttpMetastoreFactory.java @@ -16,7 +16,6 @@ import com.google.inject.Inject; import io.airlift.units.Duration; import io.trino.filesystem.TrinoFileSystemFactory; -import io.trino.plugin.hive.util.RetryDriver; import io.trino.spi.security.ConnectorIdentity; import org.weakref.jmx.Flatten; import org.weakref.jmx.Managed; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreConfig.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreConfig.java index b4511e45c17b..ae4c96b10b83 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreConfig.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreConfig.java @@ -22,7 +22,6 @@ import io.airlift.configuration.validation.FileExists; import io.airlift.units.Duration; import io.airlift.units.MinDuration; -import io.trino.plugin.hive.util.RetryDriver; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.Min; import jakarta.validation.constraints.NotNull; From 7bdf517063fa9ef2c2857587bd34daae12eb3f7b Mon Sep 17 00:00:00 2001 From: David Phillips Date: Wed, 31 Jul 2024 10:52:25 -0700 Subject: [PATCH 12/12] Move Avro utility method to ThriftMetastoreUtil --- .../trino/plugin/hive/metastore/MetastoreUtil.java | 12 ------------ .../hive/metastore/thrift/BridgingHiveMetastore.java | 1 - .../hive/metastore/thrift/ThriftMetastoreUtil.java | 9 +++++++++ 3 files changed, 9 insertions(+), 13 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/MetastoreUtil.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/MetastoreUtil.java index 561a4a336ff7..fa851d819ed8 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/MetastoreUtil.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/MetastoreUtil.java @@ -89,10 +89,7 @@ import static io.trino.hive.thrift.metastore.hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS; import static io.trino.hive.thrift.metastore.hive_metastoreConstants.META_TABLE_PARTITION_COLUMN_TYPES; import static io.trino.metastore.Partitions.makePartName; -import static io.trino.plugin.hive.HiveMetadata.AVRO_SCHEMA_LITERAL_KEY; -import static io.trino.plugin.hive.HiveMetadata.AVRO_SCHEMA_URL_KEY; import static io.trino.plugin.hive.HiveSplitManager.PRESTO_OFFLINE; -import static io.trino.plugin.hive.HiveStorageFormat.AVRO; import static io.trino.plugin.hive.metastore.SparkMetastoreUtil.getSparkBasicStatistics; import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMN_COMMENTS; import static io.trino.plugin.hive.util.SerdeConstants.SERIALIZATION_LIB; @@ -243,15 +240,6 @@ public static ProtectMode getProtectMode(Table table) return getProtectMode(table.getParameters()); } - public static boolean isAvroTableWithSchemaSet(Table table) - { - return AVRO.getSerde().equals(table.getStorage().getStorageFormat().getSerDeNullable()) && - ((table.getParameters().get(AVRO_SCHEMA_URL_KEY) != null || - (table.getStorage().getSerdeParameters().get(AVRO_SCHEMA_URL_KEY) != null)) || - (table.getParameters().get(AVRO_SCHEMA_LITERAL_KEY) != null || - (table.getStorage().getSerdeParameters().get(AVRO_SCHEMA_LITERAL_KEY) != null))); - } - public static String makePartitionName(Table table, Partition partition) { return makePartitionName(table.getPartitionColumns(), partition.getValues()); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java index 807b39fac17e..e66c5fdf76e1 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java @@ -57,7 +57,6 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.metastore.Table.TABLE_COMMENT; import static io.trino.plugin.hive.HiveMetadata.TRINO_QUERY_ID_NAME; -import static io.trino.plugin.hive.metastore.MetastoreUtil.isAvroTableWithSchemaSet; import static io.trino.plugin.hive.metastore.MetastoreUtil.metastoreFunctionName; import static io.trino.plugin.hive.metastore.MetastoreUtil.verifyCanDropColumn; import static io.trino.plugin.hive.metastore.thrift.ThriftMetastoreUtil.csvSchemaFields; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreUtil.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreUtil.java index 7fd79ede9317..1e377956c03d 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreUtil.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastoreUtil.java @@ -974,4 +974,13 @@ public static DataOperationType toDataOperationType(AcidOperation acidOperation) default -> throw new IllegalStateException("No metastore operation for ACID operation " + acidOperation); }; } + + public static boolean isAvroTableWithSchemaSet(Table table) + { + return AVRO.getSerde().equals(table.getStorage().getStorageFormat().getSerDeNullable()) && + ((table.getParameters().get(AVRO_SCHEMA_URL_KEY) != null || + (table.getStorage().getSerdeParameters().get(AVRO_SCHEMA_URL_KEY) != null)) || + (table.getParameters().get(AVRO_SCHEMA_LITERAL_KEY) != null || + (table.getStorage().getSerdeParameters().get(AVRO_SCHEMA_LITERAL_KEY) != null))); + } }