diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java index 5bb9390d8623..f46133d0f9dc 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java @@ -41,7 +41,6 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.ql.Context.Operation; import org.apache.hadoop.hive.ql.Context.RewritePolicy; @@ -58,7 +57,6 @@ import org.apache.hadoop.mapred.OutputCommitter; import org.apache.hadoop.mapred.TaskAttemptContext; import org.apache.hadoop.mapred.TaskAttemptID; -import org.apache.hadoop.mapreduce.JobID; import org.apache.hadoop.mapreduce.TaskType; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.ContentFile; @@ -84,7 +82,6 @@ import org.apache.iceberg.mr.hive.compaction.IcebergCompactionUtil; import org.apache.iceberg.mr.hive.writer.HiveIcebergWriter; import org.apache.iceberg.mr.hive.writer.WriterRegistry; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Splitter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -106,7 +103,6 @@ public class HiveIcebergOutputCommitter extends OutputCommitter { private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class); private static final Splitter TABLE_NAME_SPLITTER = Splitter.on(".."); - private static final String FOR_COMMIT_EXTENSION = ".forCommit"; private static final String CONFLICT_DETECTION_FILTER = "Conflict detection Filter Expression: {}"; private ExecutorService workerPool; @@ -159,7 +155,7 @@ public void commitTask(TaskAttemptContext originalContext) throws IOException { .run(output -> { Table table = HiveTableUtil.deserializeTable(context.getJobConf(), output); if (table != null) { - String fileForCommitLocation = generateFileForCommitLocation(table.location(), jobConf, + String fileForCommitLocation = HiveTableUtil.fileForCommitLocation(table.location(), jobConf, attemptID.getJobID(), attemptID.getTaskID().getId()); if (writers.get(output) != null) { List dataFiles = Lists.newArrayList(); @@ -288,7 +284,7 @@ public void commitJobs(List originalContextList, Operation operation final Collection jobContexts = outputs.get(output); final Table table = output.table; jobContexts.forEach(jobContext -> jobLocations.add( - generateJobLocation(table.location(), jobConf, jobContext.getJobID())) + HiveTableUtil.jobLocation(table.location(), jobConf, jobContext.getJobID())) ); commitTable(table.io(), fileExecutor, output, jobContexts, operation); }); @@ -360,7 +356,7 @@ public void abortJobs(List originalContextList) throws IOException { for (JobContext jobContext : outputs.get(output)) { LOG.info("Cleaning job for jobID: {}, table: {}", jobContext.getJobID(), output); Table table = output.table; - String jobLocation = generateJobLocation(table.location(), jobConf, jobContext.getJobID()); + String jobLocation = HiveTableUtil.jobLocation(table.location(), jobConf, jobContext.getJobID()); jobLocations.add(jobLocation); // list jobLocation to get number of forCommit files // we do this because map/reduce num in jobConf is unreliable and we have no access to vertex status info @@ -404,7 +400,8 @@ private static Set listForCommits(JobConf jobConf, String jobLocatio FileStatus[] children = path.getFileSystem(jobConf).listStatus(path); LOG.debug("Listing the job location: {} yielded these files: {}", jobLocation, Arrays.toString(children)); return Arrays.stream(children) - .filter(child -> !child.isDirectory() && child.getPath().getName().endsWith(FOR_COMMIT_EXTENSION)) + .filter(child -> !child.isDirectory() && + child.getPath().getName().endsWith(HiveTableUtil.FOR_COMMIT_EXTENSION)) .collect(Collectors.toSet()); } @@ -451,7 +448,7 @@ private void commitTable(FileIO io, ExecutorService executor, OutputTable output } LOG.info("Committing job has started for table: {}, using location: {}", - table, generateJobLocation(outputTable.table.location(), conf, jobContext.getJobID())); + table, HiveTableUtil.jobLocation(outputTable.table.location(), conf, jobContext.getJobID())); int numTasks = SessionStateUtil.getCommitInfo(conf, name) .map(info -> info.get(jobContext.getJobID().toString())) @@ -771,7 +768,7 @@ private static FilesForCommit collectResults(int numTasks, ExecutorService execu .executeWith(executor) .retry(3) .run(taskId -> { - final String taskFileName = generateFileForCommitLocation(location, conf, jobContext.getJobID(), taskId); + String taskFileName = HiveTableUtil.fileForCommitLocation(location, conf, jobContext.getJobID(), taskId); final FilesForCommit files = readFileForCommit(taskFileName, io); LOG.debug("Found Iceberg commitTask manifest file: {}\n{}", taskFileName, files); @@ -787,34 +784,6 @@ private static FilesForCommit collectResults(int numTasks, ExecutorService execu mergedAndDeletedFiles); } - /** - * Generates the job temp location based on the job configuration. - * Currently it uses TABLE_LOCATION/temp/QUERY_ID-jobId. - * @param location The location of the table - * @param conf The job's configuration - * @param jobId The JobID for the task - * @return The file to store the results - */ - @VisibleForTesting - static String generateJobLocation(String location, Configuration conf, JobID jobId) { - String queryId = conf.get(HiveConf.ConfVars.HIVE_QUERY_ID.varname); - return location + "/temp/" + queryId + "-" + jobId; - } - - /** - * Generates file location based on the task configuration and a specific task id. - * This file will be used to store the data required to generate the Iceberg commit. - * Currently it uses TABLE_LOCATION/temp/QUERY_ID-jobId/task-[0..numTasks).forCommit. - * @param location The location of the table - * @param conf The job's configuration - * @param jobId The jobId for the task - * @param taskId The taskId for the commit file - * @return The file to store the results - */ - private static String generateFileForCommitLocation(String location, Configuration conf, JobID jobId, int taskId) { - return generateJobLocation(location, conf, jobId) + "/task-" + taskId + FOR_COMMIT_EXTENSION; - } - private static void createFileForCommit(FilesForCommit writeResult, String location, FileIO io) throws IOException { OutputFile fileForCommit = io.newOutputFile(location); try (ObjectOutputStream oos = new ObjectOutputStream(fileForCommit.createOrOverwrite())) { @@ -855,7 +824,7 @@ public static List getOutputFiles(List jobContexts) thro for (JobContext jobContext : outputs.get(output)) { Table table = output.table; FileSystem fileSystem = new Path(table.location()).getFileSystem(jobConf); - String jobLocation = generateJobLocation(table.location(), jobConf, jobContext.getJobID()); + String jobLocation = HiveTableUtil.jobLocation(table.location(), jobConf, jobContext.getJobID()); // list jobLocation to get number of forCommit files // we do this because map/reduce num in jobConf is unreliable // and we have no access to vertex status info @@ -903,7 +872,7 @@ public static List> getOutputContentFiles(List jobCon .run(output -> { for (JobContext jobContext : outputs.get(output)) { Table table = output.table; - String jobLocation = generateJobLocation(table.location(), jobConf, jobContext.getJobID()); + String jobLocation = HiveTableUtil.jobLocation(table.location(), jobConf, jobContext.getJobID()); // list jobLocation to get number of forCommit files // we do this because map/reduce num in jobConf is unreliable // and we have no access to vertex status info diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java index 446a02154f89..6f572afd3b85 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy; import org.apache.hadoop.hive.ql.io.IOConstants; import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.mapreduce.JobID; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFiles; @@ -81,6 +82,7 @@ public class HiveTableUtil { private static final Logger LOG = LoggerFactory.getLogger(HiveTableUtil.class); static final String TABLE_EXTENSION = ".table"; + static final String FOR_COMMIT_EXTENSION = ".forCommit"; private HiveTableUtil() { } @@ -294,12 +296,44 @@ public Configuration get() { } } - private static String generateTableObjectLocation(String tableLocation, Configuration conf) { - return tableLocation + "/temp/" + conf.get(HiveConf.ConfVars.HIVE_QUERY_ID.varname) + TABLE_EXTENSION; + /** + * Generates the file location for the serialized table object. + * @param location The location of the table. + * @param conf The configuration containing the query ID. + * @return The file path for the serialized table object. + */ + private static String tableObjectLocation(String location, Configuration conf) { + String queryId = conf.get(HiveConf.ConfVars.HIVE_QUERY_ID.varname); + return location + "/temp/" + queryId + TABLE_EXTENSION; + } + + /** + * Generates the job temp location based on the job configuration. + * @param location The location of the table. + * @param conf The job's configuration. + * @param jobId The JobID for the task. + * @return The directory path for the job's temporary location. + */ + public static String jobLocation(String location, Configuration conf, JobID jobId) { + String queryId = conf.get(HiveConf.ConfVars.HIVE_QUERY_ID.varname); + return location + "/temp/" + queryId + "-" + jobId; + } + + /** + * Generates file location based on the task configuration and a specific task id. + * This file will be used to store the data required to generate the Iceberg commit. + * @param location The location of the table. + * @param conf The job's configuration. + * @param jobId The jobId for the task. + * @param taskId The taskId for the commit file. + * @return The file path for storing the commit data. + */ + static String fileForCommitLocation(String location, Configuration conf, JobID jobId, int taskId) { + return jobLocation(location, conf, jobId) + "/task-" + taskId + FOR_COMMIT_EXTENSION; } static void createFileForTableObject(Table table, Configuration conf) { - String filePath = generateTableObjectLocation(table.location(), conf); + String filePath = tableObjectLocation(table.location(), conf); String bytes = serializeTable(table, conf, null, null); OutputFile serializedTableFile = table.io().newOutputFile(filePath); try (ObjectOutputStream oos = new ObjectOutputStream(serializedTableFile.createOrOverwrite())) { @@ -311,7 +345,7 @@ static void createFileForTableObject(Table table, Configuration conf) { } static void cleanupTableObjectFile(String location, Configuration configuration) { - String filePath = generateTableObjectLocation(location, configuration); + String filePath = tableObjectLocation(location, configuration); Path toDelete = new Path(filePath); try { FileSystem fs = Util.getFs(toDelete, configuration); @@ -322,7 +356,7 @@ static void cleanupTableObjectFile(String location, Configuration configuration) } private static Table readTableObjectFromFile(String location, Configuration config) { - String filePath = generateTableObjectLocation(location, config); + String filePath = tableObjectLocation(location, config); try (FileIO io = new HadoopFileIO(config)) { try (ObjectInputStream ois = new ObjectInputStream(io.newInputFile(filePath).newStream())) { return SerializationUtil.deserializeFromBase64((String) ois.readObject()); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerStub.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerStub.java deleted file mode 100644 index 69d61a58dc32..000000000000 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerStub.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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 org.apache.iceberg.mr.hive; - -import java.util.concurrent.Phaser; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * HiveIcebergStorageHandlerStub is used only for unit tests. - * Currently, we use it to achieve a specific thread interleaving to simulate conflicts in concurrent writes - * deterministically. - */ -public class HiveIcebergStorageHandlerStub extends HiveIcebergStorageHandler { - private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergStorageHandlerStub.class); - - @Override - public HiveIcebergOutputCommitter getOutputCommitter() { - - try { - LOG.debug(" Using HiveIcebergStorageHandlerStub for unit tests"); - if (TestUtilPhaser.isInstantiated()) { - Phaser testUtilPhaser = TestUtilPhaser.getInstance().getPhaser(); - LOG.debug("Activating the Phaser Barrier for thread: {} ", Thread.currentThread().getName()); - testUtilPhaser.arriveAndAwaitAdvance(); - LOG.debug("Breaking the Phaser Barrier and deregistering the phaser for thread: {} ", - Thread.currentThread().getName()); - } - } catch (Exception e) { - throw new RuntimeException("Phaser failed: ", e); - } - - return new HiveIcebergOutputCommitter(); - } - -} diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerWithEngineBase.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerWithEngineBase.java index 99f159d73853..31cd880d4df5 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerWithEngineBase.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerWithEngineBase.java @@ -23,8 +23,10 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; -import java.util.stream.IntStream; +import java.util.function.Predicate; import org.apache.hadoop.hive.common.StatsSetupConst; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.exec.mr.ExecMapper; @@ -33,14 +35,23 @@ import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.data.Record; -import org.apache.iceberg.mr.TestHelper; -import org.apache.iceberg.mr.hive.TestTables.TestTableType; +import org.apache.iceberg.hadoop.ConfigProperties; +import org.apache.iceberg.mr.TestHelper.RecordsBuilder; +import org.apache.iceberg.mr.hive.test.TestHiveShell; +import org.apache.iceberg.mr.hive.test.TestTables; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.concurrent.HiveIcebergStorageHandlerStub; +import org.apache.iceberg.mr.hive.test.concurrent.TestUtilPhaser; +import org.apache.iceberg.mr.hive.test.concurrent.WithMockedStorageHandler; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; +import org.apache.iceberg.relocated.com.google.common.base.Throwables; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Tasks; import org.apache.thrift.TException; import org.junit.After; import org.junit.AfterClass; @@ -52,87 +63,147 @@ import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import static org.apache.iceberg.mr.hive.TestTables.ALL_TABLE_TYPES; -import static org.apache.iceberg.mr.hive.TestTables.TestTableType.HIVE_CATALOG; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; -import static org.junit.Assume.assumeTrue; import static org.junit.runners.Parameterized.Parameter; import static org.junit.runners.Parameterized.Parameters; @RunWith(Parameterized.class) public abstract class HiveIcebergStorageHandlerWithEngineBase { + private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergStorageHandlerWithEngineBase.class); + public static final String RETRY_STRATEGIES = "overlay,reoptimize,reexecute_lost_am,dagsubmit,recompile_without_cbo,write_conflict"; public static final String RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT = "overlay,reoptimize,reexecute_lost_am," + "dagsubmit,recompile_without_cbo"; - protected static final Schema ORDER_SCHEMA = new Schema( + protected static final Schema ORDER_SCHEMA = + new Schema( required(1, "order_id", Types.LongType.get()), required(2, "customer_id", Types.LongType.get()), required(3, "total", Types.DoubleType.get()), required(4, "product_id", Types.LongType.get()) - ); + ); - protected static final List ORDER_RECORDS = TestHelper.RecordsBuilder.newInstance(ORDER_SCHEMA) + protected static final List ORDER_RECORDS = + RecordsBuilder.newInstance(ORDER_SCHEMA) .add(100L, 0L, 11.11d, 1L) .add(101L, 0L, 22.22d, 2L) .add(102L, 1L, 33.33d, 3L) .build(); - protected static final Schema PRODUCT_SCHEMA = new Schema( + protected static final Schema PRODUCT_SCHEMA = + new Schema( optional(1, "id", Types.LongType.get()), optional(2, "name", Types.StringType.get()), optional(3, "price", Types.DoubleType.get()) - ); + ); - protected static final List PRODUCT_RECORDS = TestHelper.RecordsBuilder.newInstance(PRODUCT_SCHEMA) + protected static final List PRODUCT_RECORDS = + RecordsBuilder.newInstance(PRODUCT_SCHEMA) .add(1L, "skirt", 11.11d) .add(2L, "tee", 22.22d) .add(3L, "watch", 33.33d) .build(); protected static final List SUPPORTED_TYPES = - ImmutableList.of(Types.BooleanType.get(), Types.IntegerType.get(), Types.LongType.get(), - Types.FloatType.get(), Types.DoubleType.get(), Types.DateType.get(), Types.TimestampType.withZone(), - Types.TimestampType.withoutZone(), Types.StringType.get(), Types.BinaryType.get(), - Types.DecimalType.of(3, 1), Types.UUIDType.get(), Types.FixedType.ofLength(5), - Types.TimeType.get()); - - protected static final Map STATS_MAPPING = ImmutableMap.of( - StatsSetupConst.NUM_FILES, SnapshotSummary.TOTAL_DATA_FILES_PROP, - StatsSetupConst.ROW_COUNT, SnapshotSummary.TOTAL_RECORDS_PROP, - StatsSetupConst.TOTAL_SIZE, SnapshotSummary.TOTAL_FILE_SIZE_PROP - ); + ImmutableList.of( + Types.BooleanType.get(), Types.IntegerType.get(), Types.LongType.get(), + Types.FloatType.get(), Types.DoubleType.get(), Types.DateType.get(), Types.TimestampType.withZone(), + Types.TimestampType.withoutZone(), Types.StringType.get(), Types.BinaryType.get(), + Types.DecimalType.of(3, 1), Types.UUIDType.get(), Types.FixedType.ofLength(5), + Types.TimeType.get() + ); + + protected static final Map STATS_MAPPING = + ImmutableMap.of( + StatsSetupConst.NUM_FILES, SnapshotSummary.TOTAL_DATA_FILES_PROP, + StatsSetupConst.ROW_COUNT, SnapshotSummary.TOTAL_RECORDS_PROP, + StatsSetupConst.TOTAL_SIZE, SnapshotSummary.TOTAL_FILE_SIZE_PROP + ); @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") public static Collection parameters() { Collection testParams = Lists.newArrayList(); - // Run tests with every FileFormat for a single Catalog (HiveCatalog) + // HiveCatalog combinations for (FileFormat fileFormat : HiveIcebergStorageHandlerTestUtils.FILE_FORMATS) { - IntStream.of(2, 1).forEach(formatVersion -> { - testParams.add(new Object[]{fileFormat, HIVE_CATALOG, false, formatVersion}); - // test for vectorization=ON in case of ORC and PARQUET format - if (fileFormat != FileFormat.METADATA) { - testParams.add(new Object[]{fileFormat, HIVE_CATALOG, true, formatVersion}); - } - }); + addHiveCatalogParams(testParams, fileFormat); + } + + // Other catalogs (PARQUET only) + for (TestTableType testTableType : TestTables.ALL_TABLE_TYPES) { + addNonHiveCatalogParams(testParams, testTableType); } - // Run tests for every Catalog for a single FileFormat (PARQUET), skip HiveCatalog tests as they are added before - for (TestTableType testTableType : ALL_TABLE_TYPES) { - if (testTableType != HIVE_CATALOG) { - testParams.add(new Object[]{FileFormat.PARQUET, testTableType, false, 1}); + return testParams; + } + + private static void addHiveCatalogParams( + Collection params, FileFormat fileFormat) { + + List vectorizationModes = + fileFormat == FileFormat.METADATA ? + List.of(false) : + List.of(false, true); + + for (int formatVersion = 1; formatVersion <= 3; formatVersion++) { + for (boolean isVectorized : vectorizationModes) { + params.add(new Object[]{ + fileFormat, TestTableType.HIVE_CATALOG, isVectorized, formatVersion + }); + } + } + } + + private static void addNonHiveCatalogParams( + Collection params, TestTableType testTableType) { + + if (testTableType == TestTableType.HIVE_CATALOG) { + return; + } + // Non-Hive catalogs: PARQUET, non-vectorized, formatVersion=1 + params.add(new Object[]{ + FileFormat.PARQUET, testTableType, false, 1 + }); + } + + /** + * Helper method for child test classes to filter the base parameters. + * @param filter predicate to filter test parameters + * @return filtered collection of test parameters + */ + protected static Collection getParameters(Predicate filter) { + Collection testParams = Lists.newArrayList(); + + for (Object[] params : parameters()) { + TestParams tp = TestParams.from(params); + if (filter.test(tp)) { + testParams.add(params); } } return testParams; } + public record TestParams( + FileFormat fileFormat, TestTableType testTableType, boolean isVectorized, int formatVersion) { + + public static TestParams from(Object[] params) { + return new TestParams( + (FileFormat) params[0], + (TestTableType) params[1], + (boolean) params[2], + (int) params[3] + ); + } + } + protected static TestHiveShell shell; protected TestTables testTables; @@ -155,6 +226,9 @@ public static Collection parameters() { @Rule public Timeout timeout = new Timeout(500_000, TimeUnit.MILLISECONDS); + @Rule + public WithMockedStorageHandler.Rule mockedStorageHandlerRule = new WithMockedStorageHandler.Rule(); + @BeforeClass public static void beforeClass() { shell = HiveIcebergStorageHandlerTestUtils.shell(); @@ -167,21 +241,13 @@ public static void afterClass() throws Exception { @Before public void before() throws IOException { - validateTestParams(); testTables = HiveIcebergStorageHandlerTestUtils.testTables(shell, testTableType, temp); HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); // Fetch task conversion might kick in for certain queries preventing vectorization code path to be used, so // we turn it off explicitly to achieve better coverage. - if (isVectorized) { - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); - } else { - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "more"); - } - } - - protected void validateTestParams() { - assumeTrue(formatVersion == 1); + HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, + isVectorized ? "none" : "more"); } @After @@ -210,4 +276,84 @@ protected void validateBasicStats(Table icebergTable, String dbName, String tabl Assert.assertEquals(summary.get(entry.getValue()), hmsParams.get(entry.getKey())); } } + + /** + * Executes multiple SQL queries concurrently with controlled synchronization for testing. + * + *

This method supports two synchronization modes: + *

    + *
  • Ext-locking mode ({@code useExtLocking=true}): Queries execute in strict sequential + * order (sql[0], then sql[1], then sql[2], ...) to verify that external table locking prevents + * concurrent execution. Each query waits for its turn before starting.
  • + *
  • Barrier synchronization mode ({@code useExtLocking=false}): Queries execute concurrently, + * then commit in order to test optimistic concurrency control and retry on write conflicts.
  • + *
+ * + *

Uses {@link TestUtilPhaser} and {@link HiveIcebergStorageHandlerStub} to coordinate thread + * execution order deterministically. + * + * @param useExtLocking if true, enables external locking mode with sequential execution; + * if false, enables concurrent execution with ordered commits + * @param retryStrategies comma-separated list of Hive query retry strategies to enable + * @param sql array of SQL queries to execute. If single query provided, it's executed by all threads. + * If multiple queries provided, each thread executes sql[i] where i is the thread index. + * @throws Exception if any query execution fails + */ + protected void executeConcurrently( + boolean useExtLocking, String retryStrategies, String... sql) throws Exception { + + int nThreads = sql.length > 1 ? sql.length : 2; + TestUtilPhaser testUtilPhaser = TestUtilPhaser.getInstance(); + + try (ExecutorService executor = + Executors.newVirtualThreadPerTaskExecutor()) { + Tasks.range(nThreads) + .executeWith(executor) + .run(i -> { + LOG.debug("Thread {} started for query index {}", Thread.currentThread().getName(), i); + + TestUtilPhaser.ThreadContext.setQueryIndex(i); + + if (useExtLocking) { + TestUtilPhaser.ThreadContext.setUseExtLocking(true); + } else { + testUtilPhaser.register(); + } + + HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp); + HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); + HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); + HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_OPTIMIZE_METADATA_DELETE, false); + HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, + retryStrategies); + + if (useExtLocking) { + HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_TXN_EXT_LOCKING_ENABLED, true); + shell.getHiveConf().setBoolean(ConfigProperties.LOCK_HIVE_ENABLED, false); + + // Wait for turn: sql[0] waits for phase 0 -> 1, sql[1] for 1 -> 2, etc. + // Phase advances when previous query validates snapshot. + LOG.debug("Thread {} (queryIndex={}) waiting for turn", + Thread.currentThread().getName(), i); + testUtilPhaser.awaitTurn(); + } + + try { + shell.executeStatement(sql.length > 1 ? sql[i] : sql[0]); + LOG.debug("Thread {} (queryIndex={}) completed statement execution", + Thread.currentThread().getName(), i); + } finally { + shell.closeSession(); + } + }); + } catch (Exception ex) { + Throwable root = Throwables.getRootCause(ex); + if (root instanceof Exception) { + throw (Exception) root; + } + throw new RuntimeException(root); + } finally { + TestUtilPhaser.destroyInstance(); + } + } } diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestConflictingDataFiles.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestConflictingDataFiles.java index fb5b44eb062a..288ab402daca 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestConflictingDataFiles.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestConflictingDataFiles.java @@ -20,139 +20,98 @@ package org.apache.iceberg.mr.hive; +import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.concurrent.Executors; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.data.Record; import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.hadoop.ConfigProperties; -import org.apache.iceberg.hive.HiveTableOperations; import org.apache.iceberg.mr.TestHelper; -import org.apache.iceberg.relocated.com.google.common.base.Throwables; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.concurrent.WithMockedStorageHandler; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.Tasks; -import org.junit.After; import org.junit.Assert; import org.junit.Assume; import org.junit.Before; import org.junit.Test; -import org.mockito.MockedStatic; +import org.junit.runners.Parameterized.Parameters; -import static org.apache.iceberg.mr.hive.HiveIcebergStorageHandlerTestUtils.init; import static org.apache.iceberg.types.Types.NestedField.required; -import static org.mockito.Mockito.CALLS_REAL_METHODS; -import static org.mockito.Mockito.mockStatic; +@WithMockedStorageHandler public class TestConflictingDataFiles extends HiveIcebergStorageHandlerWithEngineBase { - private final String storageHandlerStub = "'org.apache.iceberg.mr.hive.HiveIcebergStorageHandlerStub'"; + public static final String STORAGE_HANDLER_STUB = + "'org.apache.iceberg.mr.hive.test.concurrent.HiveIcebergStorageHandlerStub'"; + + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.fileFormat() == FileFormat.PARQUET && + p.testTableType() == TestTableType.HIVE_CATALOG && + p.isVectorized()); + } @Before - public void setUpTables() throws NoSuchMethodException { + public void setUpTables() { PartitionSpec spec = PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA).identity("last_name") .bucket("customer_id", 16).build(); -// Method method = HiveTableOperations.class.getDeclaredMethod("setStorageHandler", Map.class, Boolean.TYPE); -// method.setAccessible(true); + // create and insert an initial batch of records + testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, + spec, fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, + formatVersion, Collections.emptyMap(), STORAGE_HANDLER_STUB); - try (MockedStatic tableOps = mockStatic(HiveTableOperations.class, CALLS_REAL_METHODS)) { -// tableOps.when(() -> method.invoke(null, anyMap(), eq(true))) -// .thenAnswer(invocation -> null); - // create and insert an initial batch of records - testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, spec, - fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, 2, Collections.emptyMap(), - storageHandlerStub); - } // insert one more batch so that we have multiple data files within the same partition shell.executeStatement(testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1, TableIdentifier.of("default", "customers"), false)); - TestUtilPhaser.getInstance(); - } - - @After - public void destroyTestSetUp() { - TestUtilPhaser.destroyInstance(); - } - - @Override - protected void validateTestParams() { - Assume.assumeTrue(fileFormat.equals(FileFormat.PARQUET) && isVectorized && - testTableType.equals(TestTables.TestTableType.HIVE_CATALOG)); } @Test - public void testSingleFilterUpdate() { - String[] singleFilterQuery = new String[] { "UPDATE customers SET first_name='Changed' WHERE last_name='Taylor'", - "UPDATE customers SET first_name='Changed' WHERE last_name='Donnel'" }; + public void testSingleFilterUpdate() throws Exception { + Assume.assumeTrue(formatVersion >= 2); - try { - Tasks.range(2).executeWith(Executors.newFixedThreadPool(2)).run(i -> { - TestUtilPhaser.getInstance().getPhaser().register(); - init(shell, testTables, temp); - HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, - RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT); - shell.executeStatement(singleFilterQuery[i]); - shell.closeSession(); - }); - List objects = - shell.executeStatement("SELECT * FROM customers ORDER BY customer_id, last_name, first_name"); - Assert.assertEquals(12, objects.size()); - List expected = TestHelper.RecordsBuilder.newInstance(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) - .add(1L, "Joanna", "Pierce") - .add(1L, "Changed", "Taylor") - .add(2L, "Changed", "Donnel") - .add(2L, "Susan", "Morrison") - .add(2L, "Bob", "Silver") - .add(2L, "Joanna", "Silver") - .add(3L, "Marci", "Barna") - .add(3L, "Blake", "Burr") - .add(3L, "Trudy", "Henderson") - .add(3L, "Trudy", "Johnson") - .add(4L, "Laci", "Zold") - .add(5L, "Peti", "Rozsaszin").build(); - HiveIcebergTestUtils.validateData(expected, - HiveIcebergTestUtils.valueForRow(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, objects), 0); - - } catch (Throwable ex) { - Throwable cause = Throwables.getRootCause(ex); - Assert.fail(String.valueOf(cause)); - } + String[] sql = new String[] { + "UPDATE customers SET first_name='Changed' WHERE last_name='Taylor'", + "UPDATE customers SET first_name='Changed' WHERE last_name='Donnel'" + }; + executeConcurrently(false, RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT, sql); + + List objects = + shell.executeStatement("SELECT * FROM customers ORDER BY customer_id, last_name, first_name"); + Assert.assertEquals(12, objects.size()); + List expected = TestHelper.RecordsBuilder.newInstance(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) + .add(1L, "Joanna", "Pierce") + .add(1L, "Changed", "Taylor") + .add(2L, "Changed", "Donnel") + .add(2L, "Susan", "Morrison") + .add(2L, "Bob", "Silver") + .add(2L, "Joanna", "Silver") + .add(3L, "Marci", "Barna") + .add(3L, "Blake", "Burr") + .add(3L, "Trudy", "Henderson") + .add(3L, "Trudy", "Johnson") + .add(4L, "Laci", "Zold") + .add(5L, "Peti", "Rozsaszin").build(); + HiveIcebergTestUtils.validateData(expected, + HiveIcebergTestUtils.valueForRow(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, objects), 0); } @Test - public void testMultiFiltersUpdate() { + public void testMultiFiltersUpdate() throws Exception { + Assume.assumeTrue(formatVersion >= 2); - String[] multiFilterQuery = - new String[] { "UPDATE customers SET first_name='Changed' WHERE last_name='Henderson' OR last_name='Johnson'", - "UPDATE customers SET first_name='Changed' WHERE last_name='Taylor' AND customer_id=1" }; - - try { - Tasks.range(2).executeWith(Executors.newFixedThreadPool(2)).run(i -> { - TestUtilPhaser.getInstance().getPhaser().register(); - init(shell, testTables, temp); - HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, - RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT); - shell.executeStatement(multiFilterQuery[i]); - shell.closeSession(); - }); - } catch (Throwable ex) { - // If retry succeeds then it should not throw an ValidationException. - Throwable cause = Throwables.getRootCause(ex); - Assert.assertTrue(cause instanceof ValidationException); - if (cause.getMessage().matches("^Found.*conflicting.*files(.*)")) { - Assert.fail(); - } - } + String[] sql = new String[] { + "UPDATE customers SET first_name='Changed' WHERE last_name='Henderson' OR last_name='Johnson'", + "UPDATE customers SET first_name='Changed' WHERE last_name='Taylor' AND customer_id=1" + }; + executeConcurrently(false, RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT, sql); List objects = shell.executeStatement("SELECT * FROM customers ORDER BY customer_id, last_name, first_name"); @@ -176,29 +135,13 @@ public void testMultiFiltersUpdate() { } @Test - public void testDeleteFilters() { - String[] sql = new String[] { "DELETE FROM customers WHERE last_name='Taylor'", + public void testDeleteFilters() throws Exception { + String[] sql = new String[] { + "DELETE FROM customers WHERE last_name='Taylor'", "DELETE FROM customers WHERE last_name='Donnel'", - "DELETE FROM customers WHERE last_name='Henderson' OR last_name='Johnson'" }; - - try { - Tasks.range(3).executeWith(Executors.newFixedThreadPool(3)).run(i -> { - TestUtilPhaser.getInstance().getPhaser().register(); - init(shell, testTables, temp); - HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, - RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT); - shell.executeStatement(sql[i]); - shell.closeSession(); - }); - } catch (Throwable ex) { - Throwable cause = Throwables.getRootCause(ex); - Assert.assertTrue(cause instanceof ValidationException); - if (cause.getMessage().matches("^Found.*conflicting.*files(.*)")) { - Assert.fail(); - } - } + "DELETE FROM customers WHERE last_name='Henderson' OR last_name='Johnson'" + }; + executeConcurrently(false, RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT, sql); List objects = shell.executeStatement("SELECT * FROM customers ORDER BY customer_id, last_name, first_name"); @@ -215,32 +158,49 @@ public void testDeleteFilters() { .build(); HiveIcebergTestUtils.validateData(expected, HiveIcebergTestUtils.valueForRow(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, objects), 0); - TestUtilPhaser.destroyInstance(); } @Test - public void testConflictingUpdates() { - String[] singleFilterQuery = new String[] { "UPDATE customers SET first_name='Changed' WHERE last_name='Taylor'", - "UPDATE customers SET first_name='Changed' WHERE last_name='Taylor'" }; + public void testConflictingDeletes() throws Exception { + String[] sql = new String[]{ + "DELETE FROM customers WHERE customer_id=3 or first_name='Joanna'", + "DELETE FROM customers WHERE last_name='Johnson'" + }; try { - Tasks.range(2).executeWith(Executors.newFixedThreadPool(2)).run(i -> { - TestUtilPhaser.getInstance().getPhaser().register(); - init(shell, testTables, temp); - HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, - RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT); - shell.executeStatement(singleFilterQuery[i]); - shell.closeSession(); - }); - } catch (Throwable ex) { - // Since there is a conflict it should throw ValidationException - Throwable cause = Throwables.getRootCause(ex); - Assert.assertTrue(cause instanceof ValidationException); - Assert.assertTrue(cause.getMessage().matches("^Found.*conflicting" + ".*files(.*)")); + executeConcurrently(false, RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT, sql); + } catch (ValidationException ex) { + if (formatVersion == 2) { + Assert.fail("Unexpected ValidationException for format version 2"); + } + Assert.assertTrue(ex.getMessage().startsWith( + formatVersion == 3 ? "Found concurrently added DV" : "Found conflicting files")); } + List objects = + shell.executeStatement("SELECT * FROM customers ORDER BY customer_id, last_name, first_name"); + Assert.assertEquals(6, objects.size()); + List expected = TestHelper.RecordsBuilder.newInstance(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) + .add(1L, "Sharon", "Taylor") + .add(2L, "Jake", "Donnel") + .add(2L, "Susan", "Morrison") + .add(2L, "Bob", "Silver") + .add(4L, "Laci", "Zold") + .add(5L, "Peti", "Rozsaszin") + .build(); + HiveIcebergTestUtils.validateData(expected, + HiveIcebergTestUtils.valueForRow(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, objects), 0); + } + + @Test + public void testConflictingUpdates() { + String sql = "UPDATE customers SET first_name='Changed' " + + "WHERE last_name='Taylor'"; + + Throwable ex = Assert.assertThrows(ValidationException.class, + () -> executeConcurrently(false, RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT, sql)); + Assert.assertTrue(ex.getMessage().startsWith("Found conflicting files")); + List objects = shell.executeStatement("SELECT * FROM customers ORDER BY customer_id, last_name, first_name"); Assert.assertEquals(12, objects.size()); @@ -263,8 +223,47 @@ public void testConflictingUpdates() { } @Test - public void testConcurrentInsertAndInsertOverwrite() { - Assume.assumeTrue(formatVersion == 2); + public void testConflictingUpdateAndDelete() { + Assume.assumeTrue(formatVersion >= 2); + + String[] sql = new String[]{ + "DELETE FROM customers WHERE customer_id=3 or first_name='Joanna'", + "UPDATE customers SET last_name='Changed' WHERE customer_id=3 or first_name='Joanna'" + }; + + Throwable ex = Assert.assertThrows(ValidationException.class, + () -> executeConcurrently(false, RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT, sql)); + Assert.assertTrue(ex.getMessage().startsWith("Found new conflicting delete files")); + + List res = shell.executeStatement("SELECT count(*) FROM customers " + + "WHERE customer_id != 3 and first_name != 'Joanna'"); + Assert.assertEquals(6L, res.getFirst()[0]); + } + + @Test + public void testConflictingMergeInserts() { + testTables.createTable(shell, "source", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, + PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1); + + testTables.createTable(shell, "target", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, + PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, + formatVersion, Collections.emptyMap(), STORAGE_HANDLER_STUB); + + String sql = "MERGE INTO target t USING source src ON t.customer_id = src.customer_id " + + "WHEN NOT MATCHED THEN " + + "INSERT values (src.customer_id, src.first_name, src.last_name)"; + + Throwable ex = Assert.assertThrows(ValidationException.class, + () -> executeConcurrently(false, RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT, sql)); + Assert.assertTrue(ex.getMessage().startsWith("Found conflicting files")); + + List res = shell.executeStatement("SELECT count(*) FROM target"); + Assert.assertEquals(6L, res.getFirst()[0]); + } + + @Test + public void testConcurrentInsertAndOverwrite() throws Exception { + Assume.assumeTrue(formatVersion >= 2); Schema schema = new Schema( required(1, "i", Types.IntegerType.get()), @@ -282,31 +281,13 @@ public void testConcurrentInsertAndInsertOverwrite() { .add(40, 40) .add(30, 30) .build(), - formatVersion); - - String[] singleFilterQuery = new String[] { "INSERT INTO ice_t SELECT i*100, p*100 FROM ice_t", - "INSERT OVERWRITE TABLE ice_t SELECT i+1, p+1 FROM ice_t" }; - - Tasks.range(2).executeWith(Executors.newFixedThreadPool(2)).run(i -> { - if (i == 1) { - try { - Thread.sleep(100); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - init(shell, testTables, temp); - HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); - - HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_TXN_EXT_LOCKING_ENABLED, true); - shell.getHiveConf().setBoolean(ConfigProperties.LOCK_HIVE_ENABLED, false); - - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, - RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT); - shell.executeStatement(singleFilterQuery[i]); - shell.closeSession(); - }); + formatVersion, Collections.emptyMap(), STORAGE_HANDLER_STUB); + + String[] sql = new String[] { + "INSERT INTO ice_t SELECT i*100, p*100 FROM ice_t", + "INSERT OVERWRITE TABLE ice_t SELECT i+1, p+1 FROM ice_t" + }; + executeConcurrently(true, RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT, sql); List objects = shell.executeStatement("SELECT * FROM ice_t"); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestDeserializer.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestDeserializer.java index e976483b81a1..33ae1c74e78e 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestDeserializer.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestDeserializer.java @@ -33,6 +33,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.hive.HiveVersion; import org.apache.iceberg.mr.hive.serde.objectinspector.IcebergObjectInspector; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Assume; diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergBranchOperation.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergBranchOperation.java index 335c463423e8..3d515a98ddda 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergBranchOperation.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergBranchOperation.java @@ -20,23 +20,29 @@ package org.apache.iceberg.mr.hive; import java.io.IOException; +import java.util.Collection; import java.util.concurrent.TimeUnit; import org.apache.iceberg.FileFormat; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; import org.assertj.core.api.Assertions; import org.junit.Assert; -import org.junit.Assume; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; -import static org.apache.iceberg.mr.hive.HiveIcebergTestUtils.timestampAfterSnapshot; +import static org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils.timestampAfterSnapshot; public class TestHiveIcebergBranchOperation extends HiveIcebergStorageHandlerWithEngineBase { - @Override - protected void validateTestParams() { - Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG && formatVersion == 2); + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.fileFormat() == FileFormat.PARQUET && + p.testTableType() == TestTableType.HIVE_CATALOG && + p.isVectorized() && + p.formatVersion() == 2); } @Test diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCRUD.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCRUD.java index e51015164612..1826d241a012 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCRUD.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCRUD.java @@ -21,9 +21,7 @@ import java.io.IOException; import java.util.List; -import java.util.concurrent.Executors; import java.util.stream.StreamSupport; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; @@ -34,20 +32,19 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.data.Record; import org.apache.iceberg.deletes.PositionDelete; -import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.mr.TestHelper; -import org.apache.iceberg.relocated.com.google.common.base.Throwables; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.Tasks; import org.apache.thrift.TException; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; -import static org.apache.iceberg.mr.hive.HiveIcebergStorageHandlerTestUtils.init; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; @@ -56,10 +53,6 @@ */ public class TestHiveIcebergCRUD extends HiveIcebergStorageHandlerWithEngineBase { - @Override - protected void validateTestParams() { - } - @Test public void testReadAndWriteFormatV2UnpartitionedWithEqDelete() throws IOException { Assume.assumeTrue("Reading V2 tables with eq delete files are only supported currently in " + @@ -602,142 +595,10 @@ public void testUpdateForSupportedTypes() throws IOException { } } - @Test - public void testConcurrent2Deletes() { - Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG); - - testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, - formatVersion); - String sql = "DELETE FROM customers WHERE customer_id=3 or first_name='Joanna'"; - - try { - Tasks.range(2) - .executeWith(Executors.newFixedThreadPool(2)) - .run(i -> { - init(shell, testTables, temp); - HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, - RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT); - shell.executeStatement(sql); - shell.closeSession(); - }); - } catch (Throwable ex) { - Assert.assertEquals(1, (int) formatVersion); - Throwable cause = Throwables.getRootCause(ex); - Assert.assertTrue(cause instanceof ValidationException); - Assert.assertTrue(cause.getMessage().startsWith("Found conflicting files")); - } - List res = shell.executeStatement("SELECT * FROM customers"); - Assert.assertEquals(4, res.size()); - } - - @Test - public void testConcurrent2Updates() { - Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG); - - testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, - formatVersion); - String sql = "UPDATE customers SET last_name='Changed' WHERE customer_id=3 or first_name='Joanna'"; - try { - Tasks.range(2) - .executeWith(Executors.newFixedThreadPool(2)) - .run(i -> { - init(shell, testTables, temp); - HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, - RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT); - shell.executeStatement(sql); - shell.closeSession(); - }); - } catch (Throwable ex) { - Throwable cause = Throwables.getRootCause(ex); - Assert.assertTrue(cause instanceof ValidationException); - Assert.assertTrue(cause.getMessage().matches("^Found.*conflicting.*files(.*)")); - } - List res = shell.executeStatement("SELECT * FROM customers WHERE last_name='Changed'"); - Assert.assertEquals(5, res.size()); - } - - @Test - public void testConcurrentUpdateAndDelete() { - Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG && formatVersion == 2); - - testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, - formatVersion); - String[] sql = new String[]{ - "DELETE FROM customers WHERE customer_id=3 or first_name='Joanna'", - "UPDATE customers SET last_name='Changed' WHERE customer_id=3 or first_name='Joanna'" - }; - - boolean deleteFirst = false; - try { - Tasks.range(2) - .executeWith(Executors.newFixedThreadPool(2)) - .run(i -> { - init(shell, testTables, temp); - HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, - RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT); - shell.executeStatement(sql[i]); - shell.closeSession(); - }); - } catch (Throwable ex) { - Throwable cause = Throwables.getRootCause(ex); - Assert.assertTrue(cause instanceof ValidationException); - Assert.assertTrue(cause.getMessage().matches("^Found.*conflicting.*files(.*)")); - deleteFirst = cause.getMessage().contains("conflicting delete"); - } - List res = shell.executeStatement("SELECT * FROM customers WHERE last_name='Changed'"); - Assert.assertEquals(deleteFirst ? 0 : 5, res.size()); - } - - @Test - public void testConcurrent2MergeInserts() { - Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG); - - testTables.createTable(shell, "source", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1); - testTables.createTable(shell, "target", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, - PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, - formatVersion); - - String sql = "MERGE INTO target t USING source s on t.customer_id = s.customer_id WHEN Not MATCHED THEN " + - "INSERT values (s.customer_id, s.first_name, s.last_name)"; - try { - Tasks.range(2) - .executeWith(Executors.newFixedThreadPool(2)) - .run(i -> { - init(shell, testTables, temp); - HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, - RETRY_STRATEGIES_WITHOUT_WRITE_CONFLICT); - shell.executeStatement(sql); - shell.closeSession(); - }); - } catch (Throwable ex) { - Throwable cause = Throwables.getRootCause(ex); - Assert.assertTrue(cause instanceof ValidationException); - Assert.assertTrue(cause.getMessage().startsWith("Found conflicting files")); - } - List res = shell.executeStatement("SELECT * FROM target"); - Assert.assertEquals(6, res.size()); - } - @Test public void testMultiInsert() { Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG); + testTableType == TestTableType.HIVE_CATALOG); testTables.createTable(shell, "source", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCTAS.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCTAS.java index 4077fd66f0b3..b9c3e302e457 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCTAS.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergCTAS.java @@ -20,6 +20,7 @@ package org.apache.iceberg.mr.hive; import java.io.IOException; +import java.util.Collection; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -35,13 +36,16 @@ import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.mr.InputFormatConfig; import org.apache.iceberg.mr.TestHelper; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.thrift.TException; import org.junit.Assert; -import org.junit.Assume; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; import static org.apache.iceberg.types.Types.NestedField.optional; @@ -50,10 +54,12 @@ */ public class TestHiveIcebergCTAS extends HiveIcebergStorageHandlerWithEngineBase { - @Override - protected void validateTestParams() { - Assume.assumeTrue(HiveIcebergSerDe.CTAS_EXCEPTION_MSG, testTableType == TestTables.TestTableType.HIVE_CATALOG && - isVectorized && formatVersion == 1); + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.testTableType() == TestTableType.HIVE_CATALOG && + p.isVectorized() && + p.formatVersion() == 2); } @Test @@ -377,7 +383,7 @@ public void testCTASForAllColumnTypes() { public void testCTASAndCTLTWithAuth() { shell.setHiveSessionValue("hive.security.authorization.enabled", true); shell.setHiveSessionValue("hive.security.authorization.manager", - "org.apache.iceberg.mr.hive.CustomTestHiveAuthorizerFactory"); + "org.apache.iceberg.mr.hive.test.CustomTestHiveAuthorizerFactory"); shell.setHiveSessionValue("hive.security.authorization.tables.on.storagehandlers", true); TableIdentifier identifier = TableIdentifier.of("default", "customers"); String query = String.format("CREATE EXTERNAL TABLE customers (" + diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergComplexTypeWrites.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergComplexTypeWrites.java index fbc77d5c4ad6..a45cb3d12818 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergComplexTypeWrites.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergComplexTypeWrites.java @@ -20,6 +20,7 @@ package org.apache.iceberg.mr.hive; import java.io.IOException; +import java.util.Collection; import java.util.List; import java.util.Map; import org.apache.iceberg.Schema; @@ -27,11 +28,13 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.mr.TestHelper; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; import static org.apache.iceberg.types.Types.NestedField.required; @@ -40,6 +43,12 @@ */ public class TestHiveIcebergComplexTypeWrites extends HiveIcebergStorageHandlerWithEngineBase { + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.isVectorized() && p.formatVersion() == 2); + } + @Test public void testWriteArrayOfPrimitivesInTable() throws IOException { Schema schema = new Schema(required(1, "id", Types.LongType.get()), diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java index ac8eeeb71b2f..50f1c9dc1ce9 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergExpireSnapshots.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.text.SimpleDateFormat; +import java.util.Collection; import java.util.Date; import java.util.List; import org.apache.commons.collections4.IterableUtils; @@ -32,9 +33,12 @@ import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.data.Record; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.junit.Assert; -import org.junit.Assume; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; import static org.apache.iceberg.TableProperties.MAX_SNAPSHOT_AGE_MS; import static org.apache.iceberg.TableProperties.MIN_SNAPSHOTS_TO_KEEP; @@ -44,10 +48,13 @@ */ public class TestHiveIcebergExpireSnapshots extends HiveIcebergStorageHandlerWithEngineBase { - @Override - protected void validateTestParams() { - Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG && formatVersion == 2); + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.fileFormat() == FileFormat.PARQUET && + p.testTableType() == TestTableType.HIVE_CATALOG && + p.isVectorized() && + p.formatVersion() == 2); } @Test diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergInserts.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergInserts.java index 73e0f66d8cf7..83b9455c190d 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergInserts.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergInserts.java @@ -20,6 +20,7 @@ package org.apache.iceberg.mr.hive; import java.io.IOException; +import java.util.Collection; import java.util.List; import java.util.Map; import org.apache.iceberg.AssertHelpers; @@ -33,6 +34,9 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.data.Record; import org.apache.iceberg.mr.TestHelper; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -41,6 +45,7 @@ import org.junit.Assert; import org.junit.Assume; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; import static org.apache.iceberg.NullOrder.NULLS_FIRST; import static org.apache.iceberg.NullOrder.NULLS_LAST; @@ -55,6 +60,12 @@ */ public class TestHiveIcebergInserts extends HiveIcebergStorageHandlerWithEngineBase { + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.isVectorized() && p.formatVersion() == 2); + } + @Test public void testSortedInsert() throws IOException { TableIdentifier identifier = TableIdentifier.of("default", "sort_table"); @@ -158,7 +169,7 @@ public void testInsert() throws IOException { @Test public void testInsertIntoORCFile() throws IOException { Assume.assumeTrue("Testing the create table ... stored as ORCFILE syntax is enough for a single scenario.", - testTableType == TestTables.TestTableType.HIVE_CATALOG && fileFormat == FileFormat.ORC); + testTableType == TestTableType.HIVE_CATALOG && fileFormat == FileFormat.ORC); shell.executeStatement("CREATE TABLE t2(c0 DOUBLE , c1 DOUBLE , c2 DECIMAL) STORED BY " + "ICEBERG STORED AS ORCFILE"); shell.executeStatement("INSERT INTO t2(c1, c0) VALUES(0.1803113419993464, 0.9381388537256228)"); @@ -173,7 +184,7 @@ public void testInsertIntoORCFile() throws IOException { @Test public void testStoredByIcebergInTextFile() { Assume.assumeTrue("Testing the create table ... stored as TEXTFILE syntax is enough for a single scenario.", - testTableType == TestTables.TestTableType.HIVE_CATALOG && fileFormat == FileFormat.ORC); + testTableType == TestTableType.HIVE_CATALOG && fileFormat == FileFormat.ORC); AssertHelpers.assertThrows("Create table should not work with textfile", IllegalArgumentException.class, "Unsupported fileformat", () -> @@ -472,7 +483,7 @@ public void testStructMapWithNull() throws IOException { @Test public void testWriteWithDefaultWriteFormat() { Assume.assumeTrue("Testing the default file format is enough for a single scenario.", - testTableType == TestTables.TestTableType.HIVE_CATALOG && fileFormat == FileFormat.ORC); + testTableType == TestTableType.HIVE_CATALOG && fileFormat == FileFormat.ORC); TableIdentifier identifier = TableIdentifier.of("default", "customers"); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergMigration.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergMigration.java index 526b788f638d..29f9f1f75c4a 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergMigration.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergMigration.java @@ -20,6 +20,7 @@ package org.apache.iceberg.mr.hive; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Locale; @@ -35,12 +36,14 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.mr.InputFormatConfig; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.thrift.TException; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; import org.mockito.ArgumentMatchers; import org.mockito.MockedStatic; import org.mockito.Mockito; @@ -52,6 +55,12 @@ */ public class TestHiveIcebergMigration extends HiveIcebergStorageHandlerWithEngineBase { + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.isVectorized() && p.formatVersion() == 2); + } + @Test public void testMigrateHiveTableWithPrimitiveTypeColumnsToIceberg() throws TException, InterruptedException { shell.setHiveSessionValue(InputFormatConfig.SCHEMA_AUTO_CONVERSION, "true"); @@ -91,7 +100,7 @@ public void testMigrateHiveTableWithPrimitiveTypeColumnsToIceberg() throws TExce public void testMigrateHiveTableWithUnsupportedPrimitiveTypeColumnToIceberg() { // enough to test once Assume.assumeTrue(fileFormat == FileFormat.ORC && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG); + testTableType == TestTableType.HIVE_CATALOG); TableIdentifier identifier = TableIdentifier.of("default", "tbl_unsupportedtypes"); shell.executeStatement(String.format("CREATE EXTERNAL TABLE %s (" + "char_col CHAR(10)) STORED AS %s %s %s", identifier.name(), fileFormat.name(), @@ -259,7 +268,7 @@ public void testRollbackMigratePartitionedBucketedHiveTableToIceberg() throws TE public void testMigrationFailsForUnsupportedSourceFileFormat() { // enough to test once Assume.assumeTrue(fileFormat == FileFormat.ORC && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG); + testTableType == TestTableType.HIVE_CATALOG); String tableName = "tbl_unsupported"; List formats = ImmutableList.of("TEXTFILE", "JSONFILE", "RCFILE", "SEQUENCEFILE"); formats.forEach(format -> { @@ -278,7 +287,7 @@ public void testMigrationFailsForUnsupportedSourceFileFormat() { public void testMigrationFailsForManagedTable() { // enough to test once Assume.assumeTrue(fileFormat == FileFormat.ORC && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG); + testTableType == TestTableType.HIVE_CATALOG); String tableName = "tbl_unsupported"; shell.executeStatement("CREATE MANAGED TABLE " + tableName + " (a int) STORED AS " + fileFormat + " " + testTables.locationForCreateTableSQL(TableIdentifier.of("default", tableName)) + @@ -289,8 +298,8 @@ public void testMigrationFailsForManagedTable() { () -> shell.executeStatement("ALTER TABLE " + tableName + " convert to iceberg")); } - private Table validateMigration(String tableName) throws TException, InterruptedException { - return validateMigration(tableName, null); + private void validateMigration(String tableName) throws TException, InterruptedException { + validateMigration(tableName, null); } private Table validateMigration(String tableName, String tblProperties) diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java index 4f65903e12a5..99aab0613432 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java @@ -43,6 +43,7 @@ import org.apache.iceberg.mr.Catalogs; import org.apache.iceberg.mr.InputFormatConfig; import org.apache.iceberg.mr.TestHelper; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.apache.iceberg.mr.hive.writer.HiveIcebergWriter; import org.apache.iceberg.mr.hive.writer.WriterBuilder; import org.apache.iceberg.mr.hive.writer.WriterRegistry; diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergPartitions.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergPartitions.java index f62e86201165..b0360112e7af 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergPartitions.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergPartitions.java @@ -24,6 +24,7 @@ import java.time.LocalDateTime; import java.time.OffsetDateTime; import java.time.ZoneOffset; +import java.util.Collection; import java.util.List; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionKey; @@ -32,11 +33,14 @@ import org.apache.iceberg.Table; import org.apache.iceberg.data.Record; import org.apache.iceberg.mr.TestHelper; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; @@ -47,6 +51,12 @@ */ public class TestHiveIcebergPartitions extends HiveIcebergStorageHandlerWithEngineBase { + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.formatVersion() == 2); + } + @Test public void testPartitionPruning() throws IOException { Schema salesSchema = new Schema( diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergRestrictDataFiles.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergRestrictDataFiles.java index e9d6950ef460..b21af0ff28b2 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergRestrictDataFiles.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergRestrictDataFiles.java @@ -20,18 +20,33 @@ package org.apache.iceberg.mr.hive; import java.io.IOException; +import java.util.Collection; import java.util.Collections; import java.util.List; import org.apache.commons.collections4.ListUtils; import org.apache.iceberg.AssertHelpers; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.junit.BeforeClass; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ICEBERG_ALLOW_DATAFILES_IN_TABLE_LOCATION_ONLY; public class TestHiveIcebergRestrictDataFiles extends HiveIcebergStorageHandlerWithEngineBase { + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.fileFormat() == FileFormat.PARQUET && + p.testTableType() == TestTableType.HIVE_CATALOG && + p.isVectorized() && + p.formatVersion() == 2); + } + @BeforeClass public static void beforeClass() { shell = HiveIcebergStorageHandlerTestUtils.shell( diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergRollback.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergRollback.java index 6ba9a04d1dd2..746381600c92 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergRollback.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergRollback.java @@ -21,14 +21,19 @@ import java.io.IOException; +import java.util.Collection; import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableUtil; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; /** @@ -36,10 +41,13 @@ */ public class TestHiveIcebergRollback extends HiveIcebergStorageHandlerWithEngineBase { - @Override - protected void validateTestParams() { - Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG && formatVersion == 2); + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.fileFormat() == FileFormat.PARQUET && + p.testTableType() == TestTableType.HIVE_CATALOG && + p.isVectorized() && + p.formatVersion() == 2); } @Test @@ -48,12 +56,10 @@ public void testRollbackToTimestamp() throws IOException, InterruptedException { Table table = testTables.createTableWithVersions(shell, identifier.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 3); - /* TODO: re-add test case when Iceberg issue https://github.com/apache/iceberg/issues/5507 is resolved. shell.executeStatement("ALTER TABLE " + identifier.name() + " EXECUTE ROLLBACK('" + HiveIcebergTestUtils.timestampAfterSnapshot(table, 2) + "')"); Assert.assertEquals(5, shell.executeStatement("SELECT * FROM " + identifier.name()).size()); Assert.assertEquals(3, table.history().size()); - */ shell.executeStatement("ALTER TABLE " + identifier.name() + " EXECUTE ROLLBACK('" + HiveIcebergTestUtils.timestampAfterSnapshot(table, 1) + "')"); Assert.assertEquals(4, shell.executeStatement("SELECT * FROM " + identifier.name()).size()); @@ -72,13 +78,11 @@ public void testRollbackToVersion() throws IOException, InterruptedException { Table table = testTables.createTableWithVersions(shell, identifier.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 3); - /* TODO: re-add test case when Iceberg issue https://github.com/apache/iceberg/issues/5507 is resolved. shell.executeStatement("ALTER TABLE " + identifier.name() + " EXECUTE ROLLBACK(" + table.history().get(2).snapshotId() + ")"); Assert.assertEquals(5, shell.executeStatement("SELECT * FROM " + identifier.name()).size()); table.refresh(); Assert.assertEquals(3, table.history().size()); - */ shell.executeStatement("ALTER TABLE " + identifier.name() + " EXECUTE ROLLBACK(" + table.history().get(1).snapshotId() + ")"); Assert.assertEquals(4, shell.executeStatement("SELECT * FROM " + identifier.name()).size()); @@ -94,7 +98,7 @@ public void testRollbackToVersion() throws IOException, InterruptedException { @Test public void testRevertRollback() throws IOException, InterruptedException { Assume.assumeTrue("Rollback revert is only supported for tables from Hive Catalog", - testTableType.equals(TestTables.TestTableType.HIVE_CATALOG)); + testTableType.equals(TestTableType.HIVE_CATALOG)); TableIdentifier identifier = TableIdentifier.of("default", "source"); Table table = testTables.createTableWithVersions(shell, identifier.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, fileFormat, diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSchemaEvolution.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSchemaEvolution.java index 774fbfb52dff..91f0f0133b8f 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSchemaEvolution.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSchemaEvolution.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.math.BigDecimal; +import java.util.Collection; import java.util.Comparator; import java.util.List; import org.apache.hadoop.hive.metastore.api.FieldSchema; @@ -31,6 +32,9 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.hive.HiveSchemaUtil; import org.apache.iceberg.mr.TestHelper; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; @@ -38,6 +42,7 @@ import org.junit.Assert; import org.junit.Assume; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; @@ -50,6 +55,12 @@ */ public class TestHiveIcebergSchemaEvolution extends HiveIcebergStorageHandlerWithEngineBase { + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.formatVersion() == 2); + } + @Test public void testDescribeTable() throws IOException { testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, fileFormat, @@ -90,7 +101,6 @@ public void testAlterChangeColumn() throws IOException { Assert.assertArrayEquals(new Object[]{0L, "Brown"}, result.get(0)); Assert.assertArrayEquals(new Object[]{1L, "Green"}, result.get(1)); Assert.assertArrayEquals(new Object[]{2L, "Pink"}, result.get(2)); - } @Test @@ -264,7 +274,7 @@ public void testDropColumnFromIcebergTable() throws IOException { // drop a column icebergTable.updateSchema().deleteColumn("last_name").commit(); - if (testTableType != TestTables.TestTableType.HIVE_CATALOG) { + if (testTableType != TestTableType.HIVE_CATALOG) { // We need to update columns for non-Hive catalogs shell.executeStatement("ALTER TABLE customers UPDATE COLUMNS"); } @@ -304,7 +314,7 @@ public void testAddColumnToIcebergTable() throws IOException { // Add a new column (age long) to the Iceberg table. icebergTable.updateSchema().addColumn("age", Types.LongType.get()).commit(); - if (testTableType != TestTables.TestTableType.HIVE_CATALOG) { + if (testTableType != TestTableType.HIVE_CATALOG) { // We need to update columns for non-Hive catalogs shell.executeStatement("ALTER TABLE customers UPDATE COLUMNS"); } @@ -371,7 +381,7 @@ public void testAddRequiredColumnToIcebergTable() throws IOException { // Add a new required column (age long) to the Iceberg table. icebergTable.updateSchema().allowIncompatibleChanges().addRequiredColumn("age", Types.LongType.get()).commit(); - if (testTableType != TestTables.TestTableType.HIVE_CATALOG) { + if (testTableType != TestTableType.HIVE_CATALOG) { // We need to update columns for non-Hive catalogs shell.executeStatement("ALTER TABLE customers UPDATE COLUMNS"); } @@ -591,7 +601,7 @@ public void testRenameColumnInIcebergTable() throws IOException { // Rename the last_name column to family_name icebergTable.updateSchema().renameColumn("last_name", "family_name").commit(); - if (testTableType != TestTables.TestTableType.HIVE_CATALOG) { + if (testTableType != TestTableType.HIVE_CATALOG) { // We need to update columns for non-Hive catalogs shell.executeStatement("ALTER TABLE customers UPDATE COLUMNS"); } @@ -814,7 +824,7 @@ public void testUpdateColumnTypeInIcebergTable() throws IOException, TException, // Do this check only for Hive catalog, as this is the only case when the column types are updated // in the metastore. In case of other catalog types, the table is not updated in the metastore, // so no point in checking the column types. - if (TestTables.TestTableType.HIVE_CATALOG.equals(this.testTableType)) { + if (TestTableType.HIVE_CATALOG.equals(this.testTableType)) { table = shell.metastore().getTable("default", "types_table"); Assert.assertNotNull(table); Assert.assertNotNull(table.getSd()); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSelects.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSelects.java index 8a1cf740067b..f82a2f449ade 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSelects.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSelects.java @@ -20,6 +20,7 @@ package org.apache.iceberg.mr.hive; import java.io.IOException; +import java.util.Collection; import java.util.List; import java.util.stream.Collectors; import org.apache.iceberg.FileFormat; @@ -29,11 +30,13 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.mr.InputFormatConfig; import org.apache.iceberg.mr.TestHelper; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Test; +import org.junit.runners.Parameterized; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; @@ -46,6 +49,12 @@ */ public class TestHiveIcebergSelects extends HiveIcebergStorageHandlerWithEngineBase { + @Parameterized.Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.formatVersion() == 2); + } + @Test public void testScanTable() throws IOException { testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, fileFormat, @@ -242,7 +251,7 @@ public void testMultiColumnPruning() throws IOException { */ @Test public void testVectorizedOrcMultipleSplits() throws Exception { - assumeTrue(isVectorized && FileFormat.ORC.equals(fileFormat)); + assumeTrue(isVectorized && FileFormat.ORC == fileFormat); // This data will be held by a ~870kB ORC file List records = TestHelper.generateRandomRecords(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSetCurrentSnapshot.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSetCurrentSnapshot.java index 35f252443142..7d376001c570 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSetCurrentSnapshot.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSetCurrentSnapshot.java @@ -20,24 +20,31 @@ package org.apache.iceberg.mr.hive; import java.io.IOException; +import java.util.Collection; import java.util.List; import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.junit.Assert; -import org.junit.Assume; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; /** * Tests setting current snapshot feature */ public class TestHiveIcebergSetCurrentSnapshot extends HiveIcebergStorageHandlerWithEngineBase { - @Override - protected void validateTestParams() { - Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG && formatVersion == 2); + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.fileFormat() == FileFormat.PARQUET && + p.testTableType() == TestTableType.HIVE_CATALOG && + p.isVectorized() && + p.formatVersion() == 2); } @Test diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSnapshotOperations.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSnapshotOperations.java index 55badc7eb5c6..b483df82e431 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSnapshotOperations.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSnapshotOperations.java @@ -22,6 +22,9 @@ import java.io.IOException; import java.util.List; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.mr.hive.test.TestHiveShell; +import org.apache.iceberg.mr.hive.test.TestTables; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.After; import org.junit.Assert; @@ -29,7 +32,7 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import static org.apache.iceberg.mr.hive.TestTables.TestTableType.HIVE_CATALOG; +import static org.apache.iceberg.mr.hive.test.TestTables.TestTableType.HIVE_CATALOG; import static org.junit.Assert.assertEquals; public class TestHiveIcebergSnapshotOperations { diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStatistics.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStatistics.java index 7031f323d292..fc5a2b981f9d 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStatistics.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStatistics.java @@ -32,6 +32,10 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.hadoop.ConfigProperties; +import org.apache.iceberg.mr.hive.test.TestTables; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -42,6 +46,7 @@ import org.junit.Before; import org.junit.Test; import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; /** @@ -53,13 +58,13 @@ public class TestHiveIcebergStatistics extends HiveIcebergStorageHandlerWithEngi @Parameterized.Parameter(4) public String statsSource; - @Parameterized.Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}, statsSource={4}") + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}, statsSource={4}") public static Collection parameters() { - Collection baseParams = HiveIcebergStorageHandlerWithEngineBase.parameters(); - Collection testParams = Lists.newArrayList(); - for (String statsSource : new String[]{"iceberg", "metastore"}) { - for (Object[] params : baseParams) { + + for (Object[] params : HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.isVectorized() && p.formatVersion() == 2)) { + for (String statsSource : new String[]{"iceberg", "metastore"}) { testParams.add(ArrayUtils.add(params, statsSource)); } } @@ -73,8 +78,7 @@ public void setStatsSource() { @Test public void testAnalyzeTableComputeStatistics() throws IOException, TException, InterruptedException { - Assume.assumeTrue(statsSource.equals("iceberg") || - testTableType == TestTables.TestTableType.HIVE_CATALOG); + Assume.assumeTrue(statsSource.equals("iceberg") || testTableType == TestTableType.HIVE_CATALOG); String dbName = "default"; String tableName = "customers"; @@ -113,7 +117,7 @@ public void testStatsWithInsert() { testTables.createTable(shell, identifier.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, PartitionSpec.unpartitioned(), fileFormat, ImmutableList.of()); - if (testTableType != TestTables.TestTableType.HIVE_CATALOG) { + if (testTableType != TestTableType.HIVE_CATALOG) { // If the location is set and we have to gather stats, then we have to update the table stats now shell.executeStatement("ANALYZE TABLE " + identifier + " COMPUTE STATISTICS FOR COLUMNS"); } @@ -133,7 +137,7 @@ public void testStatsWithInsert() { @Test public void testStatsWithPessimisticLockInsert() { - Assume.assumeTrue(testTableType == TestTables.TestTableType.HIVE_CATALOG); + Assume.assumeTrue(testTableType == TestTableType.HIVE_CATALOG); TableIdentifier identifier = getTableIdentifierWithPessimisticLock("false"); String insert = testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, identifier, false); shell.executeStatement(insert); @@ -144,7 +148,7 @@ public void testStatsWithPessimisticLockInsert() { @Test public void testStatsWithPessimisticLockInsertWhenHiveLockEnabled() { - Assume.assumeTrue(testTableType == TestTables.TestTableType.HIVE_CATALOG); + Assume.assumeTrue(testTableType == TestTableType.HIVE_CATALOG); TableIdentifier identifier = getTableIdentifierWithPessimisticLock("true"); String insert = testTables.getInsertQuery(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, identifier, false); AssertHelpers.assertThrows( @@ -191,7 +195,7 @@ public void testStatsWithPartitionedInsert() { testTables.createTable(shell, identifier.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, spec, fileFormat, ImmutableList.of()); - if (testTableType != TestTables.TestTableType.HIVE_CATALOG) { + if (testTableType != TestTableType.HIVE_CATALOG) { // If the location is set and we have to gather stats, then we have to update the table stats now shell.executeStatement("ANALYZE TABLE " + identifier + " COMPUTE STATISTICS FOR COLUMNS"); } @@ -206,7 +210,7 @@ public void testStatsWithPartitionedInsert() { @Test public void testStatsWithCTAS() { - Assume.assumeTrue(HiveIcebergSerDe.CTAS_EXCEPTION_MSG, testTableType == TestTables.TestTableType.HIVE_CATALOG); + Assume.assumeTrue(HiveIcebergSerDe.CTAS_EXCEPTION_MSG, testTableType == TestTableType.HIVE_CATALOG); shell.executeStatement("CREATE TABLE source (id bigint, name string) PARTITIONED BY (dept string) STORED AS ORC"); shell.executeStatement(testTables.getInsertQuery( @@ -225,7 +229,7 @@ public void testStatsWithCTAS() { @Test public void testStatsWithPartitionedCTAS() { - Assume.assumeTrue(HiveIcebergSerDe.CTAS_EXCEPTION_MSG, testTableType == TestTables.TestTableType.HIVE_CATALOG); + Assume.assumeTrue(HiveIcebergSerDe.CTAS_EXCEPTION_MSG, testTableType == TestTableType.HIVE_CATALOG); shell.executeStatement("CREATE TABLE source (id bigint, name string) PARTITIONED BY (dept string) STORED AS ORC"); shell.executeStatement(testTables.getInsertQuery( @@ -252,7 +256,7 @@ public void testStatsWithPartitionedCTAS() { @Test public void testStatsRemoved() throws IOException { Assume.assumeTrue("Only HiveCatalog can remove stats which become obsolete", - testTableType == TestTables.TestTableType.HIVE_CATALOG); + testTableType == TestTableType.HIVE_CATALOG); TableIdentifier identifier = TableIdentifier.of("default", "customers"); @@ -307,7 +311,7 @@ public void testMergeStatsWithInsert() { testTables.createTable(shell, identifier.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, PartitionSpec.unpartitioned(), fileFormat, ImmutableList.of()); - if (testTableType != TestTables.TestTableType.HIVE_CATALOG) { + if (testTableType != TestTableType.HIVE_CATALOG) { // If the location is set and we have to gather stats, then we have to update the table stats now shell.executeStatement("ANALYZE TABLE " + identifier + " COMPUTE STATISTICS FOR COLUMNS"); } diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java index 677936fbcc7c..320e79e6368e 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java @@ -35,6 +35,11 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.mr.InputFormatConfig; +import org.apache.iceberg.mr.hive.test.TestHiveShell; +import org.apache.iceberg.mr.hive.test.TestTables; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -64,12 +69,12 @@ public static Collection parameters() { // Run tests with every FileFormat for a single Catalog (HiveCatalog) for (FileFormat fileFormat : HiveIcebergStorageHandlerTestUtils.FILE_FORMATS) { - testParams.add(new Object[] {fileFormat, TestTables.TestTableType.HIVE_CATALOG}); + testParams.add(new Object[] {fileFormat, TestTableType.HIVE_CATALOG}); } // Run tests for every Catalog for a single FileFormat (PARQUET) - skip HiveCatalog tests as they are added before - for (TestTables.TestTableType testTableType : TestTables.ALL_TABLE_TYPES) { - if (!TestTables.TestTableType.HIVE_CATALOG.equals(testTableType)) { + for (TestTableType testTableType : TestTables.ALL_TABLE_TYPES) { + if (!TestTableType.HIVE_CATALOG.equals(testTableType)) { testParams.add(new Object[]{FileFormat.PARQUET, testTableType}); } } @@ -85,7 +90,7 @@ public static Collection parameters() { public FileFormat fileFormat; @Parameter(1) - public TestTables.TestTableType testTableType; + public TestTableType testTableType; @Rule public TemporaryFolder temp = new TemporaryFolder(); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java index b6d39bd01a25..3d12cfb1179a 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java @@ -71,6 +71,12 @@ import org.apache.iceberg.mr.Catalogs; import org.apache.iceberg.mr.InputFormatConfig; import org.apache.iceberg.mr.TestHelper; +import org.apache.iceberg.mr.hive.test.CustomTestHiveAuthorizerFactory; +import org.apache.iceberg.mr.hive.test.TestHiveShell; +import org.apache.iceberg.mr.hive.test.TestTables; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; @@ -149,7 +155,7 @@ public class TestHiveIcebergStorageHandlerNoScan { @Parameters(name = "catalog={0}") public static Collection parameters() { Collection testParams = Lists.newArrayList(); - for (TestTables.TestTableType testTableType : TestTables.ALL_TABLE_TYPES) { + for (TestTableType testTableType : TestTables.ALL_TABLE_TYPES) { testParams.add(new Object[] {testTableType}); } return testParams; @@ -160,7 +166,7 @@ public static Collection parameters() { private TestTables testTables; @Parameter(0) - public TestTables.TestTableType testTableType; + public TestTableType testTableType; @Rule public TemporaryFolder temp = new TemporaryFolder(); @@ -432,7 +438,7 @@ public void testSetPartitionTransformCaseSensitive() { @Test public void testInvalidCreateWithPartitionTransform() { - Assume.assumeTrue("Test on hive catalog is enough", testTableType == TestTables.TestTableType.HIVE_CATALOG); + Assume.assumeTrue("Test on hive catalog is enough", testTableType == TestTableType.HIVE_CATALOG); String query = String.format("CREATE EXTERNAL TABLE customers (customer_id BIGINT, first_name STRING, last_name " + "STRING) PARTITIONED BY spec(TRUNCATE(2, last_name)) STORED AS ORC"); Assertions.assertThatThrownBy(() -> shell.executeStatement(query)) @@ -499,8 +505,9 @@ public void testCreateDropTable() throws TException, IOException, InterruptedExc public void testCreateDropTableNonDefaultCatalog() { TableIdentifier identifier = TableIdentifier.of("default", "customers"); String catalogName = "nondefaultcatalog"; - testTables.properties().entrySet() - .forEach(e -> shell.setHiveSessionValue(e.getKey().replace(testTables.catalog, catalogName), e.getValue())); + testTables.properties().forEach((key, value) -> shell.setHiveSessionValue( + key.replace(testTables.getCatalog(), catalogName), + value)); String createSql = "CREATE EXTERNAL TABLE " + identifier + " (customer_id BIGINT, first_name STRING COMMENT 'This is first name'," + " last_name STRING COMMENT 'This is last name')" + @@ -626,7 +633,7 @@ public void testDeleteBackingTable() throws TException, IOException, Interrupted @Test public void testDropTableWithCorruptedMetadata() throws TException, IOException, InterruptedException { Assume.assumeTrue("Only HiveCatalog attempts to load the Iceberg table prior to dropping it.", - testTableType == TestTables.TestTableType.HIVE_CATALOG); + testTableType == TestTableType.HIVE_CATALOG); // create test table TableIdentifier identifier = TableIdentifier.of("default", "customers"); @@ -700,8 +707,8 @@ public void testCreateTableError() { "'='" + testTables.catalogName() + "')")); - if (testTableType != TestTables.TestTableType.HADOOP_CATALOG && - testTableType != TestTables.TestTableType.CUSTOM_CATALOG) { + if (testTableType != TestTableType.HADOOP_CATALOG && + testTableType != TestTableType.CUSTOM_CATALOG) { assertThatThrownBy .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Failed to execute Hive query") @@ -719,7 +726,7 @@ public void testCreateTableAboveExistingTable() throws IOException { testTables.createIcebergTable(shell.getHiveConf(), "customers", COMPLEX_SCHEMA, FileFormat.PARQUET, Collections.emptyMap(), Collections.emptyList()); - if (testTableType == TestTables.TestTableType.HIVE_CATALOG) { + if (testTableType == TestTableType.HIVE_CATALOG) { // In HiveCatalog we just expect an exception since the table is already exists Assertions.assertThatThrownBy( () -> @@ -749,7 +756,7 @@ public void testCreateTableAboveExistingTable() throws IOException { @Test public void testFormatVersion() throws IOException { - Assume.assumeTrue(testTableType != TestTables.TestTableType.HIVE_CATALOG); + Assume.assumeTrue(testTableType != TestTableType.HIVE_CATALOG); TableIdentifier tbl = TableIdentifier.of("default", "customers"); // Create the Iceberg table testTables.createIcebergTable(shell.getHiveConf(), "customers", COMPLEX_SCHEMA, FileFormat.PARQUET, @@ -1110,7 +1117,7 @@ public void testIcebergAndHmsTableProperties() throws Exception { @Test public void testIcebergHMSPropertiesTranslation() throws Exception { Assume.assumeTrue("Iceberg - HMS property translation is only relevant for HiveCatalog", - testTableType == TestTables.TestTableType.HIVE_CATALOG); + testTableType == TestTableType.HIVE_CATALOG); TableIdentifier identifier = TableIdentifier.of("default", "customers"); @@ -1218,7 +1225,7 @@ public void testDropTableWithoutPurge() throws IOException, TException, Interrup @Test public void testDropHiveTableWithoutUnderlyingTable() throws IOException { Assume.assumeFalse("Not relevant for HiveCatalog", - testTableType.equals(TestTables.TestTableType.HIVE_CATALOG)); + testTableType.equals(TestTableType.HIVE_CATALOG)); TableIdentifier identifier = TableIdentifier.of("default", "customers"); // Create the Iceberg table in non-HiveCatalog @@ -1470,7 +1477,7 @@ public void testAlterTableChangeColumnTypeAndComment() throws TException, Interr @Test public void testMetaHookWithUndefinedAlterOperationType() throws Exception { Assume.assumeTrue("Enough to check for one type only", - testTableType.equals(TestTables.TestTableType.HIVE_CATALOG)); + testTableType.equals(TestTableType.HIVE_CATALOG)); TableIdentifier identifier = TableIdentifier.of("default", "customers"); testTables.createTable(shell, identifier.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, SPEC, FileFormat.PARQUET, ImmutableList.of()); @@ -1558,7 +1565,7 @@ public void testAuthzURIWithAuthEnabledWithMetadataLocation(boolean masked) thro shell.getHiveConf().setBoolean(HIVE_ICEBERG_MASK_DEFAULT_LOCATION.varname, masked); shell.setHiveSessionValue("hive.security.authorization.enabled", true); shell.setHiveSessionValue("hive.security.authorization.manager", - "org.apache.iceberg.mr.hive.CustomTestHiveAuthorizerFactory"); + "org.apache.iceberg.mr.hive.test.CustomTestHiveAuthorizerFactory"); TableIdentifier source = TableIdentifier.of("default", "source"); Table sourceTable = testTables.createTable(shell, source.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, PartitionSpec.unpartitioned(), FileFormat.PARQUET, ImmutableList.of()); @@ -1589,7 +1596,7 @@ public void testAuthzURIWithAuthEnabledWithMetadataLocation(boolean masked) thro @Test public void testAuthzURIWithAuthEnabledAndMockCommandAuthorizerMasked() throws HiveException, TException, InterruptedException { - Assume.assumeTrue(testTableType.equals(TestTables.TestTableType.HIVE_CATALOG)); + Assume.assumeTrue(testTableType.equals(TestTableType.HIVE_CATALOG)); testAuthzURIWithAuthEnabledAndMockCommandAuthorizer(true); } @@ -1604,7 +1611,7 @@ public void testAuthzURIWithAuthEnabledAndMockCommandAuthorizer(boolean masked) shell.getHiveConf().setBoolean(HIVE_ICEBERG_MASK_DEFAULT_LOCATION.varname, masked); shell.setHiveSessionValue("hive.security.authorization.enabled", true); shell.setHiveSessionValue("hive.security.authorization.manager", - "org.apache.iceberg.mr.hive.CustomTestHiveAuthorizerFactory"); + "org.apache.iceberg.mr.hive.test.CustomTestHiveAuthorizerFactory"); TableIdentifier target = TableIdentifier.of("default", "target"); Table table = testTables.createTable(shell, target.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, PartitionSpec.unpartitioned(), FileFormat.PARQUET, ImmutableList.of()); @@ -1637,7 +1644,7 @@ public void testAuthzURIWithAuthEnabledAndMockCommandAuthorizer(boolean masked) @Test public void testAuthzURIWithAuthEnabledMasked() throws TException, URISyntaxException, InterruptedException { - Assume.assumeTrue(testTableType.equals(TestTables.TestTableType.HIVE_CATALOG)); + Assume.assumeTrue(testTableType.equals(TestTableType.HIVE_CATALOG)); testAuthzURIWithAuthEnabled(true); } @@ -1672,7 +1679,7 @@ public void testAuthzURIWithAuthEnabled(boolean masked) throws TException, Inter @Test public void testCreateTableWithMetadataLocation() throws IOException { Assume.assumeTrue("Create with metadata location is only supported for Hive Catalog tables", - testTableType.equals(TestTables.TestTableType.HIVE_CATALOG)); + testTableType.equals(TestTableType.HIVE_CATALOG)); TableIdentifier sourceIdentifier = TableIdentifier.of("default", "source"); Table sourceTable = testTables.createTable(shell, sourceIdentifier.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, @@ -1705,7 +1712,7 @@ public void testCreateTableWithMetadataLocation() throws IOException { @Test public void testAlterTableWithMetadataLocation() throws IOException { Assume.assumeTrue("Alter table with metadata location is only supported for Hive Catalog tables", - testTableType.equals(TestTables.TestTableType.HIVE_CATALOG)); + testTableType.equals(TestTableType.HIVE_CATALOG)); TableIdentifier tableIdentifier = TableIdentifier.of("default", "source"); // create a test table with some dummy data Table table = @@ -1774,7 +1781,7 @@ public void testAlterTableToIcebergAndMetadataLocation() throws IOException { @Test public void testCTLT() throws TException, InterruptedException { Assume.assumeTrue(" CTLT target table must be a HiveCatalog table", - testTableType == TestTables.TestTableType.HIVE_CATALOG); + testTableType == TestTableType.HIVE_CATALOG); // Create a normal table and add some data shell.executeStatement("CREATE TABLE source(a int)"); shell.executeStatement("insert into source values(1)"); @@ -1802,7 +1809,7 @@ public void testCTLT() throws TException, InterruptedException { @Test public void testCTLTHiveCatalogValidation() throws TException, InterruptedException { Assume.assumeTrue(" CTLT target table works on HiveCatalog table", - testTableType != TestTables.TestTableType.HIVE_CATALOG); + testTableType != TestTableType.HIVE_CATALOG); // Create a normal table and add some data shell.executeStatement("CREATE TABLE source(a int)"); @@ -1858,7 +1865,7 @@ public void testParquetHiveCatalogValidation() throws TException, InterruptedExc @Test public void testConcurrentIcebergCommitsAndHiveAlterTableCalls() throws Exception { - Assume.assumeTrue(testTableType.equals(TestTables.TestTableType.HIVE_CATALOG)); + Assume.assumeTrue(testTableType.equals(TestTableType.HIVE_CATALOG)); TableIdentifier identifier = TableIdentifier.of("default", "customers"); testTables.createTable( @@ -1929,7 +1936,7 @@ public void testConcurrentIcebergCommitsAndHiveAlterTableCalls() throws Exceptio @Test public void testCreateTableWithMetadataLocationWithoutSchema() throws IOException, TException, InterruptedException { Assume.assumeTrue("Create with metadata location is only supported for Hive Catalog tables", - testTableType.equals(TestTables.TestTableType.HIVE_CATALOG)); + testTableType.equals(TestTableType.HIVE_CATALOG)); TableIdentifier sourceIdentifier = TableIdentifier.of("default", "source"); PartitionSpec spec = PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA).identity("customer_id").build(); @@ -1991,7 +1998,7 @@ private void verifyAlterTableAddColumnsTests() throws Exception { @Test public void checkIcebergTableLocation() throws TException, InterruptedException, IOException { - Assume.assumeTrue("This test is only for hive catalog", testTableType == TestTables.TestTableType.HIVE_CATALOG); + Assume.assumeTrue("This test is only for hive catalog", testTableType == TestTableType.HIVE_CATALOG); String dBName = "testdb"; String tableName = "tbl"; @@ -2026,7 +2033,7 @@ public void checkIcebergTableLocation() throws TException, InterruptedException, @Test public void testSyncProperties() throws TException, InterruptedException { - Assume.assumeTrue("This test is only for hive catalog", testTableType == TestTables.TestTableType.HIVE_CATALOG); + Assume.assumeTrue("This test is only for hive catalog", testTableType == TestTableType.HIVE_CATALOG); // Test create v2 iceberg table and check iceberg properties & hms properties TableIdentifier identifier = TableIdentifier.of("default", "customers_v2"); @@ -2133,7 +2140,7 @@ private String getCurrentSnapshotForHiveCatalogTable(org.apache.iceberg.Table ic @Test public void testCreateTableWithPercentInName() throws IOException { - Assume.assumeTrue("This test is only for hive catalog", testTableType == TestTables.TestTableType.HIVE_CATALOG); + Assume.assumeTrue("This test is only for hive catalog", testTableType == TestTableType.HIVE_CATALOG); TableIdentifier identifier = TableIdentifier.of("default", "[|]#&%_@"); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java index 8d44dbb32fa2..0d14b6732e91 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java @@ -37,6 +37,10 @@ import org.apache.iceberg.common.DynFields; import org.apache.iceberg.data.Record; import org.apache.iceberg.mr.TestHelper; +import org.apache.iceberg.mr.hive.test.TestHiveShell; +import org.apache.iceberg.mr.hive.test.TestTables; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; import org.junit.After; @@ -109,7 +113,7 @@ public void before() throws IOException { dateFormat.ifPresent(ThreadLocal::remove); localTimeZone.ifPresent(ThreadLocal::remove); - this.testTables = HiveIcebergStorageHandlerTestUtils.testTables(shell, TestTables.TestTableType.HIVE_CATALOG, temp); + this.testTables = HiveIcebergStorageHandlerTestUtils.testTables(shell, TestTableType.HIVE_CATALOG, temp); HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp); } diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithMultipleCatalogs.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithMultipleCatalogs.java index b40425a8ea8e..9b367a9d89dd 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithMultipleCatalogs.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithMultipleCatalogs.java @@ -30,7 +30,11 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.mr.InputFormatConfig; -import org.apache.iceberg.mr.hive.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.TestHiveShell; +import org.apache.iceberg.mr.hive.test.TestTables; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.After; import org.junit.AfterClass; @@ -43,7 +47,7 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import static org.apache.iceberg.mr.hive.TestTables.TestTableType.HIVE_CATALOG; +import static org.apache.iceberg.mr.hive.test.TestTables.TestTableType.HIVE_CATALOG; @RunWith(Parameterized.class) public class TestHiveIcebergStorageHandlerWithMultipleCatalogs { diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTagOperation.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTagOperation.java index b65c3f376bbe..465b109dcce0 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTagOperation.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTagOperation.java @@ -20,25 +20,31 @@ package org.apache.iceberg.mr.hive; import java.io.IOException; +import java.util.Collection; import java.util.List; import java.util.concurrent.TimeUnit; import org.apache.iceberg.FileFormat; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.Assert; -import org.junit.Assume; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; -import static org.apache.iceberg.mr.hive.HiveIcebergTestUtils.timestampAfterSnapshot; +import static org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils.timestampAfterSnapshot; public class TestHiveIcebergTagOperation extends HiveIcebergStorageHandlerWithEngineBase { - @Override - protected void validateTestParams() { - Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG && formatVersion == 2); + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.fileFormat() == FileFormat.PARQUET && + p.testTableType() == TestTableType.HIVE_CATALOG && + p.isVectorized() && + p.formatVersion() == 2); } @Test diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTimeTravel.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTimeTravel.java index e4c2490d5ce8..6552b8705607 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTimeTravel.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTimeTravel.java @@ -20,26 +20,32 @@ package org.apache.iceberg.mr.hive; import java.io.IOException; +import java.util.Collection; import java.util.List; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HistoryEntry; import org.apache.iceberg.Table; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; import org.apache.iceberg.types.Types; import org.junit.Assert; -import org.junit.Assume; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; -import static org.apache.iceberg.mr.hive.HiveIcebergTestUtils.timestampAfterSnapshot; +import static org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils.timestampAfterSnapshot; /** * Tests covering the time travel feature, aka reading from a table as of a certain snapshot. */ public class TestHiveIcebergTimeTravel extends HiveIcebergStorageHandlerWithEngineBase { - @Override - protected void validateTestParams() { - Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG && formatVersion == 2); + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.fileFormat() == FileFormat.PARQUET && + p.testTableType() == TestTableType.HIVE_CATALOG && + p.isVectorized() && + p.formatVersion() == 2); } @Test diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTruncateTable.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTruncateTable.java index 4f56c812e09b..85cbb87905d3 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTruncateTable.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTruncateTable.java @@ -20,6 +20,7 @@ package org.apache.iceberg.mr.hive; import java.io.IOException; +import java.util.Collection; import java.util.List; import java.util.Map; import org.apache.iceberg.AssertHelpers; @@ -29,15 +30,24 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.data.Record; import org.apache.iceberg.mr.TestHelper; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergTestUtils; import org.apache.thrift.TException; import org.junit.Assert; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; /** * Tests truncate table feature on Iceberg tables. */ public class TestHiveIcebergTruncateTable extends HiveIcebergStorageHandlerWithEngineBase { + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.isVectorized() && p.formatVersion() == 2); + } + @Test public void testTruncateTable() throws IOException, TException, InterruptedException { // Create an Iceberg table with some records in it then execute a truncate table command. diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTypes.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTypes.java index 3ef0adc63784..a1e39045ca88 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTypes.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergTypes.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.math.BigDecimal; +import java.util.Collection; import java.util.List; import java.util.Map; import org.apache.iceberg.Schema; @@ -30,6 +31,7 @@ import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; import static org.apache.iceberg.types.Types.NestedField.required; @@ -38,6 +40,12 @@ */ public class TestHiveIcebergTypes extends HiveIcebergStorageHandlerWithEngineBase { + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.formatVersion() == 2); + } + @Test public void testDecimalTableWithPredicateLiterals() throws IOException { Schema schema = new Schema(required(1, "decimal_field", Types.DecimalType.of(7, 2))); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergVariant.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergVariant.java index 7ec594a609cf..71efa4d60899 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergVariant.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergVariant.java @@ -20,6 +20,7 @@ package org.apache.iceberg.mr.hive; import java.io.IOException; +import java.util.Collection; import java.util.List; import java.util.stream.StreamSupport; import org.apache.hadoop.fs.Path; @@ -35,6 +36,7 @@ import org.apache.parquet.schema.MessageType; import org.junit.Assert; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assume.assumeTrue; @@ -42,6 +44,12 @@ public class TestHiveIcebergVariant extends HiveIcebergStorageHandlerWithEngineBase { private static final String TYPED_VALUE_FIELD = "typed_value"; + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.formatVersion() == 3); + } + @Test public void testVariantSelectProjection() throws IOException { assumeParquetNonVectorized(); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergWriteMetadataCleanup.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergWriteMetadataCleanup.java index 6c1b050f1241..8469a13501c8 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergWriteMetadataCleanup.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergWriteMetadataCleanup.java @@ -33,6 +33,10 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.hive.IcebergCatalogProperties; import org.apache.iceberg.hive.IcebergTableProperties; +import org.apache.iceberg.mr.hive.test.TestHiveShell; +import org.apache.iceberg.mr.hive.test.TestTables; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -71,7 +75,7 @@ public static void afterClass() throws Exception { @Before public void before() throws IOException { - testTables = HiveIcebergStorageHandlerTestUtils.testTables(shell, TestTables.TestTableType.HIVE_CATALOG, temp); + testTables = HiveIcebergStorageHandlerTestUtils.testTables(shell, TestTableType.HIVE_CATALOG, temp); HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, true); } diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestOptimisticRetry.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestOptimisticRetry.java index 6094db84608e..d9249af73c91 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestOptimisticRetry.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestOptimisticRetry.java @@ -20,190 +20,124 @@ package org.apache.iceberg.mr.hive; +import java.util.Collection; +import java.util.Collections; import java.util.List; -import java.util.concurrent.Executors; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.relocated.com.google.common.base.Throwables; -import org.apache.iceberg.util.Tasks; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.mr.hive.test.concurrent.WithMockedStorageHandler; +import org.apache.iceberg.mr.hive.test.utils.HiveIcebergStorageHandlerTestUtils; import org.junit.Assert; -import org.junit.Assume; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; -import static org.apache.iceberg.mr.hive.HiveIcebergStorageHandlerTestUtils.init; +import static org.apache.iceberg.mr.hive.TestConflictingDataFiles.STORAGE_HANDLER_STUB; +@WithMockedStorageHandler public class TestOptimisticRetry extends HiveIcebergStorageHandlerWithEngineBase { - @Override - protected void validateTestParams() { - Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && - testTableType == TestTables.TestTableType.HIVE_CATALOG && formatVersion == 2); + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.fileFormat() == FileFormat.PARQUET && + p.testTableType() == TestTableType.HIVE_CATALOG && + p.isVectorized()); } @Test - public void testConcurrentOverlappingUpdates() { + public void testConcurrentOverlappingUpdates() throws Exception { testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, - formatVersion); - String sql = "UPDATE customers SET last_name='Changed' WHERE customer_id=3 or first_name='Joanna'"; + formatVersion, Collections.emptyMap(), STORAGE_HANDLER_STUB); - try { - Tasks.range(2) - .executeWith(Executors.newFixedThreadPool(2)) - .run(i -> { - init(shell, testTables, temp); - HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, - RETRY_STRATEGIES); - shell.executeStatement(sql); - shell.closeSession(); - }); - } catch (Throwable ex) { - // If retry succeeds then it should not throw an ValidationException. - Throwable cause = Throwables.getRootCause(ex); - if (cause instanceof ValidationException && cause.getMessage().matches("^Found.*conflicting.*files(.*)")) { - Assert.fail(); - } - } - - List res = shell.executeStatement("SELECT * FROM customers WHERE last_name='Changed'"); - Assert.assertEquals(5, res.size()); + String sql = "UPDATE customers SET last_name='Changed' WHERE customer_id=3 or first_name='Joanna'"; + executeConcurrently(false, RETRY_STRATEGIES, sql); + List res = shell.executeStatement("SELECT count(*) FROM customers WHERE last_name='Changed'"); + Assert.assertEquals(5L, res.getFirst()[0]); } @Test - public void testConcurrentOverwriteAndUpdate() { - TestUtilPhaser.getInstance(); + public void testConcurrentOverwriteAndUpdate() throws Exception { testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, - formatVersion); - String[] sql = - new String[] {"INSERT OVERWRITE table customers SELECT * FROM customers where last_name='Taylor'", - "UPDATE customers SET first_name='Changed' WHERE last_name='Taylor'"}; - - // The query shouldn't throw exception but rather retry & commit. - Tasks.range(2).executeWith(Executors.newFixedThreadPool(2)).run(i -> { - TestUtilPhaser.getInstance().getPhaser().register(); - init(shell, testTables, temp); - HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, RETRY_STRATEGIES); - shell.executeStatement(sql[i]); - shell.closeSession(); - }); - TestUtilPhaser.destroyInstance(); - } + formatVersion, Collections.emptyMap(), STORAGE_HANDLER_STUB); + + String[] sql = new String[] { + "INSERT OVERWRITE table customers SELECT * FROM customers WHERE last_name='Taylor'", + "UPDATE customers SET first_name='Changed' WHERE last_name='Taylor'" + }; + executeConcurrently(false, RETRY_STRATEGIES, sql); + List res = shell.executeStatement("SELECT count(*) FROM customers"); + Assert.assertEquals(1L, res.getFirst()[0]); + + res = shell.executeStatement("SELECT count(*) FROM customers WHERE first_name='Changed'"); + Assert.assertEquals(1L, res.getFirst()[0]); + } @Test - public void testNonOverlappingConcurrent2Updates() { + public void testNonOverlappingConcurrent2Updates() throws Exception { testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_2, - formatVersion); - String[] sql = new String[]{"UPDATE customers SET last_name='Changed' WHERE customer_id=3 or first_name='Joanna'", - "UPDATE customers SET last_name='Changed2' WHERE customer_id=2 and first_name='Jake'"}; - - try { - Tasks.range(2) - .executeWith(Executors.newFixedThreadPool(2)) - .run(i -> { - init(shell, testTables, temp); - HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, - RETRY_STRATEGIES); - shell.executeStatement(sql[i]); - shell.closeSession(); - }); - } catch (Throwable ex) { - // If retry succeeds then it should not throw an ValidationException. - Throwable cause = Throwables.getRootCause(ex); - if (cause instanceof ValidationException && cause.getMessage().matches("^Found.*conflicting.*files(.*)")) { - Assert.fail(); - } - } - - List res = shell.executeStatement("SELECT * FROM customers WHERE last_name='Changed'"); - Assert.assertEquals(5, res.size()); - - res = shell.executeStatement("SELECT * FROM customers WHERE last_name='Changed2'"); - Assert.assertEquals(1, res.size()); + formatVersion, Collections.emptyMap(), STORAGE_HANDLER_STUB); + + String[] sql = new String[]{ + "UPDATE customers SET last_name='Changed' WHERE customer_id=3 or first_name='Joanna'", + "UPDATE customers SET last_name='Changed2' WHERE customer_id=2 and first_name='Jake'" + }; + executeConcurrently(false, RETRY_STRATEGIES, sql); + + List res = shell.executeStatement("SELECT count(*) FROM customers WHERE last_name='Changed'"); + Assert.assertEquals(5L, res.getFirst()[0]); + + res = shell.executeStatement("SELECT count(*) FROM customers WHERE last_name='Changed2'"); + Assert.assertEquals(1L, res.getFirst()[0]); } @Test - public void testConcurrent2MergeInserts() { + public void testConcurrent2MergeInserts() throws Exception { testTables.createTable(shell, "source", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.OTHER_CUSTOMER_RECORDS_1); + testTables.createTable(shell, "target", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, - formatVersion); + formatVersion, Collections.emptyMap(), STORAGE_HANDLER_STUB); - String sql = "MERGE INTO target t USING source s on t.customer_id = s.customer_id WHEN Not MATCHED THEN " + - "INSERT values (s.customer_id, s.first_name, s.last_name)"; - try { - Tasks.range(2) - .executeWith(Executors.newFixedThreadPool(2)) - .run(i -> { - init(shell, testTables, temp); - HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, - RETRY_STRATEGIES); - shell.executeStatement(sql); - shell.closeSession(); - }); - } catch (Throwable ex) { - // If retry succeeds then it should not throw an ValidationException. - Throwable cause = Throwables.getRootCause(ex); - if (cause instanceof ValidationException && cause.getMessage().matches("^Found.*conflicting.*files(.*)")) { - Assert.fail(); - } - } - List res = shell.executeStatement("SELECT * FROM target"); - Assert.assertEquals(6, res.size()); + String sql = "MERGE INTO target t USING source s on t.customer_id = s.customer_id " + + "WHEN NOT MATCHED THEN " + + "INSERT VALUES (s.customer_id, s.first_name, s.last_name)"; + executeConcurrently(false, RETRY_STRATEGIES, sql); + + List res = shell.executeStatement("SELECT count(*) FROM target"); + Assert.assertEquals(6L, res.getFirst()[0]); } @Test - public void testConcurrent2MergeUpdates() { - testTables.createTable(shell, "merge_update_source", - HiveIcebergStorageHandlerTestUtils.USER_CLICKS_SCHEMA, PartitionSpec.unpartitioned(), - fileFormat, HiveIcebergStorageHandlerTestUtils.USER_CLICKS_RECORDS_1, - 2); - testTables.createTable(shell, "merge_update_target", - HiveIcebergStorageHandlerTestUtils.USER_CLICKS_SCHEMA, PartitionSpec.unpartitioned(), - fileFormat, HiveIcebergStorageHandlerTestUtils.USER_CLICKS_RECORDS_2, - 2); - - String query1 = "merge into merge_update_target using ( select * from merge_update_source) " + - "sub on sub.name = merge_update_target.name when matched then update set age=15"; - String query2 = "merge into merge_update_target using ( select * from merge_update_source) " + - "sub on sub.age = merge_update_target.age when matched then update set age=15"; - - String[] mergeQueryList = new String[] {query1, query2}; - try { - Tasks.range(2) - .executeWith(Executors.newFixedThreadPool(2)) - .run(i -> { - init(shell, testTables, temp); - HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); - HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, - RETRY_STRATEGIES); - shell.executeStatement(mergeQueryList[i]); - shell.closeSession(); - }); - } catch (Throwable ex) { - // If retry succeeds then it should not throw an ValidationException. - Throwable cause = Throwables.getRootCause(ex); - if (cause instanceof ValidationException && cause.getMessage().matches("^Found.*conflicting.*files(.*)")) { - Assert.fail(); - } - } - List res = shell.executeStatement("SELECT * FROM merge_update_target where age = 15"); - Assert.assertEquals(2, res.size()); - } + public void testConcurrent2MergeUpdates() throws Exception { + testTables.createTable(shell, "source", + HiveIcebergStorageHandlerTestUtils.USER_CLICKS_SCHEMA, PartitionSpec.unpartitioned(), + fileFormat, HiveIcebergStorageHandlerTestUtils.USER_CLICKS_RECORDS_1, + formatVersion); + testTables.createTable(shell, "target", + HiveIcebergStorageHandlerTestUtils.USER_CLICKS_SCHEMA, PartitionSpec.unpartitioned(), + fileFormat, HiveIcebergStorageHandlerTestUtils.USER_CLICKS_RECORDS_2, + formatVersion, Collections.emptyMap(), STORAGE_HANDLER_STUB); + + String query1 = "MERGE INTO target t USING source src ON t.name = src.name " + + "WHEN MATCHED THEN " + + "UPDATE SET age=15"; + + String query2 = "MERGE INTO target t USING source src ON t.age = src.age " + + "WHEN MATCHED THEN " + + "UPDATE SET age=15"; + + String[] sql = new String[] {query1, query2}; + executeConcurrently(false, RETRY_STRATEGIES, sql); + + List res = shell.executeStatement("SELECT count(*) FROM target where age = 15"); + Assert.assertEquals(2L, res.getFirst()[0]); + } } diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestUtilPhaser.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestUtilPhaser.java deleted file mode 100644 index 169ae9a1b123..000000000000 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestUtilPhaser.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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 org.apache.iceberg.mr.hive; - -import java.util.concurrent.Phaser; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TestUtilPhaser { - - private static final Logger LOG = LoggerFactory.getLogger(TestUtilPhaser.class); - private static TestUtilPhaser instance; - private final Phaser phaser; - - private TestUtilPhaser() { - phaser = new Phaser(); - } - - public static synchronized TestUtilPhaser getInstance() { - if (instance == null) { - LOG.info("UnitTestConcurrency: Instantiating the Phaser barrier"); - instance = new TestUtilPhaser(); - } - return instance; - } - - public Phaser getPhaser() { - return phaser; - } - - public static synchronized boolean isInstantiated() { - return instance != null; - } - - public static synchronized void destroyInstance() { - if (instance != null) { - instance.getPhaser().forceTermination(); - LOG.info("UnitTestConcurrency: Destroying the Phaser barrier"); - instance = null; - } - } -} diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/CustomTestHiveAuthorizerFactory.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/CustomTestHiveAuthorizerFactory.java similarity index 97% rename from iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/CustomTestHiveAuthorizerFactory.java rename to iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/CustomTestHiveAuthorizerFactory.java index 11eb4fcf4c75..2b640c5f125f 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/CustomTestHiveAuthorizerFactory.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/CustomTestHiveAuthorizerFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iceberg.mr.hive; +package org.apache.iceberg.mr.hive.test; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider; diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveShell.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/TestHiveShell.java similarity index 98% rename from iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveShell.java rename to iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/TestHiveShell.java index b9a1edeec880..c6f134aaa3ea 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveShell.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/TestHiveShell.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iceberg.mr.hive; +package org.apache.iceberg.mr.hive.test; import java.util.Collections; import java.util.List; @@ -37,6 +37,7 @@ import org.apache.hive.service.cli.session.HiveSession; import org.apache.hive.service.server.HiveServer2; import org.apache.iceberg.hive.TestHiveMetastore; +import org.apache.iceberg.mr.hive.HiveIcebergQueryLifeTimeHook; import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestTables.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/TestTables.java similarity index 99% rename from iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestTables.java rename to iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/TestTables.java index a5e75b3d8ac6..6d1941b1e453 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestTables.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/TestTables.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iceberg.mr.hive; +package org.apache.iceberg.mr.hive.test; import java.io.File; import java.io.IOException; @@ -56,6 +56,7 @@ import org.apache.iceberg.mr.InputFormatConfig; import org.apache.iceberg.mr.TestCatalogs; import org.apache.iceberg.mr.TestHelper; +import org.apache.iceberg.mr.hive.HiveIcebergStorageHandler; import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -77,6 +78,7 @@ public abstract class TestTables { private final Tables tables; protected final TemporaryFolder temp; + protected final String catalog; protected TestTables(Tables tables, TemporaryFolder temp, String catalogName) { @@ -89,6 +91,10 @@ protected TestTables(Catalog catalog, TemporaryFolder temp, String catalogName) this(new CatalogToTables(catalog), temp, catalogName); } + public String getCatalog() { + return catalog; + } + public Map properties() { return Collections.emptyMap(); } diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/concurrent/HiveIcebergStorageHandlerStub.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/concurrent/HiveIcebergStorageHandlerStub.java new file mode 100644 index 000000000000..a9bd1af75118 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/concurrent/HiveIcebergStorageHandlerStub.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.iceberg.mr.hive.test.concurrent; + +import java.io.IOException; +import java.time.Duration; +import java.util.List; +import org.apache.commons.lang3.ThreadUtils; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.mapred.JobContext; +import org.apache.iceberg.mr.hive.HiveIcebergOutputCommitter; +import org.apache.iceberg.mr.hive.HiveIcebergStorageHandler; +import org.apache.iceberg.mr.hive.test.concurrent.TestUtilPhaser.ThreadContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Test stub for HiveIcebergStorageHandler that coordinates concurrent write testing. + * + *

Supports two synchronization modes: + *

    + *
  • Ext-locking mode: Tests external table locking. Counter ensures queries start sequentially, + * next query starts when current reaches validateCurrentSnapshot. Phaser verifies lock prevents + * concurrent execution.
  • + *
  • Barrier mode: Tests optimistic concurrency. All queries start concurrently, sync at barrier + * in commitJobs, then commit sequentially by index. Supports retry on conflict.
  • + *
+ */ +public class HiveIcebergStorageHandlerStub extends HiveIcebergStorageHandler { + + private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergStorageHandlerStub.class); + + @Override + public HiveIcebergOutputCommitter getOutputCommitter() { + + return new HiveIcebergOutputCommitter() { + @Override + public void commitJobs(List originalContextList, Context.Operation operation) throws IOException { + int queryIndex = ThreadContext.getQueryIndex(); + LOG.debug("Thread {} (queryIndex={}) entered commitJobs", Thread.currentThread(), queryIndex); + + waitForAllWritesToComplete(); + LOG.debug("Thread {} (queryIndex={}) starting commitJobs", Thread.currentThread(), queryIndex); + + super.commitJobs(originalContextList, operation); + + // Barrier Mode: If the commit succeeded, release the turn so the next thread can proceed. + // If commit failed (exception thrown), turn is NOT released, allowing retry logic to + // reuse the current turn. + if (TestUtilPhaser.isInstantiated() && !ThreadContext.useExtLocking()) { + TestUtilPhaser.getInstance().completeTurn(); + LOG.debug("Thread {} (queryIndex={}) committed and released turn", + Thread.currentThread(), queryIndex); + } + } + + private static void waitForAllWritesToComplete() { + if (!TestUtilPhaser.isInstantiated()) { + return; + } + + int queryIndex = ThreadContext.getQueryIndex(); + boolean useExtLocking = ThreadContext.useExtLocking(); + + if (useExtLocking) { + // Ext-locking mode: Table lock should have prevented this query from starting concurrently. + // Sleep briefly to ensure that if the lock is broken, the concurrent query has time + // to incorrectly advance the phase. + ThreadUtils.sleepQuietly(Duration.ofMillis(500)); + int phase = TestUtilPhaser.getInstance().getBarrier().getPhase(); + + if (phase != queryIndex + 1) { + throw new IllegalStateException( + String.format("External locking violation: query sql[%d] expected phase %d, got %d", + queryIndex, queryIndex + 1, phase)); + } + } else { + // Barrier mode: Optimistic concurrency. + // 1. Barrier: Wait for ALL queries to reach commit stage to maximize conflict potential. + // 2. Sequential Commit: Execute commits one-by-one to control ordering. + if (!ThreadContext.isSynced()) { + LOG.debug("Thread {} (queryIndex={}) waiting at barrier", Thread.currentThread(), queryIndex); + TestUtilPhaser.getInstance().getBarrier().arriveAndAwaitAdvance(); + ThreadContext.markSynced(); + } + + LOG.debug("Thread {} (queryIndex={}) waiting for commit turn", Thread.currentThread(), queryIndex); + TestUtilPhaser.getInstance().awaitTurn(); + } + } + }; + } + + @Override + public void validateCurrentSnapshot(TableDesc tableDesc) { + super.validateCurrentSnapshot(tableDesc); + + if (!TestUtilPhaser.isInstantiated()) { + return; + } + + if (ThreadContext.useExtLocking()) { + // Ext-locking mode: Release the turn so the next sequential query can proceed + // from 'awaitTurn' to start its own execution. + TestUtilPhaser.getInstance().completeTurn(); + LOG.debug("Thread {} (queryIndex={}) validated snapshot and released turn", + Thread.currentThread(), ThreadContext.getQueryIndex()); + } + } +} diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/concurrent/TestUtilPhaser.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/concurrent/TestUtilPhaser.java new file mode 100644 index 000000000000..479504617c02 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/concurrent/TestUtilPhaser.java @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.iceberg.mr.hive.test.concurrent; + +import java.util.concurrent.Phaser; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Test utility for coordinating concurrent query execution in multi-threaded tests. + * + *

Uses {@link Phaser}s for synchronization: + *

    + *
  • barrier: Synchronizes threads at a barrier (used in Barrier mode). + *
  • turn: Enforces sequential execution order (0 -> 1 -> 2 ...). + *
+ *
    + *
  • Ext-locking mode: turn ensures sequential execution, barrier verification. + *
  • Barrier mode: barrier syncs threads, turn enforces sequential commits. + *
+ * + * @see HiveIcebergStorageHandlerStub + */ +public class TestUtilPhaser { + + private static final Logger LOG = LoggerFactory.getLogger(TestUtilPhaser.class); + + private static TestUtilPhaser instance; + + private final Phaser barrier; + private final Phaser turn; + + private TestUtilPhaser() { + barrier = new Phaser() { + @Override + protected boolean onAdvance(int phase, int registeredParties) { + // Prevent termination even if registered parties drop to zero + return false; + } + }; + // Initialize with 1 party to drive the phase advances + turn = new Phaser(1); + } + + /** + * Registers this thread with the barrier phaser. + */ + public void register() { + barrier.register(); + } + + public static synchronized TestUtilPhaser getInstance() { + if (instance == null) { + LOG.debug("Instantiating TestUtilPhaser for concurrent test synchronization"); + instance = new TestUtilPhaser(); + } + return instance; + } + + public Phaser getBarrier() { + return barrier; + } + + public static synchronized boolean isInstantiated() { + return instance != null; + } + + /** + * Waits for this query's turn to execute/commit. + * Uses turn to block until the phase reaches (queryIndex). + * Query 0 proceeds immediately (phase 0). + * Query 1 waits for phase 0 -> 1. + */ + public void awaitTurn() { + int queryIndex = ThreadContext.getQueryIndex(); + if (queryIndex > 0) { + // Wait for phase to advance from (queryIndex - 1) to queryIndex + // awaitAdvance returns immediately if current phase != arg + // So we wait on (queryIndex - 1) + turn.awaitAdvance(queryIndex - 1); + } + } + + /** + * Signals this query completed, allowing next query to proceed. + * In ext-locking mode, temporarily registers then deregisters to advance barrier phase. + * Advances turn to allow next query to start. + */ + public void completeTurn() { + if (ThreadContext.useExtLocking()) { + // Ext-locking mode: register and immediately arriveAndDeregister to advance barrier phase + barrier.register(); + barrier.arriveAndDeregister(); + } + + // Advance execution phase: queryIndex -> queryIndex + 1 + turn.arrive(); + } + + public Phaser[] getResources() { + return new Phaser[] {barrier, turn}; + } + + public static synchronized void destroyInstance() { + if (instance != null) { + for (Phaser phaser : instance.getResources()) { + phaser.forceTermination(); + } + LOG.debug("Destroying TestUtilPhaser and clearing thread context"); + instance = null; + } + ThreadContext.clear(); + } + + /** + * Thread-local context for per-thread synchronization state. + * + *

Fields: + *

    + *
  • queryIndex: Position in SQL array (sql[0]=0, sql[1]=1, ...)
  • + *
  • useExtLocking: Whether external locking mode is enabled
  • + *
  • synced: Whether this thread has synced at barrier (for retry detection in barrier mode)
  • + *
+ */ + public static final class ThreadContext { + private static final ThreadLocal QUERY_INDEX = new ThreadLocal<>(); + private static final ThreadLocal USE_EXT_LOCKING = new ThreadLocal<>(); + private static final ThreadLocal SYNCED = new ThreadLocal<>(); + + private ThreadContext() { + } + + public static void setQueryIndex(int queryIndex) { + QUERY_INDEX.set(queryIndex); + } + + public static int getQueryIndex() { + Integer idx = QUERY_INDEX.get(); + return idx != null ? idx : -1; + } + + public static void setUseExtLocking(boolean useExtLocking) { + USE_EXT_LOCKING.set(useExtLocking); + } + + public static boolean useExtLocking() { + Boolean extLocking = USE_EXT_LOCKING.get(); + return extLocking != null && extLocking; + } + + public static void markSynced() { + SYNCED.set(true); + } + + public static boolean isSynced() { + Boolean synced = SYNCED.get(); + return synced != null && synced; + } + + private static void clear() { + QUERY_INDEX.remove(); + USE_EXT_LOCKING.remove(); + SYNCED.remove(); + } + } +} diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/concurrent/WithMockedStorageHandler.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/concurrent/WithMockedStorageHandler.java new file mode 100644 index 000000000000..540537e73451 --- /dev/null +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/concurrent/WithMockedStorageHandler.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.iceberg.mr.hive.test.concurrent; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; +import java.lang.reflect.Method; +import java.util.Map; +import org.apache.iceberg.hive.HMSTablePropertyHelper; +import org.junit.rules.TestRule; +import org.junit.runner.Description; +import org.junit.runners.model.Statement; +import org.mockito.MockedStatic; + +import static org.mockito.ArgumentMatchers.anyMap; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.CALLS_REAL_METHODS; +import static org.mockito.Mockito.mockStatic; + +/** + * Annotation to indicate that a test method or test class should run with a mocked + * HMSTablePropertyHelper.setStorageHandler. This prevents the storage handler from being + * overwritten during table creation, allowing custom storage handlers to be used in tests. + * + *

When applied at the class level, all test methods in the class (including their @Before + * setup methods) will run with the mock active. + * + *

When applied at the method level, only that specific test method (including @Before for + * that test) will run with the mock active. + */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.METHOD, ElementType.TYPE}) +public @interface WithMockedStorageHandler { + + /** + * JUnit Rule that mocks HMSTablePropertyHelper.setStorageHandler for tests annotated with + * {@link WithMockedStorageHandler}. This allows tests to use custom storage handlers without + * having them overwritten by the default Iceberg storage handler. + */ + class Rule implements TestRule { + + private static final Method setStorageHandlerMethod; + + static { + try { + setStorageHandlerMethod = HMSTablePropertyHelper.class + .getDeclaredMethod("setStorageHandler", Map.class, Boolean.TYPE); + setStorageHandlerMethod.setAccessible(true); + } catch (NoSuchMethodException e) { + throw new RuntimeException("Failed to initialize WithMockedStorageHandler.Rule", e); + } + } + + @Override + public Statement apply(Statement base, Description description) { + // Check for annotation at method level OR class level + WithMockedStorageHandler annotation = description.getAnnotation(WithMockedStorageHandler.class); + if (annotation == null) { + annotation = description.getTestClass().getAnnotation(WithMockedStorageHandler.class); + } + + if (annotation == null) { + // No annotation, run test normally + return base; + } + + // Annotation present, wrap with mock (covers @Before + @Test) + return new Statement() { + @Override + public void evaluate() throws Throwable { + try (MockedStatic tableOps = + mockStatic(HMSTablePropertyHelper.class, CALLS_REAL_METHODS)) { + tableOps.when(() -> setStorageHandlerMethod.invoke(null, anyMap(), eq(true))) + .thenAnswer(invocation -> null); + base.evaluate(); + } + } + }; + } + } +} diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/utils/HiveIcebergStorageHandlerTestUtils.java similarity index 77% rename from iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java rename to iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/utils/HiveIcebergStorageHandlerTestUtils.java index 98ed9cf2b7d5..6c200984ddff 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/utils/HiveIcebergStorageHandlerTestUtils.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iceberg.mr.hive; +package org.apache.iceberg.mr.hive.test.utils; import java.io.IOException; import java.util.Collections; @@ -30,46 +30,48 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.mr.Catalogs; import org.apache.iceberg.mr.TestHelper; +import org.apache.iceberg.mr.hive.test.TestHiveShell; +import org.apache.iceberg.mr.hive.test.TestTables; import org.apache.iceberg.types.Types; import org.junit.rules.TemporaryFolder; import static org.apache.iceberg.types.Types.NestedField.optional; public class HiveIcebergStorageHandlerTestUtils { - static final FileFormat[] FILE_FORMATS = + public static final FileFormat[] FILE_FORMATS = new FileFormat[] {FileFormat.AVRO, FileFormat.ORC, FileFormat.PARQUET}; - static final Schema CUSTOMER_SCHEMA = new Schema( + public static final Schema CUSTOMER_SCHEMA = new Schema( optional(1, "customer_id", Types.LongType.get()), optional(2, "first_name", Types.StringType.get(), "This is first name"), optional(3, "last_name", Types.StringType.get(), "This is last name") ); - static final Schema CUSTOMER_SCHEMA_WITH_UPPERCASE = new Schema( + public static final Schema CUSTOMER_SCHEMA_WITH_UPPERCASE = new Schema( optional(1, "CustomER_Id", Types.LongType.get()), optional(2, "First_name", Types.StringType.get()), optional(3, "Last_name", Types.StringType.get()) ); - static final Schema USER_CLICKS_SCHEMA = new Schema( + public static final Schema USER_CLICKS_SCHEMA = new Schema( optional(1, "name", Types.StringType.get()), optional(2, "age", Types.IntegerType.get()), optional(3, "num_clicks", Types.IntegerType.get()) ); - static final List CUSTOMER_RECORDS = TestHelper.RecordsBuilder.newInstance(CUSTOMER_SCHEMA) + public static final List CUSTOMER_RECORDS = TestHelper.RecordsBuilder.newInstance(CUSTOMER_SCHEMA) .add(0L, "Alice", "Brown") .add(1L, "Bob", "Green") .add(2L, "Trudy", "Pink") .build(); - static final List OTHER_CUSTOMER_RECORDS_1 = TestHelper.RecordsBuilder.newInstance(CUSTOMER_SCHEMA) + public static final List OTHER_CUSTOMER_RECORDS_1 = TestHelper.RecordsBuilder.newInstance(CUSTOMER_SCHEMA) .add(3L, "Marci", "Barna") .add(4L, "Laci", "Zold") .add(5L, "Peti", "Rozsaszin") .build(); - static final List OTHER_CUSTOMER_RECORDS_2 = TestHelper.RecordsBuilder.newInstance(CUSTOMER_SCHEMA) + public static final List OTHER_CUSTOMER_RECORDS_2 = TestHelper.RecordsBuilder.newInstance(CUSTOMER_SCHEMA) .add(1L, "Joanna", "Pierce") .add(1L, "Sharon", "Taylor") .add(2L, "Joanna", "Silver") @@ -81,14 +83,14 @@ public class HiveIcebergStorageHandlerTestUtils { .add(3L, "Trudy", "Henderson") .build(); - static final List USER_CLICKS_RECORDS_1 = TestHelper.RecordsBuilder + public static final List USER_CLICKS_RECORDS_1 = TestHelper.RecordsBuilder .newInstance(USER_CLICKS_SCHEMA) .add("amy", 35, 12341234) .add("bob", 66, 123471) .add("cal", 21, 431) .build(); - static final List USER_CLICKS_RECORDS_2 = TestHelper.RecordsBuilder + public static final List USER_CLICKS_RECORDS_2 = TestHelper.RecordsBuilder .newInstance(USER_CLICKS_SCHEMA) .add("amy", 52, 22323) .add("drake", 44, 34222) @@ -99,11 +101,11 @@ private HiveIcebergStorageHandlerTestUtils() { // Empty constructor for the utility class } - static TestHiveShell shell() { + public static TestHiveShell shell() { return shell(Collections.emptyMap()); } - static TestHiveShell shell(Map configs) { + public static TestHiveShell shell(Map configs) { TestHiveShell shell = new TestHiveShell(); shell.setHiveConfValue("hive.notification.event.poll.interval", "-1"); shell.setHiveConfValue("hive.tez.exec.print.summary", "true"); @@ -116,21 +118,21 @@ static TestHiveShell shell(Map configs) { return shell; } - static TestTables testTables(TestHiveShell shell, TestTables.TestTableType testTableType, TemporaryFolder temp) + public static TestTables testTables(TestHiveShell shell, TestTables.TestTableType testTableType, TemporaryFolder temp) throws IOException { return testTables(shell, testTableType, temp, Catalogs.ICEBERG_DEFAULT_CATALOG_NAME); } - static TestTables testTables(TestHiveShell shell, TestTables.TestTableType testTableType, TemporaryFolder temp, + public static TestTables testTables(TestHiveShell shell, TestTables.TestTableType testTableType, TemporaryFolder temp, String catalogName) throws IOException { return testTableType.instance(shell.metastore().hiveConf(), temp, catalogName); } - static void init(TestHiveShell shell, TestTables testTables, TemporaryFolder temp) { + public static void init(TestHiveShell shell, TestTables testTables, TemporaryFolder temp) { init(shell, testTables, temp, "tez"); } - static void init(TestHiveShell shell, TestTables testTables, TemporaryFolder temp, String engine) { + public static void init(TestHiveShell shell, TestTables testTables, TemporaryFolder temp, String engine) { shell.getSession(); for (Map.Entry property : testTables.properties().entrySet()) { @@ -146,7 +148,7 @@ static void init(TestHiveShell shell, TestTables testTables, TemporaryFolder tem DefaultMetricsSystem.instance().init("TestMetrics"); } - static void close(TestHiveShell shell) throws Exception { + public static void close(TestHiveShell shell) throws Exception { shell.closeSession(); shell.metastore().reset(); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/utils/HiveIcebergTestUtils.java similarity index 98% rename from iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java rename to iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/utils/HiveIcebergTestUtils.java index 0e0104720aad..7d32c684c141 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/test/utils/HiveIcebergTestUtils.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iceberg.mr.hive; +package org.apache.iceberg.mr.hive.test.utils; import java.io.File; import java.io.IOException; @@ -40,7 +40,6 @@ import java.util.List; import java.util.Map; import java.util.UUID; -import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.common.type.HiveDecimal; @@ -77,6 +76,8 @@ import org.apache.iceberg.hadoop.HadoopOutputFile; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.mr.hive.HiveTableUtil; +import org.apache.iceberg.mr.hive.test.TestHiveShell; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.ArrayUtil; @@ -206,7 +207,7 @@ public static List valueForRow(Schema schema, List rows) { record.set(i, row[i]); } return record; - }).collect(Collectors.toList()); + }).toList(); } /** @@ -291,12 +292,12 @@ public static void validateFiles(Table table, Configuration conf, JobID jobId, i files .filter(Files::isRegularFile) .filter(path -> !path.getFileName().toString().startsWith(".")) - .collect(Collectors.toList()); + .toList(); } Assert.assertEquals(dataFileNum, dataFiles.size()); Assert.assertFalse( - new File(HiveIcebergOutputCommitter.generateJobLocation(table.location(), conf, jobId)).exists()); + new File(HiveTableUtil.jobLocation(table.location(), conf, jobId)).exists()); } /** @@ -346,7 +347,7 @@ public static DeleteFile createEqualityDeleteFile(Table table, String deleteFile FileFormat fileFormat, List rowsToDelete) throws IOException { List equalityFieldIds = equalityFields.stream() .map(id -> table.schema().findField(id).fieldId()) - .collect(Collectors.toList()); + .toList(); Schema eqDeleteRowSchema = table.schema().select(equalityFields.toArray(new String[]{})); FileAppenderFactory appenderFactory = new GenericAppenderFactory(table.schema(), table.spec(), diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/vector/TestHiveIcebergVectorization.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/vector/TestHiveIcebergVectorization.java index dfad82947172..2a8b0bdc6ed1 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/vector/TestHiveIcebergVectorization.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/vector/TestHiveIcebergVectorization.java @@ -19,7 +19,7 @@ package org.apache.iceberg.mr.hive.vector; -import java.util.Collections; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -53,20 +53,30 @@ import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.mr.TestHelper; import org.apache.iceberg.mr.hive.HiveIcebergStorageHandlerWithEngineBase; -import org.apache.iceberg.mr.hive.TestTables; import org.apache.iceberg.mr.hive.serde.objectinspector.IcebergObjectInspector; +import org.apache.iceberg.mr.hive.test.TestTables.TestTableType; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.junit.Assume.assumeTrue; public class TestHiveIcebergVectorization extends HiveIcebergStorageHandlerWithEngineBase { + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") + public static Collection parameters() { + return HiveIcebergStorageHandlerWithEngineBase.getParameters(p -> + p.testTableType() == TestTableType.HIVE_CATALOG && + p.isVectorized() && + p.formatVersion() == 2); + } + /** * Tests the row iterator implementation (HiveRow, HiveBatchContext.RowIterator) along with HiveValueConverter by * reading in values from all supported types via VRBs, and iterating on its records 1-by-1 while comparing with the @@ -75,7 +85,7 @@ public class TestHiveIcebergVectorization extends HiveIcebergStorageHandlerWithE */ @Test public void testRowIterator() throws Exception { - assumeTrue("Tests a format-independent feature", isVectorized && FileFormat.ORC.equals(fileFormat)); + assumeTrue("Tests a format-independent feature", fileFormat == FileFormat.ORC); // Create a table with sample data with all supported types, those unsupported for vectorization are commented out Schema allSchema = new Schema( @@ -140,21 +150,23 @@ public void testRowIterator() throws Exception { */ @Test public void testHiveDeleteFilterWithEmptyBatches() { - Map props = Maps.newHashMap(); - props.put("parquet.block.size", "8192"); - props.put("parquet.page.row.count.limit", "20"); - testVectorizedReadWithDeleteFilter(props); + testVectorizedReadWithDeleteFilter( + ImmutableMap.of( + "parquet.block.size", "8192", + "parquet.page.row.count.limit", "20") + ); } @Test public void testHiveDeleteFilter() { - testVectorizedReadWithDeleteFilter(Collections.emptyMap()); + testVectorizedReadWithDeleteFilter( + ImmutableMap.of() + ); } private void testVectorizedReadWithDeleteFilter(Map props) { // The Avro "vectorized" case should actually serve as compareTo scenario to non-vectorized reading, because // there's no vectorization for Avro and it falls back to the non-vectorized implementation - Assume.assumeTrue(isVectorized && testTableType == TestTables.TestTableType.HIVE_CATALOG); // Minimal schema to minimize resource footprint of what's coming next... Schema schema = new Schema( @@ -207,8 +219,7 @@ private void testVectorizedReadWithDeleteFilter(Map props) { @Test public void testHiveDeleteFilterWithFilteredParquetBlock() { - Assume.assumeTrue( - isVectorized && testTableType == TestTables.TestTableType.HIVE_CATALOG && fileFormat == FileFormat.PARQUET); + Assume.assumeTrue(fileFormat == FileFormat.PARQUET); Schema schema = new Schema( optional(1, "customer_id", Types.LongType.get()),