diff --git a/iceberg/checkstyle/checkstyle.xml b/iceberg/checkstyle/checkstyle.xml index 1da57070f866..aef7d6fff6f8 100644 --- a/iceberg/checkstyle/checkstyle.xml +++ b/iceberg/checkstyle/checkstyle.xml @@ -119,6 +119,7 @@ java.util.Collections.*, java.util.stream.Collectors.*, org.apache.commons.lang3.Validate.*, + org.apache.hadoop.hive.ql.metadata.VirtualColumn.*, org.apache.iceberg.expressions.Expressions.*, org.apache.iceberg.expressions.Expression.Operation.*, org.apache.iceberg.IsolationLevel.*, diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java index 819ad626855e..e15e556f12a8 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java @@ -52,6 +52,7 @@ private InputFormatConfig() { public static final String SERIALIZED_TABLE_PREFIX = "iceberg.mr.serialized.table."; public static final String TABLE_CATALOG_PREFIX = "iceberg.mr.table.catalog."; public static final String LOCALITY = "iceberg.mr.locality"; + public static final String WRITE_FANOUT_ENABLED = "write.fanout.enabled"; public static final String CTAS_TABLE_NAME = "iceberg.mr.ctas.table.name"; public static final String SELECTED_COLUMNS = "iceberg.mr.selected.columns"; 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 f22e66c06232..44049a30f4fa 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 @@ -22,7 +22,6 @@ import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; -import java.util.AbstractMap.SimpleImmutableEntry; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -36,6 +35,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -83,10 +83,11 @@ 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.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.relocated.com.google.common.collect.Multimap; +import org.apache.iceberg.relocated.com.google.common.collect.Multimaps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.iceberg.util.Tasks; @@ -137,6 +138,7 @@ public void commitTask(TaskAttemptContext originalContext) throws IOException { JobConf jobConf = context.getJobConf(); Set mergedPaths = getCombinedLocations(jobConf); Set outputs = HiveIcebergStorageHandler.outputTables(context.getJobConf()); + Map> writers = Optional.ofNullable(WriterRegistry.writers(attemptID)) .orElseGet(() -> { LOG.info("CommitTask found no writers for output tables: {}, attemptID: {}", outputs, attemptID); @@ -161,6 +163,7 @@ public void commitTask(TaskAttemptContext originalContext) throws IOException { List deleteFiles = Lists.newArrayList(); List replacedDataFiles = Lists.newArrayList(); Set referencedDataFiles = Sets.newHashSet(); + for (HiveIcebergWriter writer : writers.get(output)) { FilesForCommit files = writer.files(); dataFiles.addAll(files.dataFiles()); @@ -168,8 +171,8 @@ public void commitTask(TaskAttemptContext originalContext) throws IOException { replacedDataFiles.addAll(files.replacedDataFiles()); referencedDataFiles.addAll(files.referencedDataFiles()); } - createFileForCommit(new FilesForCommit(dataFiles, deleteFiles, replacedDataFiles, referencedDataFiles, - mergedPaths), + createFileForCommit( + new FilesForCommit(dataFiles, deleteFiles, replacedDataFiles, referencedDataFiles, mergedPaths), fileForCommitLocation, table.io()); } else { LOG.info("CommitTask found no writer for specific table: {}, attemptID: {}", output, attemptID); @@ -226,7 +229,6 @@ private static class OutputTable { private final String catalogName; private final String tableName; private final Table table; - private List jobContexts; private OutputTable(String catalogName, String tableName, Table table) { this.catalogName = catalogName; @@ -234,10 +236,6 @@ private OutputTable(String catalogName, String tableName, Table table) { this.table = table; } - public void setJobContexts(List jobContexts) { - this.jobContexts = ImmutableList.copyOf(jobContexts); - } - @Override public boolean equals(Object o) { if (this == o) { @@ -266,33 +264,35 @@ public void commitJobs(List originalContextList, Operation operation List jobContextList = originalContextList.stream() .map(TezUtil::enrichContextWithVertexId) .collect(Collectors.toList()); - List outputs = collectOutputs(jobContextList); + Multimap outputs = collectOutputs(jobContextList); + JobConf jobConf = jobContextList.get(0).getJobConf(); long startTime = System.currentTimeMillis(); String ids = jobContextList.stream() .map(jobContext -> jobContext.getJobID().toString()).collect(Collectors.joining(",")); LOG.info("Committing job(s) {} has started", ids); + ExecutorService fileExecutor = fileExecutor(jobConf); + ExecutorService tableExecutor = tableExecutor(jobConf, outputs.keySet().size()); + Collection jobLocations = new ConcurrentLinkedQueue<>(); - ExecutorService fileExecutor = fileExecutor(jobContextList.get(0).getJobConf()); - ExecutorService tableExecutor = tableExecutor(jobContextList.get(0).getJobConf(), outputs.size()); try { // Commits the changes for the output tables in parallel - Tasks.foreach(outputs) + Tasks.foreach(outputs.keySet()) .throwFailureWhenFinished() .stopOnFailure() .executeWith(tableExecutor) .run(output -> { - Table table = output.table; - jobLocations.addAll( - output.jobContexts.stream().map(jobContext -> - generateJobLocation(table.location(), jobContext.getJobConf(), jobContext.getJobID())) - .collect(Collectors.toList())); - commitTable(table.io(), fileExecutor, output, operation); + final Collection jobContexts = outputs.get(output); + final Table table = output.table; + jobContexts.forEach(jobContext -> jobLocations.add( + generateJobLocation(table.location(), jobConf, jobContext.getJobID())) + ); + commitTable(table.io(), fileExecutor, output, jobContexts, operation); }); // Cleanup any merge input files. - cleanMergeTaskInputFiles(jobContextList, tableExecutor); + cleanMergeTaskInputFiles(jobContextList, fileExecutor); } finally { fileExecutor.shutdown(); if (tableExecutor != null) { @@ -307,27 +307,24 @@ public void commitJobs(List originalContextList, Operation operation } } - private List collectOutputs(List jobContextList) { - return jobContextList.stream() - .flatMap(jobContext -> HiveIcebergStorageHandler.outputTables(jobContext.getJobConf()).stream() - .map(output -> new OutputTable( - HiveIcebergStorageHandler.catalogName(jobContext.getJobConf(), output), - output, - SessionStateUtil.getResource(jobContext.getJobConf(), output) + private Multimap collectOutputs(List jobContextList) { + Multimap outputs = + Multimaps.newListMultimap(Maps.newHashMap(), Lists::newArrayList); + for (JobContext jobContext : jobContextList) { + for (String output : HiveIcebergStorageHandler.outputTables(jobContext.getJobConf())) { + Table table = SessionStateUtil.getResource(jobContext.getJobConf(), output) .filter(o -> o instanceof Table).map(o -> (Table) o) // fall back to getting the serialized table from the config - .orElseGet(() -> HiveIcebergStorageHandler.table(jobContext.getJobConf(), output)))) - .filter(output -> Objects.nonNull(output.table)) - .map(output -> new SimpleImmutableEntry<>(output, jobContext))) - .collect( - Collectors.groupingBy(Map.Entry::getKey, - Collectors.mapping(Map.Entry::getValue, Collectors.toList())) - ).entrySet().stream().map( - kv -> { - kv.getKey().setJobContexts(kv.getValue()); - return kv.getKey(); - }) - .collect(Collectors.toList()); + .orElseGet(() -> HiveIcebergStorageHandler.table(jobContext.getJobConf(), output)); + if (table != null) { + String catalogName = HiveIcebergStorageHandler.catalogName(jobContext.getJobConf(), output); + outputs.put(new OutputTable(catalogName, output, table), jobContext); + } else { + LOG.info("Found no table object in QueryState or conf for: {}. Skipping job commit.", output); + } + } + } + return outputs; } /** @@ -346,43 +343,42 @@ public void abortJobs(List originalContextList) throws IOException { List jobContextList = originalContextList.stream() .map(TezUtil::enrichContextWithVertexId) .collect(Collectors.toList()); - List outputs = collectOutputs(jobContextList); + Multimap outputs = collectOutputs(jobContextList); + JobConf jobConf = jobContextList.get(0).getJobConf(); String ids = jobContextList.stream() .map(jobContext -> jobContext.getJobID().toString()).collect(Collectors.joining(",")); LOG.info("Job(s) {} are aborted. Data file cleaning started", ids); - Collection jobLocations = new ConcurrentLinkedQueue<>(); + ExecutorService fileExecutor = fileExecutor(jobConf); + ExecutorService tableExecutor = tableExecutor(jobConf, outputs.keySet().size()); - ExecutorService fileExecutor = fileExecutor(jobContextList.get(0).getJobConf()); - ExecutorService tableExecutor = tableExecutor(jobContextList.get(0).getJobConf(), outputs.size()); + Collection jobLocations = new ConcurrentLinkedQueue<>(); try { // Cleans up the changes for the output tables in parallel - Tasks.foreach(outputs.stream().flatMap(kv -> kv.jobContexts.stream() - .map(jobContext -> new SimpleImmutableEntry<>(kv.table, jobContext)))) + Tasks.foreach(outputs.keySet()) .suppressFailureWhenFinished() .executeWith(tableExecutor) .onFailure((output, exc) -> LOG.warn("Failed cleanup table {} on abort job", output, exc)) .run(output -> { - JobContext jobContext = output.getValue(); - JobConf jobConf = jobContext.getJobConf(); - LOG.info("Cleaning job for jobID: {}, table: {}", jobContext.getJobID(), output); - - Table table = output.getKey(); - String jobLocation = generateJobLocation(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 - int numTasks = listForCommits(jobConf, jobLocation).size(); - FilesForCommit results = collectResults(numTasks, fileExecutor, table.location(), jobContext, - table.io(), false); - // Check if we have files already written and remove data and delta files if there are any - Tasks.foreach(results.allFiles()) - .retry(3) - .suppressFailureWhenFinished() - .executeWith(fileExecutor) - .onFailure((file, exc) -> LOG.warn("Failed to remove data file {} on abort job", file.path(), exc)) - .run(file -> table.io().deleteFile(file.path().toString())); + 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()); + 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 + int numTasks = listForCommits(jobConf, jobLocation).size(); + FilesForCommit results = collectResults(numTasks, fileExecutor, table.location(), jobContext, + table.io(), false); + // Check if we have files already written and remove data and delta files if there are any + Tasks.foreach(results.allFiles()) + .retry(3) + .suppressFailureWhenFinished() + .executeWith(fileExecutor) + .onFailure((file, exc) -> LOG.warn("Failed to remove data file {} on abort job", file.path(), exc)) + .run(file -> table.io().deleteFile(file.path().toString())); + } }, IOException.class); } finally { fileExecutor.shutdown(); @@ -409,7 +405,7 @@ public void abortJobs(List originalContextList) throws IOException { */ private Set listForCommits(JobConf jobConf, String jobLocation) throws IOException { Path path = new Path(jobLocation); - LOG.debug("Listing job location to get forCommits for abort: {}", jobLocation); + LOG.debug("Listing job location to get commitTask manifest files for abort: {}", jobLocation); FileStatus[] children = path.getFileSystem(jobConf).listStatus(path); LOG.debug("Listing the job location: {} yielded these files: {}", jobLocation, Arrays.toString(children)); return Arrays.stream(children) @@ -423,11 +419,14 @@ private Set listForCommits(JobConf jobConf, String jobLocation) thro * @param executor The executor used to read the forCommit files * @param outputTable The table used for loading from the catalog */ - private void commitTable(FileIO io, ExecutorService executor, OutputTable outputTable, Operation operation) { + private void commitTable(FileIO io, ExecutorService executor, OutputTable outputTable, + Collection jobContexts, Operation operation) { String name = outputTable.tableName; + Properties catalogProperties = new Properties(); catalogProperties.put(Catalogs.NAME, name); catalogProperties.put(Catalogs.LOCATION, outputTable.table.location()); + if (outputTable.catalogName != null) { catalogProperties.put(InputFormatConfig.CATALOG_NAME, outputTable.catalogName); } @@ -443,7 +442,7 @@ private void commitTable(FileIO io, ExecutorService executor, OutputTable output Long snapshotId = null; Expression filterExpr = Expressions.alwaysTrue(); - for (JobContext jobContext : outputTable.jobContexts) { + for (JobContext jobContext : jobContexts) { JobConf conf = jobContext.getJobConf(); table = Optional.ofNullable(table).orElse(Catalogs.loadTable(conf, catalogProperties)); branchName = conf.get(InputFormatConfig.OUTPUT_TABLE_SNAPSHOT_REF); @@ -488,20 +487,20 @@ private void commitTable(FileIO io, ExecutorService executor, OutputTable output if (filesForCommit.isEmpty()) { LOG.info( "Not creating a new commit for table: {}, jobIDs: {}, since there were no new files to add", - table, outputTable.jobContexts.stream().map(JobContext::getJobID) + table, jobContexts.stream().map(JobContext::getJobID) .map(String::valueOf).collect(Collectors.joining(","))); } else { commitWrite(table, branchName, snapshotId, startTime, filesForCommit, operation, filterExpr); } } else { - RewritePolicy rewritePolicy = RewritePolicy.fromString(outputTable.jobContexts.stream() + RewritePolicy rewritePolicy = RewritePolicy.fromString(jobContexts.stream() .findAny() .map(x -> x.getJobConf().get(ConfVars.REWRITE_POLICY.varname)) .orElse(RewritePolicy.DEFAULT.name())); if (rewritePolicy != RewritePolicy.DEFAULT) { - String partitionPath = outputTable.jobContexts.stream() + String partitionPath = jobContexts.stream() .findAny() .map(x -> x.getJobConf().get(IcebergCompactionService.PARTITION_PATH)) .orElse(null); @@ -753,8 +752,10 @@ private static FilesForCommit collectResults(int numTasks, ExecutorService execu .executeWith(executor) .retry(3) .run(taskId -> { - String taskFileName = generateFileForCommitLocation(location, conf, jobContext.getJobID(), taskId); - FilesForCommit files = readFileForCommit(taskFileName, io); + final String taskFileName = generateFileForCommitLocation(location, conf, jobContext.getJobID(), taskId); + final FilesForCommit files = readFileForCommit(taskFileName, io); + LOG.debug("Found Iceberg commitTask manifest file: {}\n{}", taskFileName, files); + dataFiles.addAll(files.dataFiles()); deleteFiles.addAll(files.deleteFiles()); replacedDataFiles.addAll(files.replacedDataFiles()); @@ -798,14 +799,14 @@ private static void createFileForCommit(FilesForCommit writeResult, String locat try (ObjectOutputStream oos = new ObjectOutputStream(fileForCommit.createOrOverwrite())) { oos.writeObject(writeResult); } - LOG.debug("Iceberg committed file is created {}", fileForCommit); + LOG.debug("Created Iceberg commitTask manifest file: {}\n{}", location, writeResult); } private static FilesForCommit readFileForCommit(String fileForCommitLocation, FileIO io) { try (ObjectInputStream ois = new ObjectInputStream(io.newInputFile(fileForCommitLocation).newStream())) { return (FilesForCommit) ois.readObject(); } catch (ClassNotFoundException | IOException e) { - throw new NotFoundException("Can not read or parse committed file: %s", fileForCommitLocation); + throw new NotFoundException("Can not read or parse commitTask manifest file: %s", fileForCommitLocation); } } @@ -816,55 +817,54 @@ private static FilesForCommit readFileForCommit(String fileForCommitLocation, Fi * @throws IOException Throws IOException */ public List getOutputFiles(List jobContexts) throws IOException { - List outputs = collectOutputs(jobContexts); - ExecutorService fileExecutor = fileExecutor(jobContexts.get(0).getJobConf()); - ExecutorService tableExecutor = tableExecutor(jobContexts.get(0).getJobConf(), outputs.size()); + Multimap outputs = collectOutputs(jobContexts); + JobConf jobConf = jobContexts.get(0).getJobConf(); + + ExecutorService fileExecutor = fileExecutor(jobConf); + ExecutorService tableExecutor = tableExecutor(jobConf, outputs.keySet().size()); + Map> parentDirToDataFile = Maps.newConcurrentMap(); Map> parentDirToDeleteFile = Maps.newConcurrentMap(); try { - Tasks.foreach(outputs.stream().flatMap(kv -> kv.jobContexts.stream() - .map(jobContext -> new SimpleImmutableEntry<>(kv.table, jobContext)))) - .suppressFailureWhenFinished() - .executeWith(tableExecutor) - .onFailure((output, exc) -> LOG.warn("Failed to retrieve merge input file for the table {}", output, exc)) - .run(output -> { - JobContext jobContext = output.getValue(); - JobConf jobConf = jobContext.getJobConf(); - LOG.info("Cleaning job for jobID: {}, table: {}", jobContext.getJobID(), output); - - Table table = output.getKey(); - FileSystem fileSystem = new Path(table.location()).getFileSystem(jobConf); - String jobLocation = generateJobLocation(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 - int numTasks = listForCommits(jobConf, jobLocation).size(); - FilesForCommit results = collectResults(numTasks, fileExecutor, table.location(), jobContext, - table.io(), false); - for (DataFile dataFile : results.dataFiles()) { - Path filePath = new Path(dataFile.path().toString()); - FileStatus fileStatus = fileSystem.getFileStatus(filePath); - parentDirToDataFile.computeIfAbsent(filePath.getParent(), k -> Lists.newArrayList()).add(fileStatus); - } - for (DeleteFile deleteFile : results.deleteFiles()) { - Path filePath = new Path(deleteFile.path().toString()); - FileStatus fileStatus = fileSystem.getFileStatus(filePath); - parentDirToDeleteFile.computeIfAbsent(filePath.getParent(), - k -> Lists.newArrayList()).add(fileStatus); - } - }, IOException.class); + Tasks.foreach(outputs.keySet()) + .suppressFailureWhenFinished() + .executeWith(tableExecutor) + .onFailure((output, exc) -> LOG.warn("Failed to retrieve merge input file for the table {}", output, exc)) + .run(output -> { + for (JobContext jobContext : outputs.get(output)) { + LOG.info("Cleaning job for jobID: {}, table: {}", jobContext.getJobID(), output); + Table table = output.table; + FileSystem fileSystem = new Path(table.location()).getFileSystem(jobConf); + String jobLocation = generateJobLocation(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 + int numTasks = listForCommits(jobConf, jobLocation).size(); + FilesForCommit results = collectResults(numTasks, fileExecutor, table.location(), jobContext, + table.io(), false); + for (DataFile dataFile : results.dataFiles()) { + Path filePath = new Path(dataFile.path().toString()); + parentDirToDataFile.computeIfAbsent(filePath.getParent(), k -> Lists.newArrayList()) + .add(fileSystem.getFileStatus(filePath)); + } + for (DeleteFile deleteFile : results.deleteFiles()) { + Path filePath = new Path(deleteFile.path().toString()); + parentDirToDeleteFile.computeIfAbsent(filePath.getParent(), k -> Lists.newArrayList()) + .add(fileSystem.getFileStatus(filePath)); + } + } + }, IOException.class); } finally { fileExecutor.shutdown(); if (tableExecutor != null) { tableExecutor.shutdown(); } } - List dataFiles = Lists.newArrayList(); - dataFiles.addAll(parentDirToDataFile.values().stream() - .flatMap(List::stream).collect(Collectors.toList())); - dataFiles.addAll(parentDirToDeleteFile.values().stream() - .flatMap(List::stream).collect(Collectors.toList())); - return dataFiles; + return Stream.concat( + parentDirToDataFile.values().stream(), + parentDirToDeleteFile.values().stream()) + .flatMap(List::stream) + .collect(Collectors.toList()); } /** @@ -874,32 +874,33 @@ public List getOutputFiles(List jobContexts) throws IOEx * @throws IOException Throws IOException */ public List getOutputContentFiles(List jobContexts) throws IOException { - List outputs = collectOutputs(jobContexts); - ExecutorService fileExecutor = fileExecutor(jobContexts.get(0).getJobConf()); - ExecutorService tableExecutor = tableExecutor(jobContexts.get(0).getJobConf(), outputs.size()); + Multimap outputs = collectOutputs(jobContexts); + JobConf jobConf = jobContexts.get(0).getJobConf(); + + ExecutorService fileExecutor = fileExecutor(jobConf); + ExecutorService tableExecutor = tableExecutor(jobConf, outputs.keySet().size()); + Collection files = new ConcurrentLinkedQueue<>(); try { - Tasks.foreach(outputs.stream().flatMap(kv -> kv.jobContexts.stream() - .map(jobContext -> new SimpleImmutableEntry<>(kv.table, jobContext)))) - .suppressFailureWhenFinished() - .executeWith(tableExecutor) - .onFailure((output, exc) -> LOG.warn("Failed to retrieve merge input file for the table {}", output, exc)) - .run(output -> { - JobContext jobContext = output.getValue(); - JobConf jobConf = jobContext.getJobConf(); - LOG.info("Cleaning job for jobID: {}, table: {}", jobContext.getJobID(), output); - - Table table = output.getKey(); - String jobLocation = generateJobLocation(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 - int numTasks = listForCommits(jobConf, jobLocation).size(); - FilesForCommit results = collectResults(numTasks, fileExecutor, table.location(), jobContext, - table.io(), false); - files.addAll(results.dataFiles()); - files.addAll(results.deleteFiles()); - }, IOException.class); + Tasks.foreach(outputs.keySet()) + .suppressFailureWhenFinished() + .executeWith(tableExecutor) + .onFailure((output, exc) -> LOG.warn("Failed to retrieve merge input file for the table {}", output, exc)) + .run(output -> { + 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()); + // 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 + int numTasks = listForCommits(jobConf, jobLocation).size(); + FilesForCommit results = collectResults(numTasks, fileExecutor, table.location(), jobContext, + table.io(), false); + files.addAll(results.dataFiles()); + files.addAll(results.deleteFiles()); + } + }, IOException.class); } finally { fileExecutor.shutdown(); if (tableExecutor != null) { @@ -909,30 +910,25 @@ public List getOutputContentFiles(List jobContexts) thr return Lists.newArrayList(files); } - private void cleanMergeTaskInputFiles(List jobContexts, - ExecutorService tableExecutor) throws IOException { + private void cleanMergeTaskInputFiles(List jobContexts, ExecutorService fileExecutor) throws IOException { // Merge task has merged several files into one. Hence we need to remove the stale files. // At this stage the file is written and task-committed, but the old files are still present. - for (JobContext jobContext : jobContexts) { - JobConf jobConf = jobContext.getJobConf(); - if (jobConf.getInputFormat().getClass().isAssignableFrom(CombineHiveInputFormat.class)) { - MapWork mrwork = Utilities.getMapWork(jobConf); - if (mrwork != null) { - List mergedPaths = mrwork.getInputPaths(); - if (mergedPaths != null) { - Tasks.foreach(mergedPaths) - .retry(3) - .executeWith(tableExecutor) - .run(path -> { - FileSystem fs = path.getFileSystem(jobConf); - if (fs.exists(path)) { - fs.delete(path, true); - } - }, IOException.class); + Stream mergedPaths = jobContexts.stream() + .map(JobContext::getJobConf) + .filter(jobConf -> jobConf.getInputFormat().getClass().isAssignableFrom(CombineHiveInputFormat.class)) + .map(Utilities::getMapWork).filter(Objects::nonNull) + .map(MapWork::getInputPaths).filter(Objects::nonNull) + .flatMap(List::stream); + + Tasks.foreach(mergedPaths) + .retry(3) + .executeWith(fileExecutor) + .run(path -> { + FileSystem fs = path.getFileSystem(jobContexts.get(0).getJobConf()); + if (fs.exists(path)) { + fs.delete(path, true); } - } - } - } + }, IOException.class); } /** @@ -942,8 +938,7 @@ private void cleanMergeTaskInputFiles(List jobContexts, * @param branchName the name of the branch * @return The generated Optional JobContext list or empty if not presents. */ - static List generateJobContext(Configuration configuration, String tableName, - String branchName) { + static List generateJobContext(Configuration configuration, String tableName, String branchName) { JobConf jobConf = new JobConf(configuration); Optional> commitInfoMap = SessionStateUtil.getCommitInfo(jobConf, tableName); diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputFormat.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputFormat.java index 30289912b1cf..0c51ad17a162 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputFormat.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputFormat.java @@ -41,10 +41,8 @@ import org.apache.iceberg.mr.mapred.Container; import org.apache.parquet.hadoop.ParquetOutputFormat; -public class HiveIcebergOutputFormat implements OutputFormat>, +public class HiveIcebergOutputFormat implements OutputFormat>, HiveOutputFormat> { - private static final String DELETE_FILE_THREAD_POOL_SIZE = "iceberg.delete.file.thread.pool.size"; - private static final int DELETE_FILE_THREAD_POOL_SIZE_DEFAULT = 10; @Override public FileSinkOperator.RecordWriter getHiveRecordWriter(JobConf jc, Path finalOutPath, Class valueClass, @@ -68,16 +66,14 @@ private static HiveIcebergWriter writer(JobConf jc) { // It gets the config from the FileSinkOperator which has its own config for every target table Table table = HiveIcebergStorageHandler.table(jc, jc.get(hive_metastoreConstants.META_TABLE_NAME)); String tableName = jc.get(Catalogs.NAME); - int poolSize = jc.getInt(DELETE_FILE_THREAD_POOL_SIZE, DELETE_FILE_THREAD_POOL_SIZE_DEFAULT); setWriterLevelConfiguration(jc, table); return WriterBuilder.builderFor(table) .queryId(jc.get(HiveConf.ConfVars.HIVE_QUERY_ID.varname)) .tableName(tableName) .attemptID(taskAttemptID) - .poolSize(poolSize) .operation(HiveCustomStorageHandlerUtils.getWriteOperation(jc::get, tableName)) - .isFanoutEnabled(!HiveCustomStorageHandlerUtils.getWriteOperationIsSorted(jc::get, tableName)) + .hasOrdering(HiveCustomStorageHandlerUtils.getWriteOperationIsSorted(jc::get, tableName)) .isMergeTask(HiveCustomStorageHandlerUtils.isMergeTaskEnabled(jc::get, tableName)) .build(); } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java index 38c0e5bb2e00..b72751b1188b 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java @@ -175,7 +175,8 @@ private static Schema projectedSchema(Configuration configuration, String tableN return projectedSchema; } } - if (IcebergTableUtil.isCopyOnWriteMode(operation, configuration::get)) { + boolean isCOW = IcebergTableUtil.isCopyOnWriteMode(operation, configuration::get); + if (isCOW) { return IcebergAcidUtil.createSerdeSchemaForDelete(tableSchema.columns()); } switch (operation) { diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index 4b7f87f0206d..50280a8a0abe 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -209,6 +209,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.hive.ql.metadata.VirtualColumn.FILE_PATH; +import static org.apache.hadoop.hive.ql.metadata.VirtualColumn.PARTITION_HASH; +import static org.apache.hadoop.hive.ql.metadata.VirtualColumn.PARTITION_PROJECTION; +import static org.apache.hadoop.hive.ql.metadata.VirtualColumn.PARTITION_SPEC_ID; +import static org.apache.hadoop.hive.ql.metadata.VirtualColumn.ROW_POSITION; + public class HiveIcebergStorageHandler implements HiveStoragePredicateHandler, HiveStorageHandler { private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergStorageHandler.class); @@ -222,13 +228,15 @@ public class HiveIcebergStorageHandler implements HiveStoragePredicateHandler, H public static final String TABLE_DEFAULT_LOCATION = "TABLE_DEFAULT_LOCATION"; - private static final List ACID_VIRTUAL_COLS = ImmutableList.of(VirtualColumn.PARTITION_SPEC_ID, - VirtualColumn.PARTITION_HASH, VirtualColumn.FILE_PATH, VirtualColumn.ROW_POSITION, - VirtualColumn.PARTITION_PROJECTION); - private static final List ACID_VIRTUAL_COLS_AS_FIELD_SCHEMA = ACID_VIRTUAL_COLS.stream() - .map(v -> new FieldSchema(v.getName(), v.getTypeInfo().getTypeName(), "")) - .collect(Collectors.toList()); + private static final List ACID_VIRTUAL_COLS = ImmutableList.of( + PARTITION_SPEC_ID, PARTITION_HASH, FILE_PATH, ROW_POSITION, PARTITION_PROJECTION); + + private static final List ACID_VIRTUAL_COLS_AS_FIELD_SCHEMA = schema(ACID_VIRTUAL_COLS); + + private static final List POSITION_DELETE_ORDERING = + orderBy(PARTITION_SPEC_ID, PARTITION_HASH, FILE_PATH, ROW_POSITION); + private static final List EMPTY_ORDERING = ImmutableList.of(); private Configuration conf; @@ -1208,7 +1216,7 @@ public List acidSelectColumns(org.apache.hadoop.hive.ql.metadata.Ta @Override public FieldSchema getRowId() { - VirtualColumn rowId = VirtualColumn.ROW_POSITION; + VirtualColumn rowId = ROW_POSITION; return new FieldSchema(rowId.getName(), rowId.getTypeInfo().getTypeName(), ""); } @@ -1216,11 +1224,14 @@ public FieldSchema getRowId() { public List acidSortColumns(org.apache.hadoop.hive.ql.metadata.Table table, Operation operation) { switch (operation) { case DELETE: - return ACID_VIRTUAL_COLS_AS_FIELD_SCHEMA; + return IcebergTableUtil.isFanoutEnabled(table.getParameters()) ? + EMPTY_ORDERING : POSITION_DELETE_ORDERING; + case MERGE: + return POSITION_DELETE_ORDERING; default: // For update operations we use the same sort order defined by // {@link #createDPContext(HiveConf, org.apache.hadoop.hive.ql.metadata.Table)} - return ImmutableList.of(); + return EMPTY_ORDERING; } } @@ -2151,14 +2162,7 @@ public boolean supportsMergeFiles() { @Override public List getMergeTaskInputFiles(Properties properties) throws IOException { - String tableName = properties.getProperty(Catalogs.NAME); - String snapshotRef = properties.getProperty(Catalogs.SNAPSHOT_REF); - Configuration configuration = SessionState.getSessionConf(); - List originalContextList = HiveIcebergOutputCommitter - .generateJobContext(configuration, tableName, snapshotRef); - List jobContextList = originalContextList.stream() - .map(TezUtil::enrichContextWithVertexId) - .collect(Collectors.toList()); + List jobContextList = IcebergMergeTaskProperties.getJobContexts(properties); if (jobContextList.isEmpty()) { return Collections.emptyList(); } @@ -2183,4 +2187,14 @@ public boolean hasUndergonePartitionEvolution(org.apache.hadoop.hive.ql.metadata Table table = IcebergTableUtil.getTable(conf, hmsTable.getTTable()); return hasUndergonePartitionEvolution(table); } + + private static List schema(List exprs) { + return exprs.stream().map(v -> + new FieldSchema(v.getName(), v.getTypeInfo().getTypeName(), "")) + .collect(Collectors.toList()); + } + + private static List orderBy(VirtualColumn... exprs) { + return schema(Arrays.asList(exprs)); + } } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergMergeTaskProperties.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergMergeTaskProperties.java index e3e6df6ceba1..0e8befac8871 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergMergeTaskProperties.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergMergeTaskProperties.java @@ -45,14 +45,7 @@ public Path getTmpLocation() { @Override public Properties getSplitProperties() throws IOException { - String tableName = properties.getProperty(Catalogs.NAME); - String snapshotRef = properties.getProperty(Catalogs.SNAPSHOT_REF); - Configuration configuration = SessionState.getSessionConf(); - List originalContextList = HiveIcebergOutputCommitter - .generateJobContext(configuration, tableName, snapshotRef); - List jobContextList = originalContextList.stream() - .map(TezUtil::enrichContextWithVertexId) - .collect(Collectors.toList()); + List jobContextList = getJobContexts(properties); if (jobContextList.isEmpty()) { return null; } @@ -62,4 +55,15 @@ public Properties getSplitProperties() throws IOException { pathToContentFile.put(new Path(String.valueOf(contentFile.path())), contentFile)); return pathToContentFile; } + + static List getJobContexts(Properties properties) { + String tableName = properties.getProperty(Catalogs.NAME); + String snapshotRef = properties.getProperty(Catalogs.SNAPSHOT_REF); + Configuration configuration = SessionState.getSessionConf(); + + return HiveIcebergOutputCommitter.generateJobContext(configuration, tableName, snapshotRef) + .stream() + .map(TezUtil::enrichContextWithVertexId) + .collect(Collectors.toList()); + } } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java index cd5d5da3b8ad..88dd006b7721 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java @@ -78,6 +78,7 @@ import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.StructProjection; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -351,6 +352,10 @@ public static boolean isCopyOnWriteMode(Context.Operation operation, BinaryOpera return RowLevelOperationMode.COPY_ON_WRITE.modeName().equalsIgnoreCase(mode); } + public static boolean isFanoutEnabled(Map props) { + return PropertyUtil.propertyAsBoolean(props, InputFormatConfig.WRITE_FANOUT_ENABLED, true); + } + public static void performMetadataDelete(Table icebergTable, String branchName, SearchArgument sarg) { Expression exp = HiveIcebergFilterFactory.generateFilterExpression(sarg); DeleteFiles deleteFiles = icebergTable.newDelete(); diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveFileWriterFactory.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveFileWriterFactory.java index 3459ea5b1e43..a24ff33bf4bc 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveFileWriterFactory.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveFileWriterFactory.java @@ -56,6 +56,10 @@ class HiveFileWriterFactory extends BaseFileWriterFactory { positionDeleteRowSchema); } + static Builder builderFor(Table table) { + return new Builder(table); + } + @Override protected void configureDataWrite(Avro.DataWriteBuilder builder) { builder.createWriterFunc(DataWriter::create); @@ -100,4 +104,49 @@ protected void configureEqualityDelete(ORC.DeleteWriteBuilder deleteWriteBuilder protected void configurePositionDelete(ORC.DeleteWriteBuilder deleteWriteBuilder) { deleteWriteBuilder.createWriterFunc(GenericOrcWriter::buildWriter); } + + static class Builder { + private final Table table; + private FileFormat dataFileFormat; + private Schema dataSchema; + private FileFormat deleteFileFormat; + private Schema positionDeleteRowSchema; + + Builder(Table table) { + this.table = table; + } + + Builder dataFileFormat(FileFormat newDataFileFormat) { + this.dataFileFormat = newDataFileFormat; + return this; + } + + Builder dataSchema(Schema newDataSchema) { + this.dataSchema = newDataSchema; + return this; + } + + Builder deleteFileFormat(FileFormat newDeleteFileFormat) { + this.deleteFileFormat = newDeleteFileFormat; + return this; + } + + Builder positionDeleteRowSchema(Schema newPositionDeleteRowSchema) { + this.positionDeleteRowSchema = newPositionDeleteRowSchema; + return this; + } + + HiveFileWriterFactory build() { + return new HiveFileWriterFactory( + table, + dataFileFormat, + dataSchema, + null, + deleteFileFormat, + null, + null, + null, + positionDeleteRowSchema); + } + } } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergCopyOnWriteRecordWriter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergCopyOnWriteRecordWriter.java index 142f73c85491..f3c04c279e74 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergCopyOnWriteRecordWriter.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergCopyOnWriteRecordWriter.java @@ -21,22 +21,18 @@ import java.io.IOException; import java.util.List; -import java.util.Map; import org.apache.hadoop.io.Writable; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.deletes.PositionDelete; -import org.apache.iceberg.io.ClusteredDataWriter; import org.apache.iceberg.io.DataWriteResult; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.mr.hive.FilesForCommit; import org.apache.iceberg.mr.hive.IcebergAcidUtil; +import org.apache.iceberg.mr.hive.writer.WriterBuilder.Context; import org.apache.iceberg.mr.mapred.Container; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -47,13 +43,12 @@ class HiveIcebergCopyOnWriteRecordWriter extends HiveIcebergWriterBase { private final GenericRecord rowDataTemplate; private final List replacedDataFiles; - HiveIcebergCopyOnWriteRecordWriter(Schema schema, Map specs, int currentSpecId, - FileWriterFactory fileWriterFactory, OutputFileFactory fileFactory, FileIO io, - long targetFileSize) { - super(schema, specs, io, - new ClusteredDataWriter<>(fileWriterFactory, fileFactory, io, targetFileSize)); - this.currentSpecId = currentSpecId; - this.rowDataTemplate = GenericRecord.create(schema); + HiveIcebergCopyOnWriteRecordWriter(Table table, HiveFileWriterFactory writerFactory, + OutputFileFactory deleteFileFactory, Context context) { + super(table, newDataWriter(table, writerFactory, deleteFileFactory, context)); + + this.currentSpecId = table.spec().specId(); + this.rowDataTemplate = GenericRecord.create(table.schema()); this.replacedDataFiles = Lists.newArrayList(); } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergDeleteWriter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergDeleteWriter.java index 101de089821b..064021118f2d 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergDeleteWriter.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergDeleteWriter.java @@ -21,23 +21,19 @@ import java.io.IOException; import java.util.List; -import java.util.Map; import java.util.Set; import org.apache.hadoop.io.Writable; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.deletes.PositionDelete; -import org.apache.iceberg.io.ClusteredPositionDeleteWriter; import org.apache.iceberg.io.DeleteWriteResult; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.mr.hive.FilesForCommit; import org.apache.iceberg.mr.hive.IcebergAcidUtil; +import org.apache.iceberg.mr.hive.writer.WriterBuilder.Context; import org.apache.iceberg.mr.mapred.Container; class HiveIcebergDeleteWriter extends HiveIcebergWriterBase { @@ -46,14 +42,13 @@ class HiveIcebergDeleteWriter extends HiveIcebergWriterBase { private final boolean skipRowData; private final boolean isMergeTask; - HiveIcebergDeleteWriter(Schema schema, Map specs, - FileWriterFactory writerFactory, OutputFileFactory fileFactory, FileIO io, - long targetFileSize, boolean skipRowData, boolean isMergeTask) { - super(schema, specs, io, - new ClusteredPositionDeleteWriter<>(writerFactory, fileFactory, io, targetFileSize)); - rowDataTemplate = GenericRecord.create(schema); - this.skipRowData = skipRowData; - this.isMergeTask = isMergeTask; + HiveIcebergDeleteWriter(Table table, HiveFileWriterFactory writerFactory, + OutputFileFactory deleteFileFactory, Context context) { + super(table, newDeleteWriter(table, writerFactory, deleteFileFactory, context)); + + this.rowDataTemplate = GenericRecord.create(table.schema()); + this.skipRowData = context.skipRowData(); + this.isMergeTask = context.isMergeTask(); } @Override diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergRecordWriter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergRecordWriter.java index b43376ec7f99..0f1989a3ac0f 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergRecordWriter.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergRecordWriter.java @@ -21,31 +21,25 @@ import java.io.IOException; import java.util.List; -import java.util.Map; import org.apache.hadoop.io.Writable; import org.apache.iceberg.DataFile; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.iceberg.data.Record; -import org.apache.iceberg.io.ClusteredDataWriter; import org.apache.iceberg.io.DataWriteResult; -import org.apache.iceberg.io.FanoutDataWriter; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.PartitioningWriter; import org.apache.iceberg.mr.hive.FilesForCommit; +import org.apache.iceberg.mr.hive.writer.WriterBuilder.Context; import org.apache.iceberg.mr.mapred.Container; class HiveIcebergRecordWriter extends HiveIcebergWriterBase { private final int currentSpecId; - HiveIcebergRecordWriter(Schema schema, Map specs, int currentSpecId, - FileWriterFactory fileWriterFactory, OutputFileFactory fileFactory, FileIO io, - long targetFileSize, boolean fanoutEnabled) { - super(schema, specs, io, getIcebergDataWriter(fileWriterFactory, fileFactory, io, targetFileSize, fanoutEnabled)); - this.currentSpecId = currentSpecId; + HiveIcebergRecordWriter(Table table, HiveFileWriterFactory fileWriterFactory, + OutputFileFactory dataFileFactory, Context context) { + super(table, newDataWriter(table, fileWriterFactory, dataFileFactory, context)); + + this.currentSpecId = table.spec().specId(); } @Override @@ -59,11 +53,4 @@ public FilesForCommit files() { List dataFiles = ((DataWriteResult) writer.result()).dataFiles(); return FilesForCommit.onlyData(dataFiles); } - - private static PartitioningWriter getIcebergDataWriter(FileWriterFactory fileWriterFactory, - OutputFileFactory fileFactory, FileIO io, - long targetFileSize, boolean fanoutEnabled) { - return fanoutEnabled ? new FanoutDataWriter<>(fileWriterFactory, fileFactory, io, targetFileSize) - : new ClusteredDataWriter<>(fileWriterFactory, fileFactory, io, targetFileSize); - } } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergWriter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergWriter.java index bf2e17a82142..0a3cce06611b 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergWriter.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergWriter.java @@ -22,7 +22,6 @@ import java.io.IOException; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapred.Reporter; import org.apache.iceberg.data.Record; import org.apache.iceberg.mr.hive.FilesForCommit; @@ -30,15 +29,14 @@ public interface HiveIcebergWriter extends FileSinkOperator.RecordWriter, org.apache.hadoop.mapred.RecordWriter> { + FilesForCommit files(); - void close(boolean abort) throws IOException; - void write(Writable row) throws IOException; default void close(Reporter reporter) throws IOException { close(false); } - default void write(NullWritable key, Container value) throws IOException { + default void write(NullWritable key, Container value) throws IOException { write(value); } } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergWriterBase.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergWriterBase.java index f286a0fa90dd..1f07bdf584d2 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergWriterBase.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/HiveIcebergWriterBase.java @@ -23,12 +23,22 @@ import java.util.Map; import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.iceberg.data.InternalRecordWrapper; import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.DeleteGranularity; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.io.ClusteredDataWriter; +import org.apache.iceberg.io.ClusteredPositionDeleteWriter; +import org.apache.iceberg.io.DataWriteResult; +import org.apache.iceberg.io.DeleteWriteResult; +import org.apache.iceberg.io.FanoutDataWriter; +import org.apache.iceberg.io.FanoutPositionOnlyDeleteWriter; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.io.PartitioningWriter; import org.apache.iceberg.mr.hive.FilesForCommit; +import org.apache.iceberg.mr.hive.writer.WriterBuilder.Context; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.Tasks; import org.slf4j.Logger; @@ -44,11 +54,10 @@ abstract class HiveIcebergWriterBase implements HiveIcebergWriter { protected final Map partitionKeys; protected final PartitioningWriter writer; - HiveIcebergWriterBase(Schema schema, Map specs, FileIO io, - PartitioningWriter writer) { - this.io = io; - this.wrapper = new InternalRecordWrapper(schema.asStruct()); - this.specs = specs; + HiveIcebergWriterBase(Table table, PartitioningWriter writer) { + this.io = table.io(); + this.wrapper = new InternalRecordWrapper(table.schema().asStruct()); + this.specs = table.specs(); this.partitionKeys = Maps.newHashMapWithExpectedSize(specs.size()); this.writer = writer; } @@ -65,10 +74,12 @@ public void close(boolean abort) throws IOException { .suppressFailureWhenFinished() .onFailure((file, exception) -> LOG.debug("Failed on to remove file {} on abort", file, exception)) .run(file -> io.deleteFile(file.path().toString())); + LOG.warn("HiveIcebergWriter is closed with abort"); } - LOG.info("HiveIcebergWriter is closed with abort={}. Created {} data files and {} delete files", abort, + LOG.info("Created {} data files and {} delete files", result.dataFiles().size(), result.deleteFiles().size()); + LOG.debug(result.toString()); } protected PartitionKey partition(Record row, int specId) { @@ -77,4 +88,39 @@ protected PartitionKey partition(Record row, int specId) { partitionKey.partition(wrapper.wrap(row)); return partitionKey; } + + // use a fanout writer only if enabled and the input is unordered and the table is partitioned + static PartitioningWriter newDataWriter( + Table table, HiveFileWriterFactory writers, OutputFileFactory files, Context context) { + + FileIO io = table.io(); + boolean useFanoutWriter = context.useFanoutWriter(); + long targetFileSize = context.targetDataFileSize(); + + if (table.spec().isPartitioned() && useFanoutWriter) { + return new FanoutDataWriter<>(writers, files, io, targetFileSize); + } else { + return new ClusteredDataWriter<>(writers, files, io, targetFileSize); + } + } + + // the spec requires position deletes to be ordered by file and position + // use a fanout writer if the input is unordered no matter whether fanout writers are enabled + // clustered writers assume that the position deletes are already ordered by file and position + static PartitioningWriter, DeleteWriteResult> newDeleteWriter( + Table table, HiveFileWriterFactory writers, OutputFileFactory files, Context context) { + + FileIO io = table.io(); + boolean inputOrdered = context.inputOrdered(); + long targetFileSize = context.targetDeleteFileSize(); + DeleteGranularity deleteGranularity = context.deleteGranularity(); + + if (inputOrdered) { + return new ClusteredPositionDeleteWriter<>( + writers, files, io, targetFileSize, deleteGranularity); + } else { + return new FanoutPositionOnlyDeleteWriter<>( + writers, files, io, targetFileSize, deleteGranularity); + } + } } diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java index c31cfde8cfdf..02405025be24 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/writer/WriterBuilder.java @@ -25,11 +25,9 @@ import org.apache.hadoop.hive.ql.Context.Operation; import org.apache.hadoop.mapred.TaskAttemptID; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.mr.hive.IcebergTableUtil; import org.apache.iceberg.util.PropertyUtil; @@ -40,23 +38,22 @@ public class WriterBuilder { private final Table table; + private final Context context; private String tableName; private TaskAttemptID attemptID; private String queryId; - private int poolSize; private Operation operation; - private boolean fanoutEnabled; - private boolean isMergeTask; // A task may write multiple output files using multiple writers. Each of them must have a unique operationId. private static AtomicInteger operationNum = new AtomicInteger(0); // To specify whether to write the actual row data while writing the delete files. public static final String ICEBERG_DELETE_SKIPROWDATA = "iceberg.delete.skiprowdata"; - public static final String ICEBERG_DELETE_SKIPROWDATA_DEFAULT = "true"; + public static final boolean ICEBERG_DELETE_SKIPROWDATA_DEFAULT = true; private WriterBuilder(Table table) { this.table = table; + this.context = new Context(table.properties()); } public static WriterBuilder builderFor(Table table) { @@ -78,75 +75,60 @@ public WriterBuilder queryId(String newQueryId) { return this; } - public WriterBuilder poolSize(int newPoolSize) { - this.poolSize = newPoolSize; - return this; - } - public WriterBuilder operation(Operation newOperation) { this.operation = newOperation; return this; } - public WriterBuilder isFanoutEnabled(boolean isFanoutEnabled) { - this.fanoutEnabled = isFanoutEnabled; + public WriterBuilder hasOrdering(boolean inputOrdered) { + context.inputOrdered = inputOrdered; + if (IcebergTableUtil.isFanoutEnabled(table.properties()) && !inputOrdered) { + context.useFanoutWriter = true; + } return this; } - public WriterBuilder isMergeTask(boolean mergeTaskEnabled) { - this.isMergeTask = mergeTaskEnabled; + public WriterBuilder isMergeTask(boolean isMergeTaskEnabled) { + context.isMergeTask = isMergeTaskEnabled; return this; } public HiveIcebergWriter build() { - Map properties = table.properties(); - - String dataFileFormatName = properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); - FileFormat dataFileFormat = FileFormat.valueOf(dataFileFormatName.toUpperCase(Locale.ENGLISH)); - - String deleteFileFormatName = properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName); - FileFormat deleteFileFormat = FileFormat.valueOf(deleteFileFormatName.toUpperCase(Locale.ENGLISH)); - - long targetFileSize = PropertyUtil.propertyAsLong(table.properties(), TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, - TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT); - - boolean skipRowData = - Boolean.parseBoolean(properties.getOrDefault(ICEBERG_DELETE_SKIPROWDATA, ICEBERG_DELETE_SKIPROWDATA_DEFAULT)); - - Schema dataSchema = table.schema(); - FileIO io = table.io(); - Map specs = table.specs(); - int currentSpecId = table.spec().specId(); int partitionId = attemptID.getTaskID().getId(); int taskId = attemptID.getId(); String operationId = queryId + "-" + attemptID.getJobID() + "-" + operationNum.incrementAndGet(); - OutputFileFactory outputFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId) - .format(dataFileFormat) - .operationId("data-" + operationId) + + OutputFileFactory dataFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId) + .format(context.dataFileFormat()) + .operationId(operationId) .build(); - OutputFileFactory deleteOutputFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId) - .format(deleteFileFormat) - .operationId("delete-" + operationId) + OutputFileFactory deleteFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId) + .format(context.deleteFileFormat()) + .operationId(operationId) + .suffix("pos-deletes") .build(); - HiveFileWriterFactory writerFactory = - new HiveFileWriterFactory(table, dataFileFormat, dataSchema, null, deleteFileFormat, null, null, null, - skipRowData ? null : dataSchema); + HiveFileWriterFactory writerFactory = HiveFileWriterFactory.builderFor(table) + .dataFileFormat(context.dataFileFormat()) + .dataSchema(table.schema()) + .deleteFileFormat(context.deleteFileFormat()) + .positionDeleteRowSchema(context.skipRowData() || !context.inputOrdered() ? + // SortingPositionOnlyDeleteWriter doesn't support rawData in delete schema + null : table.schema()) + .build(); HiveIcebergWriter writer; - if (IcebergTableUtil.isCopyOnWriteMode(operation, properties::getOrDefault)) { - writer = new HiveIcebergCopyOnWriteRecordWriter(dataSchema, specs, currentSpecId, writerFactory, - outputFileFactory, io, targetFileSize); + boolean isCOW = IcebergTableUtil.isCopyOnWriteMode(operation, table.properties()::getOrDefault); + if (isCOW) { + writer = new HiveIcebergCopyOnWriteRecordWriter(table, writerFactory, dataFileFactory, context); } else { switch (operation) { case DELETE: - writer = new HiveIcebergDeleteWriter(dataSchema, specs, writerFactory, deleteOutputFileFactory, - io, targetFileSize, skipRowData, isMergeTask); + writer = new HiveIcebergDeleteWriter(table, writerFactory, deleteFileFactory, context); break; case OTHER: - writer = new HiveIcebergRecordWriter(dataSchema, specs, currentSpecId, writerFactory, outputFileFactory, - io, targetFileSize, fanoutEnabled); + writer = new HiveIcebergRecordWriter(table, writerFactory, dataFileFactory, context); break; default: // Update and Merge should be splitted to inserts and deletes @@ -158,4 +140,74 @@ public HiveIcebergWriter build() { WriterRegistry.registerWriter(attemptID, tableName, writer); return writer; } + + static class Context { + + private final FileFormat dataFileFormat; + private final long targetDataFileSize; + private final FileFormat deleteFileFormat; + private final long targetDeleteFileSize; + private final DeleteGranularity deleteGranularity; + private boolean useFanoutWriter; + private boolean inputOrdered; + private boolean isMergeTask; + private final boolean skipRowData; + + Context(Map properties) { + String dataFileFormatName = + properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); + this.dataFileFormat = FileFormat.valueOf(dataFileFormatName.toUpperCase(Locale.ENGLISH)); + + this.targetDataFileSize = PropertyUtil.propertyAsLong(properties, + TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT); + + String deleteFileFormatName = + properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName); + this.deleteFileFormat = FileFormat.valueOf(deleteFileFormatName.toUpperCase(Locale.ENGLISH)); + + this.targetDeleteFileSize = PropertyUtil.propertyAsLong(properties, + TableProperties.DELETE_TARGET_FILE_SIZE_BYTES, TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT); + + this.skipRowData = PropertyUtil.propertyAsBoolean(properties, + ICEBERG_DELETE_SKIPROWDATA, ICEBERG_DELETE_SKIPROWDATA_DEFAULT); + + this.deleteGranularity = DeleteGranularity.PARTITION; + } + + FileFormat dataFileFormat() { + return dataFileFormat; + } + + long targetDataFileSize() { + return targetDataFileSize; + } + + FileFormat deleteFileFormat() { + return deleteFileFormat; + } + + long targetDeleteFileSize() { + return targetDeleteFileSize; + } + + DeleteGranularity deleteGranularity() { + return deleteGranularity; + } + + boolean useFanoutWriter() { + return useFanoutWriter; + } + + boolean inputOrdered() { + return inputOrdered; + } + + boolean isMergeTask() { + return isMergeTask; + } + + boolean skipRowData() { + return skipRowData; + } + } } 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 bc3c948c4ad4..c18a035d3933 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 @@ -734,6 +734,31 @@ public void testConcurrent2MergeInserts() { Assert.assertEquals(6, res.size()); } + @Test + public void testMultiInsert() { + Assume.assumeTrue(fileFormat == FileFormat.PARQUET && isVectorized && + testTableType == TestTables.TestTableType.HIVE_CATALOG); + + testTables.createTable(shell, "source", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, + PartitionSpec.unpartitioned(), fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS); + testTables.createTable(shell, "alice", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, + PartitionSpec.unpartitioned(), fileFormat, null, formatVersion); + testTables.createTable(shell, "green", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, + PartitionSpec.unpartitioned(), fileFormat, null, formatVersion); + + String sql = "FROM source " + + "INSERT INTO alice " + + " SELECT * WHERE first_name='Alice'" + + "INSERT INTO green " + + " SELECT * WHERE last_name='Green'"; + shell.executeStatement(sql); + + List res = shell.executeStatement("SELECT * FROM alice"); + Assert.assertEquals(1, res.size()); + res = shell.executeStatement("SELECT * FROM green"); + Assert.assertEquals(1, res.size()); + } + private static PositionDelete positionDelete(CharSequence path, long pos, T row) { PositionDelete positionDelete = PositionDelete.create(); return positionDelete.set(path, pos, row); diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_avro.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_avro.q.out index 38d0bc838386..bee3291151d2 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_avro.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_avro.q.out @@ -50,8 +50,8 @@ POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product -Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[65][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product PREHOOK: query: delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_orc.q.out index 2111826b9bf8..b17ea857b98f 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_orc.q.out @@ -50,8 +50,8 @@ POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product -Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[65][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product PREHOOK: query: delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_parquet.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_parquet.q.out index 43e01378a2cb..d02e9896b0e9 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_parquet.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_partitioned_parquet.q.out @@ -50,8 +50,8 @@ POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product -Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[65][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product PREHOOK: query: delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_unpartitioned_parquet.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_unpartitioned_parquet.q.out index 85b3b2a7b56d..50410166a68d 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_unpartitioned_parquet.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_unpartitioned_parquet.q.out @@ -50,8 +50,8 @@ POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product -Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[65][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[63][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product PREHOOK: query: delete from tbl_ice where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice @@ -181,9 +181,9 @@ POSTHOOK: Output: default@tbl_ice_with_nulls Warning: Shuffle Join MERGEJOIN[57][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product Warning: Shuffle Join MERGEJOIN[59][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product -Warning: Shuffle Join MERGEJOIN[59][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product -Warning: Shuffle Join MERGEJOIN[62][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product +Warning: Shuffle Join MERGEJOIN[57][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[59][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product PREHOOK: query: delete from tbl_ice_with_nulls where id in (select id from tbl_ice_with_nulls where id > 9) or name in (select name from tbl_ice_with_nulls where name = 'sdf') PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice_with_nulls diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_atomic_merge_update.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_atomic_merge_update.q.out index 403fb63f2c09..13c2c4bddc17 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_atomic_merge_update.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_atomic_merge_update.q.out @@ -62,8 +62,8 @@ POSTHOOK: query: INSERT INTO display (skey, language_id, hierarchy_display) VALU POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@display -Warning: Shuffle Join MERGEJOIN[67][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[68][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 8' is a cross product +Warning: Shuffle Join MERGEJOIN[65][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[66][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 7' is a cross product PREHOOK: query: MERGE INTO display USING ( SELECT distinct display_skey, display, display as orig_display FROM ( diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_merge_schema.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_merge_schema.q.out index 6f0930fae007..b4880f9179b3 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_merge_schema.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_merge_schema.q.out @@ -62,8 +62,8 @@ POSTHOOK: query: INSERT INTO display (skey, language_id, hierarchy_display) VALU POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@display -Warning: Shuffle Join MERGEJOIN[67][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[68][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 8' is a cross product +Warning: Shuffle Join MERGEJOIN[65][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[66][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 7' is a cross product PREHOOK: query: explain vectorization only detail MERGE INTO display USING ( SELECT distinct display_skey, display, display as orig_display FROM ( @@ -160,13 +160,12 @@ STAGE PLANS: Stage: Stage-4 Tez Edges: - Reducer 2 <- Map 1 (XPROD_EDGE), Map 10 (XPROD_EDGE) + Reducer 2 <- Map 1 (XPROD_EDGE), Map 9 (XPROD_EDGE) Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Union 4 (CONTAINS) - Reducer 5 <- Map 10 (SIMPLE_EDGE), Union 4 (SIMPLE_EDGE) + Reducer 5 <- Map 9 (SIMPLE_EDGE), Union 4 (SIMPLE_EDGE) Reducer 6 <- Reducer 5 (SIMPLE_EDGE) - Reducer 7 <- Reducer 5 (SIMPLE_EDGE) - Reducer 8 <- Map 1 (XPROD_EDGE), Map 10 (XPROD_EDGE) - Reducer 9 <- Reducer 8 (SIMPLE_EDGE), Union 4 (CONTAINS) + Reducer 7 <- Map 1 (XPROD_EDGE), Map 9 (XPROD_EDGE) + Reducer 8 <- Reducer 7 (SIMPLE_EDGE), Union 4 (CONTAINS) Vertices: Map 1 Map Operator Tree: @@ -205,7 +204,7 @@ STAGE PLANS: dataColumns: s_key:bigint, year:int partitionColumnCount: 0 scratchColumnTypeNames: [] - Map 10 + Map 9 Map Operator Tree: TableScan Vectorization: native: true @@ -306,66 +305,42 @@ STAGE PLANS: Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez] IS true - reduceColumnNullOrder: zzzzz - reduceColumnSortOrder: +++++ - allNative: false - usesVectorUDFAdaptor: false - vectorized: true - rowBatchContext: - dataColumnCount: 10 - dataColumns: KEY.reducesinkkey0:int, KEY.reducesinkkey1:bigint, KEY.reducesinkkey2:string, KEY.reducesinkkey3:bigint, KEY.reducesinkkey4:string, VALUE._col1:string, VALUE._col2:string, VALUE._col3:int, VALUE._col4:string, VALUE._col5:string - partitionColumnCount: 0 - scratchColumnTypeNames: [bigint] - Reduce Operator Tree: - Select Vectorization: - className: VectorSelectOperator - native: true - projectedOutputColumnNums: [0, 1, 2, 3, 4, 10, 5, 6, 7, 8, 9] - selectExpressions: ConstantVectorExpression(val 1090969) -> 10:bigint - File Sink Vectorization: - className: VectorFileSinkOperator - native: false - Reducer 7 - Execution mode: vectorized - Reduce Vectorization: - enabled: true - enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez] IS true - reduceColumnNullOrder: zzzzz - reduceColumnSortOrder: +++++ + reduceColumnNullOrder: zzzz + reduceColumnSortOrder: ++++ allNative: false usesVectorUDFAdaptor: true vectorized: true rowBatchContext: - dataColumnCount: 6 - dataColumns: KEY._col0:int, KEY._col1:bigint, KEY._col2:string, KEY._col3:bigint, KEY._col4:string, VALUE._col0:bigint + dataColumnCount: 5 + dataColumns: KEY._col0:int, KEY._col1:bigint, KEY._col2:string, KEY._col3:bigint, VALUE._col0:bigint partitionColumnCount: 0 scratchColumnTypeNames: [] Reduce Operator Tree: Group By Vectorization: - aggregators: VectorUDAFCountMerge(col 5:bigint) -> bigint + aggregators: VectorUDAFCountMerge(col 4:bigint) -> bigint className: VectorGroupByOperator groupByMode: MERGEPARTIAL - keyExpressions: col 0:int, col 1:bigint, col 2:string, col 3:bigint, col 4:string + keyExpressions: col 0:int, col 1:bigint, col 2:string, col 3:bigint native: false vectorProcessingMode: MERGE_PARTIAL projectedOutputColumnNums: [0] Filter Vectorization: className: VectorFilterOperator native: true - predicateExpression: FilterLongColGreaterLongScalar(col 5:bigint, val 1) + predicateExpression: FilterLongColGreaterLongScalar(col 4:bigint, val 1) Select Vectorization: className: VectorSelectOperator native: true - projectedOutputColumnNums: [6] - selectExpressions: VectorUDFAdaptor(cardinality_violation(_col0,_col1,_col2,_col3,_col4)) -> 6:int + projectedOutputColumnNums: [5] + selectExpressions: VectorUDFAdaptor(cardinality_violation(_col0,_col1,_col2,_col3)) -> 5:int File Sink Vectorization: className: VectorFileSinkOperator native: false - Reducer 8 + Reducer 7 MergeJoin Vectorization: enabled: false enableConditionsNotMet: Vectorizing MergeJoin Supported IS false - Reducer 9 + Reducer 8 Execution mode: vectorized Reduce Vectorization: enabled: true @@ -411,8 +386,8 @@ STAGE PLANS: Stage: Stage-7 -Warning: Shuffle Join MERGEJOIN[67][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[68][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 8' is a cross product +Warning: Shuffle Join MERGEJOIN[65][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[66][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 7' is a cross product PREHOOK: query: MERGE INTO display USING ( SELECT distinct display_skey, display, display as orig_display FROM ( diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_truncate_partition_with_evolution.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_truncate_partition_with_evolution.q.out index 1c3ff07105ea..eee345606329 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_truncate_partition_with_evolution.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_truncate_partition_with_evolution.q.out @@ -78,9 +78,6 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-2 Tez -#### A masked pattern was here #### - Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -93,31 +90,18 @@ STAGE PLANS: predicate: (a = 22) (type: boolean) Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), b (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col6 - Statistics: Num rows: 2 Data size: 958 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ - Statistics: Num rows: 2 Data size: 958 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col6 (type: string) - Execution mode: vectorized - Reducer 2 + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), 22 (type: int), b (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 966 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 966 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.test_ice_int Execution mode: vectorized - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), 22 (type: int), VALUE._col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 2 Data size: 966 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 2 Data size: 966 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.test_ice_int Stage: Stage-3 Dependency Collection @@ -289,9 +273,6 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-2 Tez -#### A masked pattern was here #### - Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -304,31 +285,18 @@ STAGE PLANS: predicate: (a = 226784902765739L) (type: boolean) Statistics: Num rows: 2 Data size: 198 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), b (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col6 - Statistics: Num rows: 2 Data size: 958 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ - Statistics: Num rows: 2 Data size: 958 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col6 (type: string) + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), 226784902765739L (type: bigint), b (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 974 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 974 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.test_ice_bigint Execution mode: vectorized - Reducer 2 - Execution mode: vectorized - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), 226784902765739L (type: bigint), VALUE._col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 2 Data size: 974 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 2 Data size: 974 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.test_ice_bigint Stage: Stage-3 Dependency Collection @@ -526,9 +494,6 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-2 Tez -#### A masked pattern was here #### - Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -541,31 +506,18 @@ STAGE PLANS: predicate: (b = 'ddd') (type: boolean) Statistics: Num rows: 3 Data size: 297 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), a (type: bigint) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 3 Data size: 1188 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ - Statistics: Num rows: 3 Data size: 1188 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col5 (type: bigint) - Execution mode: vectorized - Reducer 2 + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), a (type: bigint), 'ddd' (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 3 Data size: 1449 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 1449 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.test_ice_str Execution mode: vectorized - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), VALUE._col0 (type: bigint), 'ddd' (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 3 Data size: 1449 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 3 Data size: 1449 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.test_ice_str Stage: Stage-3 Dependency Collection @@ -783,9 +735,6 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-2 Tez -#### A masked pattern was here #### - Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -798,31 +747,18 @@ STAGE PLANS: predicate: (b = DATE'2022-02-07') (type: boolean) Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), a (type: bigint) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col5 (type: bigint) - Execution mode: vectorized - Reducer 2 + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), a (type: bigint), DATE'2022-02-07' (type: date) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 452 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 452 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.test_ice_date Execution mode: vectorized - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), VALUE._col0 (type: bigint), DATE'2022-02-07' (type: date) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 452 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 452 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.test_ice_date Stage: Stage-3 Dependency Collection @@ -987,9 +923,6 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-2 Tez -#### A masked pattern was here #### - Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -1002,31 +935,18 @@ STAGE PLANS: predicate: (a = 1.156748927566759E11D) (type: boolean) Statistics: Num rows: 2 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), b (type: date) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col6 - Statistics: Num rows: 2 Data size: 888 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ - Statistics: Num rows: 2 Data size: 888 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col6 (type: date) + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), 1.156748927566759E11D (type: double), b (type: date) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 2 Data size: 904 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 904 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.test_ice_double Execution mode: vectorized - Reducer 2 - Execution mode: vectorized - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), 1.156748927566759E11D (type: double), VALUE._col0 (type: date) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 2 Data size: 904 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 2 Data size: 904 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.test_ice_double Stage: Stage-3 Dependency Collection @@ -1185,9 +1105,6 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-2 Tez -#### A masked pattern was here #### - Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -1200,30 +1117,18 @@ STAGE PLANS: predicate: ((a = 1.156748927566759E11D) and (b = DATE'2022-02-07')) (type: boolean) Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 1 Data size: 388 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ - Statistics: Num rows: 1 Data size: 388 Basic stats: COMPLETE Column stats: COMPLETE - Execution mode: vectorized - Reducer 2 + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), 1.156748927566759E11D (type: double), DATE'2022-02-07' (type: date) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 1 Data size: 452 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 452 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.test_ice_double_date Execution mode: vectorized - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), 1.156748927566759E11D (type: double), DATE'2022-02-07' (type: date) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 452 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 452 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.test_ice_double_date Stage: Stage-3 Dependency Collection diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/llap_iceberg_read_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/llap_iceberg_read_orc.q.out index 46bf3a38bcba..fdb6f3aae503 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/llap_iceberg_read_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/llap_iceberg_read_orc.q.out @@ -98,8 +98,8 @@ POSTHOOK: query: INSERT INTO display (skey, language_id, hierarchy_display) VALU POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@display -Warning: Shuffle Join MERGEJOIN[67][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[68][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 8' is a cross product +Warning: Shuffle Join MERGEJOIN[65][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[66][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 7' is a cross product PREHOOK: query: MERGE INTO display USING ( SELECT distinct display_skey, display, display as orig_display FROM ( diff --git a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_orc.q.out index 3dc08d856bc1..dcb3ecd97145 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_orc.q.out @@ -68,10 +68,8 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) Reducer 3 <- Reducer 2 (SIMPLE_EDGE) - Reducer 4 <- Reducer 2 (SIMPLE_EDGE) - Reducer 5 <- Reducer 2 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -91,7 +89,7 @@ STAGE PLANS: Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: string), _col2 (type: int) Execution mode: vectorized - Map 6 + Map 4 Map Operator Tree: TableScan alias: target_ice @@ -133,12 +131,14 @@ STAGE PLANS: expressions: _col7 (type: int), _col6 (type: bigint), _col2 (type: string), _col5 (type: bigint), _col3 (type: string), _col10 (type: int), _col9 (type: string), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 Statistics: Num rows: 4 Data size: 1449 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ + File Output Operator + compressed: false Statistics: Num rows: 4 Data size: 1449 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col5 (type: int), _col6 (type: string), _col7 (type: int) + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.target_ice Filter Operator predicate: ((_col10 = _col1) and (_col10 <= 100)) (type: boolean) Statistics: Num rows: 1 Data size: 579 Basic stats: COMPLETE Column stats: COMPLETE @@ -146,12 +146,14 @@ STAGE PLANS: expressions: _col7 (type: int), _col6 (type: bigint), _col2 (type: string), _col5 (type: bigint), _col3 (type: string), _col10 (type: int), _col9 (type: string), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ + File Output Operator + compressed: false Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col5 (type: int), _col6 (type: string), _col7 (type: int) + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.target_ice Filter Operator predicate: ((_col10 = _col1) and (_col10 <= 100)) (type: boolean) Statistics: Num rows: 1 Data size: 579 Basic stats: COMPLETE Column stats: COMPLETE @@ -186,67 +188,37 @@ STAGE PLANS: predicate: (_col10 = _col1) (type: boolean) Statistics: Num rows: 5 Data size: 1929 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col2 (type: string), _col3 (type: string), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: int) - outputColumnNames: _col2, _col3, _col5, _col6, _col7 + expressions: _col2 (type: string), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: int) + outputColumnNames: _col2, _col5, _col6, _col7 Statistics: Num rows: 5 Data size: 1929 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: count() - keys: _col7 (type: int), _col6 (type: bigint), _col2 (type: string), _col5 (type: bigint), _col3 (type: string) + keys: _col7 (type: int), _col6 (type: bigint), _col2 (type: string), _col5 (type: bigint) minReductionHashAggr: 0.4 mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 1196 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ - Map-reduce partition columns: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - Statistics: Num rows: 4 Data size: 1196 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col5 (type: bigint) + key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint) + null sort order: zzzz + sort order: ++++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint) + Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col4 (type: bigint) Reducer 3 - Execution mode: vectorized - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 4 Data size: 1449 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 4 Data size: 1449 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.target_ice - Reducer 4 - Execution mode: vectorized - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.target_ice - Reducer 5 Execution mode: vectorized Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: KEY._col0 (type: int), KEY._col1 (type: bigint), KEY._col2 (type: string), KEY._col3 (type: bigint), KEY._col4 (type: string) + keys: KEY._col0 (type: int), KEY._col1 (type: bigint), KEY._col2 (type: string), KEY._col3 (type: bigint) mode: mergepartial - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 1196 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: (_col5 > 1L) (type: boolean) - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE + predicate: (_col4 > 1L) (type: boolean) + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: cardinality_violation(_col0,_col1,_col2,_col3,_col4) (type: int) + expressions: cardinality_violation(_col0,_col1,_col2,_col3) (type: int) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator diff --git a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_partitioned_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_partitioned_orc.q.out index 635eeecc5596..499fd6ecbaad 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_partitioned_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_partitioned_orc.q.out @@ -68,12 +68,10 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE) Reducer 3 <- Reducer 2 (SIMPLE_EDGE) Reducer 4 <- Reducer 2 (SIMPLE_EDGE) Reducer 5 <- Reducer 2 (SIMPLE_EDGE) - Reducer 6 <- Reducer 2 (SIMPLE_EDGE) - Reducer 7 <- Reducer 2 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -93,7 +91,7 @@ STAGE PLANS: Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: string), _col2 (type: int) Execution mode: vectorized - Map 8 + Map 6 Map Operator Tree: TableScan alias: target_ice @@ -135,12 +133,14 @@ STAGE PLANS: expressions: _col7 (type: int), _col6 (type: bigint), _col2 (type: string), _col5 (type: bigint), _col3 (type: string), _col10 (type: int), _col9 (type: string), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 Statistics: Num rows: 4 Data size: 1449 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ + File Output Operator + compressed: false Statistics: Num rows: 4 Data size: 1449 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col5 (type: int), _col6 (type: string), _col7 (type: int) + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.target_ice Filter Operator predicate: ((_col10 = _col1) and (_col10 <= 100)) (type: boolean) Statistics: Num rows: 1 Data size: 579 Basic stats: COMPLETE Column stats: COMPLETE @@ -148,12 +148,14 @@ STAGE PLANS: expressions: _col7 (type: int), _col6 (type: bigint), _col2 (type: string), _col5 (type: bigint), _col3 (type: string), _col10 (type: int), _col9 (type: string), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ + File Output Operator + compressed: false Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col5 (type: int), _col6 (type: string), _col7 (type: int) + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.target_ice Filter Operator predicate: ((_col10 = _col1) and (_col10 <= 100)) (type: boolean) Statistics: Num rows: 1 Data size: 579 Basic stats: COMPLETE Column stats: COMPLETE @@ -186,54 +188,24 @@ STAGE PLANS: predicate: (_col10 = _col1) (type: boolean) Statistics: Num rows: 5 Data size: 1929 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col2 (type: string), _col3 (type: string), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: int) - outputColumnNames: _col2, _col3, _col5, _col6, _col7 + expressions: _col2 (type: string), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: int) + outputColumnNames: _col2, _col5, _col6, _col7 Statistics: Num rows: 5 Data size: 1929 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: count() - keys: _col7 (type: int), _col6 (type: bigint), _col2 (type: string), _col5 (type: bigint), _col3 (type: string) + keys: _col7 (type: int), _col6 (type: bigint), _col2 (type: string), _col5 (type: bigint) minReductionHashAggr: 0.4 mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 1196 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ - Map-reduce partition columns: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - Statistics: Num rows: 4 Data size: 1196 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col5 (type: bigint) + key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint) + null sort order: zzzz + sort order: ++++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint) + Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col4 (type: bigint) Reducer 3 - Execution mode: vectorized - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 4 Data size: 1449 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 4 Data size: 1449 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.target_ice - Reducer 4 - Execution mode: vectorized - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.target_ice - Reducer 5 Execution mode: vectorized Reduce Operator Tree: Select Operator @@ -248,7 +220,7 @@ STAGE PLANS: output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.target_ice - Reducer 6 + Reducer 4 Execution mode: vectorized Reduce Operator Tree: Select Operator @@ -263,20 +235,20 @@ STAGE PLANS: output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.target_ice - Reducer 7 + Reducer 5 Execution mode: vectorized Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: KEY._col0 (type: int), KEY._col1 (type: bigint), KEY._col2 (type: string), KEY._col3 (type: bigint), KEY._col4 (type: string) + keys: KEY._col0 (type: int), KEY._col1 (type: bigint), KEY._col2 (type: string), KEY._col3 (type: bigint) mode: mergepartial - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 1196 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: (_col5 > 1L) (type: boolean) - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE + predicate: (_col4 > 1L) (type: boolean) + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: cardinality_violation(_col0,_col1,_col2,_col3,_col4) (type: int) + expressions: cardinality_violation(_col0,_col1,_col2,_col3) (type: int) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator diff --git a/iceberg/iceberg-handler/src/test/results/positive/metadata_delete.q.out b/iceberg/iceberg-handler/src/test/results/positive/metadata_delete.q.out index e7dcbc71ee7b..6f4f582d2217 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/metadata_delete.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/metadata_delete.q.out @@ -361,9 +361,6 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-1 Tez -#### A masked pattern was here #### - Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -379,28 +376,15 @@ STAGE PLANS: expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), a (type: int), b (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 2 Data size: 792 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ + File Output Operator + compressed: false Statistics: Num rows: 2 Data size: 792 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col5 (type: int), _col6 (type: int) - Execution mode: vectorized - Reducer 2 + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.test_delete_config Execution mode: vectorized - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), VALUE._col0 (type: int), VALUE._col1 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 2 Data size: 792 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 2 Data size: 792 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.test_delete_config Stage: Stage-2 Dependency Collection diff --git a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_orc4.q.out b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_orc4.q.out index 1090116dcafc..bc153b7b8679 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_orc4.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_orc4.q.out @@ -396,7 +396,7 @@ STAGE PLANS: #### A masked pattern was here #### Edges: Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) Reducer 4 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) Reducer 5 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) Reducer 7 <- Map 6 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE) @@ -483,11 +483,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 Statistics: Num rows: 1 Data size: 488 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: aaaaa - sort order: +++++ + null sort order: + sort order: Statistics: Num rows: 1 Data size: 488 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col5 (type: string), _col6 (type: int), _col7 (type: bigint) + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: string), _col6 (type: int), _col7 (type: bigint) Filter Operator predicate: _col3 (type: boolean) Statistics: Num rows: 1 Data size: 592 Basic stats: COMPLETE Column stats: COMPLETE @@ -552,7 +551,7 @@ STAGE PLANS: Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: int), VALUE._col2 (type: bigint) + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: string), VALUE._col5 (type: string), VALUE._col6 (type: int), VALUE._col7 (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 Statistics: Num rows: 1 Data size: 488 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator diff --git a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_orc5.q.out b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_orc5.q.out index 34fb156430b5..3e67b488b252 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_orc5.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_orc5.q.out @@ -140,7 +140,7 @@ STAGE PLANS: #### A masked pattern was here #### Edges: Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) Reducer 4 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) Reducer 5 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) Reducer 7 <- Map 6 (SIMPLE_EDGE), Map 9 (SIMPLE_EDGE) @@ -227,11 +227,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 Statistics: Num rows: 1 Data size: 504 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: aaaaa - sort order: +++++ + null sort order: + sort order: Statistics: Num rows: 1 Data size: 504 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col5 (type: string), _col6 (type: int), _col7 (type: bigint), _col8 (type: bigint), _col9 (type: double) + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: string), _col6 (type: int), _col7 (type: bigint), _col8 (type: bigint), _col9 (type: double) Filter Operator predicate: _col4 (type: boolean) Statistics: Num rows: 1 Data size: 616 Basic stats: COMPLETE Column stats: COMPLETE @@ -296,7 +295,7 @@ STAGE PLANS: Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: bigint), VALUE._col4 (type: double) + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: string), VALUE._col5 (type: string), VALUE._col6 (type: int), VALUE._col7 (type: bigint), VALUE._col8 (type: bigint), VALUE._col9 (type: double) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 Statistics: Num rows: 1 Data size: 504 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator diff --git a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_orc7.q.out b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_orc7.q.out index b5f6d65fe952..f9dc58d11c7c 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_orc7.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_orc7.q.out @@ -110,7 +110,7 @@ STAGE PLANS: #### A masked pattern was here #### Edges: Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE) - Reducer 3 <- Reducer 2 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) Reducer 5 <- Map 4 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: @@ -175,11 +175,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 13 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: aaaaa - sort order: +++++ + null sort order: + sort order: Statistics: Num rows: 1 Data size: 13 Basic stats: COMPLETE Column stats: NONE - value expressions: _col5 (type: int), _col6 (type: bigint) + value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: bigint) Filter Operator predicate: _col2 (type: boolean) Statistics: Num rows: 1 Data size: 13 Basic stats: COMPLETE Column stats: NONE @@ -214,7 +213,7 @@ STAGE PLANS: Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), VALUE._col0 (type: int), VALUE._col1 (type: bigint) + expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: string), VALUE._col3 (type: bigint), VALUE._col4 (type: string), VALUE._col5 (type: int), VALUE._col6 (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 13 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_partitioned_avro.q.out b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_partitioned_avro.q.out index fe47918ac7dd..c9938d7a2236 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_partitioned_avro.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_partitioned_avro.q.out @@ -53,8 +53,8 @@ POSTHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice -Warning: Shuffle Join MERGEJOIN[68][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[70][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[66][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[68][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product PREHOOK: query: update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice diff --git a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_partitioned_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_partitioned_orc.q.out index e7a6c6090bcf..84de34cb28b8 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_partitioned_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_partitioned_orc.q.out @@ -53,8 +53,8 @@ POSTHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice -Warning: Shuffle Join MERGEJOIN[68][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[70][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[66][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[68][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product PREHOOK: query: update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice diff --git a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_partitioned_parquet.q.out b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_partitioned_parquet.q.out index 4d8f0d994b14..ecc74a61c6c2 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_partitioned_parquet.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_partitioned_parquet.q.out @@ -53,8 +53,8 @@ POSTHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice -Warning: Shuffle Join MERGEJOIN[68][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[70][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[66][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[68][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product PREHOOK: query: update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice diff --git a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_unpartitioned_parquet.q.out b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_unpartitioned_parquet.q.out index d15647cbe477..dd3dd1fcc5c3 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_unpartitioned_parquet.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_unpartitioned_parquet.q.out @@ -53,8 +53,8 @@ POSTHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola', POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@tbl_ice -Warning: Shuffle Join MERGEJOIN[66][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product -Warning: Shuffle Join MERGEJOIN[68][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product +Warning: Shuffle Join MERGEJOIN[64][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[66][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product PREHOOK: query: update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800) PREHOOK: type: QUERY PREHOOK: Input: default@tbl_ice diff --git a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_merge_mixed.q.out b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_merge_mixed.q.out index 52dae5be001e..02c4de2e2546 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_merge_mixed.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_merge_mixed.q.out @@ -325,11 +325,10 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Map 1 <- Map 6 (BROADCAST_EDGE) + Map 1 <- Map 5 (BROADCAST_EDGE) Reducer 2 <- Map 1 (SIMPLE_EDGE) Reducer 3 <- Map 1 (SIMPLE_EDGE) Reducer 4 <- Map 1 (SIMPLE_EDGE) - Reducer 5 <- Map 1 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -372,7 +371,7 @@ STAGE PLANS: hashTableImplementationType: OPTIMIZED outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38, _col39, _col40, _col41, _col42, _col43, _col44, _col45, _col46, _col47, _col48, _col49, _col50, _col51 input vertices: - 1 Map 6 + 1 Map 5 Statistics: Num rows: 7 Data size: 7208 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col40 (type: decimal(7,2)), _col19 (type: decimal(7,2)), _col33 (type: int), _col39 (type: int), _col24 (type: int), _col21 (type: decimal(7,2)), _col26 (type: string), _col7 (type: int), _col3 (type: int), _col10 (type: decimal(7,2)), _col49 (type: decimal(7,2)), _col38 (type: int), _col28 (type: string), _col6 (type: int), _col50 (type: decimal(7,2)), _col11 (type: decimal(7,2)), _col34 (type: int), _col17 (type: decimal(7,2)), _col14 (type: decimal(7,2)), _col45 (type: decimal(7,2)), _col47 (type: decimal(7,2)), _col20 (type: decimal(7,2)), _col8 (type: int), _col35 (type: int), _col31 (type: int), _col41 (type: decimal(7,2)), _col36 (type: int), _col4 (type: int), _col48 (type: decimal(7,2)), _col5 (type: int), _col13 (type: decimal(7,2)), _col12 (type: decimal(7,2)), _col44 (type: decimal(7,2)), _col43 (type: decimal(7,2)), _col1 (type: int), _col16 (type: decimal(7,2)), _col29 (type: int), _col2 (type: int), _col15 (type: decimal(7,2)), _col18 (type: decimal(7,2)), _col25 (type: bigint), _col9 (type: int), _col30 (type: int), _col42 (type: decimal(7,2)), _col0 (type: int), _col27 (type: bigint), _col51 (type: decimal(7,2)), _col32 (type: int), _col37 (type: int), _col46 (type: decimal(7,2)) @@ -390,25 +389,25 @@ STAGE PLANS: predicate: ((_col24 = _col34) and (_col47 = _col37) and (_col36 = 2451181) and (floor((_col34 / 1000)) * 1000) BETWEEN 1000 AND 2000 and (_col30 < 0) and _col33 is null) (type: boolean) Statistics: Num rows: 1 Data size: 3160 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col4 (type: int), _col40 (type: bigint), _col6 (type: string), _col45 (type: bigint), _col12 (type: string), _col42 (type: int), _col24 (type: int), _col47 (type: int), _col2 (type: int), _col16 (type: int), _col23 (type: int), _col26 (type: int), _col48 (type: int), _col11 (type: int), _col3 (type: int), _col0 (type: decimal(7,2)), _col25 (type: decimal(7,2)), _col43 (type: decimal(7,2)), _col32 (type: decimal(7,2)), _col19 (type: decimal(7,2)), _col49 (type: decimal(7,2)), _col20 (type: decimal(7,2)), _col28 (type: decimal(7,2)), _col10 (type: decimal(7,2)), _col14 (type: decimal(7,2)), _col46 (type: decimal(7,2)) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27 + expressions: _col4 (type: int), _col40 (type: bigint), _col6 (type: string), _col45 (type: bigint), _col12 (type: string), 2451181 (type: int), _col42 (type: int), _col24 (type: int), _col47 (type: int), _col2 (type: int), _col16 (type: int), _col23 (type: int), _col26 (type: int), _col48 (type: int), _col11 (type: int), _col3 (type: int), _col0 (type: decimal(7,2)), _col25 (type: decimal(7,2)), _col43 (type: decimal(7,2)), null (type: decimal(7,2)), _col32 (type: decimal(7,2)), _col19 (type: decimal(7,2)), _col49 (type: decimal(7,2)), _col20 (type: decimal(7,2)), _col28 (type: decimal(7,2)), _col10 (type: decimal(7,2)), _col14 (type: decimal(7,2)), _col46 (type: decimal(7,2)) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27 Select Vectorization: className: VectorSelectOperator native: true - projectedOutputColumnNums: [34, 35, 36, 37, 38, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 54, 55, 56, 57, 58, 59, 60, 61] - Statistics: Num rows: 1 Data size: 1660 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ - Reduce Sink Vectorization: - className: VectorReduceSinkObjectHashOperator - keyColumns: 34:int, 35:bigint, 36:string, 37:bigint, 38:string - native: true - nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 40:int, 41:int, 42:int, 43:int, 44:int, 45:int, 46:int, 47:int, 48:int, 49:int, 50:decimal(7,2), 51:decimal(7,2), 52:decimal(7,2), 54:decimal(7,2), 55:decimal(7,2), 56:decimal(7,2), 57:decimal(7,2), 58:decimal(7,2), 59:decimal(7,2), 60:decimal(7,2), 61:decimal(7,2) - Statistics: Num rows: 1 Data size: 1660 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col6 (type: int), _col7 (type: int), _col8 (type: int), _col9 (type: int), _col10 (type: int), _col11 (type: int), _col12 (type: int), _col13 (type: int), _col14 (type: int), _col15 (type: int), _col16 (type: decimal(7,2)), _col17 (type: decimal(7,2)), _col18 (type: decimal(7,2)), _col20 (type: decimal(7,2)), _col21 (type: decimal(7,2)), _col22 (type: decimal(7,2)), _col23 (type: decimal(7,2)), _col24 (type: decimal(7,2)), _col25 (type: decimal(7,2)), _col26 (type: decimal(7,2)), _col27 (type: decimal(7,2)) + projectedOutputColumnNums: [34, 35, 36, 37, 38, 30, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 62, 54, 55, 56, 57, 58, 59, 60, 61] + selectExpressions: ConstantVectorExpression(val 2451181) -> 30:int, ConstantVectorExpression(val null) -> 62:decimal(7,2) + Statistics: Num rows: 1 Data size: 1776 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + File Sink Vectorization: + className: VectorFileSinkOperator + native: false + Statistics: Num rows: 1 Data size: 1776 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.store_sales Filter Operator Filter Vectorization: className: VectorFilterOperator @@ -422,8 +421,8 @@ STAGE PLANS: Select Vectorization: className: VectorSelectOperator native: true - projectedOutputColumnNums: [30, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 62, 54, 55, 56, 57, 58, 59, 60, 61] - selectExpressions: ConstantVectorExpression(val 2451181) -> 30:int, ConstantVectorExpression(val 0) -> 62:decimal(7,2) + projectedOutputColumnNums: [31, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 63, 54, 55, 56, 57, 58, 59, 60, 61] + selectExpressions: ConstantVectorExpression(val 2451181) -> 31:int, ConstantVectorExpression(val 0) -> 63:decimal(7,2) Statistics: Num rows: 1 Data size: 1388 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col3 (type: int), iceberg_bucket(_col2, 3) (type: int) @@ -432,11 +431,11 @@ STAGE PLANS: Map-reduce partition columns: _col3 (type: int), iceberg_bucket(_col2, 3) (type: int) Reduce Sink Vectorization: className: VectorReduceSinkMultiKeyOperator - keyColumns: 42:int, 63:int - keyExpressions: VectorUDFAdaptor(iceberg_bucket(_col2, 3)) -> 63:int + keyColumns: 42:int, 65:int + keyExpressions: VectorUDFAdaptor(iceberg_bucket(_col2, 3)) -> 65:int native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 30:int, 40:int, 41:int, 42:int, 43:int, 44:int, 45:int, 46:int, 47:int, 48:int, 49:int, 50:decimal(7,2), 51:decimal(7,2), 52:decimal(7,2), 62:decimal(7,2), 54:decimal(7,2), 55:decimal(7,2), 56:decimal(7,2), 57:decimal(7,2), 58:decimal(7,2), 59:decimal(7,2), 60:decimal(7,2), 61:decimal(7,2) + valueColumns: 31:int, 40:int, 41:int, 42:int, 43:int, 44:int, 45:int, 46:int, 47:int, 48:int, 49:int, 50:decimal(7,2), 51:decimal(7,2), 52:decimal(7,2), 63:decimal(7,2), 54:decimal(7,2), 55:decimal(7,2), 56:decimal(7,2), 57:decimal(7,2), 58:decimal(7,2), 59:decimal(7,2), 60:decimal(7,2), 61:decimal(7,2) Statistics: Num rows: 1 Data size: 1388 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int), _col9 (type: int), _col10 (type: int), _col11 (type: decimal(7,2)), _col12 (type: decimal(7,2)), _col13 (type: decimal(7,2)), _col14 (type: decimal(7,2)), _col15 (type: decimal(7,2)), _col16 (type: decimal(7,2)), _col17 (type: decimal(7,2)), _col18 (type: decimal(7,2)), _col19 (type: decimal(7,2)), _col20 (type: decimal(7,2)), _col21 (type: decimal(7,2)), _col22 (type: decimal(7,2)) Filter Operator @@ -452,8 +451,8 @@ STAGE PLANS: Select Vectorization: className: VectorSelectOperator native: true - projectedOutputColumnNums: [31, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22] - selectExpressions: ConstantVectorExpression(val 2451181) -> 31:int + projectedOutputColumnNums: [64, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22] + selectExpressions: ConstantVectorExpression(val 2451181) -> 64:int Statistics: Num rows: 3 Data size: 2900 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: _col3 (type: int), iceberg_bucket(_col2, 3) (type: int) @@ -462,11 +461,11 @@ STAGE PLANS: Map-reduce partition columns: _col3 (type: int), iceberg_bucket(_col2, 3) (type: int) Reduce Sink Vectorization: className: VectorReduceSinkMultiKeyOperator - keyColumns: 3:int, 64:int - keyExpressions: VectorUDFAdaptor(iceberg_bucket(_col2, 3)) -> 64:int + keyColumns: 3:int, 66:int + keyExpressions: VectorUDFAdaptor(iceberg_bucket(_col2, 3)) -> 66:int native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 31:int, 1:int, 2:int, 3:int, 4:int, 5:int, 6:int, 7:int, 8:int, 9:int, 10:int, 11:decimal(7,2), 12:decimal(7,2), 13:decimal(7,2), 14:decimal(7,2), 15:decimal(7,2), 16:decimal(7,2), 17:decimal(7,2), 18:decimal(7,2), 19:decimal(7,2), 20:decimal(7,2), 21:decimal(7,2), 22:decimal(7,2) + valueColumns: 64:int, 1:int, 2:int, 3:int, 4:int, 5:int, 6:int, 7:int, 8:int, 9:int, 10:int, 11:decimal(7,2), 12:decimal(7,2), 13:decimal(7,2), 14:decimal(7,2), 15:decimal(7,2), 16:decimal(7,2), 17:decimal(7,2), 18:decimal(7,2), 19:decimal(7,2), 20:decimal(7,2), 21:decimal(7,2), 22:decimal(7,2) Statistics: Num rows: 3 Data size: 2900 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int), _col9 (type: int), _col10 (type: int), _col11 (type: decimal(7,2)), _col12 (type: decimal(7,2)), _col13 (type: decimal(7,2)), _col14 (type: decimal(7,2)), _col15 (type: decimal(7,2)), _col16 (type: decimal(7,2)), _col17 (type: decimal(7,2)), _col18 (type: decimal(7,2)), _col19 (type: decimal(7,2)), _col20 (type: decimal(7,2)), _col21 (type: decimal(7,2)), _col22 (type: decimal(7,2)) Filter Operator @@ -477,12 +476,12 @@ STAGE PLANS: predicate: ((_col24 = _col34) and (_col47 = _col37) and (_col36 = 2451181) and (floor((_col34 / 1000)) * 1000) BETWEEN 1000 AND 2000 and (_col30 < 0)) (type: boolean) Statistics: Num rows: 1 Data size: 3160 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col4 (type: int), _col6 (type: string), _col12 (type: string), _col40 (type: bigint), _col45 (type: bigint) - outputColumnNames: _col4, _col6, _col12, _col40, _col45 + expressions: _col4 (type: int), _col6 (type: string), _col40 (type: bigint), _col45 (type: bigint) + outputColumnNames: _col4, _col6, _col40, _col45 Select Vectorization: className: VectorSelectOperator native: true - projectedOutputColumnNums: [34, 36, 38, 35, 37] + projectedOutputColumnNums: [34, 36, 35, 37] Statistics: Num rows: 1 Data size: 3160 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: count() @@ -490,28 +489,28 @@ STAGE PLANS: aggregators: VectorUDAFCountStar(*) -> bigint className: VectorGroupByOperator groupByMode: HASH - keyExpressions: col 34:int, col 35:bigint, col 36:string, col 37:bigint, col 38:string + keyExpressions: col 34:int, col 35:bigint, col 36:string, col 37:bigint native: false vectorProcessingMode: HASH projectedOutputColumnNums: [0] - keys: _col4 (type: int), _col40 (type: bigint), _col6 (type: string), _col45 (type: bigint), _col12 (type: string) + keys: _col4 (type: int), _col40 (type: bigint), _col6 (type: string), _col45 (type: bigint) minReductionHashAggr: 0.4 mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ - Map-reduce partition columns: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) + key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint) + null sort order: zzzz + sort order: ++++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint) Reduce Sink Vectorization: className: VectorReduceSinkMultiKeyOperator - keyColumns: 0:int, 1:bigint, 2:string, 3:bigint, 4:string + keyColumns: 0:int, 1:bigint, 2:string, 3:bigint native: true nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true - valueColumns: 5:bigint - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col5 (type: bigint) + valueColumns: 4:bigint + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col4 (type: bigint) Execution mode: vectorized Map Vectorization: enabled: true @@ -527,8 +526,8 @@ STAGE PLANS: includeColumns: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22] dataColumns: ss_sold_date_sk:int, ss_sold_time_sk:int, ss_item_sk2:int, ss_customer_sk2:int, ss_cdemo_sk:int, ss_hdemo_sk:int, ss_addr_sk:int, ss_store_sk:int, ss_promo_sk:int, ss_ticket_number:int, ss_quantity:int, ss_wholesale_cost:decimal(7,2), ss_list_price:decimal(7,2), ss_sales_price:decimal(7,2), ss_ext_discount_amt:decimal(7,2), ss_ext_sales_price:decimal(7,2), ss_ext_wholesale_cost:decimal(7,2), ss_ext_list_price:decimal(7,2), ss_ext_tax:decimal(7,2), ss_coupon_amt:decimal(7,2), ss_net_paid:decimal(7,2), ss_net_paid_inc_tax:decimal(7,2), ss_net_profit:decimal(7,2) partitionColumnCount: 0 - scratchColumnTypeNames: [double, double, bigint, bigint, bigint, bigint, bigint, bigint, string, bigint, string, bigint, bigint, bigint, bigint, bigint, bigint, bigint, bigint, bigint, bigint, bigint, decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), bigint, bigint] - Map 6 + scratchColumnTypeNames: [double, double, bigint, bigint, bigint, bigint, bigint, bigint, string, bigint, string, bigint, bigint, bigint, bigint, bigint, bigint, bigint, bigint, bigint, bigint, bigint, decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), bigint, bigint, bigint] + Map 5 Map Operator Tree: TableScan alias: store_sales @@ -584,42 +583,6 @@ STAGE PLANS: partitionColumnCount: 0 scratchColumnTypeNames: [bigint] Reducer 2 - Execution mode: vectorized - Reduce Vectorization: - enabled: true - enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez] IS true - reduceColumnNullOrder: zzzzz - reduceColumnSortOrder: +++++ - allNative: false - usesVectorUDFAdaptor: false - vectorized: true - rowBatchContext: - dataColumnCount: 26 - dataColumns: KEY.reducesinkkey0:int, KEY.reducesinkkey1:bigint, KEY.reducesinkkey2:string, KEY.reducesinkkey3:bigint, KEY.reducesinkkey4:string, VALUE._col1:int, VALUE._col2:int, VALUE._col3:int, VALUE._col4:int, VALUE._col5:int, VALUE._col6:int, VALUE._col7:int, VALUE._col8:int, VALUE._col9:int, VALUE._col10:int, VALUE._col11:decimal(7,2)/DECIMAL_64, VALUE._col12:decimal(7,2)/DECIMAL_64, VALUE._col13:decimal(7,2)/DECIMAL_64, VALUE._col15:decimal(7,2)/DECIMAL_64, VALUE._col16:decimal(7,2)/DECIMAL_64, VALUE._col17:decimal(7,2)/DECIMAL_64, VALUE._col18:decimal(7,2)/DECIMAL_64, VALUE._col19:decimal(7,2)/DECIMAL_64, VALUE._col20:decimal(7,2)/DECIMAL_64, VALUE._col21:decimal(7,2)/DECIMAL_64, VALUE._col22:decimal(7,2)/DECIMAL_64 - partitionColumnCount: 0 - scratchColumnTypeNames: [bigint, decimal(7,2)] - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), 2451181 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: int), VALUE._col5 (type: int), VALUE._col6 (type: int), VALUE._col7 (type: int), VALUE._col8 (type: int), VALUE._col9 (type: int), VALUE._col10 (type: int), VALUE._col11 (type: decimal(7,2)), VALUE._col12 (type: decimal(7,2)), VALUE._col13 (type: decimal(7,2)), null (type: decimal(7,2)), VALUE._col15 (type: decimal(7,2)), VALUE._col16 (type: decimal(7,2)), VALUE._col17 (type: decimal(7,2)), VALUE._col18 (type: decimal(7,2)), VALUE._col19 (type: decimal(7,2)), VALUE._col20 (type: decimal(7,2)), VALUE._col21 (type: decimal(7,2)), VALUE._col22 (type: decimal(7,2)) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27 - Select Vectorization: - className: VectorSelectOperator - native: true - projectedOutputColumnNums: [0, 1, 2, 3, 4, 26, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 27, 18, 19, 20, 21, 22, 23, 24, 25] - selectExpressions: ConstantVectorExpression(val 2451181) -> 26:int, ConstantVectorExpression(val null) -> 27:decimal(7,2) - Statistics: Num rows: 1 Data size: 1776 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - File Sink Vectorization: - className: VectorFileSinkOperator - native: false - Statistics: Num rows: 1 Data size: 1776 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.store_sales - Reducer 3 Execution mode: vectorized Reduce Vectorization: enabled: true @@ -654,7 +617,7 @@ STAGE PLANS: output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.store_sales - Reducer 4 + Reducer 3 Execution mode: vectorized Reduce Vectorization: enabled: true @@ -689,51 +652,51 @@ STAGE PLANS: output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.store_sales - Reducer 5 + Reducer 4 Execution mode: vectorized Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez] IS true - reduceColumnNullOrder: zzzzz - reduceColumnSortOrder: +++++ + reduceColumnNullOrder: zzzz + reduceColumnSortOrder: ++++ allNative: false usesVectorUDFAdaptor: true vectorized: true rowBatchContext: - dataColumnCount: 6 - dataColumns: KEY._col0:int, KEY._col1:bigint, KEY._col2:string, KEY._col3:bigint, KEY._col4:string, VALUE._col0:bigint + dataColumnCount: 5 + dataColumns: KEY._col0:int, KEY._col1:bigint, KEY._col2:string, KEY._col3:bigint, VALUE._col0:bigint partitionColumnCount: 0 scratchColumnTypeNames: [] Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) Group By Vectorization: - aggregators: VectorUDAFCountMerge(col 5:bigint) -> bigint + aggregators: VectorUDAFCountMerge(col 4:bigint) -> bigint className: VectorGroupByOperator groupByMode: MERGEPARTIAL - keyExpressions: col 0:int, col 1:bigint, col 2:string, col 3:bigint, col 4:string + keyExpressions: col 0:int, col 1:bigint, col 2:string, col 3:bigint native: false vectorProcessingMode: MERGE_PARTIAL projectedOutputColumnNums: [0] - keys: KEY._col0 (type: int), KEY._col1 (type: bigint), KEY._col2 (type: string), KEY._col3 (type: bigint), KEY._col4 (type: string) + keys: KEY._col0 (type: int), KEY._col1 (type: bigint), KEY._col2 (type: string), KEY._col3 (type: bigint) mode: mergepartial - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator Filter Vectorization: className: VectorFilterOperator native: true - predicateExpression: FilterLongColGreaterLongScalar(col 5:bigint, val 1) - predicate: (_col5 > 1L) (type: boolean) - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE + predicateExpression: FilterLongColGreaterLongScalar(col 4:bigint, val 1) + predicate: (_col4 > 1L) (type: boolean) + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: cardinality_violation(_col0,_col1,_col2,_col3,_col4) (type: int) + expressions: cardinality_violation(_col0,_col1,_col2,_col3) (type: int) outputColumnNames: _col0 Select Vectorization: className: VectorSelectOperator native: true - projectedOutputColumnNums: [6] - selectExpressions: VectorUDFAdaptor(cardinality_violation(_col0,_col1,_col2,_col3,_col4)) -> 6:int + projectedOutputColumnNums: [5] + selectExpressions: VectorUDFAdaptor(cardinality_violation(_col0,_col1,_col2,_col3)) -> 5:int Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false @@ -921,11 +884,10 @@ POSTHOOK: Output: default@merge_tmp_table POSTHOOK: Output: default@store_sales POSTHOOK: Output: default@store_sales Vertex dependency in root stage -Map 1 <- Map 6 (BROADCAST_EDGE) +Map 1 <- Map 5 (BROADCAST_EDGE) Reducer 2 <- Map 1 (SIMPLE_EDGE) Reducer 3 <- Map 1 (SIMPLE_EDGE) Reducer 4 <- Map 1 (SIMPLE_EDGE) -Reducer 5 <- Map 1 (SIMPLE_EDGE) Stage-6 Stats Work{} @@ -936,78 +898,66 @@ Stage-6 Dependency Collection{} Stage-4 Reducer 2 vectorized - File Output Operator [FS_66] + File Output Operator [FS_64] table:{"name:":"default.store_sales"} - Select Operator [SEL_65] (rows=1 width=1776) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27"] + Select Operator [SEL_63] + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col3","iceberg_bucket(_col2, 3)"] <-Map 1 [SIMPLE_EDGE] vectorized - SHUFFLE [RS_60] - Select Operator [SEL_56] (rows=1 width=1660) - Output:["_col0","_col1","_col2","_col3","_col4","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27"] - Filter Operator [FIL_52] (rows=1 width=3160) + SHUFFLE [RS_59] + PartitionCols:_col3, iceberg_bucket(_col2, 3) + Select Operator [SEL_55] (rows=1 width=1388) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22"] + Filter Operator [FIL_51] (rows=1 width=3160) predicate:((_col24 = _col34) and (_col47 = _col37) and (_col36 = 2451181) and (floor((_col34 / 1000)) * 1000) BETWEEN 1000 AND 2000 and (_col30 < 0) and _col33 is null) - Select Operator [SEL_51] (rows=7 width=1029) + Select Operator [SEL_49] (rows=7 width=1029) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43","_col44","_col45","_col46","_col47","_col48","_col49"] - Map Join Operator [MAPJOIN_50] (rows=7 width=1029) - Conds:SEL_49._col2, _col1=RS_48._col8, _col7(Left Outer),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43","_col44","_col45","_col46","_col47","_col48","_col49","_col50","_col51"] - <-Map 6 [BROADCAST_EDGE] vectorized - BROADCAST [RS_48] + Map Join Operator [MAPJOIN_48] (rows=7 width=1029) + Conds:SEL_47._col2, _col1=RS_46._col8, _col7(Left Outer),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43","_col44","_col45","_col46","_col47","_col48","_col49","_col50","_col51"] + <-Map 5 [BROADCAST_EDGE] vectorized + BROADCAST [RS_46] PartitionCols:_col8, _col7 - Select Operator [SEL_47] (rows=2 width=1088) + Select Operator [SEL_45] (rows=2 width=1088) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27"] - Filter Operator [FIL_46] (rows=2 width=700) + Filter Operator [FIL_44] (rows=2 width=700) predicate:((ss_sold_date_sk = 2451181) and ss_item_sk is not null and ss_customer_sk is not null) TableScan [TS_2] (rows=2 width=55###) default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","ss_sold_time_sk","ss_item_sk","ss_customer_sk","ss_cdemo_sk","ss_hdemo_sk","ss_addr_sk","ss_store_sk","ss_promo_sk","ss_ticket_number","ss_quantity","ss_wholesale_cost","ss_list_price","ss_sales_price","ss_ext_discount_amt","ss_ext_sales_price","ss_ext_wholesale_cost","ss_ext_list_price","ss_ext_tax","ss_coupon_amt","ss_net_paid","ss_net_paid_inc_tax","ss_net_profit"] - <-Select Operator [SEL_49] (rows=5 width=380) + <-Select Operator [SEL_47] (rows=5 width=380) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23"] TableScan [TS_0] (rows=5 width=372) default@ssv,s,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_time_sk","ss_item_sk2","ss_customer_sk2","ss_cdemo_sk","ss_hdemo_sk","ss_addr_sk","ss_store_sk","ss_promo_sk","ss_ticket_number","ss_quantity","ss_wholesale_cost","ss_list_price","ss_sales_price","ss_ext_discount_amt","ss_ext_sales_price","ss_ext_wholesale_cost","ss_ext_list_price","ss_ext_tax","ss_coupon_amt","ss_net_paid","ss_net_paid_inc_tax","ss_net_profit"] Reducer 3 vectorized - File Output Operator [FS_68] + File Output Operator [FS_66] table:{"name:":"default.store_sales"} - Select Operator [SEL_67] + Select Operator [SEL_65] Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col3","iceberg_bucket(_col2, 3)"] <-Map 1 [SIMPLE_EDGE] vectorized - SHUFFLE [RS_61] + SHUFFLE [RS_60] PartitionCols:_col3, iceberg_bucket(_col2, 3) - Select Operator [SEL_57] (rows=1 width=1388) + Select Operator [SEL_56] (rows=3 width=966) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22"] - Filter Operator [FIL_53] (rows=1 width=3160) - predicate:((_col24 = _col34) and (_col47 = _col37) and (_col36 = 2451181) and (floor((_col34 / 1000)) * 1000) BETWEEN 1000 AND 2000 and (_col30 < 0) and _col33 is null) - Please refer to the previous Select Operator [SEL_51] + Filter Operator [FIL_52] (rows=3 width=1688) + predicate:(_col24 is null and _col47 is null and _col36 is null) + Please refer to the previous Select Operator [SEL_49] Reducer 4 vectorized File Output Operator [FS_70] - table:{"name:":"default.store_sales"} - Select Operator [SEL_69] - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col3","iceberg_bucket(_col2, 3)"] - <-Map 1 [SIMPLE_EDGE] vectorized - SHUFFLE [RS_62] - PartitionCols:_col3, iceberg_bucket(_col2, 3) - Select Operator [SEL_58] (rows=3 width=966) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22"] - Filter Operator [FIL_54] (rows=3 width=1688) - predicate:(_col24 is null and _col47 is null and _col36 is null) - Please refer to the previous Select Operator [SEL_51] - Reducer 5 vectorized - File Output Operator [FS_74] table:{"name:":"default.merge_tmp_table"} - Select Operator [SEL_73] (rows=1 width=4) + Select Operator [SEL_69] (rows=1 width=4) Output:["_col0"] - Filter Operator [FIL_72] (rows=1 width=396) - predicate:(_col5 > 1L) - Group By Operator [GBY_71] (rows=1 width=396) - Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4 + Filter Operator [FIL_68] (rows=1 width=212) + predicate:(_col4 > 1L) + Group By Operator [GBY_67] (rows=1 width=212) + Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3 <-Map 1 [SIMPLE_EDGE] vectorized - SHUFFLE [RS_64] - PartitionCols:_col0, _col1, _col2, _col3, _col4 - Group By Operator [GBY_63] (rows=1 width=396) - Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["count()"],keys:_col4, _col40, _col6, _col45, _col12 - Select Operator [SEL_59] (rows=1 width=3160) - Output:["_col4","_col6","_col12","_col40","_col45"] - Filter Operator [FIL_55] (rows=1 width=3160) + SHUFFLE [RS_62] + PartitionCols:_col0, _col1, _col2, _col3 + Group By Operator [GBY_61] (rows=1 width=212) + Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["count()"],keys:_col4, _col40, _col6, _col45 + Select Operator [SEL_57] (rows=1 width=3160) + Output:["_col4","_col6","_col40","_col45"] + Filter Operator [FIL_53] (rows=1 width=3160) predicate:((_col24 = _col34) and (_col47 = _col37) and (_col36 = 2451181) and (floor((_col34 / 1000)) * 1000) BETWEEN 1000 AND 2000 and (_col30 < 0)) - Please refer to the previous Select Operator [SEL_51] + Please refer to the previous Select Operator [SEL_49] Stage-7 Stats Work{} Stage-3 diff --git a/iceberg/iceberg-handler/src/test/results/positive/write_iceberg_branch.q.out b/iceberg/iceberg-handler/src/test/results/positive/write_iceberg_branch.q.out index 42b9ad902dc6..bc662c426a44 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/write_iceberg_branch.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/write_iceberg_branch.q.out @@ -226,9 +226,6 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-2 Tez -#### A masked pattern was here #### - Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -242,31 +239,18 @@ STAGE PLANS: predicate: (a = 22) (type: boolean) Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), b (type: string), c (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col6, _col7 - Statistics: Num rows: 2 Data size: 962 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ - Statistics: Num rows: 2 Data size: 962 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col6 (type: string), _col7 (type: int) - Execution mode: vectorized - Reducer 2 + expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), 22 (type: int), b (type: string), c (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 + Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.ice01 Execution mode: vectorized - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), 22 (type: int), VALUE._col0 (type: string), VALUE._col1 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.ice01 Stage: Stage-3 Dependency Collection @@ -324,9 +308,6 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-2 Tez -#### A masked pattern was here #### - Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -344,15 +325,17 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col9 Statistics: Num rows: 2 Data size: 1140 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 2 Data size: 962 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ - Statistics: Num rows: 2 Data size: 962 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col5 (type: int), _col6 (type: string) + expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), 66 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 + Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.ice01 Select Operator expressions: 33 (type: int), _col9 (type: string), 66 (type: int) outputColumnNames: _col0, _col1, _col2 @@ -366,21 +349,6 @@ STAGE PLANS: serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe name: default.ice01 Execution mode: vectorized - Reducer 2 - Execution mode: vectorized - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), VALUE._col0 (type: int), VALUE._col1 (type: string), 66 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.ice01 Stage: Stage-3 Dependency Collection @@ -456,10 +424,8 @@ STAGE PLANS: Tez #### A masked pattern was here #### Edges: - Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE) Reducer 3 <- Reducer 2 (SIMPLE_EDGE) - Reducer 4 <- Reducer 2 (SIMPLE_EDGE) - Reducer 5 <- Reducer 2 (SIMPLE_EDGE) #### A masked pattern was here #### Vertices: Map 1 @@ -479,7 +445,7 @@ STAGE PLANS: Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col1 (type: string), _col2 (type: int) Execution mode: vectorized - Map 6 + Map 4 Map Operator Tree: TableScan alias: ice01 @@ -522,12 +488,14 @@ STAGE PLANS: expressions: _col7 (type: int), _col6 (type: bigint), _col2 (type: string), _col5 (type: bigint), _col3 (type: string), _col10 (type: int), _col9 (type: string), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ + File Output Operator + compressed: false Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col5 (type: int), _col6 (type: string), _col7 (type: int) + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.ice01 Filter Operator predicate: ((_col10 = _col1) and (_col10 <= 100)) (type: boolean) Statistics: Num rows: 1 Data size: 581 Basic stats: COMPLETE Column stats: COMPLETE @@ -535,12 +503,14 @@ STAGE PLANS: expressions: _col7 (type: int), _col6 (type: bigint), _col2 (type: string), _col5 (type: bigint), _col3 (type: string), _col10 (type: int), _col9 (type: string), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ + File Output Operator + compressed: false Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col5 (type: int), _col6 (type: string), _col7 (type: int) + table: + input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat + output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat + serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe + name: default.ice01 Filter Operator predicate: ((_col10 = _col1) and (_col10 <= 100)) (type: boolean) Statistics: Num rows: 1 Data size: 581 Basic stats: COMPLETE Column stats: COMPLETE @@ -575,67 +545,37 @@ STAGE PLANS: predicate: (_col10 = _col1) (type: boolean) Statistics: Num rows: 1 Data size: 581 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: _col2 (type: string), _col3 (type: string), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: int) - outputColumnNames: _col2, _col3, _col5, _col6, _col7 + expressions: _col2 (type: string), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: int) + outputColumnNames: _col2, _col5, _col6, _col7 Statistics: Num rows: 1 Data size: 581 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: count() - keys: _col7 (type: int), _col6 (type: bigint), _col2 (type: string), _col5 (type: bigint), _col3 (type: string) + keys: _col7 (type: int), _col6 (type: bigint), _col2 (type: string), _col5 (type: bigint) minReductionHashAggr: 0.4 mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - null sort order: zzzzz - sort order: +++++ - Map-reduce partition columns: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col5 (type: bigint) + key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint) + null sort order: zzzz + sort order: ++++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint) + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col4 (type: bigint) Reducer 3 - Execution mode: vectorized - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.ice01 - Reducer 4 - Execution mode: vectorized - Reduce Operator Tree: - Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: bigint), KEY.reducesinkkey4 (type: string), VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat - output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat - serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe - name: default.ice01 - Reducer 5 Execution mode: vectorized Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: KEY._col0 (type: int), KEY._col1 (type: bigint), KEY._col2 (type: string), KEY._col3 (type: bigint), KEY._col4 (type: string) + keys: KEY._col0 (type: int), KEY._col1 (type: bigint), KEY._col2 (type: string), KEY._col3 (type: bigint) mode: mergepartial - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: (_col5 > 1L) (type: boolean) - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE + predicate: (_col4 > 1L) (type: boolean) + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: cardinality_violation(_col0,_col1,_col2,_col3,_col4) (type: int) + expressions: cardinality_violation(_col0,_col1,_col2,_col3) (type: int) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java index 7c3fc9c302e6..80e51f5fc999 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java @@ -161,7 +161,7 @@ public static enum Counter { * RecordWriter. * */ - public static interface RecordWriter { + public interface RecordWriter { void write(Writable w) throws IOException; void close(boolean abort) throws IOException; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriter.java index 5066a5307860..5f7644cc594c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/DeleteRewriter.java @@ -50,7 +50,7 @@ public ParseUtils.ReparseResult rewrite(Context context, DeleteStatement deleteB sqlGenerator.append(" from "); sqlGenerator.append(sqlGenerator.getTargetTableFullName()); - sqlGenerator.appendSortBy(sqlGenerator.getSortKeys()); + sqlGenerator.appendSortKeys(); ParseUtils.ReparseResult rr = ParseUtils.parseRewrittenQuery(context, sqlGenerator.toString()); Context rewrittenCtx = rr.rewrittenCtx; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriter.java index e94e6f579e19..44b0a0d9e5a6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/MergeRewriter.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.Context.Operation; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.HiveUtils; @@ -104,7 +105,7 @@ private void handleSource(boolean hasWhenNotMatchedClause, String sourceAlias, S MultiInsertSqlGenerator sqlGenerator) { sqlGenerator.append("FROM\n"); sqlGenerator.append("(SELECT "); - sqlGenerator.appendAcidSelectColumns(Context.Operation.MERGE); + sqlGenerator.appendAcidSelectColumns(Operation.MERGE); sqlGenerator.appendAllColsOfTargetTable(); sqlGenerator.append(" FROM ").appendTargetTableName().append(") "); sqlGenerator.appendSubQueryAlias(); @@ -121,7 +122,7 @@ private void handleCardinalityViolation( //this is a tmp table and thus Session scoped and acid requires SQL statement to be serial in a // given session, i.e. the name can be fixed across all invocations String tableName = "merge_tmp_table"; - List sortKeys = sqlGenerator.getSortKeys(); + List sortKeys = sqlGenerator.getSortKeys(Operation.MERGE); sqlGenerator.append("INSERT INTO ").append(tableName) .append("\n SELECT cardinality_violation(") .append(StringUtils.join(sortKeys, ",")); @@ -159,7 +160,7 @@ private void handleCardinalityViolation( } protected void setOperation(Context context) { - context.setOperation(Context.Operation.MERGE); + context.setOperation(Operation.MERGE); } protected static class MergeWhenClauseSqlGenerator implements MergeStatement.MergeSqlGenerator { @@ -211,7 +212,7 @@ public void appendWhenMatchedUpdateClause(MergeStatement.UpdateClause updateClau sqlGenerator.append(" -- update clause").append("\n"); List valuesAndAcidSortKeys = new ArrayList<>( targetTable.getCols().size() + targetTable.getPartCols().size() + 1); - valuesAndAcidSortKeys.addAll(sqlGenerator.getSortKeys()); + valuesAndAcidSortKeys.addAll(sqlGenerator.getSortKeys(Operation.MERGE)); addValues(targetTable, targetAlias, updateClause.getNewValuesMap(), valuesAndAcidSortKeys); sqlGenerator.appendInsertBranch(hintStr, valuesAndAcidSortKeys); hintStr = null; @@ -219,7 +220,7 @@ public void appendWhenMatchedUpdateClause(MergeStatement.UpdateClause updateClau addWhereClauseOfUpdate( onClauseAsString, updateClause.getExtraPredicate(), updateClause.getDeleteExtraPredicate(), sqlGenerator); - sqlGenerator.appendSortBy(sqlGenerator.getSortKeys()); + sqlGenerator.appendSortKeys(); } protected void addValues(Table targetTable, String targetAlias, Map newValues, diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/SplitUpdateRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/SplitUpdateRewriter.java index 75e46d2c78fe..16724f42d70a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/SplitUpdateRewriter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/SplitUpdateRewriter.java @@ -107,8 +107,7 @@ public ParseUtils.ReparseResult rewrite(Context context, UpdateStatement updateB sqlGenerator.appendInsertBranch(null, insertValues); sqlGenerator.appendInsertBranch(null, deleteValues); - List sortKeys = sqlGenerator.getSortKeys(); - sqlGenerator.appendSortBy(sortKeys); + sqlGenerator.appendSortKeys(); ParseUtils.ReparseResult rr = ParseUtils.parseRewrittenQuery(context, sqlGenerator.toString()); Context rewrittenCtx = rr.rewrittenCtx; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriter.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriter.java index 6595d1b303b0..88c3622d887d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/UpdateRewriter.java @@ -83,7 +83,7 @@ public ParseUtils.ReparseResult rewrite(Context context, UpdateStatement updateB sqlGenerator.appendTargetTableName(); // Add a sort by clause so that the row ids come out in the correct order - sqlGenerator.appendSortBy(sqlGenerator.getSortKeys()); + sqlGenerator.appendSortKeys(); ParseUtils.ReparseResult rr = ParseUtils.parseRewrittenQuery(context, sqlGenerator.toString()); Context rewrittenCtx = rr.rewrittenCtx; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/MultiInsertSqlGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/MultiInsertSqlGenerator.java index ddd57a6dd074..7587daf13055 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/MultiInsertSqlGenerator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/MultiInsertSqlGenerator.java @@ -20,7 +20,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.Context.Operation; import org.apache.hadoop.hive.ql.metadata.HiveUtils; import org.apache.hadoop.hive.ql.metadata.Table; @@ -61,18 +61,14 @@ public String getTargetTableFullName() { return targetTableFullName; } - public abstract void appendAcidSelectColumns(Context.Operation operation); - - public void appendAcidSelectColumnsForDeletedRecords(Context.Operation operation) { - appendAcidSelectColumnsForDeletedRecords(operation, true); - } + public abstract void appendAcidSelectColumns(Operation operation); - public void appendAcidSelectColumnsForDeletedRecords(Context.Operation operation, boolean skipPrefix) { + public void appendAcidSelectColumnsForDeletedRecords(Operation operation, boolean skipPrefix) { throw new UnsupportedOperationException(); } - public abstract List getDeleteValues(Context.Operation operation); - public abstract List getSortKeys(); + public abstract List getDeleteValues(Operation operation); + public abstract List getSortKeys(Operation operation); public String qualify(String columnName) { if (isBlank(subQueryAlias)) { @@ -97,7 +93,7 @@ public void appendInsertBranch(String hintStr, List values) { } public void appendDeleteBranch(String hintStr) { - List deleteValues = getDeleteValues(Context.Operation.DELETE); + List deleteValues = getDeleteValues(Operation.DELETE); appendInsertBranch(hintStr, deleteValues); } @@ -124,7 +120,7 @@ public void appendPartitionCols(Table table) { queryStr.append(")"); } - public void appendSortBy(List keys) { + private void appendSortBy(List keys) { if (keys.isEmpty()) { return; } @@ -134,7 +130,7 @@ public void appendSortBy(List keys) { } public void appendSortKeys() { - appendSortBy(getSortKeys()); + appendSortBy(getSortKeys(Operation.DELETE)); } public MultiInsertSqlGenerator append(String sqlTextFragment) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NativeAcidMultiInsertSqlGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NativeAcidMultiInsertSqlGenerator.java index 729038fac6fa..87e426800442 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NativeAcidMultiInsertSqlGenerator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NativeAcidMultiInsertSqlGenerator.java @@ -19,7 +19,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.Context.Operation; import org.apache.hadoop.hive.ql.metadata.HiveUtils; import org.apache.hadoop.hive.ql.metadata.Table; @@ -34,7 +34,7 @@ public NativeAcidMultiInsertSqlGenerator(Table table, String targetTableFullName } @Override - public void appendAcidSelectColumns(Context.Operation operation) { + public void appendAcidSelectColumns(Operation operation) { queryStr.append("ROW__ID,"); for (FieldSchema fieldSchema : targetTable.getPartCols()) { String identifier = HiveUtils.unparseIdentifier(fieldSchema.getName(), this.conf); @@ -44,7 +44,7 @@ public void appendAcidSelectColumns(Context.Operation operation) { } @Override - public List getDeleteValues(Context.Operation operation) { + public List getDeleteValues(Operation operation) { List deleteValues = new ArrayList<>(1 + targetTable.getPartCols().size()); deleteValues.add(qualify("ROW__ID")); for (FieldSchema fieldSchema : targetTable.getPartCols()) { @@ -54,7 +54,7 @@ public List getDeleteValues(Context.Operation operation) { } @Override - public List getSortKeys() { + public List getSortKeys(Operation operation) { return singletonList(qualify("ROW__ID")); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NonNativeAcidMultiInsertSqlGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NonNativeAcidMultiInsertSqlGenerator.java index 6893a8810bc3..fee4e5477554 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NonNativeAcidMultiInsertSqlGenerator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/sql/NonNativeAcidMultiInsertSqlGenerator.java @@ -20,7 +20,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.Context.Operation; import org.apache.hadoop.hive.ql.metadata.HiveUtils; import org.apache.hadoop.hive.ql.metadata.Table; @@ -38,16 +38,16 @@ public NonNativeAcidMultiInsertSqlGenerator( } @Override - public void appendAcidSelectColumns(Context.Operation operation) { + public void appendAcidSelectColumns(Operation operation) { appendAcidSelectColumns(operation, false, false); } @Override - public void appendAcidSelectColumnsForDeletedRecords(Context.Operation operation, boolean skipPrefix) { + public void appendAcidSelectColumnsForDeletedRecords(Operation operation, boolean skipPrefix) { appendAcidSelectColumns(operation, true, skipPrefix); } - private void appendAcidSelectColumns(Context.Operation operation, boolean markRowIdAsDeleted, boolean skipPrefix) { + private void appendAcidSelectColumns(Operation operation, boolean markRowIdAsDeleted, boolean skipPrefix) { List acidSelectColumns = targetTable.getStorageHandler().acidSelectColumns(targetTable, operation); for (FieldSchema fieldSchema : acidSelectColumns) { boolean deletedRowId = markRowIdAsDeleted && fieldSchema.equals(targetTable.getStorageHandler().getRowId()); @@ -70,19 +70,18 @@ private void appendAcidSelectColumns(Context.Operation operation, boolean markRo } @Override - public List getDeleteValues(Context.Operation operation) { - List acidSelectColumns = targetTable.getStorageHandler().acidSelectColumns(targetTable, operation); - List deleteValues = new ArrayList<>(acidSelectColumns.size()); - for (FieldSchema fieldSchema : acidSelectColumns) { - String prefixedIdentifier = HiveUtils.unparseIdentifier(deletePrefix + fieldSchema.getName(), this.conf); - deleteValues.add(qualify(prefixedIdentifier)); - } - return deleteValues; + public List getDeleteValues(Operation operation) { + return targetTable.getStorageHandler().acidSelectColumns(targetTable, operation) + .stream() + .map(fieldSchema -> qualify( + HiveUtils.unparseIdentifier(deletePrefix + fieldSchema.getName(), this.conf))) + .collect(Collectors.toList()); } @Override - public List getSortKeys() { - return targetTable.getStorageHandler().acidSortColumns(targetTable, Context.Operation.DELETE).stream() + public List getSortKeys(Operation operation) { + return targetTable.getStorageHandler().acidSortColumns(targetTable, operation) + .stream() .map(fieldSchema -> qualify( HiveUtils.unparseIdentifier(deletePrefix + fieldSchema.getName(), this.conf))) .collect(Collectors.toList());