diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaFailureRecoveryTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaFailureRecoveryTest.java index d189e96fa4e8..88f7ee2411aa 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaFailureRecoveryTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaFailureRecoveryTest.java @@ -17,7 +17,7 @@ import io.trino.operator.RetryPolicy; import io.trino.plugin.exchange.filesystem.FileSystemExchangePlugin; import io.trino.plugin.exchange.filesystem.containers.MinioStorage; -import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.spi.ErrorType; import io.trino.testing.BaseFailureRecoveryTest; import io.trino.testing.QueryRunner; @@ -59,7 +59,7 @@ protected QueryRunner createQueryRunner( Module failureInjectionModule) throws Exception { - HiveMinioDataLake hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName)); + Hive3MinioDataLake hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName)); hiveMinioDataLake.start(); MinioStorage minioStorage = closeAfterClass(new MinioStorage("test-exchange-spooling-" + randomNameSuffix())); minioStorage.start(); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeAwsConnectorSmokeTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeAwsConnectorSmokeTest.java index e7b1529a8f1d..8fd2de505c58 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeAwsConnectorSmokeTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeAwsConnectorSmokeTest.java @@ -13,8 +13,8 @@ */ package io.trino.plugin.deltalake; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.plugin.hive.containers.HiveHadoop; -import io.trino.plugin.hive.containers.HiveMinioDataLake; import io.trino.testing.QueryRunner; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.TestInstance; @@ -29,12 +29,12 @@ public abstract class BaseDeltaLakeAwsConnectorSmokeTest extends BaseDeltaLakeConnectorSmokeTest { - protected HiveMinioDataLake hiveMinioDataLake; + protected Hive3MinioDataLake hiveMinioDataLake; @Override protected HiveHadoop createHiveHadoop() { - hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName)); + hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName)); hiveMinioDataLake.start(); return hiveMinioDataLake.getHiveHadoop(); // closed by superclass } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeCompatibility.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeCompatibility.java index 9c9cde68f35a..4c352a590b57 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeCompatibility.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeCompatibility.java @@ -13,7 +13,7 @@ */ package io.trino.plugin.deltalake; -import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; import io.trino.tpch.TpchTable; @@ -32,7 +32,7 @@ public abstract class BaseDeltaLakeCompatibility { protected final String bucketName; protected final String resourcePath; - protected HiveMinioDataLake hiveMinioDataLake; + protected Hive3MinioDataLake hiveMinioDataLake; public BaseDeltaLakeCompatibility(String resourcePath) { @@ -44,7 +44,7 @@ public BaseDeltaLakeCompatibility(String resourcePath) protected QueryRunner createQueryRunner() throws Exception { - hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName)); + hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName)); hiveMinioDataLake.start(); QueryRunner queryRunner = DeltaLakeQueryRunner.builder() diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/DeltaLakeQueryRunner.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/DeltaLakeQueryRunner.java index 01df1213f3d5..5c2ecb1d36f5 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/DeltaLakeQueryRunner.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/DeltaLakeQueryRunner.java @@ -19,8 +19,8 @@ import io.airlift.log.Level; import io.airlift.log.Logger; import io.airlift.log.Logging; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.plugin.hive.containers.HiveHadoop; -import io.trino.plugin.hive.containers.HiveMinioDataLake; import io.trino.plugin.tpch.TpchPlugin; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; @@ -259,7 +259,7 @@ public static void main(String[] args) { String bucketName = "test-bucket"; - HiveMinioDataLake hiveMinioDataLake = new HiveMinioDataLake(bucketName); + Hive3MinioDataLake hiveMinioDataLake = new Hive3MinioDataLake(bucketName); hiveMinioDataLake.start(); QueryRunner queryRunner = builder() diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/SparkDeltaLake.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/SparkDeltaLake.java index 6ac5e396d839..9742c853c5ef 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/SparkDeltaLake.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/SparkDeltaLake.java @@ -14,8 +14,8 @@ package io.trino.plugin.deltalake; import io.trino.plugin.base.util.AutoCloseableCloser; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.plugin.hive.containers.HiveHadoop; -import io.trino.plugin.hive.containers.HiveMinioDataLake; import io.trino.testing.containers.Minio; import org.testcontainers.containers.GenericContainer; @@ -26,11 +26,11 @@ public final class SparkDeltaLake implements AutoCloseable { private final AutoCloseableCloser closer = AutoCloseableCloser.create(); - private final HiveMinioDataLake hiveMinio; + private final Hive3MinioDataLake hiveMinio; public SparkDeltaLake(String bucketName) { - hiveMinio = closer.register(new HiveMinioDataLake(bucketName)); + hiveMinio = closer.register(new Hive3MinioDataLake(bucketName)); hiveMinio.start(); closer.register(new GenericContainer<>("ghcr.io/trinodb/testing/spark3-delta:" + getDockerImagesVersion())) diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeCreateTableStatistics.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeCreateTableStatistics.java index 1263d32ce54b..1bace334c40c 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeCreateTableStatistics.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeCreateTableStatistics.java @@ -18,7 +18,7 @@ import io.trino.plugin.deltalake.transactionlog.AddFileEntry; import io.trino.plugin.deltalake.transactionlog.TransactionLogAccess; import io.trino.plugin.deltalake.transactionlog.statistics.DeltaLakeFileStatistics; -import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.spi.type.DateType; import io.trino.spi.type.DecimalType; import io.trino.spi.type.DoubleType; @@ -68,7 +68,7 @@ protected QueryRunner createQueryRunner() throws Exception { this.bucketName = "delta-test-create-table-statistics-" + randomNameSuffix(); - HiveMinioDataLake hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName)); + Hive3MinioDataLake hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName)); hiveMinioDataLake.start(); return DeltaLakeQueryRunner.builder() diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeDelete.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeDelete.java index d616b3ec2045..f854110453ed 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeDelete.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeDelete.java @@ -14,7 +14,7 @@ package io.trino.plugin.deltalake; import com.google.common.collect.ImmutableSet; -import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; import org.junit.jupiter.api.Test; @@ -31,13 +31,13 @@ public class TestDeltaLakeDelete extends AbstractTestQueryFramework { private final String bucketName = "test-delta-lake-connector-test-" + randomNameSuffix(); - private HiveMinioDataLake hiveMinioDataLake; + private Hive3MinioDataLake hiveMinioDataLake; @Override protected QueryRunner createQueryRunner() throws Exception { - hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName)); + hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName)); hiveMinioDataLake.start(); return DeltaLakeQueryRunner.builder() diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeDynamicFiltering.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeDynamicFiltering.java index ab566111784d..4bfe9a7cd8f6 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeDynamicFiltering.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeDynamicFiltering.java @@ -22,7 +22,7 @@ import io.trino.metadata.QualifiedObjectName; import io.trino.metadata.Split; import io.trino.metadata.TableHandle; -import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.security.AllowAllAccessControl; import io.trino.spi.QueryId; import io.trino.spi.connector.ColumnHandle; @@ -60,14 +60,14 @@ public class TestDeltaLakeDynamicFiltering extends AbstractTestQueryFramework { private final String bucketName = "delta-lake-test-dynamic-filtering-" + randomNameSuffix(); - private HiveMinioDataLake hiveMinioDataLake; + private Hive3MinioDataLake hiveMinioDataLake; @Override protected QueryRunner createQueryRunner() throws Exception { verify(new DynamicFilterConfig().isEnableDynamicFiltering(), "this class assumes dynamic filtering is enabled by default"); - hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName)); + hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName)); hiveMinioDataLake.start(); QueryRunner queryRunner = DeltaLakeQueryRunner.builder() diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFlushMetadataCacheProcedure.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFlushMetadataCacheProcedure.java index 61d73d098800..615c6ab50f81 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFlushMetadataCacheProcedure.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFlushMetadataCacheProcedure.java @@ -14,7 +14,7 @@ package io.trino.plugin.deltalake; import io.trino.metastore.HiveMetastore; -import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; @@ -39,11 +39,11 @@ public class TestDeltaLakeFlushMetadataCacheProcedure protected QueryRunner createQueryRunner() throws Exception { - HiveMinioDataLake hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName, HIVE3_IMAGE)); + Hive3MinioDataLake hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName, HIVE3_IMAGE)); hiveMinioDataLake.start(); metastore = new BridgingHiveMetastore( testingThriftHiveMetastoreBuilder() - .metastoreClient(hiveMinioDataLake.getHiveHadoop().getHiveMetastoreEndpoint()) + .metastoreClient(hiveMinioDataLake.getHiveMetastoreEndpoint()) .build(this::closeAfterClass)); return DeltaLakeQueryRunner.builder("default") diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java index bef79413f6fd..bcd9da3bd217 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java @@ -15,7 +15,7 @@ import com.google.common.collect.ImmutableMap; import io.trino.plugin.hive.TestingHivePlugin; -import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; import org.junit.jupiter.api.AfterAll; @@ -35,14 +35,14 @@ public class TestDeltaLakeSharedHiveMetastoreWithViews extends AbstractTestQueryFramework { private final String bucketName = "delta-lake-shared-hive-with-views-" + randomNameSuffix(); - private HiveMinioDataLake hiveMinioDataLake; + private Hive3MinioDataLake hiveMinioDataLake; private String schema; @Override protected QueryRunner createQueryRunner() throws Exception { - this.hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName)); + this.hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName)); this.hiveMinioDataLake.start(); QueryRunner queryRunner = DeltaLakeQueryRunner.builder() @@ -54,7 +54,7 @@ protected QueryRunner createQueryRunner() queryRunner.installPlugin(new TestingHivePlugin(queryRunner.getCoordinator().getBaseDataDir().resolve("hive_data"))); queryRunner.createCatalog("hive", "hive", ImmutableMap.builder() .put("hive.metastore", "thrift") - .put("hive.metastore.uri", hiveMinioDataLake.getHiveHadoop().getHiveMetastoreEndpoint().toString()) + .put("hive.metastore.uri", hiveMinioDataLake.getHiveMetastoreEndpoint().toString()) .put("fs.hadoop.enabled", "false") .put("fs.native-s3.enabled", "true") .put("s3.aws-access-key", MINIO_ACCESS_KEY) @@ -67,7 +67,7 @@ protected QueryRunner createQueryRunner() schema = queryRunner.getDefaultSession().getSchema().orElseThrow(); queryRunner.execute("CREATE TABLE hive." + schema + ".hive_table (a_integer integer)"); - hiveMinioDataLake.getHiveHadoop().runOnHive("CREATE VIEW " + schema + ".hive_view AS SELECT * FROM " + schema + ".hive_table"); + hiveMinioDataLake.runOnHive("CREATE VIEW " + schema + ".hive_view AS SELECT * FROM " + schema + ".hive_table"); queryRunner.execute("CREATE TABLE delta." + schema + ".delta_table (a_varchar varchar)"); return queryRunner; @@ -82,7 +82,7 @@ protected QueryRunner createQueryRunner() public void cleanup() { assertQuerySucceeds("DROP TABLE IF EXISTS hive." + schema + ".hive_table"); - hiveMinioDataLake.getHiveHadoop().runOnHive("DROP VIEW IF EXISTS " + schema + ".hive_view"); + hiveMinioDataLake.runOnHive("DROP VIEW IF EXISTS " + schema + ".hive_view"); assertQuerySucceeds("DROP TABLE IF EXISTS delta." + schema + ".delta_table"); assertQuerySucceeds("DROP SCHEMA IF EXISTS hive." + schema); } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeUpdate.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeUpdate.java index 857f15b0c2c0..6c1cab6c0982 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeUpdate.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeUpdate.java @@ -13,7 +13,7 @@ */ package io.trino.plugin.deltalake; -import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; import org.junit.jupiter.api.Test; @@ -35,7 +35,7 @@ public TestDeltaLakeUpdate() protected QueryRunner createQueryRunner() throws Exception { - HiveMinioDataLake hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName)); + Hive3MinioDataLake hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName)); hiveMinioDataLake.start(); return DeltaLakeQueryRunner.builder() diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestPredicatePushdown.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestPredicatePushdown.java index 0d6bf3610430..e6b909b93aa0 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestPredicatePushdown.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestPredicatePushdown.java @@ -16,7 +16,7 @@ import com.google.common.collect.ContiguousSet; import io.trino.Session; import io.trino.operator.OperatorStats; -import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.spi.QueryId; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.MaterializedResult; @@ -48,13 +48,13 @@ public class TestPredicatePushdown */ private final TableResource testTable = new TableResource("custkey_15rowgroups"); - private HiveMinioDataLake hiveMinioDataLake; + private Hive3MinioDataLake hiveMinioDataLake; @Override protected QueryRunner createQueryRunner() throws Exception { - hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName)); + hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName)); hiveMinioDataLake.start(); return DeltaLakeQueryRunner.builder() diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseTestHiveOnDataLake.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseTestHiveOnDataLake.java new file mode 100644 index 000000000000..59df61051c61 --- /dev/null +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseTestHiveOnDataLake.java @@ -0,0 +1,2287 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hive; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.airlift.units.DataSize; +import io.trino.Session; +import io.trino.metastore.Column; +import io.trino.metastore.HiveColumnStatistics; +import io.trino.metastore.HiveMetastore; +import io.trino.metastore.Partition; +import io.trino.metastore.PartitionStatistics; +import io.trino.metastore.PartitionWithStatistics; +import io.trino.metastore.Table; +import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; +import io.trino.plugin.hive.s3.S3HiveQueryRunner; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.TableNotFoundException; +import io.trino.spi.predicate.NullableValue; +import io.trino.spi.predicate.TupleDomain; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.QueryRunner; +import io.trino.testing.minio.MinioClient; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; + +import java.text.DateFormat; +import java.text.SimpleDateFormat; +import java.time.Instant; +import java.time.ZoneId; +import java.time.temporal.TemporalUnit; +import java.util.Arrays; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.TimeZone; +import java.util.stream.Collectors; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.airlift.slice.Slices.utf8Slice; +import static io.airlift.units.DataSize.Unit.MEGABYTE; +import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; +import static io.trino.plugin.hive.metastore.MetastoreUtil.getHiveBasicStatistics; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static io.trino.testing.MaterializedResult.resultBuilder; +import static io.trino.testing.TestingNames.randomNameSuffix; +import static java.lang.String.format; +import static java.nio.charset.StandardCharsets.UTF_8; +import static java.time.temporal.ChronoUnit.DAYS; +import static java.time.temporal.ChronoUnit.MINUTES; +import static java.util.regex.Pattern.quote; +import static java.util.stream.Collectors.joining; +import static java.util.stream.Collectors.toSet; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; + +@TestInstance(PER_CLASS) +abstract class BaseTestHiveOnDataLake + extends AbstractTestQueryFramework +{ + private static final String HIVE_TEST_SCHEMA = "hive_datalake"; + private static final DataSize HIVE_S3_STREAMING_PART_SIZE = DataSize.of(5, MEGABYTE); + + private final HiveMinioDataLake hiveMinioDataLake; + private final String bucketName; + + private HiveMetastore metastoreClient; + + public BaseTestHiveOnDataLake(String bucketName, HiveMinioDataLake hiveMinioDataLake) + { + this.bucketName = bucketName; + this.hiveMinioDataLake = hiveMinioDataLake; + } + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + this.hiveMinioDataLake.start(); + this.metastoreClient = new BridgingHiveMetastore( + testingThriftHiveMetastoreBuilder() + .metastoreClient(hiveMinioDataLake.getHiveMetastoreEndpoint()) + .build(this::closeAfterClass)); + return S3HiveQueryRunner.builder(hiveMinioDataLake) + .addExtraProperty("sql.path", "hive.functions") + .addExtraProperty("sql.default-function-catalog", "hive") + .addExtraProperty("sql.default-function-schema", "functions") + .setHiveProperties( + ImmutableMap.builder() + .put("hive.insert-existing-partitions-behavior", "OVERWRITE") + .put("hive.non-managed-table-writes-enabled", "true") + // Below are required to enable caching on metastore + .put("hive.metastore-cache-ttl", "1d") + .put("hive.metastore-refresh-interval", "1d") + // This is required to reduce memory pressure to test writing large files + .put("s3.streaming.part-size", HIVE_S3_STREAMING_PART_SIZE.toString()) + // This is required to enable AWS Athena partition projection + .put("hive.partition-projection-enabled", "true") + .buildOrThrow()) + .build(); + } + + @BeforeAll + public void setUp() + { + computeActual(format( + "CREATE SCHEMA hive.%1$s WITH (location='s3a://%2$s/%1$s')", + HIVE_TEST_SCHEMA, + bucketName)); + computeActual("CREATE SCHEMA hive.functions"); + } + + @AfterAll + public void destroy() + throws Exception + { + hiveMinioDataLake.close(); + } + + @Test + public void testInsertOverwriteInTransaction() + { + String testTable = getFullyQualifiedTestTableName(); + computeActual(getCreateTableStatement(testTable, "partitioned_by=ARRAY['regionkey']")); + assertThatThrownBy( + () -> newTransaction() + .execute(getSession(), session -> { + getQueryRunner().execute(session, createInsertAsSelectFromTpchStatement(testTable)); + })) + .hasMessage("Overwriting existing partition in non auto commit context doesn't support DIRECT_TO_TARGET_EXISTING_DIRECTORY write mode"); + computeActual(format("DROP TABLE %s", testTable)); + } + + @Test + public void testInsertOverwriteNonPartitionedTable() + { + String testTable = getFullyQualifiedTestTableName(); + computeActual(getCreateTableStatement(testTable)); + assertInsertFailure( + testTable, + "Overwriting unpartitioned table not supported when writing directly to target directory"); + computeActual(format("DROP TABLE %s", testTable)); + } + + @Test + public void testInsertOverwriteNonPartitionedBucketedTable() + { + String testTable = getFullyQualifiedTestTableName(); + computeActual(getCreateTableStatement( + testTable, + "bucketed_by = ARRAY['nationkey']", + "bucket_count = 3")); + assertInsertFailure( + testTable, + "Overwriting unpartitioned table not supported when writing directly to target directory"); + computeActual(format("DROP TABLE %s", testTable)); + } + + @Test + public void testInsertOverwritePartitionedTable() + { + String testTable = getFullyQualifiedTestTableName(); + computeActual(getCreateTableStatement( + testTable, + "partitioned_by=ARRAY['regionkey']")); + copyTpchNationToTable(testTable); + assertOverwritePartition(testTable); + } + + @Test + public void testInsertOverwritePartitionedAndBucketedTable() + { + String testTable = getFullyQualifiedTestTableName(); + computeActual(getCreateTableStatement( + testTable, + "partitioned_by=ARRAY['regionkey']", + "bucketed_by = ARRAY['nationkey']", + "bucket_count = 3")); + copyTpchNationToTable(testTable); + assertOverwritePartition(testTable); + } + + @Test + public void testInsertOverwritePartitionedAndBucketedExternalTable() + { + String testTable = getFullyQualifiedTestTableName(); + // Store table data in data lake bucket + computeActual(getCreateTableStatement( + testTable, + "partitioned_by=ARRAY['regionkey']", + "bucketed_by = ARRAY['nationkey']", + "bucket_count = 3")); + copyTpchNationToTable(testTable); + + // Map this table as external table + String externalTableName = testTable + "_ext"; + computeActual(getCreateTableStatement( + externalTableName, + "partitioned_by=ARRAY['regionkey']", + "bucketed_by = ARRAY['nationkey']", + "bucket_count = 3", + format("external_location = 's3a://%s/%s/%s/'", this.bucketName, HIVE_TEST_SCHEMA, testTable))); + copyTpchNationToTable(testTable); + assertOverwritePartition(externalTableName); + } + + @Test + public void testSyncPartitionOnBucketRoot() + { + String tableName = "test_sync_partition_on_bucket_root_" + randomNameSuffix(); + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); + + hiveMinioDataLake.getMinioClient().putObject( + bucketName, + "hello\u0001world\nbye\u0001world".getBytes(UTF_8), + "part_key=part_val/data.txt"); + + assertUpdate("CREATE TABLE " + fullyQualifiedTestTableName + "(" + + " a varchar," + + " b varchar," + + " part_key varchar)" + + "WITH (" + + " external_location='s3://" + bucketName + "/'," + + " partitioned_by=ARRAY['part_key']," + + " format='TEXTFILE'" + + ")"); + + getQueryRunner().execute("CALL system.sync_partition_metadata(schema_name => '" + HIVE_TEST_SCHEMA + "', table_name => '" + tableName + "', mode => 'ADD')"); + + assertQuery("SELECT * FROM " + fullyQualifiedTestTableName, "VALUES ('hello', 'world', 'part_val'), ('bye', 'world', 'part_val')"); + + assertUpdate("DROP TABLE " + fullyQualifiedTestTableName); + } + + @Test + public void testSyncPartitionCaseSensitivePathVariation() + { + String tableName = "test_sync_partition_case_variation_" + randomNameSuffix(); + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); + String tableLocation = format("s3://%s/%s/%s/", bucketName, HIVE_TEST_SCHEMA, tableName); + + hiveMinioDataLake.getMinioClient().putObject( + bucketName, + "Trino\u0001rocks".getBytes(UTF_8), + HIVE_TEST_SCHEMA + "/" + tableName + "/part_key=part_val/data.txt"); + + assertUpdate("CREATE TABLE " + fullyQualifiedTestTableName + "(" + + " a varchar," + + " b varchar," + + " part_key varchar)" + + "WITH (" + + " external_location='" + tableLocation + "'," + + " partitioned_by=ARRAY['part_key']," + + " format='TEXTFILE'" + + ")"); + + getQueryRunner().execute("CALL system.sync_partition_metadata(schema_name => '" + HIVE_TEST_SCHEMA + "', table_name => '" + tableName + "', mode => 'ADD')"); + assertQuery("SELECT * FROM " + fullyQualifiedTestTableName, "VALUES ('Trino', 'rocks', 'part_val')"); + + // Move the data to a location where the partition path differs only in case + hiveMinioDataLake.getMinioClient().removeObject(bucketName, HIVE_TEST_SCHEMA + "/" + tableName + "/part_key=part_val/data.txt"); + hiveMinioDataLake.getMinioClient().putObject( + bucketName, + "Trino\u0001rocks".getBytes(UTF_8), + HIVE_TEST_SCHEMA + "/" + tableName + "/PART_KEY=part_val/data.txt"); + + getQueryRunner().execute("CALL system.sync_partition_metadata(schema_name => '" + HIVE_TEST_SCHEMA + "', table_name => '" + tableName + "', mode => 'FULL', case_sensitive => false)"); + assertQuery("SELECT * FROM " + fullyQualifiedTestTableName, "VALUES ('Trino', 'rocks', 'part_val')"); + + // Verify that syncing again the partition metadata has no negative effect (e.g. drop the partition) + getQueryRunner().execute("CALL system.sync_partition_metadata(schema_name => '" + HIVE_TEST_SCHEMA + "', table_name => '" + tableName + "', mode => 'FULL', case_sensitive => false)"); + assertQuery("SELECT * FROM " + fullyQualifiedTestTableName, "VALUES ('Trino', 'rocks', 'part_val')"); + + assertUpdate("DROP TABLE " + fullyQualifiedTestTableName); + } + + @Test + public void testSyncPartitionSpecialCharacters() + { + String tableName = "test_sync_partition_special_characters_" + randomNameSuffix(); + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); + String tableLocation = format("s3://%s/%s/%s/", bucketName, HIVE_TEST_SCHEMA, tableName); + + hiveMinioDataLake.getMinioClient().putObject( + bucketName, + "Trino\u0001rocks\u0001hyphens".getBytes(UTF_8), + HIVE_TEST_SCHEMA + "/" + tableName + "/part_key=with-hyphen/data.txt"); + hiveMinioDataLake.getMinioClient().putObject( + bucketName, + "Trino\u0001rocks\u0001dots".getBytes(UTF_8), + HIVE_TEST_SCHEMA + "/" + tableName + "/part_key=with.dot/data.txt"); + hiveMinioDataLake.getMinioClient().putObject( + bucketName, + "Trino\u0001rocks\u0001colons".getBytes(UTF_8), + HIVE_TEST_SCHEMA + "/" + tableName + "/part_key=with%3Acolon/data.txt"); + hiveMinioDataLake.getMinioClient().putObject( + bucketName, + "Trino\u0001rocks\u0001slashes".getBytes(UTF_8), + HIVE_TEST_SCHEMA + "/" + tableName + "/part_key=with%2Fslash/data.txt"); + hiveMinioDataLake.getMinioClient().putObject( + bucketName, + "Trino\u0001rocks\u0001backslashes".getBytes(UTF_8), + HIVE_TEST_SCHEMA + "/" + tableName + "/part_key=with%5Cbackslash/data.txt"); + hiveMinioDataLake.getMinioClient().putObject( + bucketName, + "Trino\u0001rocks\u0001percents".getBytes(UTF_8), + HIVE_TEST_SCHEMA + "/" + tableName + "/part_key=with%25percent/data.txt"); + + assertUpdate("CREATE TABLE " + fullyQualifiedTestTableName + "(" + + " a varchar," + + " b varchar," + + " c varchar," + + " part_key varchar)" + + "WITH (" + + " external_location='" + tableLocation + "'," + + " partitioned_by=ARRAY['part_key']," + + " format='TEXTFILE'" + + ")"); + + getQueryRunner().execute("CALL system.sync_partition_metadata(schema_name => '" + HIVE_TEST_SCHEMA + "', table_name => '" + tableName + "', mode => 'ADD')"); + assertQuery( + "SELECT * FROM " + fullyQualifiedTestTableName, + """ + VALUES + ('Trino', 'rocks', 'hyphens', 'with-hyphen'), + ('Trino', 'rocks', 'dots', 'with.dot'), + ('Trino', 'rocks', 'colons', 'with:colon'), + ('Trino', 'rocks', 'slashes', 'with/slash'), + ('Trino', 'rocks', 'backslashes', 'with\\backslash'), + ('Trino', 'rocks', 'percents', 'with%percent') + """); + + assertUpdate("DROP TABLE " + fullyQualifiedTestTableName); + } + + @Test + public void testFlushPartitionCache() + { + String tableName = "nation_" + randomNameSuffix(); + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); + String partitionColumn = "regionkey"; + + testFlushPartitionCache( + tableName, + fullyQualifiedTestTableName, + partitionColumn, + format( + "CALL system.flush_metadata_cache(schema_name => '%s', table_name => '%s', partition_columns => ARRAY['%s'], partition_values => ARRAY['0'])", + HIVE_TEST_SCHEMA, + tableName, + partitionColumn)); + } + + private void testFlushPartitionCache(String tableName, String fullyQualifiedTestTableName, String partitionColumn, String flushCacheProcedureSql) + { + // Create table with partition on regionkey + computeActual(getCreateTableStatement( + fullyQualifiedTestTableName, + format("partitioned_by=ARRAY['%s']", partitionColumn))); + copyTpchNationToTable(fullyQualifiedTestTableName); + + String queryUsingPartitionCacheTemplate = "SELECT name FROM %s WHERE %s=%s"; + String partitionValue1 = "0"; + String queryUsingPartitionCacheForValue1 = format(queryUsingPartitionCacheTemplate, fullyQualifiedTestTableName, partitionColumn, partitionValue1); + String expectedQueryResultForValue1 = "VALUES 'ALGERIA', 'MOROCCO', 'MOZAMBIQUE', 'ETHIOPIA', 'KENYA'"; + String partitionValue2 = "1"; + String queryUsingPartitionCacheForValue2 = format(queryUsingPartitionCacheTemplate, fullyQualifiedTestTableName, partitionColumn, partitionValue2); + String expectedQueryResultForValue2 = "VALUES 'ARGENTINA', 'BRAZIL', 'CANADA', 'PERU', 'UNITED STATES'"; + + // Fill partition cache and check we got expected results + assertQuery(queryUsingPartitionCacheForValue1, expectedQueryResultForValue1); + assertQuery(queryUsingPartitionCacheForValue2, expectedQueryResultForValue2); + + // Copy partition to new location and update metadata outside Trino + renamePartitionResourcesOutsideTrino(tableName, partitionColumn, partitionValue1); + renamePartitionResourcesOutsideTrino(tableName, partitionColumn, partitionValue2); + + // Should return 0 rows as we moved partition and cache is outdated. We use nonexistent partition + assertQueryReturnsEmptyResult(queryUsingPartitionCacheForValue1); + assertQueryReturnsEmptyResult(queryUsingPartitionCacheForValue2); + + // Refresh cache + getQueryRunner().execute(flushCacheProcedureSql); + + // Should return expected rows as we refresh cache + assertQuery(queryUsingPartitionCacheForValue1, expectedQueryResultForValue1); + // Should return 0 rows as we left cache untouched + assertQueryReturnsEmptyResult(queryUsingPartitionCacheForValue2); + + // Refresh cache for schema_name => 'dummy_schema', table_name => 'dummy_table' + getQueryRunner().execute(format( + "CALL system.flush_metadata_cache(schema_name => '%s', table_name => '%s')", + HIVE_TEST_SCHEMA, + tableName)); + + // Should return expected rows for all partitions + assertQuery(queryUsingPartitionCacheForValue1, expectedQueryResultForValue1); + assertQuery(queryUsingPartitionCacheForValue2, expectedQueryResultForValue2); + + computeActual(format("DROP TABLE %s", fullyQualifiedTestTableName)); + } + + @Test + public void testWriteDifferentSizes() + { + String testTable = getFullyQualifiedTestTableName(); + computeActual(format( + "CREATE TABLE %s (" + + " col1 varchar, " + + " col2 varchar, " + + " regionkey bigint) " + + " WITH (partitioned_by=ARRAY['regionkey'])", + testTable)); + + long partSizeInBytes = HIVE_S3_STREAMING_PART_SIZE.toBytes(); + + // Exercise different code paths of Hive S3 streaming upload, with upload part size 5MB: + // 1. fileSize <= 5MB (direct upload) + testWriteWithFileSize(testTable, 50, 0, partSizeInBytes); + + // 2. 5MB < fileSize <= 10MB (upload in two parts) + testWriteWithFileSize(testTable, 100, partSizeInBytes + 1, partSizeInBytes * 2); + + // 3. fileSize > 10MB (upload in three or more parts) + testWriteWithFileSize(testTable, 150, partSizeInBytes * 2 + 1, partSizeInBytes * 3); + + computeActual(format("DROP TABLE %s", testTable)); + } + + @Test + public void testEnumPartitionProjectionOnVarcharColumnWithWhitespace() + { + String tableName = "nation_" + randomNameSuffix(); + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); + + computeActual( + "CREATE TABLE " + fullyQualifiedTestTableName + " (" + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint, " + + " \"short name\" varchar(152) WITH (" + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL1', 'CZ1'] " + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short name'], " + + " partition_projection_enabled=true " + + ")"); + + assertThat( + hiveMinioDataLake + .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) + .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") + .containsPattern("[ |]+projection\\.short name\\.type[ |]+enum[ |]+") + .containsPattern("[ |]+projection\\.short name\\.values[ |]+PL1,CZ1[ |]+"); + + computeActual(createInsertStatement( + fullyQualifiedTestTableName, + ImmutableList.of( + ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'"), + ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL2'"), + ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'"), + ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ2'")))); + + assertQuery( + format("SELECT * FROM %s", getFullyQualifiedTestTableName("\"" + tableName + "$partitions\"")), + "VALUES 'PL1', 'CZ1'"); + + assertQuery( + format("SELECT name FROM %s WHERE \"short name\"='PL1'", fullyQualifiedTestTableName), + "VALUES 'POLAND_1'"); + + // No results should be returned as Partition Projection will not project partitions for this value + assertQueryReturnsEmptyResult( + format("SELECT name FROM %s WHERE \"short name\"='PL2'", fullyQualifiedTestTableName)); + + assertQuery( + format("SELECT name FROM %s WHERE \"short name\"='PL1' OR \"short name\"='CZ1'", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('CZECH_1')"); + + // Only POLAND_1 row will be returned as other value is outside of projection + assertQuery( + format("SELECT name FROM %s WHERE \"short name\"='PL1' OR \"short name\"='CZ2'", fullyQualifiedTestTableName), + "VALUES ('POLAND_1')"); + + // All values within projection range will be returned + assertQuery( + format("SELECT name FROM %s", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('CZECH_1')"); + } + + @Test + public void testEnumPartitionProjectionOnVarcharColumnWithStorageLocationTemplateCreatedOnTrino() + { + // It's important to mix case here to detect if we properly handle rewriting + // properties between Trino and Hive (e.g for Partition Projection) + String schemaName = "Hive_Datalake_MixedCase"; + String tableName = getRandomTestTableName(); + + // We create new schema to include mixed case location path and create such keys in Object Store + computeActual("CREATE SCHEMA hive.%1$s WITH (location='s3a://%2$s/%1$s')".formatted(schemaName, bucketName)); + + String storageFormat = format( + "s3a://%s/%s/%s/short_name1=${short_name1}/short_name2=${short_name2}/", + this.bucketName, + schemaName, + tableName); + computeActual( + "CREATE TABLE " + getFullyQualifiedTestTableName(schemaName, tableName) + " ( " + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint, " + + " short_name1 varchar(152) WITH (" + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL1', 'CZ1'] " + + " ), " + + " short_name2 varchar(152) WITH (" + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL2', 'CZ2'] " + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1', 'short_name2'], " + + " partition_projection_enabled=true, " + + " partition_projection_location_template='" + storageFormat + "' " + + ")"); + assertThat( + hiveMinioDataLake + .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(schemaName, tableName))) + .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") + .containsPattern("[ |]+storage\\.location\\.template[ |]+" + quote(storageFormat) + "[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+enum[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.values[ |]+PL2,CZ2[ |]+"); + testEnumPartitionProjectionOnVarcharColumnWithStorageLocationTemplate(schemaName, tableName); + } + + @Test + public void testEnumPartitionProjectionOnVarcharColumnWithStorageLocationTemplateCreatedOnHive() + { + String tableName = getRandomTestTableName(); + String storageFormat = format( + "'s3a://%s/%s/%s/short_name1=${short_name1}/short_name2=${short_name2}/'", + this.bucketName, + HIVE_TEST_SCHEMA, + tableName); + hiveMinioDataLake.runOnHive( + "CREATE TABLE " + getHiveTestTableName(tableName) + " ( " + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint " + + ") PARTITIONED BY (" + + " short_name1 varchar(152), " + + " short_name2 varchar(152)" + + ") " + + "TBLPROPERTIES ( " + + " 'projection.enabled'='true', " + + " 'storage.location.template'=" + storageFormat + ", " + + " 'projection.short_name1.type'='enum', " + + " 'projection.short_name1.values'='PL1,CZ1', " + + " 'projection.short_name2.type'='enum', " + + " 'projection.short_name2.values'='PL2,CZ2' " + + ")"); + testEnumPartitionProjectionOnVarcharColumnWithStorageLocationTemplate(HIVE_TEST_SCHEMA, tableName); + } + + private void testEnumPartitionProjectionOnVarcharColumnWithStorageLocationTemplate(String schemaName, String tableName) + { + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(schemaName, tableName); + computeActual(createInsertStatement( + fullyQualifiedTestTableName, + ImmutableList.of( + ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "'PL2'"), + ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "'CZ2'"), + ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "'PL2'"), + ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "'CZ2'")))); + + assertQuery( + format("SELECT * FROM %s", getFullyQualifiedTestTableName(schemaName, "\"" + tableName + "$partitions\"")), + "VALUES ('PL1','PL2'), ('PL1','CZ2'), ('CZ1','PL2'), ('CZ1','CZ2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2='CZ2'", fullyQualifiedTestTableName), + "VALUES 'POLAND_2'"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); + } + + @Test + public void testEnumPartitionProjectionOnVarcharColumn() + { + String tableName = getRandomTestTableName(); + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); + + computeActual( + "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint, " + + " short_name1 varchar(152) WITH (" + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL1', 'CZ1'] " + + " ), " + + " short_name2 varchar(152) WITH (" + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL2', 'CZ2']" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1', 'short_name2'], " + + " partition_projection_enabled=true " + + ")"); + + assertThat( + hiveMinioDataLake + .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) + .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+enum[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.values[ |]+PL2,CZ2[ |]+"); + + computeActual(createInsertStatement( + fullyQualifiedTestTableName, + ImmutableList.of( + ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "'PL2'"), + ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "'CZ2'"), + ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "'PL2'"), + ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "'CZ2'")))); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2='CZ2'", fullyQualifiedTestTableName), + "VALUES 'POLAND_2'"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2='CZ2' OR short_name2='PL2' )", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); + } + + @Test + public void testIntegerPartitionProjectionOnVarcharColumnWithDigitsAlignCreatedOnTrino() + { + String tableName = getRandomTestTableName(); + computeActual( + "CREATE TABLE " + getFullyQualifiedTestTableName(tableName) + " ( " + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint, " + + " short_name1 varchar(152) WITH (" + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL1', 'CZ1'] " + + " ), " + + " short_name2 varchar(152) WITH (" + + " partition_projection_type='integer', " + + " partition_projection_range=ARRAY['1', '4'], " + + " partition_projection_digits=3" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1', 'short_name2'], " + + " partition_projection_enabled=true " + + ")"); + assertThat( + hiveMinioDataLake + .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) + .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+integer[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+1,4[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.digits[ |]+3[ |]+"); + testIntegerPartitionProjectionOnVarcharColumnWithDigitsAlign(tableName); + } + + @Test + public void testIntegerPartitionProjectionOnVarcharColumnWithDigitsAlignCreatedOnHive() + { + String tableName = "nation_" + randomNameSuffix(); + hiveMinioDataLake.runOnHive( + "CREATE TABLE " + getHiveTestTableName(tableName) + " ( " + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint " + + ") " + + "PARTITIONED BY ( " + + " short_name1 varchar(152), " + + " short_name2 varchar(152)" + + ") " + + "TBLPROPERTIES " + + "( " + + " 'projection.enabled'='true', " + + " 'projection.short_name1.type'='enum', " + + " 'projection.short_name1.values'='PL1,CZ1', " + + " 'projection.short_name2.type'='integer', " + + " 'projection.short_name2.range'='1,4', " + + " 'projection.short_name2.digits'='3'" + + ")"); + testIntegerPartitionProjectionOnVarcharColumnWithDigitsAlign(tableName); + } + + private void testIntegerPartitionProjectionOnVarcharColumnWithDigitsAlign(String tableName) + { + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); + computeActual(createInsertStatement( + fullyQualifiedTestTableName, + ImmutableList.of( + ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "'001'"), + ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "'002'"), + ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "'003'"), + ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "'004'")))); + + assertQuery( + format("SELECT * FROM %s", getFullyQualifiedTestTableName("\"" + tableName + "$partitions\"")), + "VALUES ('PL1','001'), ('PL1','002'), ('PL1','003'), ('PL1','004')," + + "('CZ1','001'), ('CZ1','002'), ('CZ1','003'), ('CZ1','004')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2='002'", fullyQualifiedTestTableName), + "VALUES 'POLAND_2'"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2='002' OR short_name2='001' )", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); + } + + @Test + public void testIntegerPartitionProjectionOnIntegerColumnWithInterval() + { + String tableName = getRandomTestTableName(); + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); + + computeActual( + "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint, " + + " short_name1 varchar(152) WITH (" + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL1', 'CZ1'] " + + " ), " + + " short_name2 integer WITH (" + + " partition_projection_type='integer', " + + " partition_projection_range=ARRAY['0', '10'], " + + " partition_projection_interval=3" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1', 'short_name2'], " + + " partition_projection_enabled=true " + + ")"); + + assertThat( + hiveMinioDataLake + .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) + .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+integer[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+0,10[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.interval[ |]+3[ |]+"); + + computeActual(createInsertStatement( + fullyQualifiedTestTableName, + ImmutableList.of( + ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "0"), + ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "3"), + ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "6"), + ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "9")))); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2=3", fullyQualifiedTestTableName), + "VALUES 'POLAND_2'"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2=3 OR short_name2=0 )", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); + } + + @Test + public void testIntegerPartitionProjectionOnIntegerColumnWithDefaults() + { + String tableName = getRandomTestTableName(); + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); + + computeActual( + "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint, " + + " short_name1 varchar(152) WITH (" + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL1', 'CZ1'] " + + " ), " + + " short_name2 integer WITH (" + + " partition_projection_type='integer', " + + " partition_projection_range=ARRAY['1', '4']" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1', 'short_name2'], " + + " partition_projection_enabled=true " + + ")"); + + assertThat( + hiveMinioDataLake + .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) + .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+integer[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+1,4[ |]+"); + + computeActual(createInsertStatement( + fullyQualifiedTestTableName, + ImmutableList.of( + ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "1"), + ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "2"), + ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "3"), + ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "4")))); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2=2", fullyQualifiedTestTableName), + "VALUES 'POLAND_2'"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2=2 OR short_name2=1 )", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); + } + + @Test + public void testDatePartitionProjectionOnDateColumnWithDefaults() + { + String tableName = "nation_" + randomNameSuffix(); + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); + + computeActual( + "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint, " + + " short_name1 varchar(152) WITH (" + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL1', 'CZ1'] " + + " ), " + + " short_name2 date WITH (" + + " partition_projection_type='date', " + + " partition_projection_format='yyyy-MM-dd', " + + " partition_projection_range=ARRAY['2001-1-22', '2001-1-25']" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1', 'short_name2'], " + + " partition_projection_enabled=true " + + ")"); + + assertThat( + hiveMinioDataLake + .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) + .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+date[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.format[ |]+yyyy-MM-dd[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+2001-1-22,2001-1-25[ |]+"); + + computeActual(createInsertStatement( + fullyQualifiedTestTableName, + ImmutableList.of( + ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "DATE '2001-1-22'"), + ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "DATE '2001-1-23'"), + ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "DATE '2001-1-24'"), + ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "DATE '2001-1-25'"), + ImmutableList.of("'CZECH_3'", "'Comment'", "4", "5", "'CZ1'", "DATE '2001-1-26'")))); + + assertQuery( + format("SELECT * FROM %s", getFullyQualifiedTestTableName("\"" + tableName + "$partitions\"")), + "VALUES ('PL1','2001-1-22'), ('PL1','2001-1-23'), ('PL1','2001-1-24'), ('PL1','2001-1-25')," + + "('CZ1','2001-1-22'), ('CZ1','2001-1-23'), ('CZ1','2001-1-24'), ('CZ1','2001-1-25')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2=(DATE '2001-1-23')", fullyQualifiedTestTableName), + "VALUES 'POLAND_2'"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2=(DATE '2001-1-23') OR short_name2=(DATE '2001-1-22') )", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name2 > DATE '2001-1-23'", fullyQualifiedTestTableName), + "VALUES ('CZECH_1'), ('CZECH_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name2 >= DATE '2001-1-23' AND short_name2 <= DATE '2001-1-25'", fullyQualifiedTestTableName), + "VALUES ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); + } + + @Test + public void testDatePartitionProjectionOnTimestampColumnWithInterval() + { + String tableName = getRandomTestTableName(); + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); + + computeActual( + "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint, " + + " short_name1 varchar(152) WITH (" + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL1', 'CZ1'] " + + " ), " + + " short_name2 timestamp WITH (" + + " partition_projection_type='date', " + + " partition_projection_format='yyyy-MM-dd HH:mm:ss', " + + " partition_projection_range=ARRAY['2001-1-22 00:00:00', '2001-1-22 00:00:06'], " + + " partition_projection_interval=2, " + + " partition_projection_interval_unit='SECONDS'" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1', 'short_name2'], " + + " partition_projection_enabled=true " + + ")"); + + assertThat( + hiveMinioDataLake + .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) + .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+date[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.format[ |]+yyyy-MM-dd HH:mm:ss[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+2001-1-22 00:00:00,2001-1-22 00:00:06[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.interval[ |]+2[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.interval\\.unit[ |]+seconds[ |]+"); + + computeActual(createInsertStatement( + fullyQualifiedTestTableName, + ImmutableList.of( + ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "TIMESTAMP '2001-1-22 00:00:00'"), + ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "TIMESTAMP '2001-1-22 00:00:02'"), + ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "TIMESTAMP '2001-1-22 00:00:04'"), + ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "TIMESTAMP '2001-1-22 00:00:06'"), + ImmutableList.of("'CZECH_3'", "'Comment'", "4", "5", "'CZ1'", "TIMESTAMP '2001-1-22 00:00:08'")))); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2=(TIMESTAMP '2001-1-22 00:00:02')", fullyQualifiedTestTableName), + "VALUES 'POLAND_2'"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2=(TIMESTAMP '2001-1-22 00:00:00') OR short_name2=(TIMESTAMP '2001-1-22 00:00:02') )", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name2 > TIMESTAMP '2001-1-22 00:00:02'", fullyQualifiedTestTableName), + "VALUES ('CZECH_1'), ('CZECH_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name2 >= TIMESTAMP '2001-1-22 00:00:02' AND short_name2 <= TIMESTAMP '2001-1-22 00:00:06'", fullyQualifiedTestTableName), + "VALUES ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); + } + + @Test + public void testDatePartitionProjectionOnTimestampColumnWithIntervalExpressionCreatedOnTrino() + { + String tableName = getRandomTestTableName(); + String dateProjectionFormat = "yyyy-MM-dd HH:mm:ss"; + computeActual( + "CREATE TABLE " + getFullyQualifiedTestTableName(tableName) + " ( " + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint, " + + " short_name1 varchar(152) WITH (" + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL1', 'CZ1'] " + + " ), " + + " short_name2 timestamp WITH (" + + " partition_projection_type='date', " + + " partition_projection_format='" + dateProjectionFormat + "', " + + // We set range to -5 minutes to NOW in order to be sure it will grab all test dates + // which range is -4 minutes till now. Also, we have to consider max no. of partitions 1k + " partition_projection_range=ARRAY['NOW-5MINUTES', 'NOW'], " + + " partition_projection_interval=1, " + + " partition_projection_interval_unit='SECONDS'" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1', 'short_name2'], " + + " partition_projection_enabled=true " + + ")"); + assertThat( + hiveMinioDataLake + .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) + .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.format[ |]+" + quote(dateProjectionFormat) + "[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+NOW-5MINUTES,NOW[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.interval[ |]+1[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.interval\\.unit[ |]+seconds[ |]+"); + testDatePartitionProjectionOnTimestampColumnWithIntervalExpression(tableName, dateProjectionFormat); + } + + @Test + public void testDatePartitionProjectionOnTimestampColumnWithIntervalExpressionCreatedOnHive() + { + String tableName = getRandomTestTableName(); + String dateProjectionFormat = "yyyy-MM-dd HH:mm:ss"; + hiveMinioDataLake.runOnHive( + "CREATE TABLE " + getHiveTestTableName(tableName) + " ( " + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint " + + ") " + + "PARTITIONED BY (" + + " short_name1 varchar(152), " + + " short_name2 timestamp " + + ") " + + "TBLPROPERTIES ( " + + " 'projection.enabled'='true', " + + " 'projection.short_name1.type'='enum', " + + " 'projection.short_name1.values'='PL1,CZ1', " + + " 'projection.short_name2.type'='date', " + + " 'projection.short_name2.format'='" + dateProjectionFormat + "', " + + // We set range to -5 minutes to NOW in order to be sure it will grab all test dates + // which range is -4 minutes till now. Also, we have to consider max no. of partitions 1k + " 'projection.short_name2.range'='NOW-5MINUTES,NOW', " + + " 'projection.short_name2.interval'='1', " + + " 'projection.short_name2.interval.unit'='SECONDS'" + + ")"); + testDatePartitionProjectionOnTimestampColumnWithIntervalExpression(tableName, dateProjectionFormat); + } + + private void testDatePartitionProjectionOnTimestampColumnWithIntervalExpression(String tableName, String dateProjectionFormat) + { + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); + + Instant dayToday = Instant.now(); + DateFormat dateFormat = new SimpleDateFormat(dateProjectionFormat); + dateFormat.setTimeZone(TimeZone.getTimeZone(ZoneId.of("UTC"))); + String minutesNowFormatted = moveDate(dateFormat, dayToday, MINUTES, 0); + String minutes1AgoFormatter = moveDate(dateFormat, dayToday, MINUTES, -1); + String minutes2AgoFormatted = moveDate(dateFormat, dayToday, MINUTES, -2); + String minutes3AgoFormatted = moveDate(dateFormat, dayToday, MINUTES, -3); + String minutes4AgoFormatted = moveDate(dateFormat, dayToday, MINUTES, -4); + + computeActual(createInsertStatement( + fullyQualifiedTestTableName, + ImmutableList.of( + ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "TIMESTAMP '" + minutesNowFormatted + "'"), + ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "TIMESTAMP '" + minutes1AgoFormatter + "'"), + ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "TIMESTAMP '" + minutes2AgoFormatted + "'"), + ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "TIMESTAMP '" + minutes3AgoFormatted + "'"), + ImmutableList.of("'CZECH_3'", "'Comment'", "4", "5", "'CZ1'", "TIMESTAMP '" + minutes4AgoFormatted + "'")))); + + assertQuery( + format("SELECT name FROM %s WHERE short_name2 > ( TIMESTAMP '%s' ) AND short_name2 <= ( TIMESTAMP '%s' )", fullyQualifiedTestTableName, minutes4AgoFormatted, minutes1AgoFormatter), + "VALUES ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); + } + + @Test + public void testDatePartitionProjectionOnVarcharColumnWithHoursInterval() + { + String tableName = getRandomTestTableName(); + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); + + computeActual( + "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint, " + + " short_name1 varchar(152) WITH (" + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL1', 'CZ1'] " + + " ), " + + " short_name2 varchar WITH (" + + " partition_projection_type='date', " + + " partition_projection_format='yyyy-MM-dd HH', " + + " partition_projection_range=ARRAY['2001-01-22 00', '2001-01-22 06'], " + + " partition_projection_interval=2, " + + " partition_projection_interval_unit='HOURS'" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1', 'short_name2'], " + + " partition_projection_enabled=true " + + ")"); + + assertThat( + hiveMinioDataLake + .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) + .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+date[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.format[ |]+yyyy-MM-dd HH[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+2001-01-22 00,2001-01-22 06[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.interval[ |]+2[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.interval\\.unit[ |]+hours[ |]+"); + + computeActual(createInsertStatement( + fullyQualifiedTestTableName, + ImmutableList.of( + ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "'2001-01-22 00'"), + ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "'2001-01-22 02'"), + ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "'2001-01-22 04'"), + ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "'2001-01-22 06'"), + ImmutableList.of("'CZECH_3'", "'Comment'", "4", "5", "'CZ1'", "'2001-01-22 08'")))); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2='2001-01-22 02'", fullyQualifiedTestTableName), + "VALUES 'POLAND_2'"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2='2001-01-22 00' OR short_name2='2001-01-22 02' )", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name2 > '2001-01-22 02'", fullyQualifiedTestTableName), + "VALUES ('CZECH_1'), ('CZECH_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name2 >= '2001-01-22 02' AND short_name2 <= '2001-01-22 06'", fullyQualifiedTestTableName), + "VALUES ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); + } + + @Test + public void testDatePartitionProjectionOnVarcharColumnWithDaysInterval() + { + String tableName = getRandomTestTableName(); + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); + + computeActual( + "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint, " + + " short_name1 varchar(152) WITH (" + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL1', 'CZ1'] " + + " ), " + + " short_name2 varchar WITH (" + + " partition_projection_type='date', " + + " partition_projection_format='yyyy-MM-dd', " + + " partition_projection_range=ARRAY['2001-01-01', '2001-01-07'], " + + " partition_projection_interval=2, " + + " partition_projection_interval_unit='DAYS'" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1', 'short_name2'], " + + " partition_projection_enabled=true " + + ")"); + + assertThat( + hiveMinioDataLake + .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) + .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+date[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.format[ |]+yyyy-MM-dd[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+2001-01-01,2001-01-07[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.interval[ |]+2[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.interval\\.unit[ |]+days[ |]+"); + + computeActual(createInsertStatement( + fullyQualifiedTestTableName, + ImmutableList.of( + ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "'2001-01-01'"), + ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "'2001-01-03'"), + ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "'2001-01-05'"), + ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "'2001-01-07'"), + ImmutableList.of("'CZECH_3'", "'Comment'", "4", "5", "'CZ1'", "'2001-01-09'")))); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2='2001-01-03'", fullyQualifiedTestTableName), + "VALUES 'POLAND_2'"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2='2001-01-01' OR short_name2='2001-01-03' )", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name2 > '2001-01-03'", fullyQualifiedTestTableName), + "VALUES ('CZECH_1'), ('CZECH_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name2 >= '2001-01-03' AND short_name2 <= '2001-01-07'", fullyQualifiedTestTableName), + "VALUES ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); + } + + @Test + public void testDatePartitionProjectionOnVarcharColumnWithIntervalExpression() + { + String tableName = getRandomTestTableName(); + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); + String dateProjectionFormat = "yyyy-MM-dd"; + + computeActual( + "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint, " + + " short_name1 varchar(152) WITH (" + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL1', 'CZ1'] " + + " ), " + + " short_name2 varchar WITH (" + + " partition_projection_type='date', " + + " partition_projection_format='" + dateProjectionFormat + "', " + + " partition_projection_range=ARRAY['NOW-3DAYS', 'NOW']" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1', 'short_name2'], " + + " partition_projection_enabled=true " + + ")"); + + assertThat( + hiveMinioDataLake + .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) + .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+date[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.format[ |]+" + quote(dateProjectionFormat) + "[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+NOW-3DAYS,NOW[ |]+"); + + Instant dayToday = Instant.now(); + DateFormat dateFormat = new SimpleDateFormat(dateProjectionFormat); + dateFormat.setTimeZone(TimeZone.getTimeZone(ZoneId.of("UTC"))); + String dayTodayFormatted = moveDate(dateFormat, dayToday, DAYS, 0); + String day1AgoFormatter = moveDate(dateFormat, dayToday, DAYS, -1); + String day2AgoFormatted = moveDate(dateFormat, dayToday, DAYS, -2); + String day3AgoFormatted = moveDate(dateFormat, dayToday, DAYS, -3); + String day4AgoFormatted = moveDate(dateFormat, dayToday, DAYS, -4); + + computeActual(createInsertStatement( + fullyQualifiedTestTableName, + ImmutableList.of( + ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "'" + dayTodayFormatted + "'"), + ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "'" + day1AgoFormatter + "'"), + ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "'" + day2AgoFormatted + "'"), + ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "'" + day3AgoFormatted + "'"), + ImmutableList.of("'CZECH_3'", "'Comment'", "4", "5", "'CZ1'", "'" + day4AgoFormatted + "'")))); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2='%s'", fullyQualifiedTestTableName, day1AgoFormatter), + "VALUES 'POLAND_2'"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2='%s' OR short_name2='%s' )", fullyQualifiedTestTableName, dayTodayFormatted, day1AgoFormatter), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name2 > '%s'", fullyQualifiedTestTableName, day2AgoFormatted), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name2 >= '%s' AND short_name2 <= '%s'", fullyQualifiedTestTableName, day4AgoFormatted, day1AgoFormatter), + "VALUES ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2')"); + + assertQuery( + format("SELECT name FROM %s", fullyQualifiedTestTableName), + "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); + } + + private String moveDate(DateFormat format, Instant today, TemporalUnit unit, int move) + { + return format.format(new Date(today.plus(move, unit).toEpochMilli())); + } + + @Test + public void testDatePartitionProjectionFormatTextWillNotCauseIntervalRequirement() + { + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(); + + computeActual( + "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint, " + + " short_name1 varchar WITH (" + + " partition_projection_type='date', " + + " partition_projection_format='''start''yyyy-MM-dd''end''''s''', " + + " partition_projection_range=ARRAY['start2001-01-01end''s', 'start2001-01-07end''s'] " + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1'], " + + " partition_projection_enabled=true " + + ")"); + } + + @Test + public void testInjectedPartitionProjectionOnVarcharColumn() + { + String tableName = getRandomTestTableName(); + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); + + computeActual( + "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint, " + + " short_name1 varchar(152) WITH (" + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL1', 'CZ1'] " + + " ), " + + " short_name2 varchar(152) WITH (" + + " partition_projection_type='injected'" + + " ) " + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1', 'short_name2'], " + + " partition_projection_enabled=true " + + ")"); + + assertThat( + hiveMinioDataLake + .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) + .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") + .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") + .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+injected[ |]+"); + + computeActual(createInsertStatement( + fullyQualifiedTestTableName, + ImmutableList.of( + ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "'001'"), + ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "'002'"), + ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "'003'"), + ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "'004'")))); + + assertQuery( + format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2='002'", fullyQualifiedTestTableName), + "VALUES 'POLAND_2'"); + + assertThatThrownBy( + () -> getQueryRunner().execute( + format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2='002' OR short_name2='001' )", fullyQualifiedTestTableName))) + .hasMessage("Column projection for column 'short_name2' failed. Injected projection requires single predicate for it's column in where clause. Currently provided can't be converted to single partition."); + + assertThatThrownBy( + () -> getQueryRunner().execute( + format("SELECT name FROM %s", fullyQualifiedTestTableName))) + .hasMessage("Column projection for column 'short_name2' failed. Injected projection requires single predicate for it's column in where clause"); + + assertThatThrownBy( + () -> getQueryRunner().execute( + format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName))) + .hasMessage("Column projection for column 'short_name2' failed. Injected projection requires single predicate for it's column in where clause"); + } + + @Test + public void testPartitionProjectionInvalidTableProperties() + { + assertThatThrownBy(() -> getQueryRunner().execute( + "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + + " name varchar " + + ") WITH ( " + + " partition_projection_enabled=true " + + ")")) + .hasMessage("Partition projection cannot be enabled on a table that is not partitioned"); + + assertThatThrownBy(() -> getQueryRunner().execute( + "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + + " name varchar WITH ( " + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL1', 'CZ1']" + + " ), " + + " short_name1 varchar " + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1'], " + + " partition_projection_enabled=true " + + ")")) + .hasMessage("Partition projection cannot be defined for non-partition column: 'name'"); + + assertThatThrownBy(() -> getQueryRunner().execute( + "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + + " name varchar, " + + " short_name1 varchar WITH ( " + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL1', 'CZ1']" + + " ), " + + " short_name2 varchar " + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1', 'short_name2'], " + + " partition_projection_enabled=true " + + ")")) + .hasMessage("Column projection for column 'short_name2' failed. Projection type property missing"); + + assertThatThrownBy(() -> getQueryRunner().execute( + "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + + " name varchar, " + + " short_name1 varchar WITH (" + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL1', 'CZ1'] " + + " ), " + + " short_name2 varchar WITH (" + + " partition_projection_type='injected' " + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1', 'short_name2'], " + + " partition_projection_enabled=true, " + + " partition_projection_location_template='s3a://dummy/short_name1=${short_name1}/'" + + ")")) + .hasMessage("Partition projection location template: s3a://dummy/short_name1=${short_name1}/ " + + "is missing partition column: 'short_name2' placeholder"); + + assertThatThrownBy(() -> getQueryRunner().execute( + "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + + " name varchar, " + + " short_name1 varchar WITH (" + + " partition_projection_type='integer', " + + " partition_projection_range=ARRAY['1', '2', '3']" + + " ), " + + " short_name2 varchar WITH (" + + " partition_projection_type='enum', " + + " partition_projection_values=ARRAY['PL1', 'CZ1'] " + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1', 'short_name2'], " + + " partition_projection_enabled=true " + + ")")) + .hasMessage("Column projection for column 'short_name1' failed. Property: 'partition_projection_range' needs to be list of 2 integers"); + + assertThatThrownBy(() -> getQueryRunner().execute( + "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + + " name varchar, " + + " short_name1 varchar WITH (" + + " partition_projection_type='date', " + + " partition_projection_values=ARRAY['2001-01-01', '2001-01-02']" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1'], " + + " partition_projection_enabled=true " + + ")")) + .hasMessage("Column projection for column 'short_name1' failed. Missing required property: 'partition_projection_format'"); + + assertThatThrownBy(() -> getQueryRunner().execute( + "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + + " name varchar, " + + " short_name1 varchar WITH (" + + " partition_projection_type='date', " + + " partition_projection_format='yyyy-MM-dd HH', " + + " partition_projection_range=ARRAY['2001-01-01', '2001-01-02']" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1'], " + + " partition_projection_enabled=true " + + ")")) + .hasMessage("Column projection for column 'short_name1' failed. Property: 'partition_projection_range' needs to be a list of 2 valid dates formatted as 'yyyy-MM-dd HH' " + + "or '^\\s*NOW\\s*(([+-])\\s*([0-9]+)\\s*(DAY|HOUR|MINUTE|SECOND)S?\\s*)?$' that are sequential: Unparseable date: \"2001-01-01\""); + + assertThatThrownBy(() -> getQueryRunner().execute( + "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + + " name varchar, " + + " short_name1 varchar WITH (" + + " partition_projection_type='date', " + + " partition_projection_format='yyyy-MM-dd', " + + " partition_projection_range=ARRAY['NOW*3DAYS', '2001-01-02']" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1'], " + + " partition_projection_enabled=true " + + ")")) + .hasMessage("Column projection for column 'short_name1' failed. Property: 'partition_projection_range' needs to be a list of 2 valid dates formatted as 'yyyy-MM-dd' " + + "or '^\\s*NOW\\s*(([+-])\\s*([0-9]+)\\s*(DAY|HOUR|MINUTE|SECOND)S?\\s*)?$' that are sequential: Unparseable date: \"NOW*3DAYS\""); + + assertThatThrownBy(() -> getQueryRunner().execute( + "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + + " name varchar, " + + " short_name1 varchar WITH (" + + " partition_projection_type='date', " + + " partition_projection_format='yyyy-MM-dd', " + + " partition_projection_range=ARRAY['2001-01-02', '2001-01-01']" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1'], " + + " partition_projection_enabled=true " + + ")")) + .hasMessage("Column projection for column 'short_name1' failed. Property: 'partition_projection_range' needs to be a list of 2 valid dates formatted as 'yyyy-MM-dd' " + + "or '^\\s*NOW\\s*(([+-])\\s*([0-9]+)\\s*(DAY|HOUR|MINUTE|SECOND)S?\\s*)?$' that are sequential"); + + assertThatThrownBy(() -> getQueryRunner().execute( + "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + + " name varchar, " + + " short_name1 varchar WITH (" + + " partition_projection_type='date', " + + " partition_projection_format='yyyy-MM-dd', " + + " partition_projection_range=ARRAY['2001-01-01', '2001-01-02'], " + + " partition_projection_interval_unit='Decades'" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1'], " + + " partition_projection_enabled=true " + + ")")) + .hasMessage("Column projection for column 'short_name1' failed. Property: 'partition_projection_interval_unit' value 'Decades' is invalid. " + + "Available options: [Days, Hours, Minutes, Seconds]"); + + assertThatThrownBy(() -> getQueryRunner().execute( + "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + + " name varchar, " + + " short_name1 varchar WITH (" + + " partition_projection_type='date', " + + " partition_projection_format='yyyy-MM-dd HH', " + + " partition_projection_range=ARRAY['2001-01-01 10', '2001-01-02 10']" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1'], " + + " partition_projection_enabled=true " + + ")")) + .hasMessage("Column projection for column 'short_name1' failed. Property: 'partition_projection_interval_unit' " + + "needs to be set when provided 'partition_projection_format' is less that single-day precision. " + + "Interval defaults to 1 day or 1 month, respectively. Otherwise, interval is required"); + + assertThatThrownBy(() -> getQueryRunner().execute( + "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + + " name varchar, " + + " short_name1 varchar WITH (" + + " partition_projection_type='date', " + + " partition_projection_format='yyyy-MM-dd', " + + " partition_projection_range=ARRAY['2001-01-01 10', '2001-01-02 10']" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1'] " + + ")")) + .hasMessage("Columns partition projection properties cannot be set when 'partition_projection_enabled' is not set"); + + // Verify that ignored flag is only interpreted for pre-existing tables where configuration is loaded from metastore. + // It should not allow creating corrupted config via Trino. It's a kill switch to run away when we have compatibility issues. + assertThatThrownBy(() -> getQueryRunner().execute( + "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + + " name varchar, " + + " short_name1 varchar WITH (" + + " partition_projection_type='date', " + + " partition_projection_format='yyyy-MM-dd HH', " + + " partition_projection_range=ARRAY['2001-01-01 10', '2001-01-02 10']" + + " )" + + ") WITH ( " + + " partitioned_by=ARRAY['short_name1'], " + + " partition_projection_enabled=true, " + + " partition_projection_ignore=true " + // <-- Even if this is set we disallow creating corrupted configuration via Trino + ")")) + .hasMessage("Column projection for column 'short_name1' failed. Property: 'partition_projection_interval_unit' " + + "needs to be set when provided 'partition_projection_format' is less that single-day precision. " + + "Interval defaults to 1 day or 1 month, respectively. Otherwise, interval is required"); + } + + @Test + public void testPartitionProjectionIgnore() + { + String tableName = "nation_" + randomNameSuffix(); + String hiveTestTableName = getHiveTestTableName(tableName); + String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); + + // Create corrupted configuration + hiveMinioDataLake.runOnHive( + "CREATE TABLE " + hiveTestTableName + " ( " + + " name varchar(25) " + + ") PARTITIONED BY (" + + " date_time varchar(152) " + + ") " + + "TBLPROPERTIES ( " + + " 'projection.enabled'='true', " + + " 'projection.date_time.type'='date', " + + " 'projection.date_time.format'='yyyy-MM-dd HH', " + + " 'projection.date_time.range'='2001-01-01,2001-01-02' " + + ")"); + + // Expect invalid Partition Projection properties to fail + assertThatThrownBy(() -> getQueryRunner().execute("SELECT * FROM " + fullyQualifiedTestTableName)) + .hasMessage("Column projection for column 'date_time' failed. Property: 'partition_projection_range' needs to be a list of 2 valid dates formatted as 'yyyy-MM-dd HH' " + + "or '^\\s*NOW\\s*(([+-])\\s*([0-9]+)\\s*(DAY|HOUR|MINUTE|SECOND)S?\\s*)?$' that are sequential: Unparseable date: \"2001-01-01\""); + + // Append kill switch table property to ignore Partition Projection properties + hiveMinioDataLake.runOnHive( + "ALTER TABLE " + hiveTestTableName + " SET TBLPROPERTIES ( 'trino.partition_projection.ignore'='TRUE' )"); + // Flush cache to get new definition + computeActual("CALL system.flush_metadata_cache(schema_name => '" + HIVE_TEST_SCHEMA + "', table_name => '" + tableName + "')"); + + // Verify query execution works + computeActual(createInsertStatement( + fullyQualifiedTestTableName, + ImmutableList.of( + ImmutableList.of("'POLAND_1'", "'2022-02-01 12'"), + ImmutableList.of("'POLAND_2'", "'2022-02-01 12'"), + ImmutableList.of("'CZECH_1'", "'2022-02-01 13'"), + ImmutableList.of("'CZECH_2'", "'2022-02-01 13'")))); + + assertQuery("SELECT * FROM " + fullyQualifiedTestTableName, + "VALUES ('POLAND_1', '2022-02-01 12'), " + + "('POLAND_2', '2022-02-01 12'), " + + "('CZECH_1', '2022-02-01 13'), " + + "('CZECH_2', '2022-02-01 13')"); + assertQuery("SELECT * FROM " + fullyQualifiedTestTableName + " WHERE date_time = '2022-02-01 12'", + "VALUES ('POLAND_1', '2022-02-01 12'), ('POLAND_2', '2022-02-01 12')"); + } + + @Test + public void testAnalyzePartitionedTableWithCanonicalization() + { + String tableName = "test_analyze_table_canonicalization_" + randomNameSuffix(); + assertUpdate("CREATE TABLE %s (a_varchar varchar, month varchar) WITH (partitioned_by = ARRAY['month'])".formatted(getFullyQualifiedTestTableName(tableName))); + + assertUpdate("INSERT INTO " + getFullyQualifiedTestTableName(tableName) + " VALUES ('A', '01'), ('B', '01'), ('C', '02'), ('D', '03')", 4); + + String tableLocation = (String) computeActual("SELECT DISTINCT regexp_replace(\"$path\", '/[^/]*/[^/]*$', '') FROM " + getFullyQualifiedTestTableName(tableName)).getOnlyValue(); + + String externalTableName = "external_" + tableName; + List partitionColumnNames = List.of("month"); + assertUpdate( + """ + CREATE TABLE %s( + a_varchar varchar, + month integer) + WITH ( + partitioned_by = ARRAY['month'], + external_location='%s') + """.formatted(getFullyQualifiedTestTableName(externalTableName), tableLocation)); + + addPartitions(tableName, externalTableName, partitionColumnNames, TupleDomain.all()); + assertQuery("SELECT * FROM " + HIVE_TEST_SCHEMA + ".\"" + externalTableName + "$partitions\"", "VALUES 1, 2, 3"); + assertUpdate("ANALYZE " + getFullyQualifiedTestTableName(externalTableName), 4); + assertQuery("SHOW STATS FOR " + getFullyQualifiedTestTableName(externalTableName), + """ + VALUES + ('a_varchar', 4.0, 2.0, 0.0, null, null, null), + ('month', null, 3.0, 0.0, null, 1, 3), + (null, null, null, null, 4.0, null, null) + """); + + assertUpdate("INSERT INTO " + getFullyQualifiedTestTableName(tableName) + " VALUES ('E', '04')", 1); + addPartitions( + tableName, + externalTableName, + partitionColumnNames, + TupleDomain.fromFixedValues(Map.of("month", new NullableValue(VARCHAR, utf8Slice("04"))))); + assertUpdate("CALL system.flush_metadata_cache(schema_name => '" + HIVE_TEST_SCHEMA + "', table_name => '" + externalTableName + "')"); + assertQuery("SELECT * FROM " + HIVE_TEST_SCHEMA + ".\"" + externalTableName + "$partitions\"", "VALUES 1, 2, 3, 4"); + assertUpdate("ANALYZE " + getFullyQualifiedTestTableName(externalTableName) + " WITH (partitions = ARRAY[ARRAY['04']])", 1); + assertQuery("SHOW STATS FOR " + getFullyQualifiedTestTableName(externalTableName), + """ + VALUES + ('a_varchar', 5.0, 2.0, 0.0, null, null, null), + ('month', null, 4.0, 0.0, null, 1, 4), + (null, null, null, null, 5.0, null, null) + """); + // TODO (https://github.com/trinodb/trino/issues/15998) fix selective ANALYZE for table with non-canonical partition values + assertQueryFails("ANALYZE " + getFullyQualifiedTestTableName(externalTableName) + " WITH (partitions = ARRAY[ARRAY['4']])", ".*Partition.*not found.*"); + + assertUpdate("DROP TABLE " + getFullyQualifiedTestTableName(externalTableName)); + assertUpdate("DROP TABLE " + getFullyQualifiedTestTableName(tableName)); + } + + @Test + public void testExternalLocationWithTrailingSpace() + { + String tableName = "test_external_location_with_trailing_space_" + randomNameSuffix(); + String tableLocationDirWithTrailingSpace = tableName + " "; + String tableLocation = format("s3a://%s/%s/%s", bucketName, HIVE_TEST_SCHEMA, tableLocationDirWithTrailingSpace); + + byte[] contents = "hello\u0001world\nbye\u0001world".getBytes(UTF_8); + String targetPath = format("%s/%s/test.txt", HIVE_TEST_SCHEMA, tableLocationDirWithTrailingSpace); + hiveMinioDataLake.getMinioClient().putObject(bucketName, contents, targetPath); + + assertUpdate(format( + "CREATE TABLE %s (" + + " a varchar, " + + " b varchar) " + + "WITH (format='TEXTFILE', external_location='%s')", + tableName, + tableLocation)); + + assertQuery("SELECT a, b FROM " + tableName, "VALUES ('hello', 'world'), ('bye', 'world')"); + + String actualTableLocation = getTableLocation(tableName); + assertThat(actualTableLocation).isEqualTo(tableLocation); + + assertUpdate("DROP TABLE " + tableName); + } + + @Test + public void testCreateSchemaInvalidName() + { + assertThatThrownBy(() -> assertUpdate("CREATE SCHEMA \".\"")) + .hasMessage("Invalid object name: '.'"); + + assertThatThrownBy(() -> assertUpdate("CREATE SCHEMA \"..\"")) + .hasMessage("Invalid object name: '..'"); + + assertThatThrownBy(() -> assertUpdate("CREATE SCHEMA \"foo/bar\"")) + .hasMessage("Invalid object name: 'foo/bar'"); + } + + @Test + public void testCreateTableInvalidName() + { + assertThatThrownBy(() -> assertUpdate("CREATE TABLE " + HIVE_TEST_SCHEMA + ".\".\" (col integer)")) + .hasMessageContaining("Invalid table name"); + assertThatThrownBy(() -> assertUpdate("CREATE TABLE " + HIVE_TEST_SCHEMA + ".\"..\" (col integer)")) + .hasMessageContaining("Invalid table name"); + assertThatThrownBy(() -> assertUpdate("CREATE TABLE " + HIVE_TEST_SCHEMA + ".\"...\" (col integer)")) + .hasMessage("Invalid table name"); + + for (String tableName : Arrays.asList("foo/bar", "foo/./bar", "foo/../bar")) { + assertThatThrownBy(() -> assertUpdate("CREATE TABLE " + HIVE_TEST_SCHEMA + ".\"" + tableName + "\" (col integer)")) + .hasMessage(format("Invalid object name: '%s'", tableName)); + assertThatThrownBy(() -> assertUpdate("CREATE TABLE " + HIVE_TEST_SCHEMA + ".\"" + tableName + "\" (col) AS VALUES 1")) + .hasMessage(format("Invalid object name: '%s'", tableName)); + } + } + + @Test + public void testRenameSchemaToInvalidObjectName() + { + String schemaName = "test_rename_schema_invalid_name_" + randomNameSuffix(); + assertUpdate("CREATE SCHEMA %1$s WITH (location='s3a://%2$s/%1$s')".formatted(schemaName, bucketName)); + + for (String invalidSchemaName : Arrays.asList(".", "..", "foo/bar")) { + assertThatThrownBy(() -> assertUpdate("ALTER SCHEMA hive." + schemaName + " RENAME TO \"" + invalidSchemaName + "\"")) + .hasMessage(format("Invalid object name: '%s'", invalidSchemaName)); + } + + assertUpdate("DROP SCHEMA " + schemaName); + } + + @Test + public void testRenameTableToInvalidObjectName() + { + String tableName = "test_rename_table_invalid_name_" + randomNameSuffix(); + assertUpdate("CREATE TABLE %s (a_varchar varchar)".formatted(getFullyQualifiedTestTableName(tableName))); + + for (String invalidTableName : Arrays.asList(".", "..", "foo/bar")) { + assertThatThrownBy(() -> assertUpdate("ALTER TABLE " + getFullyQualifiedTestTableName(tableName) + " RENAME TO \"" + invalidTableName + "\"")) + .hasMessage(format("Invalid object name: '%s'", invalidTableName)); + } + + for (String invalidSchemaName : Arrays.asList(".", "..", "foo/bar")) { + assertThatThrownBy(() -> assertUpdate("ALTER TABLE " + getFullyQualifiedTestTableName(tableName) + " RENAME TO \"" + invalidSchemaName + "\".validTableName")) + .hasMessage(format("Invalid object name: '%s'", invalidSchemaName)); + } + + assertUpdate("DROP TABLE " + getFullyQualifiedTestTableName(tableName)); + } + + @Test + public void testUnpartitionedTableExternalLocationWithTrainingSlash() + { + String tableName = "test_external_location_trailing_slash_" + randomNameSuffix(); + String tableLocationWithTrailingSlash = format("s3://%s/%s/%s/", bucketName, HIVE_TEST_SCHEMA, tableName); + byte[] contents = "Trino\nSQL\non\neverything".getBytes(UTF_8); + String dataFilePath = format("%s/%s/data.txt", HIVE_TEST_SCHEMA, tableName); + hiveMinioDataLake.getMinioClient().putObject(bucketName, contents, dataFilePath); + + assertUpdate(format( + "CREATE TABLE %s (" + + " a_varchar varchar) " + + "WITH (" + + " external_location='%s'," + + " format='TEXTFILE')", + tableName, + tableLocationWithTrailingSlash)); + assertQuery("SELECT * FROM " + tableName, "VALUES 'Trino', 'SQL', 'on', 'everything'"); + + assertUpdate("DROP TABLE " + tableName); + } + + @Test + public void testUnpartitionedTableExternalLocationOnTopOfTheBucket() + { + String topBucketName = "test-hive-unpartitioned-top-of-the-bucket-" + randomNameSuffix(); + hiveMinioDataLake.getMinio().createBucket(topBucketName); + String tableName = "test_external_location_top_of_the_bucket_" + randomNameSuffix(); + + byte[] contents = "Trino\nSQL\non\neverything".getBytes(UTF_8); + hiveMinioDataLake.getMinioClient().putObject(topBucketName, contents, "data.txt"); + + assertUpdate(format( + "CREATE TABLE %s (" + + " a_varchar varchar) " + + "WITH (" + + " external_location='%s'," + + " format='TEXTFILE')", + tableName, + format("s3://%s/", topBucketName))); + assertQuery("SELECT * FROM " + tableName, "VALUES 'Trino', 'SQL', 'on', 'everything'"); + + assertUpdate("DROP TABLE " + tableName); + } + + @Test + public void testPartitionedTableExternalLocationOnTopOfTheBucket() + { + String topBucketName = "test-hive-partitioned-top-of-the-bucket-" + randomNameSuffix(); + hiveMinioDataLake.getMinio().createBucket(topBucketName); + String tableName = "test_external_location_top_of_the_bucket_" + randomNameSuffix(); + + assertUpdate(format( + "CREATE TABLE %s (" + + " a_varchar varchar, " + + " pkey integer) " + + "WITH (" + + " external_location='%s'," + + " partitioned_by=ARRAY['pkey'])", + tableName, + format("s3://%s/", topBucketName))); + assertUpdate("INSERT INTO " + tableName + " VALUES ('a', 1) , ('b', 1), ('c', 2), ('d', 2)", 4); + assertQuery("SELECT * FROM " + tableName, "VALUES ('a', 1), ('b',1), ('c', 2), ('d', 2)"); + assertUpdate("DELETE FROM " + tableName + " where pkey = 2"); + assertQuery("SELECT * FROM " + tableName, "VALUES ('a', 1), ('b',1)"); + + assertUpdate("DROP TABLE " + tableName); + } + + @Test + public void testDropStatsPartitionedTable() + { + String tableName = "test_hive_drop_stats_partitioned_table_" + randomNameSuffix(); + assertUpdate(("CREATE TABLE %s (" + + " data integer," + + " p_varchar varchar," + + " p_integer integer" + + ") " + + "WITH (" + + " partitioned_by=ARRAY['p_varchar', 'p_integer']" + + ")").formatted(getFullyQualifiedTestTableName(tableName))); + + // Drop stats for partition which does not exist + assertThat(query(format("CALL system.drop_stats('%s', '%s', ARRAY[ARRAY['partnotfound', '999']])", HIVE_TEST_SCHEMA, tableName))) + .failure().hasMessage("No partition found for name: p_varchar=partnotfound/p_integer=999"); + + assertUpdate("INSERT INTO " + getFullyQualifiedTestTableName(tableName) + " VALUES (1, 'part1', 10) , (2, 'part2', 10), (12, 'part2', 20)", 3); + + // Run analyze on the entire table + assertUpdate("ANALYZE " + getFullyQualifiedTestTableName(tableName), 3); + + assertQuery("SHOW STATS FOR " + getFullyQualifiedTestTableName(tableName), + """ + VALUES + ('data', null, 1.0, 0.0, null, 1, 12), + ('p_varchar', 15.0, 2.0, 0.0, null, null, null), + ('p_integer', null, 2.0, 0.0, null, 10, 20), + (null, null, null, null, 3.0, null, null) + """); + + assertUpdate(format("CALL system.drop_stats('%s', '%s', ARRAY[ARRAY['part1', '10']])", HIVE_TEST_SCHEMA, tableName)); + + assertQuery("SHOW STATS FOR " + getFullyQualifiedTestTableName(tableName), + """ + VALUES + ('data', null, 1.0, 0.0, null, 2, 12), + ('p_varchar', 15.0, 2.0, 0.0, null, null, null), + ('p_integer', null, 2.0, 0.0, null, 10, 20), + (null, null, null, null, 3.0, null, null) + """); + + assertUpdate("DELETE FROM " + getFullyQualifiedTestTableName(tableName) + " WHERE p_varchar ='part1' and p_integer = 10"); + + // Drop stats for partition which does not exist + assertThat(query(format("CALL system.drop_stats('%s', '%s', ARRAY[ARRAY['part1', '10']])", HIVE_TEST_SCHEMA, tableName))) + .failure().hasMessage("No partition found for name: p_varchar=part1/p_integer=10"); + + assertQuery("SHOW STATS FOR " + getFullyQualifiedTestTableName(tableName), + """ + VALUES + ('data', null, 1.0, 0.0, null, 2, 12), + ('p_varchar', 10.0, 1.0, 0.0, null, null, null), + ('p_integer', null, 2.0, 0.0, null, 10, 20), + (null, null, null, null, 2.0, null, null) + """); + assertUpdate("DROP TABLE " + getFullyQualifiedTestTableName(tableName)); + } + + @Test + public void testUnsupportedDropSchemaCascadeWithNonHiveTable() + { + String schemaName = "test_unsupported_drop_schema_cascade_" + randomNameSuffix(); + String icebergTableName = "test_dummy_iceberg_table" + randomNameSuffix(); + + hiveMinioDataLake.runOnHive("CREATE DATABASE %2$s LOCATION 's3a://%1$s/%2$s'".formatted(bucketName, schemaName)); + try { + hiveMinioDataLake.runOnHive("CREATE TABLE " + schemaName + "." + icebergTableName + " TBLPROPERTIES ('table_type'='iceberg') AS SELECT 1 a"); + + assertQueryFails("DROP SCHEMA " + schemaName + " CASCADE", "\\QCannot query Iceberg table '%s.%s'".formatted(schemaName, icebergTableName)); + + assertThat(computeActual("SHOW SCHEMAS").getOnlyColumnAsSet()).contains(schemaName); + assertThat(computeActual("SHOW TABLES FROM " + schemaName).getOnlyColumnAsSet()).contains(icebergTableName); + assertThat(hiveMinioDataLake.getMinioClient().listObjects(bucketName, schemaName).stream()).isNotEmpty(); + } + finally { + hiveMinioDataLake.runOnHive("DROP DATABASE IF EXISTS " + schemaName + " CASCADE"); + } + } + + @Test + public void testUnsupportedCommentOnHiveView() + { + String viewName = HIVE_TEST_SCHEMA + ".test_unsupported_comment_on_hive_view_" + randomNameSuffix(); + + hiveMinioDataLake.runOnHive("CREATE VIEW " + viewName + " AS SELECT 1 x"); + try { + assertQueryFails("COMMENT ON COLUMN " + viewName + ".x IS NULL", "Hive views are not supported.*"); + } + finally { + hiveMinioDataLake.runOnHive("DROP VIEW " + viewName); + } + } + + @Test + public void testCreateFunction() + { + String name = "test_" + randomNameSuffix(); + String name2 = "test_" + randomNameSuffix(); + + assertUpdate("CREATE FUNCTION " + name + "(x integer) RETURNS bigint RETURN x * 10"); + assertQuery("SELECT " + name + "(99)", "SELECT 990"); + + assertUpdate("CREATE OR REPLACE FUNCTION " + name + "(x integer) RETURNS bigint COMMENT 't42' RETURN x * 42"); + assertQuery("SELECT " + name + "(99)", "SELECT 4158"); + + assertQueryFails("SELECT " + name + "(2.9)", ".*Unexpected parameters.*"); + + assertUpdate("CREATE FUNCTION " + name + "(x double) RETURNS double COMMENT 't88' RETURN x * 8.8"); + + assertThat(query("SHOW FUNCTIONS")) + .result() + .skippingTypesCheck() + .containsAll(resultBuilder(getSession()) + .row(name, "bigint", "integer", "scalar", true, "t42") + .row(name, "double", "double", "scalar", true, "t88") + .build()); + + assertQuery("SELECT " + name + "(99)", "SELECT 4158"); + assertQuery("SELECT " + name + "(2.9)", "SELECT 25.52"); + + assertQueryFails("CREATE FUNCTION " + name + "(x int) RETURNS bigint RETURN x", "line 1:1: Function already exists"); + + assertQuery("SELECT " + name + "(99)", "SELECT 4158"); + assertQuery("SELECT " + name + "(2.9)", "SELECT 25.52"); + + assertUpdate("CREATE OR REPLACE FUNCTION " + name + "(x bigint) RETURNS bigint RETURN x * 23"); + assertUpdate("CREATE FUNCTION " + name2 + "(s varchar) RETURNS varchar RETURN 'Hello ' || s"); + + assertThat(query("SHOW FUNCTIONS")) + .result() + .skippingTypesCheck() + .containsAll(resultBuilder(getSession()) + .row(name, "bigint", "integer", "scalar", true, "t42") + .row(name, "bigint", "bigint", "scalar", true, "") + .row(name, "double", "double", "scalar", true, "t88") + .row(name2, "varchar", "varchar", "scalar", true, "") + .build()); + + assertQuery("SELECT " + name + "(99)", "SELECT 4158"); + assertQuery("SELECT " + name + "(cast(99 as bigint))", "SELECT 2277"); + assertQuery("SELECT " + name + "(2.9)", "SELECT 25.52"); + assertQuery("SELECT " + name2 + "('world')", "SELECT 'Hello world'"); + + assertQueryFails("DROP FUNCTION " + name + "(varchar)", "line 1:1: Function not found"); + assertUpdate("DROP FUNCTION " + name + "(z bigint)"); + assertUpdate("DROP FUNCTION " + name + "(double)"); + assertUpdate("DROP FUNCTION " + name + "(int)"); + assertQueryFails("DROP FUNCTION " + name + "(bigint)", "line 1:1: Function not found"); + assertUpdate("DROP FUNCTION IF EXISTS " + name + "(bigint)"); + assertUpdate("DROP FUNCTION " + name2 + "(varchar)"); + assertQueryFails("DROP FUNCTION " + name2 + "(varchar)", "line 1:1: Function not found"); + } + + private void renamePartitionResourcesOutsideTrino(String tableName, String partitionColumn, String regionKey) + { + String partitionName = format("%s=%s", partitionColumn, regionKey); + String partitionS3KeyPrefix = format("%s/%s/%s", HIVE_TEST_SCHEMA, tableName, partitionName); + String renamedPartitionSuffix = "CP"; + + // Copy whole partition to new location + MinioClient minioClient = hiveMinioDataLake.getMinioClient(); + minioClient.listObjects(bucketName, "") + .forEach(objectKey -> { + if (objectKey.startsWith(partitionS3KeyPrefix)) { + String fileName = objectKey.substring(objectKey.lastIndexOf('/')); + String destinationKey = partitionS3KeyPrefix + renamedPartitionSuffix + fileName; + minioClient.copyObject(bucketName, objectKey, bucketName, destinationKey); + } + }); + + // Delete old partition and update metadata to point to location of new copy + Table hiveTable = metastoreClient.getTable(HIVE_TEST_SCHEMA, tableName).orElseThrow(); + Partition partition = metastoreClient.getPartition(hiveTable, List.of(regionKey)).orElseThrow(); + Map> partitionStatistics = metastoreClient.getPartitionColumnStatistics( + HIVE_TEST_SCHEMA, + tableName, + ImmutableSet.of(partitionName), + partition.getColumns().stream().map(Column::getName).collect(toSet())); + + metastoreClient.dropPartition(HIVE_TEST_SCHEMA, tableName, List.of(regionKey), true); + metastoreClient.addPartitions(HIVE_TEST_SCHEMA, tableName, List.of( + new PartitionWithStatistics( + Partition.builder(partition) + .withStorage(builder -> builder.setLocation( + partition.getStorage().getLocation() + renamedPartitionSuffix)) + .build(), + partitionName, + new PartitionStatistics(getHiveBasicStatistics(partition.getParameters()), partitionStatistics.get(partitionName))))); + } + + protected void assertInsertFailure(String testTable, String expectedMessageRegExp) + { + assertInsertFailure(getSession(), testTable, expectedMessageRegExp); + } + + protected void assertInsertFailure(Session session, String testTable, String expectedMessageRegExp) + { + assertQueryFails( + session, + createInsertAsSelectFromTpchStatement(testTable), + expectedMessageRegExp); + } + + private String createInsertAsSelectFromTpchStatement(String testTable) + { + return format("INSERT INTO %s " + + "SELECT name, comment, nationkey, regionkey " + + "FROM tpch.tiny.nation", + testTable); + } + + protected String createInsertStatement(String testTable, List> data) + { + String values = data.stream() + .map(row -> String.join(", ", row)) + .collect(Collectors.joining("), (")); + return format("INSERT INTO %s VALUES (%s)", testTable, values); + } + + protected void assertOverwritePartition(String testTable) + { + computeActual(createInsertStatement( + testTable, + ImmutableList.of( + ImmutableList.of("'POLAND'", "'Test Data'", "25", "5"), + ImmutableList.of("'CZECH'", "'Test Data'", "26", "5")))); + query(format("SELECT name, comment, nationkey, regionkey FROM %s WHERE regionkey = 5", testTable)) + .assertThat() + .result() + .skippingTypesCheck() + .containsAll(resultBuilder(getSession()) + .row("POLAND", "Test Data", 25L, 5L) + .row("CZECH", "Test Data", 26L, 5L) + .build()); + + computeActual(createInsertStatement( + testTable, + ImmutableList.of( + ImmutableList.of("'POLAND'", "'Overwrite'", "25", "5")))); + query(format("SELECT name, comment, nationkey, regionkey FROM %s WHERE regionkey = 5", testTable)) + .assertThat() + .result() + .skippingTypesCheck() + .containsAll(resultBuilder(getSession()) + .row("POLAND", "Overwrite", 25L, 5L) + .build()); + computeActual(format("DROP TABLE %s", testTable)); + } + + protected String getRandomTestTableName() + { + return "nation_" + randomNameSuffix(); + } + + protected String getFullyQualifiedTestTableName() + { + return getFullyQualifiedTestTableName(getRandomTestTableName()); + } + + protected String getFullyQualifiedTestTableName(String tableName) + { + return getFullyQualifiedTestTableName(HIVE_TEST_SCHEMA, tableName); + } + + protected String getFullyQualifiedTestTableName(String schemaName, String tableName) + { + return "hive.%s.%s".formatted(schemaName, tableName); + } + + protected String getHiveTestTableName(String tableName) + { + return getHiveTestTableName(HIVE_TEST_SCHEMA, tableName); + } + + protected String getHiveTestTableName(String schemaName, String tableName) + { + return "%s.%s".formatted(schemaName, tableName); + } + + protected String getCreateTableStatement(String tableName, String... propertiesEntries) + { + return getCreateTableStatement(tableName, Arrays.asList(propertiesEntries)); + } + + protected String getCreateTableStatement(String tableName, List propertiesEntries) + { + return format( + "CREATE TABLE %s (" + + " name varchar(25), " + + " comment varchar(152), " + + " nationkey bigint, " + + " regionkey bigint) " + + (propertiesEntries.isEmpty() ? "" : propertiesEntries + .stream() + .collect(joining(",", "WITH (", ")"))), + tableName); + } + + protected void copyTpchNationToTable(String testTable) + { + computeActual(format("INSERT INTO " + testTable + " SELECT name, comment, nationkey, regionkey FROM tpch.tiny.nation")); + } + + private void testWriteWithFileSize(String testTable, int scaleFactorInThousands, long fileSizeRangeStart, long fileSizeRangeEnd) + { + String scaledColumnExpression = format("array_join(transform(sequence(1, %d), x-> array_join(repeat(comment, 1000), '')), '')", scaleFactorInThousands); + computeActual(format("INSERT INTO " + testTable + " SELECT %s, %s, regionkey FROM tpch.tiny.nation WHERE nationkey = 9", scaledColumnExpression, scaledColumnExpression)); + query(format("SELECT length(col1) FROM %s", testTable)) + .assertThat() + .result() + .skippingTypesCheck() + .containsAll(resultBuilder(getSession()) + .row(114L * scaleFactorInThousands * 1000) + .build()); + query(format("SELECT \"$file_size\" BETWEEN %d AND %d FROM %s", fileSizeRangeStart, fileSizeRangeEnd, testTable)) + .assertThat() + .result() + .skippingTypesCheck() + .containsAll(resultBuilder(getSession()) + .row(true) + .build()); + } + + private void addPartitions( + String sourceTableName, + String destinationExternalTableName, + List columnNames, + TupleDomain partitionsKeyFilter) + { + Optional> partitionNames = metastoreClient.getPartitionNamesByFilter(HIVE_TEST_SCHEMA, sourceTableName, columnNames, partitionsKeyFilter); + if (partitionNames.isEmpty()) { + // nothing to add + return; + } + Table table = metastoreClient.getTable(HIVE_TEST_SCHEMA, sourceTableName) + .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(HIVE_TEST_SCHEMA, sourceTableName))); + Map> partitionsByNames = metastoreClient.getPartitionsByNames(table, partitionNames.get()); + + metastoreClient.addPartitions( + HIVE_TEST_SCHEMA, + destinationExternalTableName, + partitionsByNames.entrySet().stream() + .map(e -> new PartitionWithStatistics( + e.getValue() + .map(p -> Partition.builder(p).setTableName(destinationExternalTableName).build()) + .orElseThrow(), + e.getKey(), + PartitionStatistics.empty())) + .collect(toImmutableList())); + } + + private String getTableLocation(String tableName) + { + return (String) computeScalar("SELECT DISTINCT regexp_replace(\"$path\", '/[^/]*$', '') FROM " + tableName); + } + + @Test + public void testInsertOverwritePartitionedAndBucketedAcidTable() + { + String testTable = getFullyQualifiedTestTableName(); + computeActual(getCreateTableStatement( + testTable, + "partitioned_by=ARRAY['regionkey']", + "bucketed_by = ARRAY['nationkey']", + "bucket_count = 3", + "format = 'ORC'", + "transactional = true")); + assertInsertFailure( + testTable, + "Overwriting existing partition in transactional tables doesn't support DIRECT_TO_TARGET_EXISTING_DIRECTORY write mode"); + } +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java index 558a45059a5e..75efa817c54b 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java @@ -13,2264 +13,21 @@ */ package io.trino.plugin.hive; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import io.airlift.units.DataSize; -import io.trino.Session; -import io.trino.metastore.Column; -import io.trino.metastore.HiveColumnStatistics; -import io.trino.metastore.HiveMetastore; -import io.trino.metastore.Partition; -import io.trino.metastore.PartitionStatistics; -import io.trino.metastore.PartitionWithStatistics; -import io.trino.metastore.Table; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.plugin.hive.containers.HiveHadoop; -import io.trino.plugin.hive.containers.HiveMinioDataLake; -import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; -import io.trino.plugin.hive.s3.S3HiveQueryRunner; -import io.trino.spi.connector.SchemaTableName; -import io.trino.spi.connector.TableNotFoundException; -import io.trino.spi.predicate.NullableValue; -import io.trino.spi.predicate.TupleDomain; -import io.trino.testing.AbstractTestQueryFramework; -import io.trino.testing.QueryRunner; -import io.trino.testing.minio.MinioClient; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; -import java.text.DateFormat; -import java.text.SimpleDateFormat; -import java.time.Instant; -import java.time.ZoneId; -import java.time.temporal.TemporalUnit; -import java.util.Arrays; -import java.util.Date; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.TimeZone; -import java.util.stream.Collectors; - -import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.airlift.slice.Slices.utf8Slice; -import static io.airlift.units.DataSize.Unit.MEGABYTE; -import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; -import static io.trino.plugin.hive.metastore.MetastoreUtil.getHiveBasicStatistics; -import static io.trino.spi.type.VarcharType.VARCHAR; -import static io.trino.testing.MaterializedResult.resultBuilder; import static io.trino.testing.TestingNames.randomNameSuffix; -import static java.lang.String.format; -import static java.nio.charset.StandardCharsets.UTF_8; -import static java.time.temporal.ChronoUnit.DAYS; -import static java.time.temporal.ChronoUnit.MINUTES; -import static java.util.regex.Pattern.quote; -import static java.util.stream.Collectors.joining; -import static java.util.stream.Collectors.toSet; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @TestInstance(PER_CLASS) -public class TestHive3OnDataLake - extends AbstractTestQueryFramework +class TestHive3OnDataLake + extends BaseTestHiveOnDataLake { - private static final String HIVE_TEST_SCHEMA = "hive_datalake"; - private static final DataSize HIVE_S3_STREAMING_PART_SIZE = DataSize.of(5, MEGABYTE); - - private String bucketName; - private HiveMinioDataLake hiveMinioDataLake; - private HiveMetastore metastoreClient; - - @Override - protected QueryRunner createQueryRunner() - throws Exception - { - this.bucketName = "test-hive-insert-overwrite-" + randomNameSuffix(); - this.hiveMinioDataLake = closeAfterClass( - new HiveMinioDataLake(bucketName, HiveHadoop.HIVE3_IMAGE)); - this.hiveMinioDataLake.start(); - this.metastoreClient = new BridgingHiveMetastore( - testingThriftHiveMetastoreBuilder() - .metastoreClient(this.hiveMinioDataLake.getHiveHadoop().getHiveMetastoreEndpoint()) - .build(this::closeAfterClass)); - return S3HiveQueryRunner.builder(hiveMinioDataLake) - .addExtraProperty("sql.path", "hive.functions") - .addExtraProperty("sql.default-function-catalog", "hive") - .addExtraProperty("sql.default-function-schema", "functions") - .setHiveProperties( - ImmutableMap.builder() - .put("hive.insert-existing-partitions-behavior", "OVERWRITE") - .put("hive.non-managed-table-writes-enabled", "true") - // Below are required to enable caching on metastore - .put("hive.metastore-cache-ttl", "1d") - .put("hive.metastore-refresh-interval", "1d") - // This is required to reduce memory pressure to test writing large files - .put("s3.streaming.part-size", HIVE_S3_STREAMING_PART_SIZE.toString()) - // This is required to enable AWS Athena partition projection - .put("hive.partition-projection-enabled", "true") - .buildOrThrow()) - .build(); - } - - @BeforeAll - public void setUp() - { - computeActual(format( - "CREATE SCHEMA hive.%1$s WITH (location='s3a://%2$s/%1$s')", - HIVE_TEST_SCHEMA, - bucketName)); - computeActual("CREATE SCHEMA hive.functions"); - } - - @Test - public void testInsertOverwriteInTransaction() - { - String testTable = getFullyQualifiedTestTableName(); - computeActual(getCreateTableStatement(testTable, "partitioned_by=ARRAY['regionkey']")); - assertThatThrownBy( - () -> newTransaction() - .execute(getSession(), session -> { - getQueryRunner().execute(session, createInsertAsSelectFromTpchStatement(testTable)); - })) - .hasMessage("Overwriting existing partition in non auto commit context doesn't support DIRECT_TO_TARGET_EXISTING_DIRECTORY write mode"); - computeActual(format("DROP TABLE %s", testTable)); - } - - @Test - public void testInsertOverwriteNonPartitionedTable() - { - String testTable = getFullyQualifiedTestTableName(); - computeActual(getCreateTableStatement(testTable)); - assertInsertFailure( - testTable, - "Overwriting unpartitioned table not supported when writing directly to target directory"); - computeActual(format("DROP TABLE %s", testTable)); - } - - @Test - public void testInsertOverwriteNonPartitionedBucketedTable() - { - String testTable = getFullyQualifiedTestTableName(); - computeActual(getCreateTableStatement( - testTable, - "bucketed_by = ARRAY['nationkey']", - "bucket_count = 3")); - assertInsertFailure( - testTable, - "Overwriting unpartitioned table not supported when writing directly to target directory"); - computeActual(format("DROP TABLE %s", testTable)); - } - - @Test - public void testInsertOverwritePartitionedTable() - { - String testTable = getFullyQualifiedTestTableName(); - computeActual(getCreateTableStatement( - testTable, - "partitioned_by=ARRAY['regionkey']")); - copyTpchNationToTable(testTable); - assertOverwritePartition(testTable); - } - - @Test - public void testInsertOverwritePartitionedAndBucketedTable() - { - String testTable = getFullyQualifiedTestTableName(); - computeActual(getCreateTableStatement( - testTable, - "partitioned_by=ARRAY['regionkey']", - "bucketed_by = ARRAY['nationkey']", - "bucket_count = 3")); - copyTpchNationToTable(testTable); - assertOverwritePartition(testTable); - } - - @Test - public void testInsertOverwritePartitionedAndBucketedExternalTable() - { - String testTable = getFullyQualifiedTestTableName(); - // Store table data in data lake bucket - computeActual(getCreateTableStatement( - testTable, - "partitioned_by=ARRAY['regionkey']", - "bucketed_by = ARRAY['nationkey']", - "bucket_count = 3")); - copyTpchNationToTable(testTable); - - // Map this table as external table - String externalTableName = testTable + "_ext"; - computeActual(getCreateTableStatement( - externalTableName, - "partitioned_by=ARRAY['regionkey']", - "bucketed_by = ARRAY['nationkey']", - "bucket_count = 3", - format("external_location = 's3a://%s/%s/%s/'", this.bucketName, HIVE_TEST_SCHEMA, testTable))); - copyTpchNationToTable(testTable); - assertOverwritePartition(externalTableName); - } - - @Test - public void testSyncPartitionOnBucketRoot() - { - String tableName = "test_sync_partition_on_bucket_root_" + randomNameSuffix(); - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); - - hiveMinioDataLake.getMinioClient().putObject( - bucketName, - "hello\u0001world\nbye\u0001world".getBytes(UTF_8), - "part_key=part_val/data.txt"); - - assertUpdate("CREATE TABLE " + fullyQualifiedTestTableName + "(" + - " a varchar," + - " b varchar," + - " part_key varchar)" + - "WITH (" + - " external_location='s3://" + bucketName + "/'," + - " partitioned_by=ARRAY['part_key']," + - " format='TEXTFILE'" + - ")"); - - getQueryRunner().execute("CALL system.sync_partition_metadata(schema_name => '" + HIVE_TEST_SCHEMA + "', table_name => '" + tableName + "', mode => 'ADD')"); - - assertQuery("SELECT * FROM " + fullyQualifiedTestTableName, "VALUES ('hello', 'world', 'part_val'), ('bye', 'world', 'part_val')"); - - assertUpdate("DROP TABLE " + fullyQualifiedTestTableName); - } - - @Test - public void testSyncPartitionCaseSensitivePathVariation() - { - String tableName = "test_sync_partition_case_variation_" + randomNameSuffix(); - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); - String tableLocation = format("s3://%s/%s/%s/", bucketName, HIVE_TEST_SCHEMA, tableName); - - hiveMinioDataLake.getMinioClient().putObject( - bucketName, - "Trino\u0001rocks".getBytes(UTF_8), - HIVE_TEST_SCHEMA + "/" + tableName + "/part_key=part_val/data.txt"); - - assertUpdate("CREATE TABLE " + fullyQualifiedTestTableName + "(" + - " a varchar," + - " b varchar," + - " part_key varchar)" + - "WITH (" + - " external_location='" + tableLocation + "'," + - " partitioned_by=ARRAY['part_key']," + - " format='TEXTFILE'" + - ")"); - - getQueryRunner().execute("CALL system.sync_partition_metadata(schema_name => '" + HIVE_TEST_SCHEMA + "', table_name => '" + tableName + "', mode => 'ADD')"); - assertQuery("SELECT * FROM " + fullyQualifiedTestTableName, "VALUES ('Trino', 'rocks', 'part_val')"); - - // Move the data to a location where the partition path differs only in case - hiveMinioDataLake.getMinioClient().removeObject(bucketName, HIVE_TEST_SCHEMA + "/" + tableName + "/part_key=part_val/data.txt"); - hiveMinioDataLake.getMinioClient().putObject( - bucketName, - "Trino\u0001rocks".getBytes(UTF_8), - HIVE_TEST_SCHEMA + "/" + tableName + "/PART_KEY=part_val/data.txt"); - - getQueryRunner().execute("CALL system.sync_partition_metadata(schema_name => '" + HIVE_TEST_SCHEMA + "', table_name => '" + tableName + "', mode => 'FULL', case_sensitive => false)"); - assertQuery("SELECT * FROM " + fullyQualifiedTestTableName, "VALUES ('Trino', 'rocks', 'part_val')"); - - // Verify that syncing again the partition metadata has no negative effect (e.g. drop the partition) - getQueryRunner().execute("CALL system.sync_partition_metadata(schema_name => '" + HIVE_TEST_SCHEMA + "', table_name => '" + tableName + "', mode => 'FULL', case_sensitive => false)"); - assertQuery("SELECT * FROM " + fullyQualifiedTestTableName, "VALUES ('Trino', 'rocks', 'part_val')"); - - assertUpdate("DROP TABLE " + fullyQualifiedTestTableName); - } - - @Test - public void testSyncPartitionSpecialCharacters() - { - String tableName = "test_sync_partition_special_characters_" + randomNameSuffix(); - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); - String tableLocation = format("s3://%s/%s/%s/", bucketName, HIVE_TEST_SCHEMA, tableName); - - hiveMinioDataLake.getMinioClient().putObject( - bucketName, - "Trino\u0001rocks\u0001hyphens".getBytes(UTF_8), - HIVE_TEST_SCHEMA + "/" + tableName + "/part_key=with-hyphen/data.txt"); - hiveMinioDataLake.getMinioClient().putObject( - bucketName, - "Trino\u0001rocks\u0001dots".getBytes(UTF_8), - HIVE_TEST_SCHEMA + "/" + tableName + "/part_key=with.dot/data.txt"); - hiveMinioDataLake.getMinioClient().putObject( - bucketName, - "Trino\u0001rocks\u0001colons".getBytes(UTF_8), - HIVE_TEST_SCHEMA + "/" + tableName + "/part_key=with%3Acolon/data.txt"); - hiveMinioDataLake.getMinioClient().putObject( - bucketName, - "Trino\u0001rocks\u0001slashes".getBytes(UTF_8), - HIVE_TEST_SCHEMA + "/" + tableName + "/part_key=with%2Fslash/data.txt"); - hiveMinioDataLake.getMinioClient().putObject( - bucketName, - "Trino\u0001rocks\u0001backslashes".getBytes(UTF_8), - HIVE_TEST_SCHEMA + "/" + tableName + "/part_key=with%5Cbackslash/data.txt"); - hiveMinioDataLake.getMinioClient().putObject( - bucketName, - "Trino\u0001rocks\u0001percents".getBytes(UTF_8), - HIVE_TEST_SCHEMA + "/" + tableName + "/part_key=with%25percent/data.txt"); - - assertUpdate("CREATE TABLE " + fullyQualifiedTestTableName + "(" + - " a varchar," + - " b varchar," + - " c varchar," + - " part_key varchar)" + - "WITH (" + - " external_location='" + tableLocation + "'," + - " partitioned_by=ARRAY['part_key']," + - " format='TEXTFILE'" + - ")"); - - getQueryRunner().execute("CALL system.sync_partition_metadata(schema_name => '" + HIVE_TEST_SCHEMA + "', table_name => '" + tableName + "', mode => 'ADD')"); - assertQuery( - "SELECT * FROM " + fullyQualifiedTestTableName, - """ - VALUES - ('Trino', 'rocks', 'hyphens', 'with-hyphen'), - ('Trino', 'rocks', 'dots', 'with.dot'), - ('Trino', 'rocks', 'colons', 'with:colon'), - ('Trino', 'rocks', 'slashes', 'with/slash'), - ('Trino', 'rocks', 'backslashes', 'with\\backslash'), - ('Trino', 'rocks', 'percents', 'with%percent') - """); - - assertUpdate("DROP TABLE " + fullyQualifiedTestTableName); - } - - @Test - public void testFlushPartitionCache() - { - String tableName = "nation_" + randomNameSuffix(); - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); - String partitionColumn = "regionkey"; - - testFlushPartitionCache( - tableName, - fullyQualifiedTestTableName, - partitionColumn, - format( - "CALL system.flush_metadata_cache(schema_name => '%s', table_name => '%s', partition_columns => ARRAY['%s'], partition_values => ARRAY['0'])", - HIVE_TEST_SCHEMA, - tableName, - partitionColumn)); - } - - private void testFlushPartitionCache(String tableName, String fullyQualifiedTestTableName, String partitionColumn, String flushCacheProcedureSql) - { - // Create table with partition on regionkey - computeActual(getCreateTableStatement( - fullyQualifiedTestTableName, - format("partitioned_by=ARRAY['%s']", partitionColumn))); - copyTpchNationToTable(fullyQualifiedTestTableName); - - String queryUsingPartitionCacheTemplate = "SELECT name FROM %s WHERE %s=%s"; - String partitionValue1 = "0"; - String queryUsingPartitionCacheForValue1 = format(queryUsingPartitionCacheTemplate, fullyQualifiedTestTableName, partitionColumn, partitionValue1); - String expectedQueryResultForValue1 = "VALUES 'ALGERIA', 'MOROCCO', 'MOZAMBIQUE', 'ETHIOPIA', 'KENYA'"; - String partitionValue2 = "1"; - String queryUsingPartitionCacheForValue2 = format(queryUsingPartitionCacheTemplate, fullyQualifiedTestTableName, partitionColumn, partitionValue2); - String expectedQueryResultForValue2 = "VALUES 'ARGENTINA', 'BRAZIL', 'CANADA', 'PERU', 'UNITED STATES'"; - - // Fill partition cache and check we got expected results - assertQuery(queryUsingPartitionCacheForValue1, expectedQueryResultForValue1); - assertQuery(queryUsingPartitionCacheForValue2, expectedQueryResultForValue2); - - // Copy partition to new location and update metadata outside Trino - renamePartitionResourcesOutsideTrino(tableName, partitionColumn, partitionValue1); - renamePartitionResourcesOutsideTrino(tableName, partitionColumn, partitionValue2); - - // Should return 0 rows as we moved partition and cache is outdated. We use nonexistent partition - assertQueryReturnsEmptyResult(queryUsingPartitionCacheForValue1); - assertQueryReturnsEmptyResult(queryUsingPartitionCacheForValue2); - - // Refresh cache - getQueryRunner().execute(flushCacheProcedureSql); - - // Should return expected rows as we refresh cache - assertQuery(queryUsingPartitionCacheForValue1, expectedQueryResultForValue1); - // Should return 0 rows as we left cache untouched - assertQueryReturnsEmptyResult(queryUsingPartitionCacheForValue2); - - // Refresh cache for schema_name => 'dummy_schema', table_name => 'dummy_table' - getQueryRunner().execute(format( - "CALL system.flush_metadata_cache(schema_name => '%s', table_name => '%s')", - HIVE_TEST_SCHEMA, - tableName)); - - // Should return expected rows for all partitions - assertQuery(queryUsingPartitionCacheForValue1, expectedQueryResultForValue1); - assertQuery(queryUsingPartitionCacheForValue2, expectedQueryResultForValue2); - - computeActual(format("DROP TABLE %s", fullyQualifiedTestTableName)); - } - - @Test - public void testWriteDifferentSizes() - { - String testTable = getFullyQualifiedTestTableName(); - computeActual(format( - "CREATE TABLE %s (" + - " col1 varchar, " + - " col2 varchar, " + - " regionkey bigint) " + - " WITH (partitioned_by=ARRAY['regionkey'])", - testTable)); - - long partSizeInBytes = HIVE_S3_STREAMING_PART_SIZE.toBytes(); - - // Exercise different code paths of Hive S3 streaming upload, with upload part size 5MB: - // 1. fileSize <= 5MB (direct upload) - testWriteWithFileSize(testTable, 50, 0, partSizeInBytes); - - // 2. 5MB < fileSize <= 10MB (upload in two parts) - testWriteWithFileSize(testTable, 100, partSizeInBytes + 1, partSizeInBytes * 2); - - // 3. fileSize > 10MB (upload in three or more parts) - testWriteWithFileSize(testTable, 150, partSizeInBytes * 2 + 1, partSizeInBytes * 3); - - computeActual(format("DROP TABLE %s", testTable)); - } - - @Test - public void testEnumPartitionProjectionOnVarcharColumnWithWhitespace() - { - String tableName = "nation_" + randomNameSuffix(); - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); - - computeActual( - "CREATE TABLE " + fullyQualifiedTestTableName + " (" + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint, " + - " \"short name\" varchar(152) WITH (" + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL1', 'CZ1'] " + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short name'], " + - " partition_projection_enabled=true " + - ")"); - - assertThat( - hiveMinioDataLake.getHiveHadoop() - .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) - .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") - .containsPattern("[ |]+projection\\.short name\\.type[ |]+enum[ |]+") - .containsPattern("[ |]+projection\\.short name\\.values[ |]+PL1,CZ1[ |]+"); - - computeActual(createInsertStatement( - fullyQualifiedTestTableName, - ImmutableList.of( - ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'"), - ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL2'"), - ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'"), - ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ2'")))); - - assertQuery( - format("SELECT * FROM %s", getFullyQualifiedTestTableName("\"" + tableName + "$partitions\"")), - "VALUES 'PL1', 'CZ1'"); - - assertQuery( - format("SELECT name FROM %s WHERE \"short name\"='PL1'", fullyQualifiedTestTableName), - "VALUES 'POLAND_1'"); - - // No results should be returned as Partition Projection will not project partitions for this value - assertQueryReturnsEmptyResult( - format("SELECT name FROM %s WHERE \"short name\"='PL2'", fullyQualifiedTestTableName)); - - assertQuery( - format("SELECT name FROM %s WHERE \"short name\"='PL1' OR \"short name\"='CZ1'", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('CZECH_1')"); - - // Only POLAND_1 row will be returned as other value is outside of projection - assertQuery( - format("SELECT name FROM %s WHERE \"short name\"='PL1' OR \"short name\"='CZ2'", fullyQualifiedTestTableName), - "VALUES ('POLAND_1')"); - - // All values within projection range will be returned - assertQuery( - format("SELECT name FROM %s", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('CZECH_1')"); - } - - @Test - public void testEnumPartitionProjectionOnVarcharColumnWithStorageLocationTemplateCreatedOnTrino() - { - // It's important to mix case here to detect if we properly handle rewriting - // properties between Trino and Hive (e.g for Partition Projection) - String schemaName = "Hive_Datalake_MixedCase"; - String tableName = getRandomTestTableName(); - - // We create new schema to include mixed case location path and create such keys in Object Store - computeActual("CREATE SCHEMA hive.%1$s WITH (location='s3a://%2$s/%1$s')".formatted(schemaName, bucketName)); - - String storageFormat = format( - "s3a://%s/%s/%s/short_name1=${short_name1}/short_name2=${short_name2}/", - this.bucketName, - schemaName, - tableName); - computeActual( - "CREATE TABLE " + getFullyQualifiedTestTableName(schemaName, tableName) + " ( " + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint, " + - " short_name1 varchar(152) WITH (" + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL1', 'CZ1'] " + - " ), " + - " short_name2 varchar(152) WITH (" + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL2', 'CZ2'] " + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1', 'short_name2'], " + - " partition_projection_enabled=true, " + - " partition_projection_location_template='" + storageFormat + "' " + - ")"); - assertThat( - hiveMinioDataLake.getHiveHadoop() - .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(schemaName, tableName))) - .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") - .containsPattern("[ |]+storage\\.location\\.template[ |]+" + quote(storageFormat) + "[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+enum[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.values[ |]+PL2,CZ2[ |]+"); - testEnumPartitionProjectionOnVarcharColumnWithStorageLocationTemplate(schemaName, tableName); - } - - @Test - public void testEnumPartitionProjectionOnVarcharColumnWithStorageLocationTemplateCreatedOnHive() - { - String tableName = getRandomTestTableName(); - String storageFormat = format( - "'s3a://%s/%s/%s/short_name1=${short_name1}/short_name2=${short_name2}/'", - this.bucketName, - HIVE_TEST_SCHEMA, - tableName); - hiveMinioDataLake.getHiveHadoop().runOnHive( - "CREATE TABLE " + getHiveTestTableName(tableName) + " ( " + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint " + - ") PARTITIONED BY (" + - " short_name1 varchar(152), " + - " short_name2 varchar(152)" + - ") " + - "TBLPROPERTIES ( " + - " 'projection.enabled'='true', " + - " 'storage.location.template'=" + storageFormat + ", " + - " 'projection.short_name1.type'='enum', " + - " 'projection.short_name1.values'='PL1,CZ1', " + - " 'projection.short_name2.type'='enum', " + - " 'projection.short_name2.values'='PL2,CZ2' " + - ")"); - testEnumPartitionProjectionOnVarcharColumnWithStorageLocationTemplate(HIVE_TEST_SCHEMA, tableName); - } - - private void testEnumPartitionProjectionOnVarcharColumnWithStorageLocationTemplate(String schemaName, String tableName) - { - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(schemaName, tableName); - computeActual(createInsertStatement( - fullyQualifiedTestTableName, - ImmutableList.of( - ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "'PL2'"), - ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "'CZ2'"), - ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "'PL2'"), - ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "'CZ2'")))); - - assertQuery( - format("SELECT * FROM %s", getFullyQualifiedTestTableName(schemaName, "\"" + tableName + "$partitions\"")), - "VALUES ('PL1','PL2'), ('PL1','CZ2'), ('CZ1','PL2'), ('CZ1','CZ2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2='CZ2'", fullyQualifiedTestTableName), - "VALUES 'POLAND_2'"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); - } - - @Test - public void testEnumPartitionProjectionOnVarcharColumn() - { - String tableName = getRandomTestTableName(); - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); - - computeActual( - "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint, " + - " short_name1 varchar(152) WITH (" + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL1', 'CZ1'] " + - " ), " + - " short_name2 varchar(152) WITH (" + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL2', 'CZ2']" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1', 'short_name2'], " + - " partition_projection_enabled=true " + - ")"); - - assertThat( - hiveMinioDataLake.getHiveHadoop() - .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) - .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+enum[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.values[ |]+PL2,CZ2[ |]+"); - - computeActual(createInsertStatement( - fullyQualifiedTestTableName, - ImmutableList.of( - ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "'PL2'"), - ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "'CZ2'"), - ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "'PL2'"), - ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "'CZ2'")))); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2='CZ2'", fullyQualifiedTestTableName), - "VALUES 'POLAND_2'"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2='CZ2' OR short_name2='PL2' )", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); - } - - @Test - public void testIntegerPartitionProjectionOnVarcharColumnWithDigitsAlignCreatedOnTrino() - { - String tableName = getRandomTestTableName(); - computeActual( - "CREATE TABLE " + getFullyQualifiedTestTableName(tableName) + " ( " + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint, " + - " short_name1 varchar(152) WITH (" + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL1', 'CZ1'] " + - " ), " + - " short_name2 varchar(152) WITH (" + - " partition_projection_type='integer', " + - " partition_projection_range=ARRAY['1', '4'], " + - " partition_projection_digits=3" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1', 'short_name2'], " + - " partition_projection_enabled=true " + - ")"); - assertThat( - hiveMinioDataLake.getHiveHadoop() - .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) - .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+integer[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+1,4[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.digits[ |]+3[ |]+"); - testIntegerPartitionProjectionOnVarcharColumnWithDigitsAlign(tableName); - } - - @Test - public void testIntegerPartitionProjectionOnVarcharColumnWithDigitsAlignCreatedOnHive() - { - String tableName = "nation_" + randomNameSuffix(); - hiveMinioDataLake.getHiveHadoop().runOnHive( - "CREATE TABLE " + getHiveTestTableName(tableName) + " ( " + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint " + - ") " + - "PARTITIONED BY ( " + - " short_name1 varchar(152), " + - " short_name2 varchar(152)" + - ") " + - "TBLPROPERTIES " + - "( " + - " 'projection.enabled'='true', " + - " 'projection.short_name1.type'='enum', " + - " 'projection.short_name1.values'='PL1,CZ1', " + - " 'projection.short_name2.type'='integer', " + - " 'projection.short_name2.range'='1,4', " + - " 'projection.short_name2.digits'='3'" + - ")"); - testIntegerPartitionProjectionOnVarcharColumnWithDigitsAlign(tableName); - } - - private void testIntegerPartitionProjectionOnVarcharColumnWithDigitsAlign(String tableName) - { - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); - computeActual(createInsertStatement( - fullyQualifiedTestTableName, - ImmutableList.of( - ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "'001'"), - ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "'002'"), - ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "'003'"), - ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "'004'")))); - - assertQuery( - format("SELECT * FROM %s", getFullyQualifiedTestTableName("\"" + tableName + "$partitions\"")), - "VALUES ('PL1','001'), ('PL1','002'), ('PL1','003'), ('PL1','004')," + - "('CZ1','001'), ('CZ1','002'), ('CZ1','003'), ('CZ1','004')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2='002'", fullyQualifiedTestTableName), - "VALUES 'POLAND_2'"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2='002' OR short_name2='001' )", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); - } - - @Test - public void testIntegerPartitionProjectionOnIntegerColumnWithInterval() - { - String tableName = getRandomTestTableName(); - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); - - computeActual( - "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint, " + - " short_name1 varchar(152) WITH (" + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL1', 'CZ1'] " + - " ), " + - " short_name2 integer WITH (" + - " partition_projection_type='integer', " + - " partition_projection_range=ARRAY['0', '10'], " + - " partition_projection_interval=3" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1', 'short_name2'], " + - " partition_projection_enabled=true " + - ")"); - - assertThat( - hiveMinioDataLake.getHiveHadoop() - .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) - .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+integer[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+0,10[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.interval[ |]+3[ |]+"); - - computeActual(createInsertStatement( - fullyQualifiedTestTableName, - ImmutableList.of( - ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "0"), - ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "3"), - ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "6"), - ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "9")))); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2=3", fullyQualifiedTestTableName), - "VALUES 'POLAND_2'"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2=3 OR short_name2=0 )", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); - } - - @Test - public void testIntegerPartitionProjectionOnIntegerColumnWithDefaults() - { - String tableName = getRandomTestTableName(); - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); - - computeActual( - "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint, " + - " short_name1 varchar(152) WITH (" + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL1', 'CZ1'] " + - " ), " + - " short_name2 integer WITH (" + - " partition_projection_type='integer', " + - " partition_projection_range=ARRAY['1', '4']" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1', 'short_name2'], " + - " partition_projection_enabled=true " + - ")"); - - assertThat( - hiveMinioDataLake.getHiveHadoop() - .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) - .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+integer[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+1,4[ |]+"); - - computeActual(createInsertStatement( - fullyQualifiedTestTableName, - ImmutableList.of( - ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "1"), - ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "2"), - ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "3"), - ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "4")))); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2=2", fullyQualifiedTestTableName), - "VALUES 'POLAND_2'"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2=2 OR short_name2=1 )", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); - } - - @Test - public void testDatePartitionProjectionOnDateColumnWithDefaults() - { - String tableName = "nation_" + randomNameSuffix(); - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); - - computeActual( - "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint, " + - " short_name1 varchar(152) WITH (" + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL1', 'CZ1'] " + - " ), " + - " short_name2 date WITH (" + - " partition_projection_type='date', " + - " partition_projection_format='yyyy-MM-dd', " + - " partition_projection_range=ARRAY['2001-1-22', '2001-1-25']" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1', 'short_name2'], " + - " partition_projection_enabled=true " + - ")"); - - assertThat( - hiveMinioDataLake.getHiveHadoop() - .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) - .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+date[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.format[ |]+yyyy-MM-dd[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+2001-1-22,2001-1-25[ |]+"); - - computeActual(createInsertStatement( - fullyQualifiedTestTableName, - ImmutableList.of( - ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "DATE '2001-1-22'"), - ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "DATE '2001-1-23'"), - ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "DATE '2001-1-24'"), - ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "DATE '2001-1-25'"), - ImmutableList.of("'CZECH_3'", "'Comment'", "4", "5", "'CZ1'", "DATE '2001-1-26'")))); - - assertQuery( - format("SELECT * FROM %s", getFullyQualifiedTestTableName("\"" + tableName + "$partitions\"")), - "VALUES ('PL1','2001-1-22'), ('PL1','2001-1-23'), ('PL1','2001-1-24'), ('PL1','2001-1-25')," + - "('CZ1','2001-1-22'), ('CZ1','2001-1-23'), ('CZ1','2001-1-24'), ('CZ1','2001-1-25')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2=(DATE '2001-1-23')", fullyQualifiedTestTableName), - "VALUES 'POLAND_2'"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2=(DATE '2001-1-23') OR short_name2=(DATE '2001-1-22') )", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name2 > DATE '2001-1-23'", fullyQualifiedTestTableName), - "VALUES ('CZECH_1'), ('CZECH_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name2 >= DATE '2001-1-23' AND short_name2 <= DATE '2001-1-25'", fullyQualifiedTestTableName), - "VALUES ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); - } - - @Test - public void testDatePartitionProjectionOnTimestampColumnWithInterval() - { - String tableName = getRandomTestTableName(); - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); - - computeActual( - "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint, " + - " short_name1 varchar(152) WITH (" + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL1', 'CZ1'] " + - " ), " + - " short_name2 timestamp WITH (" + - " partition_projection_type='date', " + - " partition_projection_format='yyyy-MM-dd HH:mm:ss', " + - " partition_projection_range=ARRAY['2001-1-22 00:00:00', '2001-1-22 00:00:06'], " + - " partition_projection_interval=2, " + - " partition_projection_interval_unit='SECONDS'" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1', 'short_name2'], " + - " partition_projection_enabled=true " + - ")"); - - assertThat( - hiveMinioDataLake.getHiveHadoop() - .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) - .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+date[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.format[ |]+yyyy-MM-dd HH:mm:ss[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+2001-1-22 00:00:00,2001-1-22 00:00:06[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.interval[ |]+2[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.interval\\.unit[ |]+seconds[ |]+"); - - computeActual(createInsertStatement( - fullyQualifiedTestTableName, - ImmutableList.of( - ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "TIMESTAMP '2001-1-22 00:00:00'"), - ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "TIMESTAMP '2001-1-22 00:00:02'"), - ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "TIMESTAMP '2001-1-22 00:00:04'"), - ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "TIMESTAMP '2001-1-22 00:00:06'"), - ImmutableList.of("'CZECH_3'", "'Comment'", "4", "5", "'CZ1'", "TIMESTAMP '2001-1-22 00:00:08'")))); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2=(TIMESTAMP '2001-1-22 00:00:02')", fullyQualifiedTestTableName), - "VALUES 'POLAND_2'"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2=(TIMESTAMP '2001-1-22 00:00:00') OR short_name2=(TIMESTAMP '2001-1-22 00:00:02') )", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name2 > TIMESTAMP '2001-1-22 00:00:02'", fullyQualifiedTestTableName), - "VALUES ('CZECH_1'), ('CZECH_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name2 >= TIMESTAMP '2001-1-22 00:00:02' AND short_name2 <= TIMESTAMP '2001-1-22 00:00:06'", fullyQualifiedTestTableName), - "VALUES ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); - } - - @Test - public void testDatePartitionProjectionOnTimestampColumnWithIntervalExpressionCreatedOnTrino() - { - String tableName = getRandomTestTableName(); - String dateProjectionFormat = "yyyy-MM-dd HH:mm:ss"; - computeActual( - "CREATE TABLE " + getFullyQualifiedTestTableName(tableName) + " ( " + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint, " + - " short_name1 varchar(152) WITH (" + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL1', 'CZ1'] " + - " ), " + - " short_name2 timestamp WITH (" + - " partition_projection_type='date', " + - " partition_projection_format='" + dateProjectionFormat + "', " + - // We set range to -5 minutes to NOW in order to be sure it will grab all test dates - // which range is -4 minutes till now. Also, we have to consider max no. of partitions 1k - " partition_projection_range=ARRAY['NOW-5MINUTES', 'NOW'], " + - " partition_projection_interval=1, " + - " partition_projection_interval_unit='SECONDS'" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1', 'short_name2'], " + - " partition_projection_enabled=true " + - ")"); - assertThat( - hiveMinioDataLake.getHiveHadoop() - .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) - .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.format[ |]+" + quote(dateProjectionFormat) + "[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+NOW-5MINUTES,NOW[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.interval[ |]+1[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.interval\\.unit[ |]+seconds[ |]+"); - testDatePartitionProjectionOnTimestampColumnWithIntervalExpression(tableName, dateProjectionFormat); - } - - @Test - public void testDatePartitionProjectionOnTimestampColumnWithIntervalExpressionCreatedOnHive() - { - String tableName = getRandomTestTableName(); - String dateProjectionFormat = "yyyy-MM-dd HH:mm:ss"; - hiveMinioDataLake.getHiveHadoop().runOnHive( - "CREATE TABLE " + getHiveTestTableName(tableName) + " ( " + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint " + - ") " + - "PARTITIONED BY (" + - " short_name1 varchar(152), " + - " short_name2 timestamp " + - ") " + - "TBLPROPERTIES ( " + - " 'projection.enabled'='true', " + - " 'projection.short_name1.type'='enum', " + - " 'projection.short_name1.values'='PL1,CZ1', " + - " 'projection.short_name2.type'='date', " + - " 'projection.short_name2.format'='" + dateProjectionFormat + "', " + - // We set range to -5 minutes to NOW in order to be sure it will grab all test dates - // which range is -4 minutes till now. Also, we have to consider max no. of partitions 1k - " 'projection.short_name2.range'='NOW-5MINUTES,NOW', " + - " 'projection.short_name2.interval'='1', " + - " 'projection.short_name2.interval.unit'='SECONDS'" + - ")"); - testDatePartitionProjectionOnTimestampColumnWithIntervalExpression(tableName, dateProjectionFormat); - } - - private void testDatePartitionProjectionOnTimestampColumnWithIntervalExpression(String tableName, String dateProjectionFormat) - { - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); - - Instant dayToday = Instant.now(); - DateFormat dateFormat = new SimpleDateFormat(dateProjectionFormat); - dateFormat.setTimeZone(TimeZone.getTimeZone(ZoneId.of("UTC"))); - String minutesNowFormatted = moveDate(dateFormat, dayToday, MINUTES, 0); - String minutes1AgoFormatter = moveDate(dateFormat, dayToday, MINUTES, -1); - String minutes2AgoFormatted = moveDate(dateFormat, dayToday, MINUTES, -2); - String minutes3AgoFormatted = moveDate(dateFormat, dayToday, MINUTES, -3); - String minutes4AgoFormatted = moveDate(dateFormat, dayToday, MINUTES, -4); - - computeActual(createInsertStatement( - fullyQualifiedTestTableName, - ImmutableList.of( - ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "TIMESTAMP '" + minutesNowFormatted + "'"), - ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "TIMESTAMP '" + minutes1AgoFormatter + "'"), - ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "TIMESTAMP '" + minutes2AgoFormatted + "'"), - ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "TIMESTAMP '" + minutes3AgoFormatted + "'"), - ImmutableList.of("'CZECH_3'", "'Comment'", "4", "5", "'CZ1'", "TIMESTAMP '" + minutes4AgoFormatted + "'")))); - - assertQuery( - format("SELECT name FROM %s WHERE short_name2 > ( TIMESTAMP '%s' ) AND short_name2 <= ( TIMESTAMP '%s' )", fullyQualifiedTestTableName, minutes4AgoFormatted, minutes1AgoFormatter), - "VALUES ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); - } - - @Test - public void testDatePartitionProjectionOnVarcharColumnWithHoursInterval() - { - String tableName = getRandomTestTableName(); - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); - - computeActual( - "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint, " + - " short_name1 varchar(152) WITH (" + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL1', 'CZ1'] " + - " ), " + - " short_name2 varchar WITH (" + - " partition_projection_type='date', " + - " partition_projection_format='yyyy-MM-dd HH', " + - " partition_projection_range=ARRAY['2001-01-22 00', '2001-01-22 06'], " + - " partition_projection_interval=2, " + - " partition_projection_interval_unit='HOURS'" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1', 'short_name2'], " + - " partition_projection_enabled=true " + - ")"); - - assertThat( - hiveMinioDataLake.getHiveHadoop() - .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) - .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+date[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.format[ |]+yyyy-MM-dd HH[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+2001-01-22 00,2001-01-22 06[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.interval[ |]+2[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.interval\\.unit[ |]+hours[ |]+"); - - computeActual(createInsertStatement( - fullyQualifiedTestTableName, - ImmutableList.of( - ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "'2001-01-22 00'"), - ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "'2001-01-22 02'"), - ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "'2001-01-22 04'"), - ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "'2001-01-22 06'"), - ImmutableList.of("'CZECH_3'", "'Comment'", "4", "5", "'CZ1'", "'2001-01-22 08'")))); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2='2001-01-22 02'", fullyQualifiedTestTableName), - "VALUES 'POLAND_2'"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2='2001-01-22 00' OR short_name2='2001-01-22 02' )", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name2 > '2001-01-22 02'", fullyQualifiedTestTableName), - "VALUES ('CZECH_1'), ('CZECH_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name2 >= '2001-01-22 02' AND short_name2 <= '2001-01-22 06'", fullyQualifiedTestTableName), - "VALUES ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); - } - - @Test - public void testDatePartitionProjectionOnVarcharColumnWithDaysInterval() - { - String tableName = getRandomTestTableName(); - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); - - computeActual( - "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint, " + - " short_name1 varchar(152) WITH (" + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL1', 'CZ1'] " + - " ), " + - " short_name2 varchar WITH (" + - " partition_projection_type='date', " + - " partition_projection_format='yyyy-MM-dd', " + - " partition_projection_range=ARRAY['2001-01-01', '2001-01-07'], " + - " partition_projection_interval=2, " + - " partition_projection_interval_unit='DAYS'" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1', 'short_name2'], " + - " partition_projection_enabled=true " + - ")"); - - assertThat( - hiveMinioDataLake.getHiveHadoop() - .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) - .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+date[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.format[ |]+yyyy-MM-dd[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+2001-01-01,2001-01-07[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.interval[ |]+2[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.interval\\.unit[ |]+days[ |]+"); - - computeActual(createInsertStatement( - fullyQualifiedTestTableName, - ImmutableList.of( - ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "'2001-01-01'"), - ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "'2001-01-03'"), - ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "'2001-01-05'"), - ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "'2001-01-07'"), - ImmutableList.of("'CZECH_3'", "'Comment'", "4", "5", "'CZ1'", "'2001-01-09'")))); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2='2001-01-03'", fullyQualifiedTestTableName), - "VALUES 'POLAND_2'"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2='2001-01-01' OR short_name2='2001-01-03' )", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name2 > '2001-01-03'", fullyQualifiedTestTableName), - "VALUES ('CZECH_1'), ('CZECH_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name2 >= '2001-01-03' AND short_name2 <= '2001-01-07'", fullyQualifiedTestTableName), - "VALUES ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); - } - - @Test - public void testDatePartitionProjectionOnVarcharColumnWithIntervalExpression() - { - String tableName = getRandomTestTableName(); - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); - String dateProjectionFormat = "yyyy-MM-dd"; - - computeActual( - "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint, " + - " short_name1 varchar(152) WITH (" + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL1', 'CZ1'] " + - " ), " + - " short_name2 varchar WITH (" + - " partition_projection_type='date', " + - " partition_projection_format='" + dateProjectionFormat + "', " + - " partition_projection_range=ARRAY['NOW-3DAYS', 'NOW']" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1', 'short_name2'], " + - " partition_projection_enabled=true " + - ")"); - - assertThat( - hiveMinioDataLake.getHiveHadoop() - .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) - .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+date[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.format[ |]+" + quote(dateProjectionFormat) + "[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.range[ |]+NOW-3DAYS,NOW[ |]+"); - - Instant dayToday = Instant.now(); - DateFormat dateFormat = new SimpleDateFormat(dateProjectionFormat); - dateFormat.setTimeZone(TimeZone.getTimeZone(ZoneId.of("UTC"))); - String dayTodayFormatted = moveDate(dateFormat, dayToday, DAYS, 0); - String day1AgoFormatter = moveDate(dateFormat, dayToday, DAYS, -1); - String day2AgoFormatted = moveDate(dateFormat, dayToday, DAYS, -2); - String day3AgoFormatted = moveDate(dateFormat, dayToday, DAYS, -3); - String day4AgoFormatted = moveDate(dateFormat, dayToday, DAYS, -4); - - computeActual(createInsertStatement( - fullyQualifiedTestTableName, - ImmutableList.of( - ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "'" + dayTodayFormatted + "'"), - ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "'" + day1AgoFormatter + "'"), - ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "'" + day2AgoFormatted + "'"), - ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "'" + day3AgoFormatted + "'"), - ImmutableList.of("'CZECH_3'", "'Comment'", "4", "5", "'CZ1'", "'" + day4AgoFormatted + "'")))); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2='%s'", fullyQualifiedTestTableName, day1AgoFormatter), - "VALUES 'POLAND_2'"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2='%s' OR short_name2='%s' )", fullyQualifiedTestTableName, dayTodayFormatted, day1AgoFormatter), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name2 > '%s'", fullyQualifiedTestTableName, day2AgoFormatted), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name2 >= '%s' AND short_name2 <= '%s'", fullyQualifiedTestTableName, day4AgoFormatted, day1AgoFormatter), - "VALUES ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2')"); - - assertQuery( - format("SELECT name FROM %s", fullyQualifiedTestTableName), - "VALUES ('POLAND_1'), ('POLAND_2'), ('CZECH_1'), ('CZECH_2')"); - } - - private String moveDate(DateFormat format, Instant today, TemporalUnit unit, int move) - { - return format.format(new Date(today.plus(move, unit).toEpochMilli())); - } - - @Test - public void testDatePartitionProjectionFormatTextWillNotCauseIntervalRequirement() - { - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(); - - computeActual( - "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint, " + - " short_name1 varchar WITH (" + - " partition_projection_type='date', " + - " partition_projection_format='''start''yyyy-MM-dd''end''''s''', " + - " partition_projection_range=ARRAY['start2001-01-01end''s', 'start2001-01-07end''s'] " + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1'], " + - " partition_projection_enabled=true " + - ")"); - } - - @Test - public void testInjectedPartitionProjectionOnVarcharColumn() - { - String tableName = getRandomTestTableName(); - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); - - computeActual( - "CREATE TABLE " + fullyQualifiedTestTableName + " ( " + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint, " + - " short_name1 varchar(152) WITH (" + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL1', 'CZ1'] " + - " ), " + - " short_name2 varchar(152) WITH (" + - " partition_projection_type='injected'" + - " ) " + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1', 'short_name2'], " + - " partition_projection_enabled=true " + - ")"); - - assertThat( - hiveMinioDataLake.getHiveHadoop() - .runOnHive("SHOW TBLPROPERTIES " + getHiveTestTableName(tableName))) - .containsPattern("[ |]+projection\\.enabled[ |]+true[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.type[ |]+enum[ |]+") - .containsPattern("[ |]+projection\\.short_name1\\.values[ |]+PL1,CZ1[ |]+") - .containsPattern("[ |]+projection\\.short_name2\\.type[ |]+injected[ |]+"); - - computeActual(createInsertStatement( - fullyQualifiedTestTableName, - ImmutableList.of( - ImmutableList.of("'POLAND_1'", "'Comment'", "0", "5", "'PL1'", "'001'"), - ImmutableList.of("'POLAND_2'", "'Comment'", "1", "5", "'PL1'", "'002'"), - ImmutableList.of("'CZECH_1'", "'Comment'", "2", "5", "'CZ1'", "'003'"), - ImmutableList.of("'CZECH_2'", "'Comment'", "3", "5", "'CZ1'", "'004'")))); - - assertQuery( - format("SELECT name FROM %s WHERE short_name1='PL1' AND short_name2='002'", fullyQualifiedTestTableName), - "VALUES 'POLAND_2'"); - - assertThatThrownBy( - () -> getQueryRunner().execute( - format("SELECT name FROM %s WHERE short_name1='PL1' AND ( short_name2='002' OR short_name2='001' )", fullyQualifiedTestTableName))) - .hasMessage("Column projection for column 'short_name2' failed. Injected projection requires single predicate for it's column in where clause. Currently provided can't be converted to single partition."); - - assertThatThrownBy( - () -> getQueryRunner().execute( - format("SELECT name FROM %s", fullyQualifiedTestTableName))) - .hasMessage("Column projection for column 'short_name2' failed. Injected projection requires single predicate for it's column in where clause"); - - assertThatThrownBy( - () -> getQueryRunner().execute( - format("SELECT name FROM %s WHERE short_name1='PL1'", fullyQualifiedTestTableName))) - .hasMessage("Column projection for column 'short_name2' failed. Injected projection requires single predicate for it's column in where clause"); - } - - @Test - public void testPartitionProjectionInvalidTableProperties() - { - assertThatThrownBy(() -> getQueryRunner().execute( - "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + - " name varchar " + - ") WITH ( " + - " partition_projection_enabled=true " + - ")")) - .hasMessage("Partition projection cannot be enabled on a table that is not partitioned"); - - assertThatThrownBy(() -> getQueryRunner().execute( - "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + - " name varchar WITH ( " + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL1', 'CZ1']" + - " ), " + - " short_name1 varchar " + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1'], " + - " partition_projection_enabled=true " + - ")")) - .hasMessage("Partition projection cannot be defined for non-partition column: 'name'"); - - assertThatThrownBy(() -> getQueryRunner().execute( - "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + - " name varchar, " + - " short_name1 varchar WITH ( " + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL1', 'CZ1']" + - " ), " + - " short_name2 varchar " + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1', 'short_name2'], " + - " partition_projection_enabled=true " + - ")")) - .hasMessage("Column projection for column 'short_name2' failed. Projection type property missing"); - - assertThatThrownBy(() -> getQueryRunner().execute( - "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + - " name varchar, " + - " short_name1 varchar WITH (" + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL1', 'CZ1'] " + - " ), " + - " short_name2 varchar WITH (" + - " partition_projection_type='injected' " + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1', 'short_name2'], " + - " partition_projection_enabled=true, " + - " partition_projection_location_template='s3a://dummy/short_name1=${short_name1}/'" + - ")")) - .hasMessage("Partition projection location template: s3a://dummy/short_name1=${short_name1}/ " + - "is missing partition column: 'short_name2' placeholder"); - - assertThatThrownBy(() -> getQueryRunner().execute( - "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + - " name varchar, " + - " short_name1 varchar WITH (" + - " partition_projection_type='integer', " + - " partition_projection_range=ARRAY['1', '2', '3']" + - " ), " + - " short_name2 varchar WITH (" + - " partition_projection_type='enum', " + - " partition_projection_values=ARRAY['PL1', 'CZ1'] " + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1', 'short_name2'], " + - " partition_projection_enabled=true " + - ")")) - .hasMessage("Column projection for column 'short_name1' failed. Property: 'partition_projection_range' needs to be list of 2 integers"); - - assertThatThrownBy(() -> getQueryRunner().execute( - "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + - " name varchar, " + - " short_name1 varchar WITH (" + - " partition_projection_type='date', " + - " partition_projection_values=ARRAY['2001-01-01', '2001-01-02']" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1'], " + - " partition_projection_enabled=true " + - ")")) - .hasMessage("Column projection for column 'short_name1' failed. Missing required property: 'partition_projection_format'"); - - assertThatThrownBy(() -> getQueryRunner().execute( - "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + - " name varchar, " + - " short_name1 varchar WITH (" + - " partition_projection_type='date', " + - " partition_projection_format='yyyy-MM-dd HH', " + - " partition_projection_range=ARRAY['2001-01-01', '2001-01-02']" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1'], " + - " partition_projection_enabled=true " + - ")")) - .hasMessage("Column projection for column 'short_name1' failed. Property: 'partition_projection_range' needs to be a list of 2 valid dates formatted as 'yyyy-MM-dd HH' " + - "or '^\\s*NOW\\s*(([+-])\\s*([0-9]+)\\s*(DAY|HOUR|MINUTE|SECOND)S?\\s*)?$' that are sequential: Unparseable date: \"2001-01-01\""); - - assertThatThrownBy(() -> getQueryRunner().execute( - "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + - " name varchar, " + - " short_name1 varchar WITH (" + - " partition_projection_type='date', " + - " partition_projection_format='yyyy-MM-dd', " + - " partition_projection_range=ARRAY['NOW*3DAYS', '2001-01-02']" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1'], " + - " partition_projection_enabled=true " + - ")")) - .hasMessage("Column projection for column 'short_name1' failed. Property: 'partition_projection_range' needs to be a list of 2 valid dates formatted as 'yyyy-MM-dd' " + - "or '^\\s*NOW\\s*(([+-])\\s*([0-9]+)\\s*(DAY|HOUR|MINUTE|SECOND)S?\\s*)?$' that are sequential: Unparseable date: \"NOW*3DAYS\""); - - assertThatThrownBy(() -> getQueryRunner().execute( - "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + - " name varchar, " + - " short_name1 varchar WITH (" + - " partition_projection_type='date', " + - " partition_projection_format='yyyy-MM-dd', " + - " partition_projection_range=ARRAY['2001-01-02', '2001-01-01']" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1'], " + - " partition_projection_enabled=true " + - ")")) - .hasMessage("Column projection for column 'short_name1' failed. Property: 'partition_projection_range' needs to be a list of 2 valid dates formatted as 'yyyy-MM-dd' " + - "or '^\\s*NOW\\s*(([+-])\\s*([0-9]+)\\s*(DAY|HOUR|MINUTE|SECOND)S?\\s*)?$' that are sequential"); - - assertThatThrownBy(() -> getQueryRunner().execute( - "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + - " name varchar, " + - " short_name1 varchar WITH (" + - " partition_projection_type='date', " + - " partition_projection_format='yyyy-MM-dd', " + - " partition_projection_range=ARRAY['2001-01-01', '2001-01-02'], " + - " partition_projection_interval_unit='Decades'" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1'], " + - " partition_projection_enabled=true " + - ")")) - .hasMessage("Column projection for column 'short_name1' failed. Property: 'partition_projection_interval_unit' value 'Decades' is invalid. " + - "Available options: [Days, Hours, Minutes, Seconds]"); - - assertThatThrownBy(() -> getQueryRunner().execute( - "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + - " name varchar, " + - " short_name1 varchar WITH (" + - " partition_projection_type='date', " + - " partition_projection_format='yyyy-MM-dd HH', " + - " partition_projection_range=ARRAY['2001-01-01 10', '2001-01-02 10']" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1'], " + - " partition_projection_enabled=true " + - ")")) - .hasMessage("Column projection for column 'short_name1' failed. Property: 'partition_projection_interval_unit' " + - "needs to be set when provided 'partition_projection_format' is less that single-day precision. " + - "Interval defaults to 1 day or 1 month, respectively. Otherwise, interval is required"); - - assertThatThrownBy(() -> getQueryRunner().execute( - "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + - " name varchar, " + - " short_name1 varchar WITH (" + - " partition_projection_type='date', " + - " partition_projection_format='yyyy-MM-dd', " + - " partition_projection_range=ARRAY['2001-01-01 10', '2001-01-02 10']" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1'] " + - ")")) - .hasMessage("Columns partition projection properties cannot be set when 'partition_projection_enabled' is not set"); - - // Verify that ignored flag is only interpreted for pre-existing tables where configuration is loaded from metastore. - // It should not allow creating corrupted config via Trino. It's a kill switch to run away when we have compatibility issues. - assertThatThrownBy(() -> getQueryRunner().execute( - "CREATE TABLE " + getFullyQualifiedTestTableName("nation_" + randomNameSuffix()) + " ( " + - " name varchar, " + - " short_name1 varchar WITH (" + - " partition_projection_type='date', " + - " partition_projection_format='yyyy-MM-dd HH', " + - " partition_projection_range=ARRAY['2001-01-01 10', '2001-01-02 10']" + - " )" + - ") WITH ( " + - " partitioned_by=ARRAY['short_name1'], " + - " partition_projection_enabled=true, " + - " partition_projection_ignore=true " + // <-- Even if this is set we disallow creating corrupted configuration via Trino - ")")) - .hasMessage("Column projection for column 'short_name1' failed. Property: 'partition_projection_interval_unit' " + - "needs to be set when provided 'partition_projection_format' is less that single-day precision. " + - "Interval defaults to 1 day or 1 month, respectively. Otherwise, interval is required"); - } - - @Test - public void testPartitionProjectionIgnore() - { - String tableName = "nation_" + randomNameSuffix(); - String hiveTestTableName = getHiveTestTableName(tableName); - String fullyQualifiedTestTableName = getFullyQualifiedTestTableName(tableName); - - // Create corrupted configuration - hiveMinioDataLake.getHiveHadoop().runOnHive( - "CREATE TABLE " + hiveTestTableName + " ( " + - " name varchar(25) " + - ") PARTITIONED BY (" + - " date_time varchar(152) " + - ") " + - "TBLPROPERTIES ( " + - " 'projection.enabled'='true', " + - " 'projection.date_time.type'='date', " + - " 'projection.date_time.format'='yyyy-MM-dd HH', " + - " 'projection.date_time.range'='2001-01-01,2001-01-02' " + - ")"); - - // Expect invalid Partition Projection properties to fail - assertThatThrownBy(() -> getQueryRunner().execute("SELECT * FROM " + fullyQualifiedTestTableName)) - .hasMessage("Column projection for column 'date_time' failed. Property: 'partition_projection_range' needs to be a list of 2 valid dates formatted as 'yyyy-MM-dd HH' " + - "or '^\\s*NOW\\s*(([+-])\\s*([0-9]+)\\s*(DAY|HOUR|MINUTE|SECOND)S?\\s*)?$' that are sequential: Unparseable date: \"2001-01-01\""); - - // Append kill switch table property to ignore Partition Projection properties - hiveMinioDataLake.getHiveHadoop().runOnHive( - "ALTER TABLE " + hiveTestTableName + " SET TBLPROPERTIES ( 'trino.partition_projection.ignore'='TRUE' )"); - // Flush cache to get new definition - computeActual("CALL system.flush_metadata_cache(schema_name => '" + HIVE_TEST_SCHEMA + "', table_name => '" + tableName + "')"); - - // Verify query execution works - computeActual(createInsertStatement( - fullyQualifiedTestTableName, - ImmutableList.of( - ImmutableList.of("'POLAND_1'", "'2022-02-01 12'"), - ImmutableList.of("'POLAND_2'", "'2022-02-01 12'"), - ImmutableList.of("'CZECH_1'", "'2022-02-01 13'"), - ImmutableList.of("'CZECH_2'", "'2022-02-01 13'")))); - - assertQuery("SELECT * FROM " + fullyQualifiedTestTableName, - "VALUES ('POLAND_1', '2022-02-01 12'), " + - "('POLAND_2', '2022-02-01 12'), " + - "('CZECH_1', '2022-02-01 13'), " + - "('CZECH_2', '2022-02-01 13')"); - assertQuery("SELECT * FROM " + fullyQualifiedTestTableName + " WHERE date_time = '2022-02-01 12'", - "VALUES ('POLAND_1', '2022-02-01 12'), ('POLAND_2', '2022-02-01 12')"); - } - - @Test - public void testAnalyzePartitionedTableWithCanonicalization() - { - String tableName = "test_analyze_table_canonicalization_" + randomNameSuffix(); - assertUpdate("CREATE TABLE %s (a_varchar varchar, month varchar) WITH (partitioned_by = ARRAY['month'])".formatted(getFullyQualifiedTestTableName(tableName))); - - assertUpdate("INSERT INTO " + getFullyQualifiedTestTableName(tableName) + " VALUES ('A', '01'), ('B', '01'), ('C', '02'), ('D', '03')", 4); - - String tableLocation = (String) computeActual("SELECT DISTINCT regexp_replace(\"$path\", '/[^/]*/[^/]*$', '') FROM " + getFullyQualifiedTestTableName(tableName)).getOnlyValue(); - - String externalTableName = "external_" + tableName; - List partitionColumnNames = List.of("month"); - assertUpdate( - """ - CREATE TABLE %s( - a_varchar varchar, - month integer) - WITH ( - partitioned_by = ARRAY['month'], - external_location='%s') - """.formatted(getFullyQualifiedTestTableName(externalTableName), tableLocation)); - - addPartitions(tableName, externalTableName, partitionColumnNames, TupleDomain.all()); - assertQuery("SELECT * FROM " + HIVE_TEST_SCHEMA + ".\"" + externalTableName + "$partitions\"", "VALUES 1, 2, 3"); - assertUpdate("ANALYZE " + getFullyQualifiedTestTableName(externalTableName), 4); - assertQuery("SHOW STATS FOR " + getFullyQualifiedTestTableName(externalTableName), - """ - VALUES - ('a_varchar', 4.0, 2.0, 0.0, null, null, null), - ('month', null, 3.0, 0.0, null, 1, 3), - (null, null, null, null, 4.0, null, null) - """); - - assertUpdate("INSERT INTO " + getFullyQualifiedTestTableName(tableName) + " VALUES ('E', '04')", 1); - addPartitions( - tableName, - externalTableName, - partitionColumnNames, - TupleDomain.fromFixedValues(Map.of("month", new NullableValue(VARCHAR, utf8Slice("04"))))); - assertUpdate("CALL system.flush_metadata_cache(schema_name => '" + HIVE_TEST_SCHEMA + "', table_name => '" + externalTableName + "')"); - assertQuery("SELECT * FROM " + HIVE_TEST_SCHEMA + ".\"" + externalTableName + "$partitions\"", "VALUES 1, 2, 3, 4"); - assertUpdate("ANALYZE " + getFullyQualifiedTestTableName(externalTableName) + " WITH (partitions = ARRAY[ARRAY['04']])", 1); - assertQuery("SHOW STATS FOR " + getFullyQualifiedTestTableName(externalTableName), - """ - VALUES - ('a_varchar', 5.0, 2.0, 0.0, null, null, null), - ('month', null, 4.0, 0.0, null, 1, 4), - (null, null, null, null, 5.0, null, null) - """); - // TODO (https://github.com/trinodb/trino/issues/15998) fix selective ANALYZE for table with non-canonical partition values - assertQueryFails("ANALYZE " + getFullyQualifiedTestTableName(externalTableName) + " WITH (partitions = ARRAY[ARRAY['4']])", ".*Partition.*not found.*"); - - assertUpdate("DROP TABLE " + getFullyQualifiedTestTableName(externalTableName)); - assertUpdate("DROP TABLE " + getFullyQualifiedTestTableName(tableName)); - } - - @Test - public void testExternalLocationWithTrailingSpace() - { - String tableName = "test_external_location_with_trailing_space_" + randomNameSuffix(); - String tableLocationDirWithTrailingSpace = tableName + " "; - String tableLocation = format("s3a://%s/%s/%s", bucketName, HIVE_TEST_SCHEMA, tableLocationDirWithTrailingSpace); - - byte[] contents = "hello\u0001world\nbye\u0001world".getBytes(UTF_8); - String targetPath = format("%s/%s/test.txt", HIVE_TEST_SCHEMA, tableLocationDirWithTrailingSpace); - hiveMinioDataLake.getMinioClient().putObject(bucketName, contents, targetPath); - - assertUpdate(format( - "CREATE TABLE %s (" + - " a varchar, " + - " b varchar) " + - "WITH (format='TEXTFILE', external_location='%s')", - tableName, - tableLocation)); - - assertQuery("SELECT a, b FROM " + tableName, "VALUES ('hello', 'world'), ('bye', 'world')"); - - String actualTableLocation = getTableLocation(tableName); - assertThat(actualTableLocation).isEqualTo(tableLocation); - - assertUpdate("DROP TABLE " + tableName); - } - - @Test - public void testCreateSchemaInvalidName() - { - assertThatThrownBy(() -> assertUpdate("CREATE SCHEMA \".\"")) - .hasMessage("Invalid object name: '.'"); - - assertThatThrownBy(() -> assertUpdate("CREATE SCHEMA \"..\"")) - .hasMessage("Invalid object name: '..'"); - - assertThatThrownBy(() -> assertUpdate("CREATE SCHEMA \"foo/bar\"")) - .hasMessage("Invalid object name: 'foo/bar'"); - } - - @Test - public void testCreateTableInvalidName() - { - assertThatThrownBy(() -> assertUpdate("CREATE TABLE " + HIVE_TEST_SCHEMA + ".\".\" (col integer)")) - .hasMessageContaining("Invalid table name"); - assertThatThrownBy(() -> assertUpdate("CREATE TABLE " + HIVE_TEST_SCHEMA + ".\"..\" (col integer)")) - .hasMessageContaining("Invalid table name"); - assertThatThrownBy(() -> assertUpdate("CREATE TABLE " + HIVE_TEST_SCHEMA + ".\"...\" (col integer)")) - .hasMessage("Invalid table name"); - - for (String tableName : Arrays.asList("foo/bar", "foo/./bar", "foo/../bar")) { - assertThatThrownBy(() -> assertUpdate("CREATE TABLE " + HIVE_TEST_SCHEMA + ".\"" + tableName + "\" (col integer)")) - .hasMessage(format("Invalid object name: '%s'", tableName)); - assertThatThrownBy(() -> assertUpdate("CREATE TABLE " + HIVE_TEST_SCHEMA + ".\"" + tableName + "\" (col) AS VALUES 1")) - .hasMessage(format("Invalid object name: '%s'", tableName)); - } - } - - @Test - public void testRenameSchemaToInvalidObjectName() - { - String schemaName = "test_rename_schema_invalid_name_" + randomNameSuffix(); - assertUpdate("CREATE SCHEMA %1$s WITH (location='s3a://%2$s/%1$s')".formatted(schemaName, bucketName)); - - for (String invalidSchemaName : Arrays.asList(".", "..", "foo/bar")) { - assertThatThrownBy(() -> assertUpdate("ALTER SCHEMA hive." + schemaName + " RENAME TO \"" + invalidSchemaName + "\"")) - .hasMessage(format("Invalid object name: '%s'", invalidSchemaName)); - } - - assertUpdate("DROP SCHEMA " + schemaName); - } - - @Test - public void testRenameTableToInvalidObjectName() - { - String tableName = "test_rename_table_invalid_name_" + randomNameSuffix(); - assertUpdate("CREATE TABLE %s (a_varchar varchar)".formatted(getFullyQualifiedTestTableName(tableName))); - - for (String invalidTableName : Arrays.asList(".", "..", "foo/bar")) { - assertThatThrownBy(() -> assertUpdate("ALTER TABLE " + getFullyQualifiedTestTableName(tableName) + " RENAME TO \"" + invalidTableName + "\"")) - .hasMessage(format("Invalid object name: '%s'", invalidTableName)); - } - - for (String invalidSchemaName : Arrays.asList(".", "..", "foo/bar")) { - assertThatThrownBy(() -> assertUpdate("ALTER TABLE " + getFullyQualifiedTestTableName(tableName) + " RENAME TO \"" + invalidSchemaName + "\".validTableName")) - .hasMessage(format("Invalid object name: '%s'", invalidSchemaName)); - } - - assertUpdate("DROP TABLE " + getFullyQualifiedTestTableName(tableName)); - } - - @Test - public void testUnpartitionedTableExternalLocationWithTrainingSlash() - { - String tableName = "test_external_location_trailing_slash_" + randomNameSuffix(); - String tableLocationWithTrailingSlash = format("s3://%s/%s/%s/", bucketName, HIVE_TEST_SCHEMA, tableName); - byte[] contents = "Trino\nSQL\non\neverything".getBytes(UTF_8); - String dataFilePath = format("%s/%s/data.txt", HIVE_TEST_SCHEMA, tableName); - hiveMinioDataLake.getMinioClient().putObject(bucketName, contents, dataFilePath); - - assertUpdate(format( - "CREATE TABLE %s (" + - " a_varchar varchar) " + - "WITH (" + - " external_location='%s'," + - " format='TEXTFILE')", - tableName, - tableLocationWithTrailingSlash)); - assertQuery("SELECT * FROM " + tableName, "VALUES 'Trino', 'SQL', 'on', 'everything'"); - - assertUpdate("DROP TABLE " + tableName); - } - - @Test - public void testUnpartitionedTableExternalLocationOnTopOfTheBucket() - { - String topBucketName = "test-hive-unpartitioned-top-of-the-bucket-" + randomNameSuffix(); - hiveMinioDataLake.getMinio().createBucket(topBucketName); - String tableName = "test_external_location_top_of_the_bucket_" + randomNameSuffix(); - - byte[] contents = "Trino\nSQL\non\neverything".getBytes(UTF_8); - hiveMinioDataLake.getMinioClient().putObject(topBucketName, contents, "data.txt"); - - assertUpdate(format( - "CREATE TABLE %s (" + - " a_varchar varchar) " + - "WITH (" + - " external_location='%s'," + - " format='TEXTFILE')", - tableName, - format("s3://%s/", topBucketName))); - assertQuery("SELECT * FROM " + tableName, "VALUES 'Trino', 'SQL', 'on', 'everything'"); - - assertUpdate("DROP TABLE " + tableName); - } - - @Test - public void testPartitionedTableExternalLocationOnTopOfTheBucket() - { - String topBucketName = "test-hive-partitioned-top-of-the-bucket-" + randomNameSuffix(); - hiveMinioDataLake.getMinio().createBucket(topBucketName); - String tableName = "test_external_location_top_of_the_bucket_" + randomNameSuffix(); - - assertUpdate(format( - "CREATE TABLE %s (" + - " a_varchar varchar, " + - " pkey integer) " + - "WITH (" + - " external_location='%s'," + - " partitioned_by=ARRAY['pkey'])", - tableName, - format("s3://%s/", topBucketName))); - assertUpdate("INSERT INTO " + tableName + " VALUES ('a', 1) , ('b', 1), ('c', 2), ('d', 2)", 4); - assertQuery("SELECT * FROM " + tableName, "VALUES ('a', 1), ('b',1), ('c', 2), ('d', 2)"); - assertUpdate("DELETE FROM " + tableName + " where pkey = 2"); - assertQuery("SELECT * FROM " + tableName, "VALUES ('a', 1), ('b',1)"); - - assertUpdate("DROP TABLE " + tableName); - } - - @Test - public void testDropStatsPartitionedTable() - { - String tableName = "test_hive_drop_stats_partitioned_table_" + randomNameSuffix(); - assertUpdate(("CREATE TABLE %s (" + - " data integer," + - " p_varchar varchar," + - " p_integer integer" + - ") " + - "WITH (" + - " partitioned_by=ARRAY['p_varchar', 'p_integer']" + - ")").formatted(getFullyQualifiedTestTableName(tableName))); - - // Drop stats for partition which does not exist - assertThat(query(format("CALL system.drop_stats('%s', '%s', ARRAY[ARRAY['partnotfound', '999']])", HIVE_TEST_SCHEMA, tableName))) - .failure().hasMessage("No partition found for name: p_varchar=partnotfound/p_integer=999"); - - assertUpdate("INSERT INTO " + getFullyQualifiedTestTableName(tableName) + " VALUES (1, 'part1', 10) , (2, 'part2', 10), (12, 'part2', 20)", 3); - - // Run analyze on the entire table - assertUpdate("ANALYZE " + getFullyQualifiedTestTableName(tableName), 3); - - assertQuery("SHOW STATS FOR " + getFullyQualifiedTestTableName(tableName), - """ - VALUES - ('data', null, 1.0, 0.0, null, 1, 12), - ('p_varchar', 15.0, 2.0, 0.0, null, null, null), - ('p_integer', null, 2.0, 0.0, null, 10, 20), - (null, null, null, null, 3.0, null, null) - """); - - assertUpdate(format("CALL system.drop_stats('%s', '%s', ARRAY[ARRAY['part1', '10']])", HIVE_TEST_SCHEMA, tableName)); - - assertQuery("SHOW STATS FOR " + getFullyQualifiedTestTableName(tableName), - """ - VALUES - ('data', null, 1.0, 0.0, null, 2, 12), - ('p_varchar', 15.0, 2.0, 0.0, null, null, null), - ('p_integer', null, 2.0, 0.0, null, 10, 20), - (null, null, null, null, 3.0, null, null) - """); - - assertUpdate("DELETE FROM " + getFullyQualifiedTestTableName(tableName) + " WHERE p_varchar ='part1' and p_integer = 10"); - - // Drop stats for partition which does not exist - assertThat(query(format("CALL system.drop_stats('%s', '%s', ARRAY[ARRAY['part1', '10']])", HIVE_TEST_SCHEMA, tableName))) - .failure().hasMessage("No partition found for name: p_varchar=part1/p_integer=10"); - - assertQuery("SHOW STATS FOR " + getFullyQualifiedTestTableName(tableName), - """ - VALUES - ('data', null, 1.0, 0.0, null, 2, 12), - ('p_varchar', 10.0, 1.0, 0.0, null, null, null), - ('p_integer', null, 2.0, 0.0, null, 10, 20), - (null, null, null, null, 2.0, null, null) - """); - assertUpdate("DROP TABLE " + getFullyQualifiedTestTableName(tableName)); - } - - @Test - public void testUnsupportedDropSchemaCascadeWithNonHiveTable() - { - String schemaName = "test_unsupported_drop_schema_cascade_" + randomNameSuffix(); - String icebergTableName = "test_dummy_iceberg_table" + randomNameSuffix(); - - hiveMinioDataLake.getHiveHadoop().runOnHive("CREATE DATABASE %2$s LOCATION 's3a://%1$s/%2$s'".formatted(bucketName, schemaName)); - try { - hiveMinioDataLake.getHiveHadoop().runOnHive("CREATE TABLE " + schemaName + "." + icebergTableName + " TBLPROPERTIES ('table_type'='iceberg') AS SELECT 1 a"); - - assertQueryFails("DROP SCHEMA " + schemaName + " CASCADE", "\\QCannot query Iceberg table '%s.%s'".formatted(schemaName, icebergTableName)); - - assertThat(computeActual("SHOW SCHEMAS").getOnlyColumnAsSet()).contains(schemaName); - assertThat(computeActual("SHOW TABLES FROM " + schemaName).getOnlyColumnAsSet()).contains(icebergTableName); - assertThat(hiveMinioDataLake.getMinioClient().listObjects(bucketName, schemaName).stream()).isNotEmpty(); - } - finally { - hiveMinioDataLake.getHiveHadoop().runOnHive("DROP DATABASE IF EXISTS " + schemaName + " CASCADE"); - } - } - - @Test - public void testUnsupportedCommentOnHiveView() - { - String viewName = HIVE_TEST_SCHEMA + ".test_unsupported_comment_on_hive_view_" + randomNameSuffix(); - - hiveMinioDataLake.getHiveHadoop().runOnHive("CREATE VIEW " + viewName + " AS SELECT 1 x"); - try { - assertQueryFails("COMMENT ON COLUMN " + viewName + ".x IS NULL", "Hive views are not supported.*"); - } - finally { - hiveMinioDataLake.getHiveHadoop().runOnHive("DROP VIEW " + viewName); - } - } - - @Test - public void testCreateFunction() - { - String name = "test_" + randomNameSuffix(); - String name2 = "test_" + randomNameSuffix(); - - assertUpdate("CREATE FUNCTION " + name + "(x integer) RETURNS bigint RETURN x * 10"); - assertQuery("SELECT " + name + "(99)", "SELECT 990"); - - assertUpdate("CREATE OR REPLACE FUNCTION " + name + "(x integer) RETURNS bigint COMMENT 't42' RETURN x * 42"); - assertQuery("SELECT " + name + "(99)", "SELECT 4158"); - - assertQueryFails("SELECT " + name + "(2.9)", ".*Unexpected parameters.*"); - - assertUpdate("CREATE FUNCTION " + name + "(x double) RETURNS double COMMENT 't88' RETURN x * 8.8"); - - assertThat(query("SHOW FUNCTIONS")) - .result() - .skippingTypesCheck() - .containsAll(resultBuilder(getSession()) - .row(name, "bigint", "integer", "scalar", true, "t42") - .row(name, "double", "double", "scalar", true, "t88") - .build()); - - assertQuery("SELECT " + name + "(99)", "SELECT 4158"); - assertQuery("SELECT " + name + "(2.9)", "SELECT 25.52"); - - assertQueryFails("CREATE FUNCTION " + name + "(x int) RETURNS bigint RETURN x", "line 1:1: Function already exists"); - - assertQuery("SELECT " + name + "(99)", "SELECT 4158"); - assertQuery("SELECT " + name + "(2.9)", "SELECT 25.52"); - - assertUpdate("CREATE OR REPLACE FUNCTION " + name + "(x bigint) RETURNS bigint RETURN x * 23"); - assertUpdate("CREATE FUNCTION " + name2 + "(s varchar) RETURNS varchar RETURN 'Hello ' || s"); - - assertThat(query("SHOW FUNCTIONS")) - .result() - .skippingTypesCheck() - .containsAll(resultBuilder(getSession()) - .row(name, "bigint", "integer", "scalar", true, "t42") - .row(name, "bigint", "bigint", "scalar", true, "") - .row(name, "double", "double", "scalar", true, "t88") - .row(name2, "varchar", "varchar", "scalar", true, "") - .build()); - - assertQuery("SELECT " + name + "(99)", "SELECT 4158"); - assertQuery("SELECT " + name + "(cast(99 as bigint))", "SELECT 2277"); - assertQuery("SELECT " + name + "(2.9)", "SELECT 25.52"); - assertQuery("SELECT " + name2 + "('world')", "SELECT 'Hello world'"); - - assertQueryFails("DROP FUNCTION " + name + "(varchar)", "line 1:1: Function not found"); - assertUpdate("DROP FUNCTION " + name + "(z bigint)"); - assertUpdate("DROP FUNCTION " + name + "(double)"); - assertUpdate("DROP FUNCTION " + name + "(int)"); - assertQueryFails("DROP FUNCTION " + name + "(bigint)", "line 1:1: Function not found"); - assertUpdate("DROP FUNCTION IF EXISTS " + name + "(bigint)"); - assertUpdate("DROP FUNCTION " + name2 + "(varchar)"); - assertQueryFails("DROP FUNCTION " + name2 + "(varchar)", "line 1:1: Function not found"); - } - - private void renamePartitionResourcesOutsideTrino(String tableName, String partitionColumn, String regionKey) - { - String partitionName = format("%s=%s", partitionColumn, regionKey); - String partitionS3KeyPrefix = format("%s/%s/%s", HIVE_TEST_SCHEMA, tableName, partitionName); - String renamedPartitionSuffix = "CP"; - - // Copy whole partition to new location - MinioClient minioClient = hiveMinioDataLake.getMinioClient(); - minioClient.listObjects(bucketName, "") - .forEach(objectKey -> { - if (objectKey.startsWith(partitionS3KeyPrefix)) { - String fileName = objectKey.substring(objectKey.lastIndexOf('/')); - String destinationKey = partitionS3KeyPrefix + renamedPartitionSuffix + fileName; - minioClient.copyObject(bucketName, objectKey, bucketName, destinationKey); - } - }); - - // Delete old partition and update metadata to point to location of new copy - Table hiveTable = metastoreClient.getTable(HIVE_TEST_SCHEMA, tableName).orElseThrow(); - Partition partition = metastoreClient.getPartition(hiveTable, List.of(regionKey)).orElseThrow(); - Map> partitionStatistics = metastoreClient.getPartitionColumnStatistics( - HIVE_TEST_SCHEMA, - tableName, - ImmutableSet.of(partitionName), - partition.getColumns().stream().map(Column::getName).collect(toSet())); - - metastoreClient.dropPartition(HIVE_TEST_SCHEMA, tableName, List.of(regionKey), true); - metastoreClient.addPartitions(HIVE_TEST_SCHEMA, tableName, List.of( - new PartitionWithStatistics( - Partition.builder(partition) - .withStorage(builder -> builder.setLocation( - partition.getStorage().getLocation() + renamedPartitionSuffix)) - .build(), - partitionName, - new PartitionStatistics(getHiveBasicStatistics(partition.getParameters()), partitionStatistics.get(partitionName))))); - } - - protected void assertInsertFailure(String testTable, String expectedMessageRegExp) - { - assertInsertFailure(getSession(), testTable, expectedMessageRegExp); - } - - protected void assertInsertFailure(Session session, String testTable, String expectedMessageRegExp) - { - assertQueryFails( - session, - createInsertAsSelectFromTpchStatement(testTable), - expectedMessageRegExp); - } - - private String createInsertAsSelectFromTpchStatement(String testTable) - { - return format("INSERT INTO %s " + - "SELECT name, comment, nationkey, regionkey " + - "FROM tpch.tiny.nation", - testTable); - } - - protected String createInsertStatement(String testTable, List> data) - { - String values = data.stream() - .map(row -> String.join(", ", row)) - .collect(Collectors.joining("), (")); - return format("INSERT INTO %s VALUES (%s)", testTable, values); - } - - protected void assertOverwritePartition(String testTable) - { - computeActual(createInsertStatement( - testTable, - ImmutableList.of( - ImmutableList.of("'POLAND'", "'Test Data'", "25", "5"), - ImmutableList.of("'CZECH'", "'Test Data'", "26", "5")))); - query(format("SELECT name, comment, nationkey, regionkey FROM %s WHERE regionkey = 5", testTable)) - .assertThat() - .result() - .skippingTypesCheck() - .containsAll(resultBuilder(getSession()) - .row("POLAND", "Test Data", 25L, 5L) - .row("CZECH", "Test Data", 26L, 5L) - .build()); - - computeActual(createInsertStatement( - testTable, - ImmutableList.of( - ImmutableList.of("'POLAND'", "'Overwrite'", "25", "5")))); - query(format("SELECT name, comment, nationkey, regionkey FROM %s WHERE regionkey = 5", testTable)) - .assertThat() - .result() - .skippingTypesCheck() - .containsAll(resultBuilder(getSession()) - .row("POLAND", "Overwrite", 25L, 5L) - .build()); - computeActual(format("DROP TABLE %s", testTable)); - } - - protected String getRandomTestTableName() - { - return "nation_" + randomNameSuffix(); - } - - protected String getFullyQualifiedTestTableName() - { - return getFullyQualifiedTestTableName(getRandomTestTableName()); - } - - protected String getFullyQualifiedTestTableName(String tableName) - { - return getFullyQualifiedTestTableName(HIVE_TEST_SCHEMA, tableName); - } - - protected String getFullyQualifiedTestTableName(String schemaName, String tableName) - { - return "hive.%s.%s".formatted(schemaName, tableName); - } - - protected String getHiveTestTableName(String tableName) - { - return getHiveTestTableName(HIVE_TEST_SCHEMA, tableName); - } - - protected String getHiveTestTableName(String schemaName, String tableName) - { - return "%s.%s".formatted(schemaName, tableName); - } - - protected String getCreateTableStatement(String tableName, String... propertiesEntries) - { - return getCreateTableStatement(tableName, Arrays.asList(propertiesEntries)); - } - - protected String getCreateTableStatement(String tableName, List propertiesEntries) - { - return format( - "CREATE TABLE %s (" + - " name varchar(25), " + - " comment varchar(152), " + - " nationkey bigint, " + - " regionkey bigint) " + - (propertiesEntries.isEmpty() ? "" : propertiesEntries - .stream() - .collect(joining(",", "WITH (", ")"))), - tableName); - } - - protected void copyTpchNationToTable(String testTable) - { - computeActual(format("INSERT INTO " + testTable + " SELECT name, comment, nationkey, regionkey FROM tpch.tiny.nation")); - } - - private void testWriteWithFileSize(String testTable, int scaleFactorInThousands, long fileSizeRangeStart, long fileSizeRangeEnd) - { - String scaledColumnExpression = format("array_join(transform(sequence(1, %d), x-> array_join(repeat(comment, 1000), '')), '')", scaleFactorInThousands); - computeActual(format("INSERT INTO " + testTable + " SELECT %s, %s, regionkey FROM tpch.tiny.nation WHERE nationkey = 9", scaledColumnExpression, scaledColumnExpression)); - query(format("SELECT length(col1) FROM %s", testTable)) - .assertThat() - .result() - .skippingTypesCheck() - .containsAll(resultBuilder(getSession()) - .row(114L * scaleFactorInThousands * 1000) - .build()); - query(format("SELECT \"$file_size\" BETWEEN %d AND %d FROM %s", fileSizeRangeStart, fileSizeRangeEnd, testTable)) - .assertThat() - .result() - .skippingTypesCheck() - .containsAll(resultBuilder(getSession()) - .row(true) - .build()); - } - - private void addPartitions( - String sourceTableName, - String destinationExternalTableName, - List columnNames, - TupleDomain partitionsKeyFilter) - { - Optional> partitionNames = metastoreClient.getPartitionNamesByFilter(HIVE_TEST_SCHEMA, sourceTableName, columnNames, partitionsKeyFilter); - if (partitionNames.isEmpty()) { - // nothing to add - return; - } - Table table = metastoreClient.getTable(HIVE_TEST_SCHEMA, sourceTableName) - .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(HIVE_TEST_SCHEMA, sourceTableName))); - Map> partitionsByNames = metastoreClient.getPartitionsByNames(table, partitionNames.get()); - - metastoreClient.addPartitions( - HIVE_TEST_SCHEMA, - destinationExternalTableName, - partitionsByNames.entrySet().stream() - .map(e -> new PartitionWithStatistics( - e.getValue() - .map(p -> Partition.builder(p).setTableName(destinationExternalTableName).build()) - .orElseThrow(), - e.getKey(), - PartitionStatistics.empty())) - .collect(toImmutableList())); - } - - private String getTableLocation(String tableName) - { - return (String) computeScalar("SELECT DISTINCT regexp_replace(\"$path\", '/[^/]*$', '') FROM " + tableName); - } + private static final String BUCKET_NAME = "test-hive-insert-overwrite-" + randomNameSuffix(); - @Test - public void testInsertOverwritePartitionedAndBucketedAcidTable() + public TestHive3OnDataLake() { - String testTable = getFullyQualifiedTestTableName(); - computeActual(getCreateTableStatement( - testTable, - "partitioned_by=ARRAY['regionkey']", - "bucketed_by = ARRAY['nationkey']", - "bucket_count = 3", - "format = 'ORC'", - "transactional = true")); - assertInsertFailure( - testTable, - "Overwriting existing partition in transactional tables doesn't support DIRECT_TO_TARGET_EXISTING_DIRECTORY write mode"); + super(BUCKET_NAME, new Hive3MinioDataLake(BUCKET_NAME, HiveHadoop.HIVE3_IMAGE)); } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive4OnDataLake.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive4OnDataLake.java new file mode 100644 index 000000000000..379de81fcc91 --- /dev/null +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive4OnDataLake.java @@ -0,0 +1,66 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hive; + +import io.trino.plugin.hive.containers.Hive4MinioDataLake; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; +import org.junit.jupiter.api.parallel.ExecutionMode; + +import static io.trino.testing.TestingNames.randomNameSuffix; +import static org.junit.jupiter.api.Assumptions.abort; + +@Execution(ExecutionMode.SAME_THREAD) // TODO Make custom hive4 image to support running queries concurrently +class TestHive4OnDataLake + extends BaseTestHiveOnDataLake +{ + private static final String BUCKET_NAME = "test-hive-insert-overwrite-" + randomNameSuffix(); + + public TestHive4OnDataLake() + { + super(BUCKET_NAME, new Hive4MinioDataLake(BUCKET_NAME)); + } + + @Override + @Test + public void testSyncPartitionOnBucketRoot() + { + // https://github.com/trinodb/trino/issues/24453 + abort("Fails with `location must not be root path`"); + } + + @Override + @Test + public void testUnpartitionedTableExternalLocationOnTopOfTheBucket() + { + // https://github.com/trinodb/trino/issues/24453 + abort("Fails with `location must not be root path`"); + } + + @Override + @Test + public void testPartitionedTableExternalLocationOnTopOfTheBucket() + { + // https://github.com/trinodb/trino/issues/24453 + abort("Fails with `location must not be root path`"); + } + + @Override + @Test + public void testInsertOverwritePartitionedAndBucketedAcidTable() + { + // https://github.com/trinodb/trino/issues/24454 + abort("Fails with `Processor has no capabilities, cannot create an ACID table`"); + } +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveAnalyzeCorruptStatistics.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveAnalyzeCorruptStatistics.java index f986576683df..0f5fc357ea67 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveAnalyzeCorruptStatistics.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveAnalyzeCorruptStatistics.java @@ -14,7 +14,7 @@ package io.trino.plugin.hive; import io.airlift.units.Duration; -import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.plugin.hive.s3.S3HiveQueryRunner; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; @@ -28,13 +28,13 @@ public class TestHiveAnalyzeCorruptStatistics extends AbstractTestQueryFramework { - private HiveMinioDataLake hiveMinioDataLake; + private Hive3MinioDataLake hiveMinioDataLake; @Override protected QueryRunner createQueryRunner() throws Exception { - hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake("test-analyze")); + hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake("test-analyze")); hiveMinioDataLake.start(); return S3HiveQueryRunner.builder(hiveMinioDataLake) 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 35433d3610da..6d55f9cd915d 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,8 +15,8 @@ import io.trino.metastore.Database; import io.trino.metastore.HiveMetastore; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.plugin.hive.containers.HiveHadoop; -import io.trino.plugin.hive.containers.HiveMinioDataLake; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.security.PrincipalType; import io.trino.testing.BaseConnectorSmokeTest; @@ -49,7 +49,7 @@ protected QueryRunner createQueryRunner() throws Exception { String bucketName = "test-hive-metastore-catalog-smoke-test-" + randomNameSuffix(); - HiveMinioDataLake hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName, HiveHadoop.HIVE3_IMAGE)); + Hive3MinioDataLake hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName, HiveHadoop.HIVE3_IMAGE)); hiveMinioDataLake.start(); // Inserting into metastore's database directly because the Hive does not expose a way to create a custom catalog @@ -57,7 +57,7 @@ protected QueryRunner createQueryRunner() QueryRunner queryRunner = HiveQueryRunner.builder() .addHiveProperty("hive.metastore", "thrift") - .addHiveProperty("hive.metastore.uri", hiveMinioDataLake.getHiveHadoop().getHiveMetastoreEndpoint().toString()) + .addHiveProperty("hive.metastore.uri", hiveMinioDataLake.getHiveMetastoreEndpoint().toString()) .addHiveProperty("hive.metastore.thrift.catalog-name", HIVE_CUSTOM_CATALOG) .addHiveProperty("fs.hadoop.enabled", "false") .addHiveProperty("fs.native-s3.enabled", "true") diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveQueryFailureRecoveryTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveQueryFailureRecoveryTest.java index 54b05e04f833..fd36b81605ec 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveQueryFailureRecoveryTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveQueryFailureRecoveryTest.java @@ -17,7 +17,7 @@ import io.trino.operator.RetryPolicy; import io.trino.plugin.exchange.filesystem.FileSystemExchangePlugin; import io.trino.plugin.exchange.filesystem.containers.MinioStorage; -import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.plugin.hive.s3.S3HiveQueryRunner; import io.trino.testing.QueryRunner; import io.trino.tpch.TpchTable; @@ -43,7 +43,7 @@ public TestHiveQueryFailureRecoveryTest() super(RetryPolicy.QUERY); } - private HiveMinioDataLake hiveMinioDataLake; + private Hive3MinioDataLake hiveMinioDataLake; private MinioStorage minioStorage; @Override @@ -55,7 +55,7 @@ protected QueryRunner createQueryRunner( throws Exception { String bucketName = "test-hive-insert-overwrite-" + randomNameSuffix(); // randomizing bucket name to ensure cached TrinoS3FileSystem objects are not reused - this.hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName)); + this.hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName)); hiveMinioDataLake.start(); this.minioStorage = closeAfterClass(new MinioStorage("test-exchange-spooling-" + randomNameSuffix())); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveTaskFailureRecoveryTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveTaskFailureRecoveryTest.java index a74f29d305d8..cca5e7a009c0 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveTaskFailureRecoveryTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveTaskFailureRecoveryTest.java @@ -17,7 +17,7 @@ import io.trino.operator.RetryPolicy; import io.trino.plugin.exchange.filesystem.FileSystemExchangePlugin; import io.trino.plugin.exchange.filesystem.containers.MinioStorage; -import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.plugin.hive.s3.S3HiveQueryRunner; import io.trino.testing.QueryRunner; import io.trino.tpch.TpchTable; @@ -43,7 +43,7 @@ public TestHiveTaskFailureRecoveryTest() super(RetryPolicy.TASK); } - private HiveMinioDataLake hiveMinioDataLake; + private Hive3MinioDataLake hiveMinioDataLake; private MinioStorage minioStorage; @Override @@ -55,7 +55,7 @@ protected QueryRunner createQueryRunner( throws Exception { String bucketName = "test-hive-insert-overwrite-" + randomNameSuffix(); // randomizing bucket name to ensure cached TrinoS3FileSystem objects are not reused - this.hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName)); + this.hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName)); hiveMinioDataLake.start(); this.minioStorage = closeAfterClass(new MinioStorage("test-exchange-spooling-" + randomNameSuffix())); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/containers/Hive3MinioDataLake.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/containers/Hive3MinioDataLake.java new file mode 100644 index 000000000000..840e18d55bf9 --- /dev/null +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/containers/Hive3MinioDataLake.java @@ -0,0 +1,74 @@ +/* + * 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.containers; + +import com.google.common.collect.ImmutableMap; + +import java.net.URI; +import java.util.Map; + +import static io.trino.plugin.hive.containers.HiveMinioDataLake.State.STARTED; +import static io.trino.testing.containers.TestContainers.getPathFromClassPathResource; + +public class Hive3MinioDataLake + extends HiveMinioDataLake +{ + private final HiveHadoop hiveHadoop; + + public Hive3MinioDataLake(String bucketName) + { + this(bucketName, HiveHadoop.HIVE3_IMAGE); + } + + public Hive3MinioDataLake(String bucketName, String hiveHadoopImage) + { + this(bucketName, ImmutableMap.of("/etc/hadoop/conf/core-site.xml", getPathFromClassPathResource("hive_minio_datalake/hive-core-site.xml")), hiveHadoopImage); + } + + public Hive3MinioDataLake(String bucketName, Map hiveHadoopFilesToMount, String hiveHadoopImage) + { + super(bucketName); + HiveHadoop.Builder hiveHadoopBuilder = HiveHadoop.builder() + .withImage(hiveHadoopImage) + .withNetwork(network) + .withFilesToMount(hiveHadoopFilesToMount); + this.hiveHadoop = closer.register(hiveHadoopBuilder.build()); + } + + @Override + public void start() + { + super.start(); + hiveHadoop.start(); + state = STARTED; + } + + @Override + public String runOnHive(String sql) + { + return hiveHadoop.runOnHive(sql); + } + + @Override + public HiveHadoop getHiveHadoop() + { + return hiveHadoop; + } + + @Override + public URI getHiveMetastoreEndpoint() + { + return hiveHadoop.getHiveMetastoreEndpoint(); + } +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/containers/Hive4HiveServer.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/containers/Hive4HiveServer.java new file mode 100644 index 000000000000..96af084ca48c --- /dev/null +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/containers/Hive4HiveServer.java @@ -0,0 +1,95 @@ +/* + * 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.containers; + +import com.google.common.collect.ImmutableSet; +import com.google.common.net.HostAndPort; +import io.airlift.log.Logger; +import io.trino.testing.containers.BaseTestContainer; +import org.testcontainers.containers.Network; + +import java.net.URI; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static io.trino.plugin.hive.containers.Hive4Metastore.HIVE4_IMAGE; + +public class Hive4HiveServer + extends BaseTestContainer +{ + public static final int HIVE_SERVER_PORT = 10000; + + private static final Logger log = Logger.get(Hive4HiveServer.class); + private static final String HOST_NAME = "hiveserver2"; + + public static Builder builder() + { + return new Builder(); + } + + public static class Builder + extends BaseTestContainer.Builder + { + private Builder() + { + this.image = HIVE4_IMAGE; + this.hostName = HOST_NAME; + this.exposePorts = ImmutableSet.of(HIVE_SERVER_PORT); + } + + @Override + public Hive4HiveServer build() + { + return new Hive4HiveServer(image, hostName, exposePorts, filesToMount, envVars, network, startupRetryLimit); + } + } + + private Hive4HiveServer( + String image, + String hostName, + Set ports, + Map filesToMount, + Map envVars, + Optional network, + int startupRetryLimit) + { + super( + image, + hostName, + ports, + filesToMount, + envVars, + network, + startupRetryLimit); + } + + @Override + public void start() + { + super.start(); + log.info("Hive container started with addresses for hive server: %s", getHiveServerEndpoint()); + } + + public String runOnHive(String query) + { + return executeInContainerFailOnError("beeline", "-u", "jdbc:hive2://localhost:%s/default".formatted(HIVE_SERVER_PORT), "-n", "hive", "-e", query); + } + + public URI getHiveServerEndpoint() + { + HostAndPort address = getMappedHostAndPortForExposedPort(HIVE_SERVER_PORT); + return URI.create(address.getHost() + ":" + address.getPort()); + } +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/containers/Hive4Metastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/containers/Hive4Metastore.java new file mode 100644 index 000000000000..c7d01ca36345 --- /dev/null +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/containers/Hive4Metastore.java @@ -0,0 +1,96 @@ +/* + * 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.containers; + +import com.google.common.collect.ImmutableSet; +import com.google.common.net.HostAndPort; +import io.airlift.log.Logger; +import io.trino.testing.containers.BaseTestContainer; +import org.testcontainers.containers.Network; + +import java.net.URI; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static io.trino.testing.TestingProperties.getDockerImagesVersion; + +public class Hive4Metastore + extends BaseTestContainer +{ + public static final String HIVE4_IMAGE = "ghcr.io/trinodb/testing/hive4.0-hive:" + getDockerImagesVersion(); + public static final int HIVE_METASTORE_PORT = 9083; + + private static final Logger log = Logger.get(HiveHadoop.class); + private static final String HOST_NAME = "metastore"; + + public static Builder builder() + { + return new Builder(); + } + + public static class Builder + extends BaseTestContainer.Builder + { + private Builder() + { + this.image = HIVE4_IMAGE; + this.hostName = HOST_NAME; + this.exposePorts = ImmutableSet.of(HIVE_METASTORE_PORT); + } + + @Override + public Hive4Metastore build() + { + return new Hive4Metastore(image, hostName, exposePorts, filesToMount, envVars, network, startupRetryLimit); + } + } + + private Hive4Metastore( + String image, + String hostName, + Set ports, + Map filesToMount, + Map envVars, + Optional network, + int startupRetryLimit) + { + super( + image, + hostName, + ports, + filesToMount, + envVars, + network, + startupRetryLimit); + } + + @Override + public void start() + { + super.start(); + log.info("Hive container started with addresses for metastore: %s", getHiveMetastoreEndpoint()); + } + + public URI getHiveMetastoreEndpoint() + { + HostAndPort address = getMappedHostAndPortForExposedPort(HIVE_METASTORE_PORT); + return URI.create("thrift://" + address.getHost() + ":" + address.getPort()); + } + + public URI getInternalHiveMetastoreEndpoint() + { + return URI.create("thrift://" + HOST_NAME + ":" + HIVE_METASTORE_PORT); + } +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/containers/Hive4MinioDataLake.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/containers/Hive4MinioDataLake.java new file mode 100644 index 000000000000..e58ea118889b --- /dev/null +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/containers/Hive4MinioDataLake.java @@ -0,0 +1,92 @@ +/* + * 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.containers; + +import com.google.common.collect.ImmutableMap; + +import java.net.URI; +import java.util.Map; +import java.util.Set; + +import static io.trino.plugin.hive.containers.Hive4HiveServer.HIVE_SERVER_PORT; +import static io.trino.plugin.hive.containers.Hive4Metastore.HIVE4_IMAGE; +import static io.trino.plugin.hive.containers.HiveMinioDataLake.State.STARTED; +import static io.trino.testing.containers.Minio.MINIO_ACCESS_KEY; +import static io.trino.testing.containers.Minio.MINIO_SECRET_KEY; +import static io.trino.testing.containers.TestContainers.getPathFromClassPathResource; + +public class Hive4MinioDataLake + extends HiveMinioDataLake +{ + private final Hive4HiveServer hiveServer; + private final Hive4Metastore hiveMetastore; + + public Hive4MinioDataLake(String bucketName) + { + super(bucketName); + String hiveImage = HIVE4_IMAGE; + Map hiveFilesToMount = ImmutableMap.of("/opt/hive/conf/hive-site.xml", getPathFromClassPathResource("hive_minio_datalake/hive4-hive-site.xml")); + // Separate hms and hiveserver(below) is created as standalone hiveserver doesn't expose embedded hms. https://github.com/apache/hive/blob/a1420ed816c315d98be7ebf05cdc3ba139a68643/packaging/src/docker/README.md?plain=1#L46. + // Run standalone metastore https://github.com/apache/hive/blob/a1420ed816c315d98be7ebf05cdc3ba139a68643/packaging/src/docker/README.md?plain=1#L105 + Hive4Metastore.Builder metastorebuilder = Hive4Metastore.builder() + .withImage(hiveImage) + .withEnvVars(Map.of("SERVICE_NAME", "metastore")) + .withNetwork(network) + .withExposePorts(Set.of(Hive4Metastore.HIVE_METASTORE_PORT)) + .withFilesToMount(hiveFilesToMount); + this.hiveMetastore = closer.register(metastorebuilder.build()); + + // Run hive server connecting to remote(above) metastore https://github.com/apache/hive/blob/a1420ed816c315d98be7ebf05cdc3ba139a68643/packaging/src/docker/README.md?plain=1#L139-L143 + Hive4HiveServer.Builder hiveHadoopBuilder = Hive4HiveServer.builder() + .withImage(hiveImage) + .withEnvVars(Map.of( + "SERVICE_NAME", "hiveserver2", + "HIVE_SERVER2_THRIFT_PORT", String.valueOf(HIVE_SERVER_PORT), + "SERVICE_OPTS", "-Xmx1G -Dhive.metastore.uris=%s".formatted(hiveMetastore.getInternalHiveMetastoreEndpoint()), + "IS_RESUME", "true", + "AWS_ACCESS_KEY_ID", MINIO_ACCESS_KEY, + "AWS_SECRET_KEY", MINIO_SECRET_KEY)) + .withNetwork(network) + .withExposePorts(Set.of(HIVE_SERVER_PORT)) + .withFilesToMount(hiveFilesToMount); + this.hiveServer = closer.register(hiveHadoopBuilder.build()); + } + + @Override + public void start() + { + super.start(); + hiveMetastore.start(); + hiveServer.start(); + state = STARTED; + } + + @Override + public String runOnHive(String sql) + { + return hiveServer.runOnHive(sql); + } + + @Override + public HiveHadoop getHiveHadoop() + { + throw new UnsupportedOperationException(); + } + + @Override + public URI getHiveMetastoreEndpoint() + { + return hiveMetastore.getHiveMetastoreEndpoint(); + } +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/containers/HiveMinioDataLake.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/containers/HiveMinioDataLake.java index 0ce79c2ff6a6..19bbc35d32a4 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/containers/HiveMinioDataLake.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/containers/HiveMinioDataLake.java @@ -19,50 +19,36 @@ import io.trino.testing.minio.MinioClient; import org.testcontainers.containers.Network; +import java.net.URI; import java.util.List; -import java.util.Map; import static com.google.common.base.Preconditions.checkState; +import static io.trino.plugin.hive.containers.HiveMinioDataLake.State.INITIAL; +import static io.trino.plugin.hive.containers.HiveMinioDataLake.State.STARTED; +import static io.trino.plugin.hive.containers.HiveMinioDataLake.State.STARTING; +import static io.trino.plugin.hive.containers.HiveMinioDataLake.State.STOPPED; import static io.trino.testing.containers.Minio.MINIO_ACCESS_KEY; import static io.trino.testing.containers.Minio.MINIO_SECRET_KEY; -import static io.trino.testing.containers.TestContainers.getPathFromClassPathResource; import static java.util.Objects.requireNonNull; import static org.testcontainers.containers.Network.newNetwork; -public class HiveMinioDataLake +public abstract class HiveMinioDataLake implements AutoCloseable { - /** - * In S3 this region is implicitly the default one. In Minio, however, - * if we set an empty region, it will accept any. - * So setting it by default to `us-east-1` simulates S3 better - */ - public static final String MINIO_DEFAULT_REGION = "us-east-1"; + private static final String MINIO_DEFAULT_REGION = "us-east-1"; private final String bucketName; private final Minio minio; - private final HiveHadoop hiveHadoop; - - private final AutoCloseableCloser closer = AutoCloseableCloser.create(); - private final Network network; - - private State state = State.INITIAL; private MinioClient minioClient; - public HiveMinioDataLake(String bucketName) - { - this(bucketName, HiveHadoop.HIVE3_IMAGE); - } - - public HiveMinioDataLake(String bucketName, String hiveHadoopImage) - { - this(bucketName, ImmutableMap.of("/etc/hadoop/conf/core-site.xml", getPathFromClassPathResource("hive_minio_datalake/hive-core-site.xml")), hiveHadoopImage); - } + protected final AutoCloseableCloser closer = AutoCloseableCloser.create(); + protected final Network network; + protected State state = INITIAL; - public HiveMinioDataLake(String bucketName, Map hiveHadoopFilesToMount, String hiveHadoopImage) + public HiveMinioDataLake(String bucketName) { this.bucketName = requireNonNull(bucketName, "bucketName is null"); - network = closer.register(newNetwork()); + this.network = closer.register(newNetwork()); this.minio = closer.register( Minio.builder() .withNetwork(network) @@ -72,30 +58,22 @@ public HiveMinioDataLake(String bucketName, Map hiveHadoopFilesT .put("MINIO_REGION", MINIO_DEFAULT_REGION) .buildOrThrow()) .build()); - - HiveHadoop.Builder hiveHadoopBuilder = HiveHadoop.builder() - .withImage(hiveHadoopImage) - .withNetwork(network) - .withFilesToMount(hiveHadoopFilesToMount); - this.hiveHadoop = closer.register(hiveHadoopBuilder.build()); } public void start() { - checkState(state == State.INITIAL, "Already started: %s", state); - state = State.STARTING; + checkState(state == INITIAL, "Already started: %s", state); + state = STARTING; minio.start(); - hiveHadoop.start(); minioClient = closer.register(minio.createMinioClient()); minio.createBucket(bucketName); - state = State.STARTED; } public void stop() throws Exception { closer.close(); - state = State.STOPPED; + state = STOPPED; } public Network getNetwork() @@ -105,7 +83,7 @@ public Network getNetwork() public MinioClient getMinioClient() { - checkState(state == State.STARTED, "Can't provide client when MinIO state is: %s", state); + checkState(state == STARTED, "Can't provide client when MinIO state is: %s", state); return minioClient; } @@ -129,10 +107,11 @@ public Minio getMinio() return minio; } - public HiveHadoop getHiveHadoop() - { - return hiveHadoop; - } + public abstract String runOnHive(String sql); + + public abstract HiveHadoop getHiveHadoop(); + + public abstract URI getHiveMetastoreEndpoint(); public String getBucketName() { @@ -146,7 +125,7 @@ public void close() stop(); } - private enum State + protected enum State { INITIAL, STARTING, 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 1fa641e51955..23a18de4e7e5 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 @@ -18,8 +18,8 @@ import io.trino.metastore.Database; import io.trino.metastore.HiveMetastore; 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.containers.HiveMinioDataLake; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.security.PrincipalType; import io.trino.testing.AbstractTestQueryFramework; @@ -52,7 +52,7 @@ protected QueryRunner createQueryRunner() throws Exception { this.bucketName = "test-hive-metastore-catalogs-" + randomNameSuffix(); - HiveMinioDataLake hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName, HiveHadoop.HIVE3_IMAGE)); + Hive3MinioDataLake hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName, HiveHadoop.HIVE3_IMAGE)); hiveMinioDataLake.start(); QueryRunner queryRunner = HiveQueryRunner.builder() @@ -75,11 +75,11 @@ protected QueryRunner createQueryRunner() return queryRunner; } - private static Map buildHiveProperties(HiveMinioDataLake hiveMinioDataLake) + private static Map buildHiveProperties(Hive3MinioDataLake hiveMinioDataLake) { return ImmutableMap.builder() .put("hive.metastore", "thrift") - .put("hive.metastore.uri", hiveMinioDataLake.getHiveHadoop().getHiveMetastoreEndpoint().toString()) + .put("hive.metastore.uri", hiveMinioDataLake.getHiveMetastoreEndpoint().toString()) .put("fs.hadoop.enabled", "false") .put("fs.native-s3.enabled", "true") .put("s3.path-style-access", "true") diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/S3HiveQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/S3HiveQueryRunner.java index 9df3a85934b3..ea246e127f5d 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/S3HiveQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/S3HiveQueryRunner.java @@ -19,6 +19,8 @@ import io.airlift.log.Logging; import io.airlift.units.Duration; import io.trino.plugin.hive.HiveQueryRunner; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; +import io.trino.plugin.hive.containers.Hive4MinioDataLake; import io.trino.plugin.hive.containers.HiveMinioDataLake; import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; import io.trino.plugin.hive.metastore.thrift.TestingTokenAwareMetastoreClientFactory; @@ -47,7 +49,7 @@ public final class S3HiveQueryRunner private S3HiveQueryRunner() {} public static QueryRunner create( - HiveMinioDataLake hiveMinioDataLake, + Hive3MinioDataLake hiveMinioDataLake, Map additionalHiveProperties) throws Exception { @@ -59,7 +61,7 @@ public static QueryRunner create( public static Builder builder(HiveMinioDataLake hiveMinioDataLake) { return builder() - .setHiveMetastoreEndpoint(hiveMinioDataLake.getHiveHadoop().getHiveMetastoreEndpoint()) + .setHiveMetastoreEndpoint(hiveMinioDataLake.getHiveMetastoreEndpoint()) .setS3Endpoint("http://" + hiveMinioDataLake.getMinio().getMinioApiEndpoint()) .setS3Region(MINIO_REGION) .setS3AccessKey(MINIO_ACCESS_KEY) @@ -173,7 +175,7 @@ public DistributedQueryRunner build() public static void main(String[] args) throws Exception { - HiveMinioDataLake hiveMinioDataLake = new HiveMinioDataLake("tpch"); + Hive3MinioDataLake hiveMinioDataLake = new Hive3MinioDataLake("tpch"); hiveMinioDataLake.start(); QueryRunner queryRunner = S3HiveQueryRunner.builder(hiveMinioDataLake) @@ -186,4 +188,24 @@ public static void main(String[] args) log.info("======== SERVER STARTED ========"); log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); } + + public static class S3Hive4QueryRunner + { + public static void main(String[] args) + throws Exception + { + Hive4MinioDataLake hiveMinioDataLake = new Hive4MinioDataLake("tpch"); + hiveMinioDataLake.start(); + + QueryRunner queryRunner = S3HiveQueryRunner.builder(hiveMinioDataLake) + .addCoordinatorProperty("http-server.http.port", "8080") + .setHiveProperties(ImmutableMap.of("hive.security", "allow-all")) + .setSkipTimezoneSetup(true) + .setInitialTables(TpchTable.getTables()) + .build(); + Logger log = Logger.get(S3Hive4QueryRunner.class); + log.info("======== SERVER STARTED ========"); + log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); + } + } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestHiveS3MinioQueries.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestHiveS3MinioQueries.java index 8c0fbaf5b091..8c8d6c172a1a 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestHiveS3MinioQueries.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestHiveS3MinioQueries.java @@ -14,7 +14,7 @@ package io.trino.plugin.hive.s3; import com.google.common.collect.ImmutableMap; -import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; import org.junit.jupiter.api.Test; @@ -31,7 +31,7 @@ public class TestHiveS3MinioQueries extends AbstractTestQueryFramework { - private HiveMinioDataLake hiveMinioDataLake; + private Hive3MinioDataLake hiveMinioDataLake; private String bucketName; @Override @@ -39,7 +39,7 @@ protected QueryRunner createQueryRunner() throws Exception { this.bucketName = "test-hive-minio-queries-" + randomNameSuffix(); - this.hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName)); + this.hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName)); this.hiveMinioDataLake.start(); return S3HiveQueryRunner.builder(hiveMinioDataLake) diff --git a/plugin/trino-hive/src/test/resources/hive_minio_datalake/hive4-hive-site.xml b/plugin/trino-hive/src/test/resources/hive_minio_datalake/hive4-hive-site.xml new file mode 100644 index 000000000000..eec52b5f6ac5 --- /dev/null +++ b/plugin/trino-hive/src/test/resources/hive_minio_datalake/hive4-hive-site.xml @@ -0,0 +1,68 @@ + + + + hive.server2.enable.doAs + false + + + hive.tez.exec.inplace.progress + false + + + hive.exec.scratchdir + /opt/hive/scratch_dir + + + hive.user.install.directory + /opt/hive/install_dir + + + tez.runtime.optimize.local.fetch + true + + + hive.exec.submit.local.task.via.child + false + + + mapreduce.framework.name + local + + + hive.metastore.warehouse.dir + /opt/hive/data/warehouse + + + metastore.metastore.event.db.notification.api.auth + false + + + + + hive.users.in.admin.role + hive + + + + + fs.s3a.access.key + accesskey + + + fs.s3a.secret.key + secretkey + + + fs.s3a.endpoint + http://minio:4566 + + + fs.s3a.path.style.access + true + + + fs.s3.impl + org.apache.hadoop.fs.s3a.S3AFileSystem + + + 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 8f817c21fabf..915ca2b9ead2 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 @@ -20,7 +20,7 @@ import io.trino.filesystem.Location; import io.trino.metastore.Database; import io.trino.plugin.base.util.Closables; -import io.trino.plugin.hive.containers.HiveMinioDataLake; +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; @@ -56,7 +56,7 @@ public static Builder builder() return new Builder("local:///"); } - public static Builder builder(HiveMinioDataLake hiveMinioDataLake) + public static Builder builder(Hive3MinioDataLake hiveMinioDataLake) { return new Builder("s3://" + hiveMinioDataLake.getBucketName() + "/") .addConnectorProperty("fs.hadoop.enabled", "false") @@ -157,7 +157,7 @@ public static void main(String[] args) Logging.initialize(); Logger log = Logger.get(HudiMinioQueryRunnerMain.class); - HiveMinioDataLake hiveMinioDataLake = new HiveMinioDataLake("test-bucket"); + Hive3MinioDataLake hiveMinioDataLake = new Hive3MinioDataLake("test-bucket"); hiveMinioDataLake.start(); QueryRunner queryRunner = builder(hiveMinioDataLake) .addCoordinatorProperty("http-server.http.port", "8080") diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiMinioConnectorSmokeTest.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiMinioConnectorSmokeTest.java index 069fb05c0523..5fa1332073cd 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiMinioConnectorSmokeTest.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiMinioConnectorSmokeTest.java @@ -13,7 +13,7 @@ */ package io.trino.plugin.hudi; -import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.plugin.hudi.testing.TpchHudiTablesInitializer; import io.trino.testing.QueryRunner; @@ -29,7 +29,7 @@ protected QueryRunner createQueryRunner() throws Exception { String bucketName = "test-hudi-connector-" + randomNameSuffix(); - HiveMinioDataLake hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName, HIVE3_IMAGE)); + Hive3MinioDataLake hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName, HIVE3_IMAGE)); hiveMinioDataLake.start(); hiveMinioDataLake.getMinioClient().ensureBucketExists(bucketName); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMinioConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMinioConnectorSmokeTest.java index 4fdeb620b748..3144bd56bb02 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMinioConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMinioConnectorSmokeTest.java @@ -17,7 +17,7 @@ import io.minio.messages.Event; import io.trino.Session; import io.trino.metastore.HiveMetastore; -import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; import io.trino.testing.QueryRunner; import io.trino.testing.minio.MinioClient; @@ -50,7 +50,7 @@ public abstract class BaseIcebergMinioConnectorSmokeTest private final String schemaName; private final String bucketName; - private HiveMinioDataLake hiveMinioDataLake; + private Hive3MinioDataLake hiveMinioDataLake; protected BaseIcebergMinioConnectorSmokeTest(FileFormat format) { @@ -63,7 +63,7 @@ protected BaseIcebergMinioConnectorSmokeTest(FileFormat format) protected QueryRunner createQueryRunner() throws Exception { - this.hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName)); + this.hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName)); this.hiveMinioDataLake.start(); return IcebergQueryRunner.builder() @@ -71,7 +71,7 @@ protected QueryRunner createQueryRunner() ImmutableMap.builder() .put("iceberg.file-format", format.name()) .put("iceberg.catalog.type", "HIVE_METASTORE") - .put("hive.metastore.uri", hiveMinioDataLake.getHiveHadoop().getHiveMetastoreEndpoint().toString()) + .put("hive.metastore.uri", hiveMinioDataLake.getHiveMetastoreEndpoint().toString()) .put("hive.metastore.thrift.client.read-timeout", "1m") // read timed out sometimes happens with the default timeout .put("fs.hadoop.enabled", "false") .put("fs.native-s3.enabled", "true") @@ -258,7 +258,7 @@ protected void dropTableFromMetastore(String tableName) { HiveMetastore metastore = new BridgingHiveMetastore( testingThriftHiveMetastoreBuilder() - .metastoreClient(hiveMinioDataLake.getHiveHadoop().getHiveMetastoreEndpoint()) + .metastoreClient(hiveMinioDataLake.getHiveMetastoreEndpoint()) .build(this::closeAfterClass)); metastore.dropTable(schemaName, tableName, false); assertThat(metastore.getTable(schemaName, tableName)).isEmpty(); @@ -269,7 +269,7 @@ protected String getMetadataLocation(String tableName) { HiveMetastore metastore = new BridgingHiveMetastore( testingThriftHiveMetastoreBuilder() - .metastoreClient(hiveMinioDataLake.getHiveHadoop().getHiveMetastoreEndpoint()) + .metastoreClient(hiveMinioDataLake.getHiveMetastoreEndpoint()) .build(this::closeAfterClass)); return metastore .getTable(schemaName, tableName).orElseThrow() diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java index 698658916e7e..ff7664af0ee0 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java @@ -21,8 +21,8 @@ import io.airlift.log.Logger; import io.airlift.log.Logging; import io.trino.plugin.exchange.filesystem.FileSystemExchangePlugin; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.plugin.hive.containers.HiveHadoop; -import io.trino.plugin.hive.containers.HiveMinioDataLake; import io.trino.plugin.iceberg.catalog.jdbc.TestingIcebergJdbcServer; import io.trino.plugin.iceberg.catalog.rest.TestingPolarisCatalog; import io.trino.plugin.iceberg.containers.NessieContainer; @@ -319,7 +319,7 @@ public static void main(String[] args) { String bucketName = "test-bucket"; @SuppressWarnings("resource") - HiveMinioDataLake hiveMinioDataLake = new HiveMinioDataLake(bucketName); + Hive3MinioDataLake hiveMinioDataLake = new Hive3MinioDataLake(bucketName); hiveMinioDataLake.start(); @SuppressWarnings("resource") diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestTrinoHive4CatalogWithHiveMetastore.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestTrinoHive4CatalogWithHiveMetastore.java new file mode 100644 index 000000000000..7c4f0ae1e245 --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestTrinoHive4CatalogWithHiveMetastore.java @@ -0,0 +1,27 @@ +/* + * 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 io.trino.plugin.hive.containers.Hive4MinioDataLake; +import io.trino.plugin.hive.containers.HiveMinioDataLake; + +public class TestTrinoHive4CatalogWithHiveMetastore + extends TestTrinoHiveCatalogWithHiveMetastore +{ + @Override + HiveMinioDataLake hiveMinioDataLake() + { + return new Hive4MinioDataLake(bucketName); + } +} 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 587f63d2accf..01f4b8a4c3ed 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 @@ -33,6 +33,7 @@ import io.trino.metastore.TableInfo; 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; @@ -87,17 +88,22 @@ public class TestTrinoHiveCatalogWithHiveMetastore { private static final Logger LOG = Logger.get(TestTrinoHiveCatalogWithHiveMetastore.class); - private AutoCloseableCloser closer = AutoCloseableCloser.create(); + private final AutoCloseableCloser closer = AutoCloseableCloser.create(); // Use MinIO for storage, since HDFS is hard to get working in a unit test private HiveMinioDataLake dataLake; private TrinoFileSystem fileSystem; - private String bucketName; + protected String bucketName; + + HiveMinioDataLake hiveMinioDataLake() + { + return new Hive3MinioDataLake(bucketName, HIVE3_IMAGE); + } @BeforeAll public void setUp() { bucketName = "test-hive-catalog-with-hms-" + randomNameSuffix(); - dataLake = closer.register(new HiveMinioDataLake(bucketName, HIVE3_IMAGE)); + dataLake = closer.register(hiveMinioDataLake()); dataLake.start(); } @@ -130,7 +136,7 @@ protected TrinoCatalog createTrinoCatalog(boolean useUniqueTableLocations) .thriftMetastoreConfig(new ThriftMetastoreConfig() // Read timed out sometimes happens with the default timeout .setReadTimeout(new Duration(1, MINUTES))) - .metastoreClient(dataLake.getHiveHadoop().getHiveMetastoreEndpoint()) + .metastoreClient(dataLake.getHiveMetastoreEndpoint()) .build(closer::register); CachingHiveMetastore metastore = createPerTransactionCache(new BridgingHiveMetastore(thriftMetastore), 1000); fileSystem = fileSystemFactory.create(SESSION); diff --git a/pom.xml b/pom.xml index 907c0c868ac7..e7da68940ea9 100644 --- a/pom.xml +++ b/pom.xml @@ -191,7 +191,7 @@ 1.12.779 4.17.0 7.7.1 - 106 + 107 1.22 11.0.1 1.15.1 diff --git a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/delta/TestDeltaFaultTolerantExecutionTest.java b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/delta/TestDeltaFaultTolerantExecutionTest.java index 0aeee0ac7346..bcffb9655fb2 100644 --- a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/delta/TestDeltaFaultTolerantExecutionTest.java +++ b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/delta/TestDeltaFaultTolerantExecutionTest.java @@ -17,7 +17,7 @@ import io.trino.plugin.deltalake.DeltaLakeQueryRunner; import io.trino.plugin.exchange.filesystem.FileSystemExchangePlugin; import io.trino.plugin.exchange.filesystem.containers.MinioStorage; -import io.trino.plugin.hive.containers.HiveMinioDataLake; +import io.trino.plugin.hive.containers.Hive3MinioDataLake; import io.trino.testing.FaultTolerantExecutionConnectorTestHelper; import io.trino.testing.QueryRunner; @@ -38,7 +38,7 @@ public TestDeltaFaultTolerantExecutionTest() protected QueryRunner createQueryRunner() throws Exception { - HiveMinioDataLake hiveMinioDataLake = closeAfterClass(new HiveMinioDataLake(bucketName)); + Hive3MinioDataLake hiveMinioDataLake = closeAfterClass(new Hive3MinioDataLake(bucketName)); hiveMinioDataLake.start(); MinioStorage minioStorage = closeAfterClass(new MinioStorage(bucketName)); minioStorage.start();