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 cdc5dbac31c3..cb7cafc610b6 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 @@ -19,7 +19,6 @@ package org.apache.iceberg.mr.hive; -import com.fasterxml.jackson.databind.ObjectMapper; import java.io.Closeable; import java.io.IOException; import java.io.Serializable; @@ -61,8 +60,8 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.metastore.HiveMetaHook; -import org.apache.hadoop.hive.metastore.HiveMetaStoreUtils; import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.AggrStats; import org.apache.hadoop.hive.metastore.api.ColumnStatistics; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; import org.apache.hadoop.hive.metastore.api.EnvironmentContext; @@ -71,21 +70,19 @@ import org.apache.hadoop.hive.metastore.api.LockType; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf; import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils; import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; -import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.Context.Operation; import org.apache.hadoop.hive.ql.Context.RewritePolicy; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.QueryState; -import org.apache.hadoop.hive.ql.ddl.DDLOperationContext; import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableDesc; import org.apache.hadoop.hive.ql.ddl.table.AlterTableType; import org.apache.hadoop.hive.ql.ddl.table.create.CreateTableDesc; import org.apache.hadoop.hive.ql.ddl.table.create.like.CreateTableLikeDesc; import org.apache.hadoop.hive.ql.ddl.table.misc.properties.AlterTableSetPropertiesDesc; import org.apache.hadoop.hive.ql.exec.ColumnInfo; -import org.apache.hadoop.hive.ql.exec.FetchOperator; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.io.IOConstants; @@ -123,26 +120,18 @@ import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.ql.session.SessionStateUtil; import org.apache.hadoop.hive.ql.stats.Partish; -import org.apache.hadoop.hive.ql.stats.StatsUtils; import org.apache.hadoop.hive.ql.util.NullOrdering; import org.apache.hadoop.hive.serde2.AbstractSerDe; -import org.apache.hadoop.hive.serde2.DefaultFetchFormatter; import org.apache.hadoop.hive.serde2.Deserializer; -import org.apache.hadoop.hive.serde2.FetchFormatter; import org.apache.hadoop.hive.serde2.SerDeException; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.StructField; import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.mapred.InputFormat; -import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobContext; import org.apache.hadoop.mapred.OutputFormat; -import org.apache.hadoop.mapred.RecordReader; -import org.apache.hadoop.mapred.Reporter; import org.apache.iceberg.BaseMetastoreTableOperations; import org.apache.iceberg.BaseTable; import org.apache.iceberg.DataFile; @@ -203,6 +192,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Splitter; import org.apache.iceberg.relocated.com.google.common.base.Throwables; +import org.apache.iceberg.relocated.com.google.common.collect.FluentIterable; 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.Iterables; @@ -290,10 +280,10 @@ public void configureInputJobProperties(TableDesc tableDesc, Map fallbackToNonVectorizedModeBasedOnProperties(tableDesc.getProperties()); boolean allowDataFilesWithinTableLocationOnly = - conf.getBoolean(HiveConf.ConfVars.HIVE_ICEBERG_ALLOW_DATAFILES_IN_TABLE_LOCATION_ONLY.varname, - HiveConf.ConfVars.HIVE_ICEBERG_ALLOW_DATAFILES_IN_TABLE_LOCATION_ONLY.defaultBoolVal); + conf.getBoolean(ConfVars.HIVE_ICEBERG_ALLOW_DATAFILES_IN_TABLE_LOCATION_ONLY.varname, + ConfVars.HIVE_ICEBERG_ALLOW_DATAFILES_IN_TABLE_LOCATION_ONLY.defaultBoolVal); - map.put(HiveConf.ConfVars.HIVE_ICEBERG_ALLOW_DATAFILES_IN_TABLE_LOCATION_ONLY.varname, + map.put(ConfVars.HIVE_ICEBERG_ALLOW_DATAFILES_IN_TABLE_LOCATION_ONLY.varname, String.valueOf(allowDataFilesWithinTableLocationOnly)); } @@ -349,7 +339,7 @@ public void configureJobConf(TableDesc tableDesc, JobConf jobConf) { "Can not handle table " + tableName + ". Its name contains '" + TABLE_NAME_SEPARATOR + "'"); if (HiveCustomStorageHandlerUtils.getWriteOperation(tableDesc.getProperties()::getProperty, tableName) != null) { HiveCustomStorageHandlerUtils.setWriteOperation(jobConf, tableName, - Context.Operation.valueOf(tableDesc.getProperties().getProperty( + Operation.valueOf(tableDesc.getProperties().getProperty( HiveCustomStorageHandlerUtils.WRITE_OPERATION_CONFIG_PREFIX + tableName))); } boolean isMergeTaskEnabled = Boolean.parseBoolean(tableDesc.getProperty( @@ -367,7 +357,7 @@ public void configureJobConf(TableDesc tableDesc, JobConf jobConf) { } } try { - if (!jobConf.getBoolean(HiveConf.ConfVars.HIVE_IN_TEST_IDE.varname, false)) { + if (!jobConf.getBoolean(ConfVars.HIVE_IN_TEST_IDE.varname, false)) { // For running unit test this won't work as maven surefire CP is different than what we have on a cluster: // it places the current projects' classes and test-classes to top instead of jars made from these... Utilities.addDependencyJars(jobConf, HiveIcebergStorageHandler.class); @@ -415,14 +405,14 @@ public DecomposedPredicate decomposePredicate(JobConf jobConf, Deserializer dese ExprNodeDesc pushedPredicate = exprNodeDesc.clone(); List subExprNodes = pushedPredicate.getChildren(); - Set skipList = Stream.of(VirtualColumn.FILE_PATH, VirtualColumn.PARTITION_SPEC_ID, - VirtualColumn.PARTITION_HASH) + Set skipList = Stream.of(FILE_PATH, PARTITION_SPEC_ID, PARTITION_HASH) .map(VirtualColumn::getName).collect(Collectors.toSet()); if (subExprNodes.removeIf(nodeDesc -> nodeDesc.getCols() != null && - nodeDesc.getCols().stream().anyMatch(skipList::contains))) { + nodeDesc.getCols().stream().anyMatch(skipList::contains))) { if (subExprNodes.size() == 1) { - pushedPredicate = subExprNodes.get(0); + pushedPredicate = (subExprNodes.get(0) instanceof ExprNodeGenericFuncDesc) ? + subExprNodes.get(0) : null; } else if (subExprNodes.isEmpty()) { pushedPredicate = null; } @@ -491,23 +481,33 @@ public void appendFiles(org.apache.hadoop.hive.metastore.api.Table table, URI fr @Override public Map getBasicStatistics(Partish partish) { + return getBasicStatistics(partish, false); + } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + private Map getBasicStatistics(Partish partish, boolean quickStats) { Map stats = Maps.newHashMap(); - if (!getStatsSource().equals(HiveMetaHook.ICEBERG)) { - return partish.getPartParameters(); - } org.apache.hadoop.hive.ql.metadata.Table hmsTable = partish.getTable(); // For write queries where rows got modified, don't fetch from cache as values could have changed. Table table = getTable(hmsTable); - Snapshot snapshot = IcebergTableUtil.getTableSnapshot(table, hmsTable); - if (snapshot != null) { + + if (snapshot == null) { + stats.put(StatsSetupConst.NUM_FILES, "0"); + stats.put(StatsSetupConst.ROW_COUNT, "0"); + stats.put(StatsSetupConst.TOTAL_SIZE, "0"); + + } else if (!getStatsSource().equals(HiveMetaHook.ICEBERG) && !quickStats) { + stats = partish.getPartParameters(); + + } else { Map summary = getPartishSummary(partish, table, snapshot); if (summary != null) { if (summary.containsKey(TOTAL_DATA_FILES_PROP)) { stats.put(StatsSetupConst.NUM_FILES, summary.get(TOTAL_DATA_FILES_PROP)); } - if (summary.containsKey(TOTAL_RECORDS_PROP)) { + if (summary.containsKey(TOTAL_RECORDS_PROP) && !quickStats) { long totalRecords = Long.parseLong(summary.get(TOTAL_RECORDS_PROP)); if (summary.containsKey(TOTAL_EQ_DELETES_PROP) && summary.containsKey(TOTAL_POS_DELETES_PROP)) { @@ -525,18 +525,21 @@ public Map getBasicStatistics(Partish partish) { stats.put(StatsSetupConst.TOTAL_SIZE, summary.get(TOTAL_FILE_SIZE_PROP)); } } - } else { - stats.put(StatsSetupConst.NUM_FILES, "0"); - stats.put(StatsSetupConst.ROW_COUNT, "0"); - stats.put(StatsSetupConst.TOTAL_SIZE, "0"); } return stats; } @Override public Map computeBasicStatistics(Partish partish) { + Map stats; if (!getStatsSource().equals(HiveMetaHook.ICEBERG)) { - return partish.getPartParameters(); + stats = partish.getPartParameters(); + + if (!StatsSetupConst.areBasicStatsUptoDate(stats)) { + // populate quick-stats + stats = getBasicStatistics(partish, true); + } + return stats; } org.apache.hadoop.hive.ql.metadata.Table hmsTable = partish.getTable(); // For write queries where rows got modified, don't fetch from cache as values could have changed. @@ -612,23 +615,21 @@ private Table getTable(org.apache.hadoop.hive.ql.metadata.Table hmsTable) { @Override public boolean canSetColStatistics(org.apache.hadoop.hive.ql.metadata.Table hmsTable) { - Table table = IcebergTableUtil.getTable(conf, hmsTable.getTTable()); - return table.currentSnapshot() != null && getStatsSource().equals(HiveMetaHook.ICEBERG); + return getStatsSource().equals(HiveMetaHook.ICEBERG); } @Override public boolean setColStatistics(org.apache.hadoop.hive.ql.metadata.Table hmsTable, List colStats) { Table tbl = IcebergTableUtil.getTable(conf, hmsTable.getTTable()); - return writeColStats(colStats.get(0), tbl); + return writeColStats(colStats, tbl); } - private boolean writeColStats(ColumnStatistics tableColStats, Table tbl) { + private boolean writeColStats(List colStats, Table tbl) { try { if (!shouldRewriteColStats(tbl)) { - checkAndMergeColStats(tableColStats, tbl); + checkAndMergeColStats(colStats, tbl); } // Currently, we are only serializing table level stats. - byte[] serializeColStats = SerializationUtils.serialize(tableColStats); StatisticsFile statisticsFile; String statsPath = tbl.location() + STATS + UUID.randomUUID(); @@ -636,7 +637,10 @@ private boolean writeColStats(ColumnStatistics tableColStats, Table tbl) { .createdBy(Constants.HIVE_ENGINE).build()) { long snapshotId = tbl.currentSnapshot().snapshotId(); long snapshotSequenceNumber = tbl.currentSnapshot().sequenceNumber(); - puffinWriter.add( + + colStats.forEach(statsObj -> { + byte[] serializeColStats = SerializationUtils.serialize(statsObj); + puffinWriter.add( new Blob( ColumnStatisticsObj.class.getSimpleName(), ImmutableList.of(1), @@ -644,8 +648,10 @@ private boolean writeColStats(ColumnStatistics tableColStats, Table tbl) { snapshotSequenceNumber, ByteBuffer.wrap(serializeColStats), PuffinCompressionCodec.NONE, - ImmutableMap.of() - )); + ImmutableMap.of("partition", + String.valueOf(statsObj.getStatsDesc().getPartName())) + )); + }); puffinWriter.finish(); statisticsFile = @@ -695,25 +701,64 @@ public List getColStatistics(org.apache.hadoop.hive.ql.meta ColumnStatistics emptyStats = new ColumnStatistics(); if (snapshot != null) { return IcebergTableUtil.getColStatsPath(table, snapshot.snapshotId()) - .map(statsPath -> readColStats(table, statsPath)) + .map(statsPath -> readColStats(table, statsPath, null).get(0)) .orElse(emptyStats).getStatsObj(); } return emptyStats.getStatsObj(); } - private ColumnStatistics readColStats(Table table, Path statsPath) { + @Override + public AggrStats getAggrColStatsFor(org.apache.hadoop.hive.ql.metadata.Table hmsTable, List colNames, + List partNames) throws MetaException { + Table table = IcebergTableUtil.getTable(conf, hmsTable.getTTable()); + Snapshot snapshot = IcebergTableUtil.getTableSnapshot(table, hmsTable); + + AggrStats emptyStats = new AggrStats(Collections.emptyList(), 0); + if (snapshot != null) { + boolean useDensityFunctionForNDVEstimation = MetastoreConf.getBoolVar(getConf(), + MetastoreConf.ConfVars.STATS_NDV_DENSITY_FUNCTION); + double ndvTuner = MetastoreConf.getDoubleVar(getConf(), MetastoreConf.ConfVars.STATS_NDV_TUNER); + + List partStats = IcebergTableUtil.getColStatsPath(table, snapshot.snapshotId()) + .map(statsPath -> readColStats(table, statsPath, Sets.newHashSet(partNames))) + .orElse(Collections.emptyList()); + + partStats.forEach(colStats -> + colStats.getStatsObj().removeIf(statsObj -> !colNames.contains(statsObj.getColName()))); + + List colStatsList = MetaStoreServerUtils.aggrPartitionStats(partStats, + MetaStoreUtils.getDefaultCatalog(conf), hmsTable.getDbName(), hmsTable.getTableName(), + partNames, colNames, + partStats.size() == partNames.size(), + useDensityFunctionForNDVEstimation, ndvTuner); + + return new AggrStats(colStatsList, partStats.size()); + } + return emptyStats; + } + + private List readColStats(Table table, Path statsPath, Set partNames) { + List colStats = Lists.newArrayList(); + try (PuffinReader reader = Puffin.read(table.io().newInputFile(statsPath.toString())).build()) { List blobMetadata = reader.fileMetadata().blobs(); + + if (partNames != null) { + blobMetadata = blobMetadata.stream() + .filter(metadata -> partNames.contains(metadata.properties().get("partition"))) + .collect(Collectors.toList()); + } Iterator it = Iterables.transform(reader.readAll(blobMetadata), Pair::second).iterator(); - if (it.hasNext()) { + LOG.info("Using col stats from : {}", statsPath); + + while (it.hasNext()) { byte[] byteBuffer = ByteBuffers.toByteArray(it.next()); - LOG.info("Using col stats from : {}", statsPath); - return SerializationUtils.deserialize(byteBuffer); + colStats.add(SerializationUtils.deserialize(byteBuffer)); } } catch (Exception e) { LOG.warn(" Unable to read col stats: ", e); } - return new ColumnStatistics(); + return colStats; } @Override @@ -722,8 +767,8 @@ public boolean canComputeQueryUsingStats(Partish partish) { if (hmsTable.getMetaTable() != null) { return false; } - if (!getStatsSource().equals(HiveMetaHook.ICEBERG) && - StatsUtils.areBasicStatsUptoDateForQueryAnswering(partish.getTable(), partish.getPartParameters())) { + if (!getStatsSource().equals(HiveMetaHook.ICEBERG) && StatsSetupConst.areBasicStatsUptoDate( + partish.getPartParameters())) { return true; } Table table = getTable(hmsTable); @@ -742,7 +787,7 @@ public boolean canComputeQueryUsingStats(Partish partish) { } private String getStatsSource() { - return HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_ICEBERG_STATS_SOURCE, HiveMetaHook.ICEBERG) + return HiveConf.getVar(conf, ConfVars.HIVE_ICEBERG_STATS_SOURCE, HiveMetaHook.ICEBERG) .toUpperCase(); } @@ -752,14 +797,35 @@ private boolean shouldRewriteColStats(Table tbl) { IcebergTableUtil.getColStatsPath(tbl).isPresent(); } - private void checkAndMergeColStats(ColumnStatistics statsObjNew, Table tbl) throws InvalidObjectException { + private void checkAndMergeColStats(List statsNew, Table tbl) throws InvalidObjectException { Long previousSnapshotId = tbl.currentSnapshot().parentId(); if (previousSnapshotId != null && canProvideColStats(tbl, previousSnapshotId)) { - ColumnStatistics statsObjOld = IcebergTableUtil.getColStatsPath(tbl, previousSnapshotId) - .map(statsPath -> readColStats(tbl, statsPath)) - .orElse(null); - if (statsObjOld != null && statsObjOld.getStatsObjSize() != 0 && !statsObjNew.getStatsObj().isEmpty()) { - MetaStoreServerUtils.mergeColStats(statsObjNew, statsObjOld); + List statsOld = IcebergTableUtil.getColStatsPath(tbl, previousSnapshotId) + .map(statsPath -> readColStats(tbl, statsPath, null)) + .orElse(Collections.emptyList()); + + boolean isTblLevel = statsNew.get(0).getStatsDesc().isIsTblLevel(); + Map oldStatsMap = Maps.newHashMap(); + + if (!isTblLevel) { + for (ColumnStatistics statsObjOld : statsOld) { + oldStatsMap.put(statsObjOld.getStatsDesc().getPartName(), statsObjOld); + } + } + for (ColumnStatistics statsObjNew : statsNew) { + String partitionKey = statsObjNew.getStatsDesc().getPartName(); + ColumnStatistics statsObjOld = isTblLevel ? + statsOld.get(0) : oldStatsMap.get(partitionKey); + + if (statsObjOld != null && statsObjOld.getStatsObjSize() != 0 && !statsObjNew.getStatsObj().isEmpty()) { + MetaStoreServerUtils.mergeColStats(statsObjNew, statsObjOld); + if (!isTblLevel) { + oldStatsMap.remove(partitionKey); + } + } + } + if (!isTblLevel) { + statsNew.addAll(oldStatsMap.values()); } } } @@ -783,34 +849,40 @@ public boolean supportsPartitionTransform() { @Override public List getPartitionTransformSpec(org.apache.hadoop.hive.ql.metadata.Table hmsTable) { - TableDesc tableDesc = Utilities.getTableDesc(hmsTable); - Table table = IcebergTableUtil.getTable(conf, tableDesc.getProperties()); - return table.spec().fields().stream().map(f -> - getTransformSpec(table, f.transform().toString().toUpperCase(), f.sourceId()) - ).collect(Collectors.toList()); + Table table = IcebergTableUtil.getTable(conf, hmsTable.getTTable()); + return table.spec().fields().stream() + .filter(f -> !f.transform().isVoid()) + .map(f -> { + TransformSpec spec = IcebergTableUtil.getTransformSpec( + table, f.transform().toString(), f.sourceId()); + spec.setFieldName(f.name()); + return spec; + }) + .collect(Collectors.toList()); + } + + @Override + public Map> getPartitionTransformSpecs( + org.apache.hadoop.hive.ql.metadata.Table hmsTable) { + Table table = IcebergTableUtil.getTable(conf, hmsTable.getTTable()); + return table.specs().entrySet().stream().flatMap(e -> + e.getValue().fields().stream() + .filter(f -> !f.transform().isVoid()) + .map(f -> { + TransformSpec spec = IcebergTableUtil.getTransformSpec( + table, f.transform().toString(), f.sourceId()); + spec.setFieldName(f.name()); + return Pair.of(e.getKey(), spec); + })) + .collect(Collectors.groupingBy( + Pair::first, Collectors.mapping(Pair::second, Collectors.toList()))); } private List getSortTransformSpec(Table table) { return table.sortOrder().fields().stream().map(s -> - getTransformSpec(table, s.transform().toString().toUpperCase(), s.sourceId()) - ).collect(Collectors.toList()); - } - - private TransformSpec getTransformSpec(Table table, String transformName, int sourceId) { - TransformSpec spec = new TransformSpec(); - spec.setColumnName(table.schema().findColumnName(sourceId)); - // if the transform name contains '[' it means it has some config params - if (transformName.contains("[")) { - spec.setTransformType(TransformSpec.TransformType - .valueOf(transformName.substring(0, transformName.indexOf("[")))); - spec.setTransformParam(Optional.of(Integer - .valueOf(transformName.substring(transformName.indexOf("[") + 1, transformName.indexOf("]"))))); - } else { - spec.setTransformType(TransformSpec.TransformType.valueOf(transformName)); - spec.setTransformParam(Optional.empty()); - } - - return spec; + IcebergTableUtil.getTransformSpec(table, s.transform().toString(), s.sourceId()) + ) + .collect(Collectors.toList()); } @Override @@ -827,8 +899,8 @@ public DynamicPartitionCtx createDPContext( Table table = IcebergTableUtil.getTable(conf, tableDesc.getProperties()); DynamicPartitionCtx dpCtx = new DynamicPartitionCtx(Maps.newLinkedHashMap(), - hiveConf.getVar(HiveConf.ConfVars.DEFAULT_PARTITION_NAME), - hiveConf.getIntVar(HiveConf.ConfVars.DYNAMIC_PARTITION_MAX_PARTS_PER_NODE)); + hiveConf.getVar(ConfVars.DEFAULT_PARTITION_NAME), + hiveConf.getIntVar(ConfVars.DYNAMIC_PARTITION_MAX_PARTS_PER_NODE)); List, ExprNodeDesc>> customSortExprs = Lists.newLinkedList(); dpCtx.setCustomSortExpressions(customSortExprs); @@ -1006,8 +1078,8 @@ public void executeOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable, hmsTable.getTableName()); AlterTableExecuteSpec.ExpireSnapshotsSpec expireSnapshotsSpec = (AlterTableExecuteSpec.ExpireSnapshotsSpec) executeSpec.getOperationParams(); - int numThreads = conf.getInt(HiveConf.ConfVars.HIVE_ICEBERG_EXPIRE_SNAPSHOT_NUMTHREADS.varname, - HiveConf.ConfVars.HIVE_ICEBERG_EXPIRE_SNAPSHOT_NUMTHREADS.defaultIntVal); + int numThreads = conf.getInt(ConfVars.HIVE_ICEBERG_EXPIRE_SNAPSHOT_NUMTHREADS.varname, + ConfVars.HIVE_ICEBERG_EXPIRE_SNAPSHOT_NUMTHREADS.defaultIntVal); expireSnapshot(icebergTable, expireSnapshotsSpec, numThreads); break; case SET_CURRENT_SNAPSHOT: @@ -1035,8 +1107,8 @@ public void executeOperation(org.apache.hadoop.hive.ql.metadata.Table hmsTable, deleteMetadataSpec.getSarg()); break; case DELETE_ORPHAN_FILES: - int numDeleteThreads = conf.getInt(HiveConf.ConfVars.HIVE_ICEBERG_EXPIRE_SNAPSHOT_NUMTHREADS.varname, - HiveConf.ConfVars.HIVE_ICEBERG_EXPIRE_SNAPSHOT_NUMTHREADS.defaultIntVal); + int numDeleteThreads = conf.getInt(ConfVars.HIVE_ICEBERG_EXPIRE_SNAPSHOT_NUMTHREADS.varname, + ConfVars.HIVE_ICEBERG_EXPIRE_SNAPSHOT_NUMTHREADS.defaultIntVal); AlterTableExecuteSpec.DeleteOrphanFilesDesc deleteOrphanFilesSpec = (AlterTableExecuteSpec.DeleteOrphanFilesDesc) executeSpec.getOperationParams(); deleteOrphanFiles(icebergTable, deleteOrphanFilesSpec.getTimestampMillis(), numDeleteThreads); @@ -1283,8 +1355,8 @@ String getPathForAuth(String locationProperty) { } String getPathForAuth(String locationProperty, String defaultTableLocation) { - boolean maskDefaultLocation = conf.getBoolean(HiveConf.ConfVars.HIVE_ICEBERG_MASK_DEFAULT_LOCATION.varname, - HiveConf.ConfVars.HIVE_ICEBERG_MASK_DEFAULT_LOCATION.defaultBoolVal); + boolean maskDefaultLocation = conf.getBoolean(ConfVars.HIVE_ICEBERG_MASK_DEFAULT_LOCATION.varname, + ConfVars.HIVE_ICEBERG_MASK_DEFAULT_LOCATION.defaultBoolVal); String location = URI.create(locationProperty).getPath(); if (!maskDefaultLocation || defaultTableLocation == null || !arePathsInSameFs(locationProperty, defaultTableLocation)) { @@ -1721,7 +1793,7 @@ private void fallbackToNonVectorizedModeBasedOnProperties(Properties tableProps) isValidMetadataTable(tableProps.getProperty(IcebergAcidUtil.META_TABLE_PROPERTY)) || hasOrcTimeInSchema(tableProps, tableSchema) || !hasParquetNestedTypeWithinListOrMap(tableProps, tableSchema)) { - conf.setBoolean(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED.varname, false); + conf.setBoolean(ConfVars.HIVE_VECTORIZATION_ENABLED.varname, false); } } @@ -1913,7 +1985,7 @@ public Map getNativeProperties(org.apache.hadoop.hive.ql.metadat } @Override - public boolean shouldOverwrite(org.apache.hadoop.hive.ql.metadata.Table mTable, Context.Operation operation) { + public boolean shouldOverwrite(org.apache.hadoop.hive.ql.metadata.Table mTable, Operation operation) { return IcebergTableUtil.isCopyOnWriteMode(operation, mTable.getParameters()::getOrDefault); } @@ -1969,61 +2041,12 @@ Boolean hasAppendsOnly(Iterable snapshots, SnapshotContext since) { return null; } - @Override - public List showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table hmstbl) - throws HiveException { - Configuration confs = context.getConf(); - JobConf job = HiveTableUtil.getPartJobConf(confs, hmstbl); - Class formatter = hmstbl.getInputFormatClass(); - - try { - InputFormat inputFormat = FetchOperator.getInputFormatFromCache(formatter, job); - InputSplit[] splits = inputFormat.getSplits(job, 1); - try (RecordReader reader = inputFormat.getRecordReader(splits[0], job, - Reporter.NULL)) { - return getPartitions(context, job, reader, hmstbl); - } - } catch (Exception e) { - throw new HiveException(e, ErrorMsg.GENERIC_ERROR, - "show partitions for table " + hmstbl.getTableName() + ". " + ErrorMsg.TABLE_NOT_PARTITIONED + - " or the table is empty "); - } - } - - private List getPartitions(DDLOperationContext context, Configuration job, - RecordReader reader, org.apache.hadoop.hive.ql.metadata.Table hmstbl) - throws Exception { - - List parts = Lists.newArrayList(); - Writable value = reader.createValue(); - WritableComparable key = reader.createKey(); - - try (FetchFormatter fetcher = new DefaultFetchFormatter()) { - fetcher.initialize(job, HiveTableUtil.getSerializationProps()); - org.apache.hadoop.hive.ql.metadata.Table metaDataPartTable = - context.getDb().getTable(hmstbl.getDbName(), hmstbl.getTableName(), "partitions", true); - Deserializer currSerDe = HiveMetaStoreUtils.getDeserializer(job, metaDataPartTable.getTTable(), - metaDataPartTable.getMetaTable(), false); - ObjectMapper mapper = new ObjectMapper(); - Table tbl = getTable(hmstbl); - while (reader.next(key, value)) { - String[] row = - fetcher.convert(currSerDe.deserialize(value), currSerDe.getObjectInspector()) - .toString().split("\t"); - parts.add(HiveTableUtil.getParseData(row[IcebergTableUtil.PART_IDX], row[IcebergTableUtil.SPEC_IDX], - mapper, tbl.spec().specId())); - } - } - Collections.sort(parts); - return parts; - } - @Override public void validatePartSpec(org.apache.hadoop.hive.ql.metadata.Table hmsTable, Map partitionSpec, - Context.RewritePolicy policy) throws SemanticException { + RewritePolicy policy) throws SemanticException { Table table = IcebergTableUtil.getTable(conf, hmsTable.getTTable()); List partitionFields = IcebergTableUtil.getPartitionFields(table, - policy != Context.RewritePolicy.PARTITION); + policy != RewritePolicy.PARTITION); validatePartSpecImpl(hmsTable, partitionSpec, partitionFields); } @@ -2109,14 +2132,14 @@ private boolean hasUndergonePartitionEvolution(Table table) { } @Override - public List getPartitions(org.apache.hadoop.hive.ql.metadata.Table table, + public List getPartitions(org.apache.hadoop.hive.ql.metadata.Table hmsTable, Map partitionSpec, boolean latestSpecOnly) throws SemanticException { - Table icebergTable = IcebergTableUtil.getTable(conf, table.getTTable()); - return IcebergTableUtil.getPartitionNames(icebergTable, partitionSpec, latestSpecOnly).stream() + Table table = IcebergTableUtil.getTable(conf, hmsTable.getTTable()); + return IcebergTableUtil.getPartitionNames(table, partitionSpec, latestSpecOnly).stream() .map(partName -> { Map partSpecMap = Maps.newLinkedHashMap(); Warehouse.makeSpecFromName(partSpecMap, new Path(partName), null); - return new DummyPartition(table, partName, partSpecMap); + return new DummyPartition(hmsTable, partName, partSpecMap); }).collect(Collectors.toList()); } @@ -2132,7 +2155,7 @@ public boolean isPartitioned(org.apache.hadoop.hive.ql.metadata.Table hmsTable) @Override public Partition getPartition(org.apache.hadoop.hive.ql.metadata.Table table, - Map partitionSpec, Context.RewritePolicy policy) throws SemanticException { + Map partitionSpec, RewritePolicy policy) throws SemanticException { validatePartSpec(table, partitionSpec, policy); return getPartitionImpl(table, partitionSpec); } @@ -2157,7 +2180,7 @@ private Partition getPartitionImpl(org.apache.hadoop.hive.ql.metadata.Table tabl public List getPartitionNames(org.apache.hadoop.hive.ql.metadata.Table hmsTable, Map partitionSpec) throws SemanticException { Table icebergTable = IcebergTableUtil.getTable(conf, hmsTable.getTTable()); - return IcebergTableUtil.getPartitionNames(icebergTable, partitionSpec, true); + return IcebergTableUtil.getPartitionNames(icebergTable, partitionSpec, false); } /** @@ -2216,7 +2239,8 @@ public boolean canPerformMetadataDelete(org.apache.hadoop.hive.ql.metadata.Table return true; } - TableScan scan = table.newScan().filter(exp).caseSensitive(false).includeColumnStats().ignoreResiduals(); + TableScan scan = table.newScan().filter(exp) + .caseSensitive(false).includeColumnStats().ignoreResiduals(); if (branchName != null) { scan.useRef(HiveUtils.getTableSnapshotRef(branchName)); } @@ -2248,15 +2272,9 @@ public List getPartitionKeys(org.apache.hadoop.hive.ql.metadata.Tab return IcebergTableUtil.getPartitionKeys(icebergTable, icebergTable.spec().specId()); } - @Override - public List getPartitionsByExpr(org.apache.hadoop.hive.ql.metadata.Table hmsTable, ExprNodeDesc filter) - throws SemanticException { - return getPartitionsByExpr(hmsTable, filter, true); - } - @Override public List getPartitionsByExpr(org.apache.hadoop.hive.ql.metadata.Table hmsTable, ExprNodeDesc filter, - boolean latestSpecOnly) throws SemanticException { + Boolean latestSpecOnly) throws SemanticException { Expression exp = HiveIcebergInputFormat.getFilterExpr(conf, (ExprNodeGenericFuncDesc) filter); if (exp == null) { return ImmutableList.of(); @@ -2265,17 +2283,23 @@ public List getPartitionsByExpr(org.apache.hadoop.hive.ql.metadata.Ta int tableSpecId = table.spec().specId(); Set partitions = Sets.newHashSet(); - TableScan scan = table.newScan().filter(exp).caseSensitive(false).includeColumnStats().ignoreResiduals(); + TableScan scan = table.newScan().filter(exp) + .caseSensitive(false).includeColumnStats().ignoreResiduals(); try (CloseableIterable tasks = scan.planFiles()) { - tasks.forEach(task -> { + FluentIterable.from(tasks).filter(task -> task.spec().isPartitioned()).forEach(task -> { DataFile file = task.file(); PartitionSpec spec = task.spec(); - if (latestSpecOnly && file.specId() == tableSpecId || !latestSpecOnly && file.specId() != tableSpecId) { + + if (latestSpecOnly == null || latestSpecOnly && file.specId() == tableSpecId || + !latestSpecOnly && file.specId() != tableSpecId) { + PartitionData partitionData = IcebergTableUtil.toPartitionData(task.partition(), spec.partitionType()); String partName = spec.partitionToPath(partitionData); + Map partSpecMap = Maps.newLinkedHashMap(); Warehouse.makeSpecFromName(partSpecMap, new Path(partName), null); + DummyPartition partition = new DummyPartition(hmsTable, partName, partSpecMap); partitions.add(partition); } @@ -2291,7 +2315,8 @@ public boolean hasDataMatchingFilterExpr(org.apache.hadoop.hive.ql.metadata.Tabl SearchArgument sarg = ConvertAstToSearchArg.create(conf, (ExprNodeGenericFuncDesc) filter); Expression exp = HiveIcebergFilterFactory.generateFilterExpression(sarg); Table table = IcebergTableUtil.getTable(conf, hmsTable.getTTable()); - TableScan scan = table.newScan().filter(exp).caseSensitive(false).includeColumnStats().ignoreResiduals(); + TableScan scan = table.newScan().filter(exp) + .caseSensitive(false).includeColumnStats().ignoreResiduals(); boolean result = false; try (CloseableIterable tasks = scan.planFiles()) { diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java index 439639167f8c..24f070eb8d57 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java @@ -19,8 +19,6 @@ package org.apache.iceberg.mr.hive; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; @@ -34,14 +32,12 @@ import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; -import org.apache.hadoop.hive.conf.Constants; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.Warehouse; import org.apache.hadoop.hive.metastore.api.FieldSchema; @@ -50,12 +46,8 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy; import org.apache.hadoop.hive.metastore.utils.FileUtils; -import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.io.IOConstants; import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.serde.serdeConstants; -import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; -import org.apache.hadoop.mapred.JobConf; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFiles; @@ -281,32 +273,4 @@ public static boolean isCtas(Properties properties) { return Boolean.parseBoolean(properties.getProperty(hive_metastoreConstants.TABLE_IS_CTAS)); } - static Properties getSerializationProps() { - Properties props = new Properties(); - props.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.tabCode); - props.put(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL"); - return props; - } - - static String getParseData(String parseData, String specId, ObjectMapper mapper, Integer currentSpecId) - throws JsonProcessingException { - Map map = mapper.readValue(parseData, Map.class); - String partString = - map.entrySet().stream() - .filter(entry -> entry.getValue() != null) - .map(java.lang.Object::toString) - .collect(Collectors.joining("/")); - String currentSpecMarker = currentSpecId.toString().equals(specId) ? "current-" : ""; - return String.format("%sspec-id=%s/%s", currentSpecMarker, specId, partString); - } - - static JobConf getPartJobConf(Configuration confs, org.apache.hadoop.hive.ql.metadata.Table tbl) { - JobConf job = new JobConf(confs); - job.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, Constants.ICEBERG_PARTITION_COLUMNS); - job.set(InputFormatConfig.TABLE_LOCATION, tbl.getPath().toString()); - job.set(InputFormatConfig.TABLE_IDENTIFIER, tbl.getFullyQualifiedName() + ".partitions"); - HiveConf.setVar(job, HiveConf.ConfVars.HIVE_FETCH_OUTPUT_SERDE, Constants.DELIMITED_JSON_SERDE); - HiveConf.setBoolVar(job, HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, false); - return job; - } } 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 35c185a29e25..c2beca41eded 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 @@ -22,6 +22,7 @@ import java.io.IOException; import java.time.ZoneId; import java.util.Collections; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Optional; @@ -473,50 +474,56 @@ public static List getPartitionKeys(Table table, int specId) { public static List getPartitionFields(Table table, boolean latestSpecOnly) { return latestSpecOnly ? table.spec().fields() : table.specs().values().stream() - .flatMap(spec -> spec.fields().stream()).distinct() + .flatMap(spec -> spec.fields().stream() + .filter(f -> !f.transform().isVoid())) + .distinct() .collect(Collectors.toList()); } /** - * Returns a Map of PartitionData as the keys and partition spec ids as the values - * @param icebergTable Iceberg table + * Returns a list of partition names satisfying the provided partition spec. + * @param table Iceberg table * @param partSpecMap Partition Spec used as the criteria for filtering - * @param allowPartialSpec When true, must return partitions which match partSpecMap exactly, otherwise partially - * @param latestSpecOnly When true, returns partitions with the latest partition spec only, otherwise with any specs - * @return Map of PartitionData and partition spec found based on the specified constraints + * @param latestSpecOnly when True, returns partitions with the current spec only, else - any specs + * @return List of partition names */ - public static Map getPartitionInfo(Table icebergTable, Map partSpecMap, - boolean allowPartialSpec, boolean latestSpecOnly) throws SemanticException, IOException { + public static List getPartitionNames(Table table, Map partSpecMap, + boolean latestSpecOnly) throws SemanticException { Expression expression = IcebergTableUtil.generateExpressionFromPartitionSpec( - icebergTable, partSpecMap, latestSpecOnly); + table, partSpecMap, latestSpecOnly); PartitionsTable partitionsTable = (PartitionsTable) MetadataTableUtils.createMetadataTableInstance( - icebergTable, MetadataTableType.PARTITIONS); + table, MetadataTableType.PARTITIONS); - Map result = Maps.newLinkedHashMap(); try (CloseableIterable fileScanTasks = partitionsTable.newScan().planFiles()) { - fileScanTasks.forEach(task -> - CloseableIterable.filter( - CloseableIterable.transform(task.asDataTask().rows(), row -> { - StructProjection data = row.get(IcebergTableUtil.PART_IDX, StructProjection.class); - Integer specId = row.get(IcebergTableUtil.SPEC_IDX, Integer.class); - return Maps.immutableEntry(IcebergTableUtil.toPartitionData(data, - Partitioning.partitionType(icebergTable), icebergTable.specs().get(specId).partitionType()), - specId); - }), entry -> { - ResidualEvaluator resEval = ResidualEvaluator.of(icebergTable.specs().get(entry.getValue()), - expression, false); - return resEval.residualFor(entry.getKey()).isEquivalentTo(Expressions.alwaysTrue()) && - (entry.getKey().size() == partSpecMap.size() || allowPartialSpec) && - (entry.getValue() == icebergTable.spec().specId() || !latestSpecOnly); - }).forEach(entry -> result.put(entry.getKey(), entry.getValue()))); - } + return FluentIterable.from(fileScanTasks) + .transformAndConcat(task -> task.asDataTask().rows()) + .transform(row -> { + StructLike data = row.get(IcebergTableUtil.PART_IDX, StructProjection.class); + PartitionSpec spec = table.specs().get(row.get(IcebergTableUtil.SPEC_IDX, Integer.class)); + return Maps.immutableEntry( + IcebergTableUtil.toPartitionData( + data, Partitioning.partitionType(table), spec.partitionType()), + spec); + }).filter(e -> { + ResidualEvaluator resEval = ResidualEvaluator.of(e.getValue(), + expression, false); + return e.getValue().isPartitioned() && + resEval.residualFor(e.getKey()).isEquivalentTo(Expressions.alwaysTrue()) && + (e.getValue().specId() == table.spec().specId() || !latestSpecOnly); + + }).transform(e -> e.getValue().partitionToPath(e.getKey())).toSortedList( + Comparator.naturalOrder()); - return result; + } catch (IOException e) { + throw new SemanticException( + String.format("Error while fetching the partitions due to: %s", e)); + } } public static long getPartitionHash(Table icebergTable, String partitionPath) throws IOException { PartitionsTable partitionsTable = (PartitionsTable) MetadataTableUtils .createMetadataTableInstance(icebergTable, MetadataTableType.PARTITIONS); + try (CloseableIterable fileScanTasks = partitionsTable.newScan().planFiles()) { return FluentIterable.from(fileScanTasks) .transformAndConcat(task -> task.asDataTask().rows()) @@ -534,19 +541,10 @@ public static long getPartitionHash(Table icebergTable, String partitionPath) th } } - public static List getPartitionNames(Table icebergTable, Map partitionSpec, - boolean latestSpecOnly) throws SemanticException { - try { - return IcebergTableUtil - .getPartitionInfo(icebergTable, partitionSpec, true, latestSpecOnly).entrySet().stream() - .map(e -> { - PartitionData partitionData = e.getKey(); - int specId = e.getValue(); - return icebergTable.specs().get(specId).partitionToPath(partitionData); - }).collect(Collectors.toList()); - } catch (IOException e) { - throw new SemanticException(String.format("Error while fetching the partitions due to: %s", e)); - } + public static TransformSpec getTransformSpec(Table table, String transformName, int sourceId) { + TransformSpec spec = TransformSpec.fromString(transformName.toUpperCase(), + table.schema().findColumnName(sourceId)); + return spec; } } diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java index e7d1cf63bf96..98ed9cf2b7d5 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java @@ -126,6 +126,10 @@ static TestTables testTables(TestHiveShell shell, TestTables.TestTableType testT return testTableType.instance(shell.metastore().hiveConf(), temp, catalogName); } + static void init(TestHiveShell shell, TestTables testTables, TemporaryFolder temp) { + init(shell, testTables, temp, "tez"); + } + static void init(TestHiveShell shell, TestTables testTables, TemporaryFolder temp, String engine) { shell.getSession(); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerWithEngineBase.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerWithEngineBase.java index 846681062402..99f159d73853 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerWithEngineBase.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerWithEngineBase.java @@ -33,8 +33,8 @@ import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.data.Record; -import org.apache.iceberg.hive.HiveVersion; import org.apache.iceberg.mr.TestHelper; +import org.apache.iceberg.mr.hive.TestTables.TestTableType; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -53,6 +53,8 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import static org.apache.iceberg.mr.hive.TestTables.ALL_TABLE_TYPES; +import static org.apache.iceberg.mr.hive.TestTables.TestTableType.HIVE_CATALOG; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; import static org.junit.Assume.assumeTrue; @@ -62,8 +64,6 @@ @RunWith(Parameterized.class) public abstract class HiveIcebergStorageHandlerWithEngineBase { - protected static final String[] EXECUTION_ENGINES = new String[] {"tez"}; - public static final String RETRY_STRATEGIES = "overlay,reoptimize,reexecute_lost_am,dagsubmit,recompile_without_cbo,write_conflict"; @@ -108,34 +108,25 @@ public abstract class HiveIcebergStorageHandlerWithEngineBase { StatsSetupConst.TOTAL_SIZE, SnapshotSummary.TOTAL_FILE_SIZE_PROP ); - @Parameters(name = "fileFormat={0}, engine={1}, catalog={2}, isVectorized={3}, formatVersion={4}") + @Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}") public static Collection parameters() { Collection testParams = Lists.newArrayList(); - String javaVersion = System.getProperty("java.specification.version"); // Run tests with every FileFormat for a single Catalog (HiveCatalog) for (FileFormat fileFormat : HiveIcebergStorageHandlerTestUtils.FILE_FORMATS) { - for (String engine : EXECUTION_ENGINES) { - IntStream.of(2, 1).forEach(formatVersion -> { - // include Tez tests only for Java 8 - if (javaVersion.equals("1.8")) { - testParams.add(new Object[]{fileFormat, engine, TestTables.TestTableType.HIVE_CATALOG, false, - formatVersion}); - // test for vectorization=ON in case of ORC and PARQUET format with Tez engine - if (fileFormat != FileFormat.METADATA && "tez".equals(engine) && HiveVersion.min(HiveVersion.HIVE_3)) { - testParams.add(new Object[]{fileFormat, engine, TestTables.TestTableType.HIVE_CATALOG, true, - formatVersion}); - } - } - }); - } + IntStream.of(2, 1).forEach(formatVersion -> { + testParams.add(new Object[]{fileFormat, HIVE_CATALOG, false, formatVersion}); + // test for vectorization=ON in case of ORC and PARQUET format + if (fileFormat != FileFormat.METADATA) { + testParams.add(new Object[]{fileFormat, HIVE_CATALOG, true, formatVersion}); + } + }); } - // Run tests for every Catalog for a single FileFormat (PARQUET) and execution engine (tez) - // skip HiveCatalog tests as they are added before - for (TestTables.TestTableType testTableType : TestTables.ALL_TABLE_TYPES) { - if (!TestTables.TestTableType.HIVE_CATALOG.equals(testTableType)) { - testParams.add(new Object[]{FileFormat.PARQUET, "tez", testTableType, false, 1}); + // Run tests for every Catalog for a single FileFormat (PARQUET), skip HiveCatalog tests as they are added before + for (TestTableType testTableType : ALL_TABLE_TYPES) { + if (testTableType != HIVE_CATALOG) { + testParams.add(new Object[]{FileFormat.PARQUET, testTableType, false, 1}); } } @@ -150,15 +141,12 @@ public static Collection parameters() { public FileFormat fileFormat; @Parameter(1) - public String executionEngine; + public TestTableType testTableType; @Parameter(2) - public TestTables.TestTableType testTableType; - - @Parameter(3) public boolean isVectorized; - @Parameter(4) + @Parameter(3) public Integer formatVersion; @Rule @@ -181,7 +169,7 @@ public static void afterClass() throws Exception { public void before() throws IOException { validateTestParams(); testTables = HiveIcebergStorageHandlerTestUtils.testTables(shell, testTableType, temp); - HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp, executionEngine); + HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); // Fetch task conversion might kick in for certain queries preventing vectorization code path to be used, so // we turn it off explicitly to achieve better coverage. diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestConflictingDataFiles.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestConflictingDataFiles.java index a07d2d5203f0..c98f0cb9b343 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestConflictingDataFiles.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestConflictingDataFiles.java @@ -98,7 +98,7 @@ public void testSingleFilterUpdate() { try { Tasks.range(2).executeWith(Executors.newFixedThreadPool(2)).run(i -> { TestUtilPhaser.getInstance().getPhaser().register(); - init(shell, testTables, temp, executionEngine); + init(shell, testTables, temp); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, @@ -141,7 +141,7 @@ public void testMultiFiltersUpdate() { try { Tasks.range(2).executeWith(Executors.newFixedThreadPool(2)).run(i -> { TestUtilPhaser.getInstance().getPhaser().register(); - init(shell, testTables, temp, executionEngine); + init(shell, testTables, temp); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, @@ -188,7 +188,7 @@ public void testDeleteFilters() { try { Tasks.range(3).executeWith(Executors.newFixedThreadPool(3)).run(i -> { TestUtilPhaser.getInstance().getPhaser().register(); - init(shell, testTables, temp, executionEngine); + init(shell, testTables, temp); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, @@ -230,7 +230,7 @@ public void testConflictingUpdates() { try { Tasks.range(2).executeWith(Executors.newFixedThreadPool(2)).run(i -> { TestUtilPhaser.getInstance().getPhaser().register(); - init(shell, testTables, temp, executionEngine); + init(shell, testTables, temp); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, @@ -299,7 +299,7 @@ public void testConcurrentInsertAndInsertOverwrite() { throw new RuntimeException(e); } } - init(shell, testTables, temp, executionEngine); + init(shell, testTables, temp); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); 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 c18a035d3933..e51015164612 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 @@ -616,7 +616,7 @@ public void testConcurrent2Deletes() { Tasks.range(2) .executeWith(Executors.newFixedThreadPool(2)) .run(i -> { - init(shell, testTables, temp, executionEngine); + init(shell, testTables, temp); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, @@ -647,7 +647,7 @@ public void testConcurrent2Updates() { Tasks.range(2) .executeWith(Executors.newFixedThreadPool(2)) .run(i -> { - init(shell, testTables, temp, executionEngine); + init(shell, testTables, temp); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, @@ -682,7 +682,7 @@ public void testConcurrentUpdateAndDelete() { Tasks.range(2) .executeWith(Executors.newFixedThreadPool(2)) .run(i -> { - init(shell, testTables, temp, executionEngine); + init(shell, testTables, temp); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, @@ -717,7 +717,7 @@ public void testConcurrent2MergeInserts() { Tasks.range(2) .executeWith(Executors.newFixedThreadPool(2)) .run(i -> { - init(shell, testTables, temp, executionEngine); + init(shell, testTables, temp); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSnapshotOperations.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSnapshotOperations.java index 7f92fcea4932..55badc7eb5c6 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSnapshotOperations.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSnapshotOperations.java @@ -43,7 +43,7 @@ public void before() throws IOException { shell = HiveIcebergStorageHandlerTestUtils.shell(); temp.create(); testTables = HiveIcebergStorageHandlerTestUtils.testTables(shell, HIVE_CATALOG, temp); - HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp, "tez"); + HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp); } @After diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStatistics.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStatistics.java index 187b9b462ff4..aaf7138056d1 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStatistics.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStatistics.java @@ -20,7 +20,9 @@ package org.apache.iceberg.mr.hive; import java.io.IOException; +import java.util.Collection; import java.util.List; +import org.apache.commons.lang3.ArrayUtils; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.StatsSetupConst; import org.apache.hadoop.hive.conf.HiveConf; @@ -36,7 +38,9 @@ import org.apache.thrift.TException; import org.junit.Assert; import org.junit.Assume; +import org.junit.Before; import org.junit.Test; +import org.junit.runners.Parameterized; /** @@ -45,8 +49,32 @@ */ public class TestHiveIcebergStatistics extends HiveIcebergStorageHandlerWithEngineBase { + @Parameterized.Parameter(4) + public String statsSource; + + @Parameterized.Parameters(name = "fileFormat={0}, catalog={1}, isVectorized={2}, formatVersion={3}, statsSource={4}") + public static Collection parameters() { + Collection baseParams = HiveIcebergStorageHandlerWithEngineBase.parameters(); + + Collection testParams = Lists.newArrayList(); + for (String statsSource : new String[]{"iceberg", "metastore"}) { + for (Object[] params : baseParams) { + testParams.add(ArrayUtils.add(params, statsSource)); + } + } + return testParams; + } + + @Before + public void setStatsSource() { + HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_ICEBERG_STATS_SOURCE, statsSource); + } + @Test public void testAnalyzeTableComputeStatistics() throws IOException, TException, InterruptedException { + Assume.assumeTrue(statsSource.equals("iceberg") || + testTableType == TestTables.TestTableType.HIVE_CATALOG); + String dbName = "default"; String tableName = "customers"; Table table = testTables @@ -174,6 +202,11 @@ public void testStatsWithPartitionedCTAS() { testTables.propertiesForCreateTableSQL( ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, fileFormat.toString())))); + if (statsSource.equals("iceberg")) { + // TODO: Propagate partition spec from CREATE statement to the ColumnStatsSemanticAnalyzer + shell.executeStatement("ANALYZE TABLE target COMPUTE STATISTICS FOR COLUMNS"); + } + checkColStat("target", "id", true); checkColStat("target", "dept", true); checkColStatMinMaxValue("target", "id", 0, 2); @@ -264,6 +297,8 @@ public void testMergeStatsWithInsert() { @Test public void testIcebergColStatsPath() throws IOException { + Assume.assumeTrue(statsSource.equals("iceberg")); + TableIdentifier identifier = TableIdentifier.of("default", "customers"); shell.setHiveSessionValue(HiveConf.ConfVars.HIVE_STATS_AUTOGATHER.varname, true); @@ -332,8 +367,6 @@ private void checkColStatMaxLengthDistinctValue(String tableName, String colName private void checkColStatMinMaxDistinctValue(String tableName, String colName, int minValue, int maxValue, int distinct, int nulls) { - - shell.executeStatement("set hive.iceberg.stats.source=metastore"); List rows = shell.executeStatement("DESCRIBE FORMATTED " + tableName + " " + colName); // Check min @@ -351,25 +384,5 @@ private void checkColStatMinMaxDistinctValue(String tableName, String colName, i // Check distinct Assert.assertEquals("distinct_count", rows.get(5)[0]); Assert.assertEquals(String.valueOf(distinct), rows.get(5)[1]); - - shell.executeStatement("set hive.iceberg.stats.source=iceberg"); - rows = shell.executeStatement("DESCRIBE FORMATTED " + tableName + " " + colName); - - // Check min - Assert.assertEquals("min", rows.get(2)[0]); - Assert.assertEquals(String.valueOf(minValue), rows.get(2)[1]); - - // Check max - Assert.assertEquals("max", rows.get(3)[0]); - Assert.assertEquals(String.valueOf(maxValue), rows.get(3)[1]); - - // Check num of nulls - Assert.assertEquals("num_nulls", rows.get(4)[0]); - Assert.assertEquals(String.valueOf(nulls), rows.get(4)[1]); - - // Check distinct - Assert.assertEquals("distinct_count", rows.get(5)[0]); - Assert.assertEquals(String.valueOf(distinct), rows.get(5)[1]); - } } diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java index a8072f45afe6..1a3856c7df80 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java @@ -1306,9 +1306,8 @@ public void testAlterTableRenamePartitionColumn() throws Exception { shell.executeStatement("ALTER TABLE default.customers SET PARTITION SPEC (region, city)"); List result = shell.executeStatement("DESCRIBE default.customers"); - Assert.assertArrayEquals(new String[] {"family_name", "VOID", null}, result.get(8)); - Assert.assertArrayEquals(new String[] {"region", "IDENTITY", null}, result.get(9)); - Assert.assertArrayEquals(new String[] {"city", "IDENTITY", null}, result.get(10)); + Assert.assertArrayEquals(new String[] {"region", "IDENTITY", null}, result.get(8)); + Assert.assertArrayEquals(new String[] {"city", "IDENTITY", null}, result.get(9)); } @Test diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java index ca1f326abed5..8d44dbb32fa2 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java @@ -110,7 +110,7 @@ public void before() throws IOException { localTimeZone.ifPresent(ThreadLocal::remove); this.testTables = HiveIcebergStorageHandlerTestUtils.testTables(shell, TestTables.TestTableType.HIVE_CATALOG, temp); - HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp, "tez"); + HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp); } @After diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithMultipleCatalogs.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithMultipleCatalogs.java index 08f60b04c8e8..b40425a8ea8e 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithMultipleCatalogs.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithMultipleCatalogs.java @@ -30,6 +30,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.mr.InputFormatConfig; +import org.apache.iceberg.mr.hive.TestTables.TestTableType; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.After; import org.junit.AfterClass; @@ -42,10 +43,11 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import static org.apache.iceberg.mr.hive.TestTables.TestTableType.HIVE_CATALOG; + @RunWith(Parameterized.class) public class TestHiveIcebergStorageHandlerWithMultipleCatalogs { - private static final String[] EXECUTION_ENGINES = new String[] { "tez" }; private static final String HIVECATALOGNAME = "table1_catalog"; private static final String OTHERCATALOGNAME = "table2_catalog"; private static TestHiveShell shell; @@ -55,14 +57,12 @@ public class TestHiveIcebergStorageHandlerWithMultipleCatalogs { @Parameterized.Parameter(1) public FileFormat fileFormat2; @Parameterized.Parameter(2) - public String executionEngine; - @Parameterized.Parameter(3) public TestTables.TestTableType testTableType1; - @Parameterized.Parameter(4) + @Parameterized.Parameter(3) public String table1CatalogName; - @Parameterized.Parameter(5) + @Parameterized.Parameter(4) public TestTables.TestTableType testTableType2; - @Parameterized.Parameter(6) + @Parameterized.Parameter(5) public String table2CatalogName; @Rule @@ -70,22 +70,16 @@ public class TestHiveIcebergStorageHandlerWithMultipleCatalogs { private TestTables testTables1; private TestTables testTables2; - @Parameterized.Parameters(name = "fileFormat1={0}, fileFormat2={1}, engine={2}, tableType1={3}, catalogName1={4}, " + - "tableType2={5}, catalogName2={6}") + @Parameterized.Parameters(name = "fileFormat1={0}, fileFormat2={1}, tableType1={2}, catalogName1={3}, " + + "tableType2={4}, catalogName2={5}") public static Collection parameters() { Collection testParams = Lists.newArrayList(); - String javaVersion = System.getProperty("java.specification.version"); // Run tests with PARQUET and ORC file formats for a two Catalogs - for (String engine : EXECUTION_ENGINES) { - // include Tez tests only for Java 8 - if (javaVersion.equals("1.8")) { - for (TestTables.TestTableType testTableType : TestTables.ALL_TABLE_TYPES) { - if (!TestTables.TestTableType.HIVE_CATALOG.equals(testTableType)) { - testParams.add(new Object[]{FileFormat.PARQUET, FileFormat.ORC, engine, - TestTables.TestTableType.HIVE_CATALOG, HIVECATALOGNAME, testTableType, OTHERCATALOGNAME}); - } - } + for (TestTableType testTableType : TestTables.ALL_TABLE_TYPES) { + if (!HIVE_CATALOG.equals(testTableType)) { + testParams.add(new Object[]{FileFormat.PARQUET, FileFormat.ORC, + HIVE_CATALOG, HIVECATALOGNAME, testTableType, OTHERCATALOGNAME}); } } return testParams; @@ -104,7 +98,7 @@ public static void afterClass() throws Exception { @Before public void before() throws IOException { testTables1 = HiveIcebergStorageHandlerTestUtils.testTables(shell, testTableType1, temp, table1CatalogName); - HiveIcebergStorageHandlerTestUtils.init(shell, testTables1, temp, executionEngine); + HiveIcebergStorageHandlerTestUtils.init(shell, testTables1, temp); testTables1.properties().entrySet().forEach(e -> shell.setHiveSessionValue(e.getKey(), e.getValue())); testTables2 = HiveIcebergStorageHandlerTestUtils.testTables(shell, testTableType2, temp, table2CatalogName); diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestOptimisticRetry.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestOptimisticRetry.java index 3c00e28695c8..6094db84608e 100644 --- a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestOptimisticRetry.java +++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestOptimisticRetry.java @@ -53,7 +53,7 @@ public void testConcurrentOverlappingUpdates() { Tasks.range(2) .executeWith(Executors.newFixedThreadPool(2)) .run(i -> { - init(shell, testTables, temp, executionEngine); + init(shell, testTables, temp); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, @@ -87,7 +87,7 @@ public void testConcurrentOverwriteAndUpdate() { // The query shouldn't throw exception but rather retry & commit. Tasks.range(2).executeWith(Executors.newFixedThreadPool(2)).run(i -> { TestUtilPhaser.getInstance().getPhaser().register(); - init(shell, testTables, temp, executionEngine); + init(shell, testTables, temp); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, RETRY_STRATEGIES); @@ -110,7 +110,7 @@ public void testNonOverlappingConcurrent2Updates() { Tasks.range(2) .executeWith(Executors.newFixedThreadPool(2)) .run(i -> { - init(shell, testTables, temp, executionEngine); + init(shell, testTables, temp); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, @@ -147,7 +147,7 @@ public void testConcurrent2MergeInserts() { Tasks.range(2) .executeWith(Executors.newFixedThreadPool(2)) .run(i -> { - init(shell, testTables, temp, executionEngine); + init(shell, testTables, temp); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, @@ -187,7 +187,7 @@ public void testConcurrent2MergeUpdates() { Tasks.range(2) .executeWith(Executors.newFixedThreadPool(2)) .run(i -> { - init(shell, testTables, temp, executionEngine); + init(shell, testTables, temp); HiveConf.setBoolVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorized); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_FETCH_TASK_CONVERSION, "none"); HiveConf.setVar(shell.getHiveConf(), HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, diff --git a/iceberg/iceberg-handler/src/test/queries/negative/show_partitions_negative_test.q b/iceberg/iceberg-handler/src/test/queries/negative/show_partitions_negative_test.q deleted file mode 100644 index 9a8a2456c010..000000000000 --- a/iceberg/iceberg-handler/src/test/queries/negative/show_partitions_negative_test.q +++ /dev/null @@ -1,10 +0,0 @@ --- Mask the totalSize value as it can have slight variability, causing test flakiness --- Mask random uuid ---! qt:replace:/(\s+uuid\s+)\S+(\s*)/$1#Masked#$2/ - ---Null case -> if partitions spec is altered. Null partitions need to be ignored. -create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2') ; -select * from default.ice2.partitions order by `partition`; -show partitions ice2; - - diff --git a/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_mixed.q b/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_mixed.q index ee1a5b196a94..02bf22a5238f 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_mixed.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/delete_iceberg_mixed.q @@ -9,8 +9,8 @@ -- create an unpartitioned table with skip delete data set to false - create table ice01 (id int, name string) Stored by Iceberg stored as ORC - TBLPROPERTIES('format-version'='2'); +create table ice01 (id int, name string) Stored by Iceberg stored as ORC +TBLPROPERTIES('format-version'='2'); -- insert some values insert into ice01 values (1, 'ABC'),(2, 'CBS'),(3, null),(4, 'POPI'),(5, 'AQWR'),(6, 'POIU'),(9, null),(8, diff --git a/iceberg/iceberg-handler/src/test/queries/positive/show_partitions_test.q b/iceberg/iceberg-handler/src/test/queries/positive/show_partitions_test.q index ea1fdc84cc4b..e401e89304d9 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/show_partitions_test.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/show_partitions_test.q @@ -6,16 +6,22 @@ set hive.vectorized.execution.enabled=false; -- Create a hive and iceberg table to compare. create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored as orc ; -insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5); create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2') ; -insert into ice1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5); ---compare hive table with iceberg table +-- Compare empty hive and iceberg tables show partitions hiveT1; describe default.ice1.partitions; -select `partition` from default.ice1.partitions order by `partition`; -show partitions ice1 ; +select * from default.ice1.partitions; +show partitions ice1; +-- Insert data +insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5); +insert into ice1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5); + +-- Compare hive and iceberg tables +show partitions hiveT1; +select `partition` from default.ice1.partitions; +show partitions ice1; explain show partitions hiveT1; explain show partitions ice1; @@ -26,16 +32,14 @@ create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part in TBLPROPERTIES("format-version"='2') ; insert into ice2 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2,10, 5); -select `partition` from default.ice2.partitions order by `partition`; +select `partition` from default.ice2.partitions; show partitions ice2; ALTER TABLE ice2 SET PARTITION SPEC (c) ; -select `partition` from default.ice2.partitions order by `partition`; +select `partition` from default.ice2.partitions; show partitions ice2; insert into ice2 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5); -select `partition` from default.ice2.partitions order by `partition`; -show partitions ice2; - - +select `partition` from default.ice2.partitions; +show partitions ice2; \ No newline at end of file diff --git a/iceberg/iceberg-handler/src/test/results/negative/show_partitions_negative_test.q.out b/iceberg/iceberg-handler/src/test/results/negative/show_partitions_negative_test.q.out deleted file mode 100644 index 3044e35ad4fc..000000000000 --- a/iceberg/iceberg-handler/src/test/results/negative/show_partitions_negative_test.q.out +++ /dev/null @@ -1,20 +0,0 @@ -PREHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2') -PREHOOK: type: CREATETABLE -PREHOOK: Output: database:default -PREHOOK: Output: default@ice2 -POSTHOOK: query: create table ice2 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2') -POSTHOOK: type: CREATETABLE -POSTHOOK: Output: database:default -POSTHOOK: Output: default@ice2 -PREHOOK: query: select * from default.ice2.partitions order by `partition` -PREHOOK: type: QUERY -PREHOOK: Input: default@ice2 -PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select * from default.ice2.partitions order by `partition` -POSTHOOK: type: QUERY -POSTHOOK: Input: default@ice2 -POSTHOOK: Output: hdfs://### HDFS PATH ### -PREHOOK: query: show partitions ice2 -PREHOOK: type: SHOWPARTITIONS -PREHOOK: Input: default@ice2 -FAILED: Execution Error, return code 40000 from org.apache.hadoop.hive.ql.ddl.DDLTask. Exception while processing show partitions for table ice2. TABLE_NOT_PARTITIONED or the table is empty diff --git a/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out index 15b3a5c67849..ae6f0ae42c4c 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/ctas_iceberg_partitioned_orc.q.out @@ -304,9 +304,11 @@ Table Parameters: iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### numFiles 2 + numPartitions 2 numRows 2 parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### + rawDataSize 0 serialization.format 1 snapshot-count 1 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out index 924a50bda374..f7b4d905bad9 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_partitioned.q.out @@ -121,77 +121,77 @@ STAGE PLANS: Map Operator Tree: TableScan alias: tbl_ice - Statistics: Num rows: 6 Data size: 582 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 588 Basic stats: COMPLETE Column stats: PARTIAL Filter Operator predicate: FILE__PATH is not null (type: boolean) - Statistics: Num rows: 6 Data size: 582 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 588 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: a (type: int), b (type: string), c (type: int), 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, _col5, _col6, _col7 - Statistics: Num rows: 6 Data size: 2910 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 2916 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 6 Data size: 2910 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 2916 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) Select Operator expressions: a (type: int), c (type: int), FILE__PATH (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 6 Data size: 1152 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1152 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 6 Data size: 1152 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1152 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) Select Operator expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), PARTITION__PROJECTION (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 6 Data size: 2862 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 2868 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 6 Data size: 2862 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 2868 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: string) Filter Operator predicate: (c > 800) (type: boolean) - Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: PARTIAL Select Operator - Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: PARTIAL Group By Operator aggregations: count() - minReductionHashAggr: 0.8333333 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: bigint) Group By Operator keys: c (type: int) - minReductionHashAggr: 0.6666666 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: PARTIAL Group By Operator aggregations: count(), count(c) - minReductionHashAggr: 0.8333333 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized Map 22 @@ -199,62 +199,62 @@ STAGE PLANS: TableScan alias: tbl_ice filterExpr: (a <= 5) (type: boolean) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (a <= 5) (type: boolean) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Select Operator expressions: a (type: int) outputColumnNames: a - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: a (type: int) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(), count(a) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Select Operator - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized Reducer 10 @@ -266,11 +266,11 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col1, _col2, _col3, _col5 - Statistics: Num rows: 9 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1353 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 9 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1353 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean) Reducer 11 Reduce Operator Tree: @@ -281,13 +281,13 @@ STAGE PLANS: 0 1 outputColumnNames: _col1, _col2, _col3, _col5, _col6 - Statistics: Num rows: 9 Data size: 1852 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1407 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col1 (type: int) - Statistics: Num rows: 9 Data size: 1852 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1407 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean), _col6 (type: bigint) Reducer 12 Reduce Operator Tree: @@ -298,23 +298,23 @@ STAGE PLANS: 0 _col1 (type: int) 1 _col0 (type: int) outputColumnNames: _col2, _col3, _col5, _col6, _col8 - Statistics: Num rows: 9 Data size: 1852 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1547 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((_col3 <> 0L) and _col5 is not null) or ((_col6 <> 0L) and _col8 is not null)) (type: boolean) - Statistics: Num rows: 9 Data size: 1852 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1547 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 9 Data size: 1852 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1547 Basic stats: COMPLETE Column stats: NONE Reducer 13 Execution mode: vectorized Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: string) outputColumnNames: _col2 - Statistics: Num rows: 9 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1547 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -334,26 +334,26 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 9 Data size: 1656 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1547 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 773 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 773 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: string) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 773 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 773 Basic stats: COMPLETE Column stats: NONE Reducer 14 Reduce Operator Tree: Merge Join Operator @@ -363,13 +363,13 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 6 Data size: 2910 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 2946 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 6 Data size: 2910 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 2946 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: string), _col7 (type: bigint) Reducer 15 Reduce Operator Tree: @@ -380,11 +380,11 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9 - Statistics: Num rows: 6 Data size: 2926 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3240 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 6 Data size: 2926 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3240 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: string), _col7 (type: bigint), _col9 (type: boolean) Reducer 16 Reduce Operator Tree: @@ -395,13 +395,13 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9, _col10 - Statistics: Num rows: 6 Data size: 2974 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3294 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col2 (type: int) - Statistics: Num rows: 6 Data size: 2974 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3294 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: string), _col7 (type: bigint), _col9 (type: boolean), _col10 (type: bigint) Reducer 17 Reduce Operator Tree: @@ -412,16 +412,16 @@ STAGE PLANS: 0 _col2 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9, _col10, _col12 - Statistics: Num rows: 6 Data size: 2998 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3623 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((_col7 <> 0L) and _col9 is not null) or ((_col10 <> 0L) and _col12 is not null)) (type: boolean) - Statistics: Num rows: 6 Data size: 2998 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3623 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 6 Data size: 2998 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3623 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: string) Reducer 18 Execution mode: vectorized @@ -429,7 +429,7 @@ STAGE PLANS: Select Operator expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col5 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 6 Data size: 2862 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3623 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -448,20 +448,20 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 6 Data size: 2862 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3623 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 3 Data size: 1431 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 1811 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col6 (type: string), _col0 (type: int), _col1 (type: string), _col2 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 3 Data size: 1455 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 1811 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 7 Data size: 3395 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 8 Data size: 5265 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 19 Execution mode: vectorized @@ -470,16 +470,16 @@ STAGE PLANS: aggregations: count(VALUE._col0) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: bigint) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: bigint) Reducer 2 Reduce Operator Tree: @@ -490,11 +490,11 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9 - Statistics: Num rows: 6 Data size: 2926 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3207 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 6 Data size: 2926 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3207 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col9 (type: boolean) Reducer 20 Execution mode: vectorized @@ -503,31 +503,31 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: _col0 (type: int), true (type: boolean) outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col1 (type: boolean) Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col1 (type: boolean) Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col1 (type: boolean) Reducer 21 Execution mode: vectorized @@ -536,11 +536,11 @@ STAGE PLANS: aggregations: count(VALUE._col0), count(VALUE._col1) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 23 Execution mode: vectorized @@ -549,11 +549,11 @@ STAGE PLANS: aggregations: count(VALUE._col0) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Reducer 24 Execution mode: vectorized @@ -562,31 +562,31 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), true (type: boolean) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean) Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean) Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean) Reducer 25 Execution mode: vectorized @@ -595,11 +595,11 @@ STAGE PLANS: aggregations: count(VALUE._col0), count(VALUE._col1) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 26 Execution mode: vectorized @@ -608,11 +608,11 @@ STAGE PLANS: aggregations: count(VALUE._col0) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Reducer 3 Reduce Operator Tree: @@ -623,17 +623,17 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9, _col10, _col11 - Statistics: Num rows: 6 Data size: 3022 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3333 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col10 (type: bigint), _col11 (type: bigint), _col9 (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col11 - Statistics: Num rows: 6 Data size: 3022 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3333 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col2 (type: int) - Statistics: Num rows: 6 Data size: 3022 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3333 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col8 (type: bigint), _col9 (type: bigint), _col11 (type: boolean) Reducer 4 Reduce Operator Tree: @@ -644,11 +644,11 @@ STAGE PLANS: 0 _col2 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col11, _col13 - Statistics: Num rows: 6 Data size: 3046 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3666 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 6 Data size: 3046 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3666 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col8 (type: bigint), _col9 (type: bigint), _col11 (type: boolean), _col13 (type: boolean) Reducer 5 Reduce Operator Tree: @@ -659,24 +659,24 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col11, _col13, _col14, _col15 - Statistics: Num rows: 6 Data size: 3142 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3768 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col8 (type: bigint), _col9 (type: bigint), _col11 (type: boolean), _col14 (type: bigint), _col15 (type: bigint), _col13 (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col11, _col12, _col13, _col15 - Statistics: Num rows: 6 Data size: 3142 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3768 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((_col11 is not null and (_col8 <> 0L)) or ((_col0 is null or (_col9 < _col8)) and null and (_col8 <> 0L) and _col11 is null) or (_col15 is not null and (_col12 <> 0L)) or ((_col2 is null or (_col13 < _col12)) and null and (_col12 <> 0L) and _col15 is null)) is null or (((_col8 = 0L) or (_col11 is null and (_col9 >= _col8) and _col0 is not null)) and ((_col12 = 0L) or (_col15 is null and (_col13 >= _col12) and _col2 is not null)))) (type: boolean) - Statistics: Num rows: 6 Data size: 3142 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 3140 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 6 Data size: 2910 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 3140 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 6 Data size: 2910 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 3140 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint), _col7 (type: string) Reducer 6 Reduce Operator Tree: @@ -687,17 +687,17 @@ STAGE PLANS: 0 _col5 (type: string) 1 _col0 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 4 Data size: 1940 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 3454 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col0 (type: int), _col1 (type: string), _col2 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 4 Data size: 1940 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 3454 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 7 Data size: 3395 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 8 Data size: 5265 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 8 Execution mode: vectorized @@ -708,7 +708,7 @@ STAGE PLANS: File Output Operator compressed: false Dp Sort State: PARTITION_SORTED - Statistics: Num rows: 7 Data size: 3395 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 8 Data size: 5265 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -723,13 +723,13 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 6 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1230 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 6 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1230 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint) Union 7 Vertex: Union 7 @@ -838,58 +838,58 @@ STAGE PLANS: Map Operator Tree: TableScan alias: tbl_ice - Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: FILE__PATH is not null (type: boolean) - Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), b (type: string), c (type: int), 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, _col5, _col6, _col7 - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), PARTITION__PROJECTION (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 2 Data size: 954 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 954 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: string) Select Operator expressions: a (type: int) outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (a is not null and FILE__PATH is not null) (type: boolean) - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), FILE__PATH (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Execution mode: vectorized Map 14 @@ -919,11 +919,11 @@ STAGE PLANS: aggregations: count(VALUE._col0), count(VALUE._col1) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 11 Reduce Operator Tree: @@ -934,20 +934,20 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col1 - Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reducer 12 Execution mode: vectorized Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: string) outputColumnNames: _col1 - Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -967,26 +967,26 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: string) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reducer 13 Execution mode: vectorized Reduce Operator Tree: @@ -994,17 +994,17 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), true (type: boolean) outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean) Reducer 2 Reduce Operator Tree: @@ -1015,11 +1015,11 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9 - Statistics: Num rows: 2 Data size: 978 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 422 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 2 Data size: 978 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 422 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col9 (type: boolean) Reducer 3 Reduce Operator Tree: @@ -1030,24 +1030,24 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9, _col10, _col11 - Statistics: Num rows: 2 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 456 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col10 (type: bigint), _col11 (type: bigint), _col9 (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col11 - Statistics: Num rows: 2 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 456 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((_col8 = 0L) or ((_col11 is not null and (_col8 <> 0L)) or ((_col0 is null or (_col9 < _col8)) and null and (_col8 <> 0L) and _col11 is null)) is null or (_col11 is null and (_col9 >= _col8) and _col0 is not null)) (type: boolean) - Statistics: Num rows: 2 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 456 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 456 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 456 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint), _col7 (type: string) Reducer 4 Reduce Operator Tree: @@ -1058,17 +1058,17 @@ STAGE PLANS: 0 _col5 (type: string) 1 _col0 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 501 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col0 (type: int), _col1 (type: string), _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 + Statistics: Num rows: 2 Data size: 501 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 712 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 6 Execution mode: vectorized @@ -1079,7 +1079,7 @@ STAGE PLANS: File Output Operator compressed: false Dp Sort State: PARTITION_SORTED - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 712 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1094,13 +1094,13 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 2 Data size: 954 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 422 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 2 Data size: 954 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 422 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: string) Reducer 8 Execution mode: vectorized @@ -1108,7 +1108,7 @@ STAGE PLANS: Select Operator expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col5 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 2 Data size: 954 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 422 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -1127,20 +1127,20 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 2 Data size: 954 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 422 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 211 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col6 (type: string), _col0 (type: int), _col1 (type: string), _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 + Statistics: Num rows: 1 Data size: 211 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 712 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 9 Reduce Operator Tree: @@ -1151,42 +1151,42 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(), count(_col0) - minReductionHashAggr: 0.5 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Group By Operator keys: _col0 (type: int) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Union 5 Vertex: Union 5 @@ -1290,58 +1290,58 @@ STAGE PLANS: Map Operator Tree: TableScan alias: tbl_ice - Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: FILE__PATH is not null (type: boolean) - Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), b (type: string), c (type: int), 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, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), PARTITION__PROJECTION (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: string) Select Operator expressions: a (type: int) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (a is not null and FILE__PATH is not null) (type: boolean) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), FILE__PATH (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Execution mode: vectorized Map 14 @@ -1371,11 +1371,11 @@ STAGE PLANS: aggregations: count(VALUE._col0), count(VALUE._col1) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 11 Execution mode: vectorized @@ -1384,17 +1384,17 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), true (type: boolean) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean) Reducer 12 Reduce Operator Tree: @@ -1405,20 +1405,20 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col1 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reducer 13 Execution mode: vectorized Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: string) outputColumnNames: _col1 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -1438,26 +1438,26 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: string) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Merge Join Operator @@ -1467,11 +1467,11 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9 - Statistics: Num rows: 1 Data size: 489 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 211 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 489 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 211 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col9 (type: boolean) Reducer 3 Reduce Operator Tree: @@ -1482,24 +1482,24 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9, _col10, _col11 - Statistics: Num rows: 1 Data size: 505 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 228 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col10 (type: bigint), _col11 (type: bigint), _col9 (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col11 - Statistics: Num rows: 1 Data size: 505 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 228 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((_col8 = 0L) or ((_col11 is not null and (_col8 <> 0L)) or ((_col0 is null or (_col9 < _col8)) and null and (_col8 <> 0L) and _col11 is null)) is null or (_col11 is null and (_col9 >= _col8) and _col0 is not null)) (type: boolean) - Statistics: Num rows: 1 Data size: 505 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 228 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 228 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 228 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint), _col7 (type: string) Reducer 4 Reduce Operator Tree: @@ -1510,17 +1510,17 @@ STAGE PLANS: 0 _col5 (type: string) 1 _col0 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 250 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col0 (type: int), _col1 (type: string), _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 + Statistics: Num rows: 1 Data size: 250 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 254 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 6 Execution mode: vectorized @@ -1531,7 +1531,7 @@ STAGE PLANS: File Output Operator compressed: false Dp Sort State: PARTITION_SORTED - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 254 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1546,13 +1546,13 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: string) Reducer 8 Execution mode: vectorized @@ -1560,7 +1560,7 @@ STAGE PLANS: Select Operator expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col5 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -1579,20 +1579,20 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col6 (type: string), _col0 (type: int), _col1 (type: string), _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 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 254 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 9 Reduce Operator Tree: @@ -1603,62 +1603,62 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col1 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(), count(_col1) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Group By Operator keys: _col1 (type: int) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: int) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: int) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Union 5 Vertex: Union 5 diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_unpartitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_unpartitioned.q.out index 85f1ec748bdf..fe2376b4f562 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_unpartitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_copy_on_write_unpartitioned.q.out @@ -988,58 +988,58 @@ STAGE PLANS: Map Operator Tree: TableScan alias: tbl_ice - Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: FILE__PATH is not null (type: boolean) - Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), b (type: string), c (type: int), 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, _col5, _col6, _col7 - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), PARTITION__PROJECTION (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 2 Data size: 954 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 954 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: string) Select Operator expressions: a (type: int) outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (a is not null and FILE__PATH is not null) (type: boolean) - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), FILE__PATH (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Execution mode: vectorized Map 13 @@ -1071,20 +1071,20 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col1 - Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reducer 11 Execution mode: vectorized Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: string) outputColumnNames: _col1 - Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -1104,26 +1104,26 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: string) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reducer 12 Execution mode: vectorized Reduce Operator Tree: @@ -1131,17 +1131,17 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), true (type: boolean) outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean) Reducer 2 Reduce Operator Tree: @@ -1152,11 +1152,11 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9 - Statistics: Num rows: 2 Data size: 978 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 422 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 2 Data size: 978 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 422 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col9 (type: boolean) Reducer 3 Reduce Operator Tree: @@ -1167,24 +1167,24 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9, _col10, _col11 - Statistics: Num rows: 2 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 456 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col10 (type: bigint), _col11 (type: bigint), _col9 (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col11 - Statistics: Num rows: 2 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 456 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((_col8 = 0L) or ((_col11 is not null and (_col8 <> 0L)) or ((_col0 is null or (_col9 < _col8)) and null and (_col8 <> 0L) and _col11 is null)) is null or (_col11 is null and (_col9 >= _col8) and _col0 is not null)) (type: boolean) - Statistics: Num rows: 2 Data size: 1010 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 456 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 456 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 456 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint), _col7 (type: string) Reducer 4 Reduce Operator Tree: @@ -1195,14 +1195,14 @@ STAGE PLANS: 0 _col5 (type: string) 1 _col0 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 501 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col0 (type: int), _col1 (type: string), _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 + Statistics: Num rows: 2 Data size: 501 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 712 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1217,13 +1217,13 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 2 Data size: 954 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 422 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 2 Data size: 954 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 422 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: string) Reducer 7 Execution mode: vectorized @@ -1231,7 +1231,7 @@ STAGE PLANS: Select Operator expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col5 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 2 Data size: 954 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 422 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -1250,17 +1250,17 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 2 Data size: 954 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 422 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 211 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col6 (type: string), _col0 (type: int), _col1 (type: string), _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 + Statistics: Num rows: 1 Data size: 211 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 712 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1275,42 +1275,42 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(), count(_col0) - minReductionHashAggr: 0.5 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Group By Operator keys: _col0 (type: int) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reducer 9 Execution mode: vectorized Reduce Operator Tree: @@ -1318,11 +1318,11 @@ STAGE PLANS: aggregations: count(VALUE._col0), count(VALUE._col1) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Union 5 Vertex: Union 5 @@ -1426,58 +1426,58 @@ STAGE PLANS: Map Operator Tree: TableScan alias: tbl_ice - Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: FILE__PATH is not null (type: boolean) - Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), b (type: string), c (type: int), 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, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), PARTITION__PROJECTION (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: string) Select Operator expressions: a (type: int) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (a is not null and FILE__PATH is not null) (type: boolean) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), FILE__PATH (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Execution mode: vectorized Map 13 @@ -1509,20 +1509,20 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col1 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reducer 11 Execution mode: vectorized Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: string) outputColumnNames: _col1 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -1542,26 +1542,26 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: string) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reducer 12 Execution mode: vectorized Reduce Operator Tree: @@ -1569,11 +1569,11 @@ STAGE PLANS: aggregations: count(VALUE._col0), count(VALUE._col1) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 2 Reduce Operator Tree: @@ -1584,11 +1584,11 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9 - Statistics: Num rows: 1 Data size: 489 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 211 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 489 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 211 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col9 (type: boolean) Reducer 3 Reduce Operator Tree: @@ -1599,24 +1599,24 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9, _col10, _col11 - Statistics: Num rows: 1 Data size: 505 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 228 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col10 (type: bigint), _col11 (type: bigint), _col9 (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col11 - Statistics: Num rows: 1 Data size: 505 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 228 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((_col8 = 0L) or ((_col11 is not null and (_col8 <> 0L)) or ((_col0 is null or (_col9 < _col8)) and null and (_col8 <> 0L) and _col11 is null)) is null or (_col11 is null and (_col9 >= _col8) and _col0 is not null)) (type: boolean) - Statistics: Num rows: 1 Data size: 505 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 228 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 228 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 228 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint), _col7 (type: string) Reducer 4 Reduce Operator Tree: @@ -1627,14 +1627,14 @@ STAGE PLANS: 0 _col5 (type: string) 1 _col0 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 250 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col0 (type: int), _col1 (type: string), _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 + Statistics: Num rows: 1 Data size: 250 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 254 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1649,13 +1649,13 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: string) Reducer 7 Execution mode: vectorized @@ -1663,7 +1663,7 @@ STAGE PLANS: Select Operator expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col5 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -1682,17 +1682,17 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col6 (type: string), _col0 (type: int), _col1 (type: string), _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 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 2 Data size: 970 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 254 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1707,62 +1707,62 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col1 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col1 (type: int) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: int) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(), count(_col1) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Select Operator expressions: _col1 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: int) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reducer 9 Execution mode: vectorized Reduce Operator Tree: @@ -1770,17 +1770,17 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), true (type: boolean) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean) Union 5 Vertex: Union 5 diff --git a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_mixed.q.out b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_mixed.q.out index 27a1d97f204e..240f6559f34f 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_mixed.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/delete_iceberg_mixed.q.out @@ -1,10 +1,10 @@ PREHOOK: query: create table ice01 (id int, name string) Stored by Iceberg stored as ORC - TBLPROPERTIES('format-version'='2') +TBLPROPERTIES('format-version'='2') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@ice01 POSTHOOK: query: create table ice01 (id int, name string) Stored by Iceberg stored as ORC - TBLPROPERTIES('format-version'='2') +TBLPROPERTIES('format-version'='2') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@ice01 @@ -84,54 +84,54 @@ Stage-4 <-Reducer 2 [CONTAINS] File Output Operator [FS_46] table:{"name:":"default.ice01"} - Select Operator [SEL_44] (rows=3 width=479) + Select Operator [SEL_44] (rows=7 width=206) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"] - Merge Join Operator [MERGEJOIN_43] (rows=3 width=479) + Merge Join Operator [MERGEJOIN_43] (rows=7 width=206) Conds:RS_59._col4=RS_65._col0(Left Semi),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"] <-Map 1 [SIMPLE_EDGE] vectorized SHUFFLE [RS_59] PartitionCols:_col4 - Select Operator [SEL_56] (rows=5 width=479) + Select Operator [SEL_56] (rows=7 width=188) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"] - Filter Operator [FIL_53] (rows=5 width=91) - predicate:((((id <= 4) and (id <> 2)) or ((id > 4) or (id = 2)) is null) and ((id <= 4) or (id <> 2) or ((id > 4) or (id = 2)) is null) and FILE__PATH is not null) - TableScan [TS_0] (rows=7 width=78) - default@ice01,ice01,Tbl:COMPLETE,Col:COMPLETE,Output:["id","name"] + Filter Operator [FIL_53] (rows=7 width=188) + predicate:(((id <= 4) or (id <> 2) or ((id > 4) or (id = 2)) is null) and (((id <= 4) and (id <> 2)) or ((id > 4) or (id = 2)) is null) and FILE__PATH is not null) + TableScan [TS_0] (rows=7 width=188) + default@ice01,ice01,Tbl:COMPLETE,Col:NONE,Output:["id","name"] <-Reducer 4 [SIMPLE_EDGE] vectorized SHUFFLE [RS_65] PartitionCols:_col0 - Group By Operator [GBY_64] (rows=3 width=184) + Group By Operator [GBY_64] (rows=3 width=4) Output:["_col0"],keys:_col0 - Select Operator [SEL_63] (rows=3 width=184) + Select Operator [SEL_63] (rows=3 width=4) Output:["_col0"] - Filter Operator [FIL_62] (rows=3 width=184) + Filter Operator [FIL_62] (rows=3 width=4) predicate:(row_number_window_0 = 1) - PTF Operator [PTF_61] (rows=6 width=184) + PTF Operator [PTF_61] (rows=7 width=4) Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col4 ASC NULLS FIRST","partition by:":"_col4"}] - Select Operator [SEL_60] (rows=6 width=184) + Select Operator [SEL_60] (rows=7 width=4) Output:["_col4"] <-Map 1 [SIMPLE_EDGE] vectorized SHUFFLE [RS_57] PartitionCols:FILE__PATH - Filter Operator [FIL_54] (rows=6 width=4) + Filter Operator [FIL_54] (rows=7 width=4) predicate:(((id > 4) or (id = 2)) and FILE__PATH is not null) Please refer to the previous TableScan [TS_0] <-Reducer 5 [CONTAINS] vectorized File Output Operator [FS_70] table:{"name:":"default.ice01"} - Select Operator [SEL_69] (rows=3 width=450) + Select Operator [SEL_69] (rows=3 width=188) Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"] - Filter Operator [FIL_68] (rows=3 width=442) + Filter Operator [FIL_68] (rows=3 width=188) predicate:(row_number_window_0 = 1) - PTF Operator [PTF_67] (rows=6 width=456) + PTF Operator [PTF_67] (rows=7 width=188) Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col4 ASC NULLS FIRST","partition by:":"_col4"}] - Select Operator [SEL_66] (rows=6 width=456) + Select Operator [SEL_66] (rows=7 width=188) Output:["_col0","_col1","_col2","_col3","_col4","_col6"] <-Map 1 [SIMPLE_EDGE] vectorized SHUFFLE [RS_58] PartitionCols:FILE__PATH - Filter Operator [FIL_55] (rows=6 width=76) - predicate:((id > 4) or (id = 2)) + Filter Operator [FIL_55] (rows=7 width=188) + predicate:((id = 2) or (id > 4)) Please refer to the previous TableScan [TS_0] PREHOOK: query: delete from ice01 where id>4 OR id=2 diff --git a/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_writes.q.out b/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_writes.q.out index b89e15109dfa..19d252b09498 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_writes.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_writes.q.out @@ -132,7 +132,7 @@ Plan optimized by CBO. Vertex dependency in root stage Reducer 2 <- Map 1 (SIMPLE_EDGE) -Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) +Reducer 3 <- Map 1 (SIMPLE_EDGE) Stage-3 Stats Work{} @@ -148,7 +148,7 @@ Stage-3 Select Operator [SEL_17] Output:["_col0","_col1","_col1"] <-Map 1 [SIMPLE_EDGE] vectorized - PARTITION_ONLY_SHUFFLE [RS_13] + SHUFFLE [RS_13] PartitionCols:_col1 Select Operator [SEL_12] (rows=22 width=87) Output:["_col0","_col1"] @@ -156,14 +156,15 @@ Stage-3 default@tbl_src,tbl_src,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b"] Reducer 3 vectorized File Output Operator [FS_21] - Select Operator [SEL_20] (rows=1 width=530) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"] - Group By Operator [GBY_19] (rows=1 width=332) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)"] - <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized - PARTITION_ONLY_SHUFFLE [RS_16] - Group By Operator [GBY_15] (rows=1 width=400) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)"] + Select Operator [SEL_20] (rows=6 width=754) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"] + Group By Operator [GBY_19] (rows=6 width=419) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)"],keys:KEY._col0 + <-Map 1 [SIMPLE_EDGE] vectorized + SHUFFLE [RS_16] + PartitionCols:_col0 + Group By Operator [GBY_15] (rows=6 width=487) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)"],keys:ccy Select Operator [SEL_14] (rows=22 width=87) Output:["a","ccy"] Please refer to the previous Select Operator [SEL_12] @@ -226,7 +227,7 @@ Plan optimized by CBO. Vertex dependency in root stage Reducer 2 <- Map 1 (SIMPLE_EDGE) -Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) +Reducer 3 <- Map 1 (SIMPLE_EDGE) Stage-3 Stats Work{} @@ -242,7 +243,7 @@ Stage-3 Select Operator [SEL_17] Output:["_col0","_col1","iceberg_bucket(_col1, 2)"] <-Map 1 [SIMPLE_EDGE] vectorized - PARTITION_ONLY_SHUFFLE [RS_13] + SHUFFLE [RS_13] PartitionCols:iceberg_bucket(_col1, 2) Select Operator [SEL_12] (rows=22 width=87) Output:["_col0","_col1"] @@ -250,14 +251,15 @@ Stage-3 default@tbl_src,tbl_src,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b"] Reducer 3 vectorized File Output Operator [FS_21] - Select Operator [SEL_20] (rows=1 width=530) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"] - Group By Operator [GBY_19] (rows=1 width=332) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)"] - <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized - PARTITION_ONLY_SHUFFLE [RS_16] - Group By Operator [GBY_15] (rows=1 width=400) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)"] + Select Operator [SEL_20] (rows=5 width=574) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"] + Group By Operator [GBY_19] (rows=5 width=336) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)"],keys:KEY._col0 + <-Map 1 [SIMPLE_EDGE] vectorized + SHUFFLE [RS_16] + PartitionCols:_col0 + Group By Operator [GBY_15] (rows=5 width=404) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)"],keys:iceberg_bucket(ccy, 2) Select Operator [SEL_14] (rows=22 width=87) Output:["a","ccy"] Please refer to the previous Select Operator [SEL_12] @@ -320,7 +322,7 @@ Plan optimized by CBO. Vertex dependency in root stage Reducer 2 <- Map 1 (SIMPLE_EDGE) -Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) +Reducer 3 <- Map 1 (SIMPLE_EDGE) Stage-3 Stats Work{} @@ -336,7 +338,7 @@ Stage-3 Select Operator [SEL_17] Output:["_col0","_col1","_col2","_col1","iceberg_bucket(_col2, 3)"] <-Map 1 [SIMPLE_EDGE] vectorized - PARTITION_ONLY_SHUFFLE [RS_13] + SHUFFLE [RS_13] PartitionCols:_col1, iceberg_bucket(_col2, 3) Select Operator [SEL_12] (rows=22 width=94) Output:["_col0","_col1","_col2"] @@ -344,14 +346,15 @@ Stage-3 default@tbl_src,tbl_src,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b","c"] Reducer 3 vectorized File Output Operator [FS_21] - Select Operator [SEL_20] (rows=1 width=794) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] - Group By Operator [GBY_19] (rows=1 width=500) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] - <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized - PARTITION_ONLY_SHUFFLE [RS_16] - Group By Operator [GBY_15] (rows=1 width=568) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)","min(c)","max(c)","count(c)","compute_bit_vector_hll(c)"] + Select Operator [SEL_20] (rows=11 width=1030) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"] + Group By Operator [GBY_19] (rows=11 width=591) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"],keys:KEY._col0, KEY._col1 + <-Map 1 [SIMPLE_EDGE] vectorized + SHUFFLE [RS_16] + PartitionCols:_col0, _col1 + Group By Operator [GBY_15] (rows=11 width=659) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)","min(c)","max(c)","count(c)","compute_bit_vector_hll(c)"],keys:ccy, iceberg_bucket(c, 3) Select Operator [SEL_14] (rows=22 width=94) Output:["a","ccy","c"] Please refer to the previous Select Operator [SEL_12] @@ -455,7 +458,7 @@ Plan optimized by CBO. Vertex dependency in root stage Reducer 2 <- Map 1 (SIMPLE_EDGE) -Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) +Reducer 3 <- Map 1 (SIMPLE_EDGE) Stage-3 Stats Work{} @@ -471,7 +474,7 @@ Stage-3 Select Operator [SEL_20] Output:["_col0","_col1","_col2","_col1","iceberg_bucket(_col2, 3)"] <-Map 1 [SIMPLE_EDGE] vectorized - PARTITION_ONLY_SHUFFLE [RS_16] + SHUFFLE [RS_16] PartitionCols:_col1, iceberg_bucket(_col2, 3) Select Operator [SEL_15] (rows=4 width=99) Output:["_col0","_col1","_col2"] @@ -481,14 +484,15 @@ Stage-3 default@tbl_src,tbl_src,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b","c"] Reducer 3 vectorized File Output Operator [FS_24] - Select Operator [SEL_23] (rows=1 width=794) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] - Group By Operator [GBY_22] (rows=1 width=500) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] - <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized - PARTITION_ONLY_SHUFFLE [RS_19] - Group By Operator [GBY_18] (rows=1 width=568) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)","min(c)","max(c)","count(c)","compute_bit_vector_hll(c)"] + Select Operator [SEL_23] (rows=4 width=1030) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"] + Group By Operator [GBY_22] (rows=4 width=591) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"],keys:KEY._col0, KEY._col1 + <-Map 1 [SIMPLE_EDGE] vectorized + SHUFFLE [RS_19] + PartitionCols:_col0, _col1 + Group By Operator [GBY_18] (rows=4 width=659) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)","min(c)","max(c)","count(c)","compute_bit_vector_hll(c)"],keys:ccy, iceberg_bucket(c, 3) Select Operator [SEL_17] (rows=4 width=99) Output:["a","ccy","c"] Please refer to the previous Select Operator [SEL_15] @@ -512,7 +516,7 @@ POSTHOOK: Output: default@tbl_target_mixed Plan optimized by CBO. Vertex dependency in root stage -Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE) +Reducer 2 <- Map 1 (SIMPLE_EDGE) Stage-3 Stats Work{} @@ -524,11 +528,11 @@ Stage-3 Stage-1 Reducer 2 vectorized File Output Operator [FS_20] - Select Operator [SEL_19] (rows=1 width=794) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] - Group By Operator [GBY_18] (rows=1 width=500) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] - <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized + Select Operator [SEL_19] (rows=1 width=1030) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"] + Group By Operator [GBY_18] (rows=1 width=591) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"],keys:KEY._col0, KEY._col1 + <-Map 1 [SIMPLE_EDGE] vectorized File Output Operator [FS_14] table:{"name:":"default.tbl_target_mixed"} Select Operator [SEL_13] (rows=1 width=99) @@ -537,9 +541,10 @@ Stage-3 predicate:((c = 100L) and (b = 'USD')) TableScan [TS_0] (rows=22 width=94) default@tbl_src,tbl_src,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b","c"] - PARTITION_ONLY_SHUFFLE [RS_17] - Group By Operator [GBY_16] (rows=1 width=568) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)","min(c)","max(c)","count(c)","compute_bit_vector_hll(c)"] + SHUFFLE [RS_17] + PartitionCols:_col0, _col1 + Group By Operator [GBY_16] (rows=1 width=659) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)","min(c)","max(c)","count(c)","compute_bit_vector_hll(c)"],keys:ccy, iceberg_bucket(c, 3) Select Operator [SEL_15] (rows=1 width=99) Output:["a","ccy","c"] Please refer to the previous Select Operator [SEL_13] @@ -686,7 +691,7 @@ Plan optimized by CBO. Vertex dependency in root stage Reducer 2 <- Map 1 (SIMPLE_EDGE) -Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) +Reducer 3 <- Map 1 (SIMPLE_EDGE) Stage-3 Stats Work{} @@ -702,7 +707,7 @@ Stage-3 Select Operator [SEL_17] Output:["_col0","_col1","iceberg_truncate(_col1, 2)"] <-Map 1 [SIMPLE_EDGE] vectorized - PARTITION_ONLY_SHUFFLE [RS_13] + SHUFFLE [RS_13] PartitionCols:iceberg_truncate(_col1, 2) Select Operator [SEL_12] (rows=22 width=87) Output:["_col0","_col1"] @@ -710,14 +715,15 @@ Stage-3 default@tbl_src,tbl_src,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b"] Reducer 3 vectorized File Output Operator [FS_21] - Select Operator [SEL_20] (rows=1 width=530) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"] - Group By Operator [GBY_19] (rows=1 width=332) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)"] - <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized - PARTITION_ONLY_SHUFFLE [RS_16] - Group By Operator [GBY_15] (rows=1 width=400) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)"] + Select Operator [SEL_20] (rows=5 width=754) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"] + Group By Operator [GBY_19] (rows=5 width=516) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)"],keys:KEY._col0 + <-Map 1 [SIMPLE_EDGE] vectorized + SHUFFLE [RS_16] + PartitionCols:_col0 + Group By Operator [GBY_15] (rows=5 width=584) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)"],keys:iceberg_truncate(ccy, 2) Select Operator [SEL_14] (rows=22 width=87) Output:["a","ccy"] Please refer to the previous Select Operator [SEL_12] @@ -780,7 +786,7 @@ Plan optimized by CBO. Vertex dependency in root stage Reducer 2 <- Map 1 (SIMPLE_EDGE) -Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) +Reducer 3 <- Map 1 (SIMPLE_EDGE) Stage-3 Stats Work{} @@ -796,7 +802,7 @@ Stage-3 Select Operator [SEL_17] Output:["_col0","_col1","iceberg_truncate(_col0, 2)"] <-Map 1 [SIMPLE_EDGE] vectorized - PARTITION_ONLY_SHUFFLE [RS_13] + SHUFFLE [RS_13] PartitionCols:iceberg_truncate(_col0, 2) Select Operator [SEL_12] (rows=22 width=87) Output:["_col0","_col1"] @@ -804,14 +810,15 @@ Stage-3 default@tbl_src,tbl_src,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b"] Reducer 3 vectorized File Output Operator [FS_21] - Select Operator [SEL_20] (rows=1 width=530) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"] - Group By Operator [GBY_19] (rows=1 width=332) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)"] - <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized - PARTITION_ONLY_SHUFFLE [RS_16] - Group By Operator [GBY_15] (rows=1 width=400) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(id)","max(id)","count(1)","count(id)","compute_bit_vector_hll(id)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)"] + Select Operator [SEL_20] (rows=21 width=754) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"] + Group By Operator [GBY_19] (rows=21 width=516) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)"],keys:KEY._col0 + <-Map 1 [SIMPLE_EDGE] vectorized + SHUFFLE [RS_16] + PartitionCols:_col0 + Group By Operator [GBY_15] (rows=21 width=584) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["min(id)","max(id)","count(1)","count(id)","compute_bit_vector_hll(id)","max(length(ccy))","avg(COALESCE(length(ccy),0))","count(ccy)","compute_bit_vector_hll(ccy)"],keys:iceberg_truncate(id, 2) Select Operator [SEL_14] (rows=22 width=87) Output:["id","ccy"] Please refer to the previous Select Operator [SEL_12] @@ -874,7 +881,7 @@ Plan optimized by CBO. Vertex dependency in root stage Reducer 2 <- Map 1 (SIMPLE_EDGE) -Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) +Reducer 3 <- Map 1 (SIMPLE_EDGE) Stage-3 Stats Work{} @@ -890,7 +897,7 @@ Stage-3 Select Operator [SEL_17] Output:["_col0","_col1","iceberg_truncate(_col1, 2)"] <-Map 1 [SIMPLE_EDGE] vectorized - PARTITION_ONLY_SHUFFLE [RS_13] + SHUFFLE [RS_13] PartitionCols:iceberg_truncate(_col1, 2) Select Operator [SEL_12] (rows=22 width=11) Output:["_col0","_col1"] @@ -898,14 +905,15 @@ Stage-3 default@tbl_src,tbl_src,Tbl:COMPLETE,Col:COMPLETE,Output:["a","c"] Reducer 3 vectorized File Output Operator [FS_21] - Select Operator [SEL_20] (rows=1 width=528) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"] - Group By Operator [GBY_19] (rows=1 width=336) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)"] - <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized - PARTITION_ONLY_SHUFFLE [RS_16] - Group By Operator [GBY_15] (rows=1 width=336) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","min(ccy)","max(ccy)","count(ccy)","compute_bit_vector_hll(ccy)"] + Select Operator [SEL_20] (rows=8 width=752) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"] + Group By Operator [GBY_19] (rows=8 width=520) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)"],keys:KEY._col0 + <-Map 1 [SIMPLE_EDGE] vectorized + SHUFFLE [RS_16] + PartitionCols:_col0 + Group By Operator [GBY_15] (rows=8 width=520) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","min(ccy)","max(ccy)","count(ccy)","compute_bit_vector_hll(ccy)"],keys:iceberg_truncate(ccy, 2) Select Operator [SEL_14] (rows=22 width=11) Output:["a","ccy"] Please refer to the previous Select Operator [SEL_12] @@ -968,7 +976,7 @@ Plan optimized by CBO. Vertex dependency in root stage Reducer 2 <- Map 1 (SIMPLE_EDGE) -Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) +Reducer 3 <- Map 1 (SIMPLE_EDGE) Stage-3 Stats Work{} @@ -984,7 +992,7 @@ Stage-3 Select Operator [SEL_18] Output:["_col0","_col1","_col2","iceberg_truncate(_col1, 2)","iceberg_truncate(_col2, 3)"] <-Map 1 [SIMPLE_EDGE] vectorized - PARTITION_ONLY_SHUFFLE [RS_14] + SHUFFLE [RS_14] PartitionCols:iceberg_truncate(_col1, 2), iceberg_truncate(_col2, 3) Select Operator [SEL_13] (rows=22 width=199) Output:["_col0","_col1","_col2"] @@ -992,14 +1000,15 @@ Stage-3 default@tbl_src,tbl_src,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b"] Reducer 3 vectorized File Output Operator [FS_22] - Select Operator [SEL_21] (rows=1 width=1005) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] - Group By Operator [GBY_20] (rows=1 width=708) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] - <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized - PARTITION_ONLY_SHUFFLE [RS_17] - Group By Operator [GBY_16] (rows=1 width=776) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(b))","avg(COALESCE(length(b),0))","count(b)","compute_bit_vector_hll(b)","min(ccy)","max(ccy)","count(ccy)","compute_bit_vector_hll(ccy)"] + Select Operator [SEL_21] (rows=5 width=1421) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"] + Group By Operator [GBY_20] (rows=5 width=1076) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","max(VALUE._col5)","avg(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"],keys:KEY._col0, KEY._col1 + <-Map 1 [SIMPLE_EDGE] vectorized + SHUFFLE [RS_17] + PartitionCols:_col0, _col1 + Group By Operator [GBY_16] (rows=5 width=1144) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["min(a)","max(a)","count(1)","count(a)","compute_bit_vector_hll(a)","max(length(b))","avg(COALESCE(length(b),0))","count(b)","compute_bit_vector_hll(b)","min(ccy)","max(ccy)","count(ccy)","compute_bit_vector_hll(ccy)"],keys:iceberg_truncate(b, 2), iceberg_truncate(ccy, 3) Select Operator [SEL_15] (rows=22 width=199) Output:["a","b","ccy"] Please refer to the previous Select Operator [SEL_13] @@ -1068,7 +1077,7 @@ Plan optimized by CBO. Vertex dependency in root stage Reducer 2 <- Map 1 (SIMPLE_EDGE) -Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) +Reducer 3 <- Map 1 (SIMPLE_EDGE) Stage-3 Stats Work{} @@ -1084,7 +1093,7 @@ Stage-3 Select Operator [SEL_16] Output:["_col0","_col1","_col2","_col2","iceberg_year(_col1)"] <-Map 1 [SIMPLE_EDGE] - PARTITION_ONLY_SHUFFLE [RS_13] + SHUFFLE [RS_13] PartitionCols:_col2, iceberg_year(_col1) Select Operator [SEL_3] (rows=1 width=240) Output:["_col0","_col1","_col2"] @@ -1096,14 +1105,15 @@ Stage-3 _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE Reducer 3 vectorized File Output Operator [FS_20] - Select Operator [SEL_19] (rows=1 width=890) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] - Group By Operator [GBY_18] (rows=1 width=596) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] - <-Map 1 [CUSTOM_SIMPLE_EDGE] - PARTITION_ONLY_SHUFFLE [RS_9] - Group By Operator [GBY_8] (rows=1 width=664) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_date)","max(date_time_date)","count(date_time_date)","compute_bit_vector_hll(date_time_date)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"] + Select Operator [SEL_19] (rows=1 width=946) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"] + Group By Operator [GBY_18] (rows=1 width=604) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"],keys:KEY._col0, KEY._col1 + <-Map 1 [SIMPLE_EDGE] + SHUFFLE [RS_9] + PartitionCols:_col0, _col1 + Group By Operator [GBY_8] (rows=1 width=672) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_date)","max(date_time_date)","count(date_time_date)","compute_bit_vector_hll(date_time_date)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"],keys:year_partition, iceberg_year(date_time_date) Select Operator [SEL_7] (rows=1 width=240) Output:["id","date_time_date","year_partition"] Please refer to the previous Select Operator [SEL_3] @@ -1153,7 +1163,7 @@ Plan optimized by CBO. Vertex dependency in root stage Reducer 2 <- Map 1 (SIMPLE_EDGE) -Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) +Reducer 3 <- Map 1 (SIMPLE_EDGE) Stage-3 Stats Work{} @@ -1169,7 +1179,7 @@ Stage-3 Select Operator [SEL_16] Output:["_col0","_col1","_col2","_col2","iceberg_year(_col1)"] <-Map 1 [SIMPLE_EDGE] - PARTITION_ONLY_SHUFFLE [RS_13] + SHUFFLE [RS_13] PartitionCols:_col2, iceberg_year(_col1) Select Operator [SEL_3] (rows=1 width=224) Output:["_col0","_col1","_col2"] @@ -1181,14 +1191,15 @@ Stage-3 _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE Reducer 3 vectorized File Output Operator [FS_20] - Select Operator [SEL_19] (rows=1 width=863) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] - Group By Operator [GBY_18] (rows=1 width=564) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] - <-Map 1 [CUSTOM_SIMPLE_EDGE] - PARTITION_ONLY_SHUFFLE [RS_9] - Group By Operator [GBY_8] (rows=1 width=632) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_timestamp)","max(date_time_timestamp)","count(date_time_timestamp)","compute_bit_vector_hll(date_time_timestamp)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"] + Select Operator [SEL_19] (rows=1 width=919) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"] + Group By Operator [GBY_18] (rows=1 width=572) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"],keys:KEY._col0, KEY._col1 + <-Map 1 [SIMPLE_EDGE] + SHUFFLE [RS_9] + PartitionCols:_col0, _col1 + Group By Operator [GBY_8] (rows=1 width=640) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_timestamp)","max(date_time_timestamp)","count(date_time_timestamp)","compute_bit_vector_hll(date_time_timestamp)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"],keys:year_partition, iceberg_year(date_time_timestamp) Select Operator [SEL_7] (rows=1 width=224) Output:["id","date_time_timestamp","year_partition"] Please refer to the previous Select Operator [SEL_3] @@ -1238,7 +1249,7 @@ Plan optimized by CBO. Vertex dependency in root stage Reducer 2 <- Map 1 (SIMPLE_EDGE) -Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) +Reducer 3 <- Map 1 (SIMPLE_EDGE) Stage-3 Stats Work{} @@ -1254,7 +1265,7 @@ Stage-3 Select Operator [SEL_16] Output:["_col0","_col1","_col2","_col2","iceberg_month(_col1)"] <-Map 1 [SIMPLE_EDGE] - PARTITION_ONLY_SHUFFLE [RS_13] + SHUFFLE [RS_13] PartitionCols:_col2, iceberg_month(_col1) Select Operator [SEL_3] (rows=1 width=240) Output:["_col0","_col1","_col2"] @@ -1266,14 +1277,15 @@ Stage-3 _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE Reducer 3 vectorized File Output Operator [FS_20] - Select Operator [SEL_19] (rows=1 width=890) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] - Group By Operator [GBY_18] (rows=1 width=596) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] - <-Map 1 [CUSTOM_SIMPLE_EDGE] - PARTITION_ONLY_SHUFFLE [RS_9] - Group By Operator [GBY_8] (rows=1 width=664) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_date)","max(date_time_date)","count(date_time_date)","compute_bit_vector_hll(date_time_date)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"] + Select Operator [SEL_19] (rows=1 width=946) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"] + Group By Operator [GBY_18] (rows=1 width=604) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"],keys:KEY._col0, KEY._col1 + <-Map 1 [SIMPLE_EDGE] + SHUFFLE [RS_9] + PartitionCols:_col0, _col1 + Group By Operator [GBY_8] (rows=1 width=672) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_date)","max(date_time_date)","count(date_time_date)","compute_bit_vector_hll(date_time_date)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"],keys:year_partition, iceberg_month(date_time_date) Select Operator [SEL_7] (rows=1 width=240) Output:["id","date_time_date","year_partition"] Please refer to the previous Select Operator [SEL_3] @@ -1323,7 +1335,7 @@ Plan optimized by CBO. Vertex dependency in root stage Reducer 2 <- Map 1 (SIMPLE_EDGE) -Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) +Reducer 3 <- Map 1 (SIMPLE_EDGE) Stage-3 Stats Work{} @@ -1339,7 +1351,7 @@ Stage-3 Select Operator [SEL_16] Output:["_col0","_col1","_col2","_col2","iceberg_month(_col1)"] <-Map 1 [SIMPLE_EDGE] - PARTITION_ONLY_SHUFFLE [RS_13] + SHUFFLE [RS_13] PartitionCols:_col2, iceberg_month(_col1) Select Operator [SEL_3] (rows=1 width=224) Output:["_col0","_col1","_col2"] @@ -1351,14 +1363,15 @@ Stage-3 _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE Reducer 3 vectorized File Output Operator [FS_20] - Select Operator [SEL_19] (rows=1 width=863) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] - Group By Operator [GBY_18] (rows=1 width=564) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] - <-Map 1 [CUSTOM_SIMPLE_EDGE] - PARTITION_ONLY_SHUFFLE [RS_9] - Group By Operator [GBY_8] (rows=1 width=632) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_timestamp)","max(date_time_timestamp)","count(date_time_timestamp)","compute_bit_vector_hll(date_time_timestamp)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"] + Select Operator [SEL_19] (rows=1 width=919) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"] + Group By Operator [GBY_18] (rows=1 width=572) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"],keys:KEY._col0, KEY._col1 + <-Map 1 [SIMPLE_EDGE] + SHUFFLE [RS_9] + PartitionCols:_col0, _col1 + Group By Operator [GBY_8] (rows=1 width=640) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_timestamp)","max(date_time_timestamp)","count(date_time_timestamp)","compute_bit_vector_hll(date_time_timestamp)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"],keys:year_partition, iceberg_month(date_time_timestamp) Select Operator [SEL_7] (rows=1 width=224) Output:["id","date_time_timestamp","year_partition"] Please refer to the previous Select Operator [SEL_3] @@ -1408,7 +1421,7 @@ Plan optimized by CBO. Vertex dependency in root stage Reducer 2 <- Map 1 (SIMPLE_EDGE) -Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) +Reducer 3 <- Map 1 (SIMPLE_EDGE) Stage-3 Stats Work{} @@ -1424,7 +1437,7 @@ Stage-3 Select Operator [SEL_16] Output:["_col0","_col1","_col2","_col2","iceberg_day(_col1)"] <-Map 1 [SIMPLE_EDGE] - PARTITION_ONLY_SHUFFLE [RS_13] + SHUFFLE [RS_13] PartitionCols:_col2, iceberg_day(_col1) Select Operator [SEL_3] (rows=1 width=240) Output:["_col0","_col1","_col2"] @@ -1436,14 +1449,15 @@ Stage-3 _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE Reducer 3 vectorized File Output Operator [FS_20] - Select Operator [SEL_19] (rows=1 width=890) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] - Group By Operator [GBY_18] (rows=1 width=596) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] - <-Map 1 [CUSTOM_SIMPLE_EDGE] - PARTITION_ONLY_SHUFFLE [RS_9] - Group By Operator [GBY_8] (rows=1 width=664) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_date)","max(date_time_date)","count(date_time_date)","compute_bit_vector_hll(date_time_date)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"] + Select Operator [SEL_19] (rows=1 width=946) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"] + Group By Operator [GBY_18] (rows=1 width=604) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"],keys:KEY._col0, KEY._col1 + <-Map 1 [SIMPLE_EDGE] + SHUFFLE [RS_9] + PartitionCols:_col0, _col1 + Group By Operator [GBY_8] (rows=1 width=672) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_date)","max(date_time_date)","count(date_time_date)","compute_bit_vector_hll(date_time_date)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"],keys:year_partition, iceberg_day(date_time_date) Select Operator [SEL_7] (rows=1 width=240) Output:["id","date_time_date","year_partition"] Please refer to the previous Select Operator [SEL_3] @@ -1493,7 +1507,7 @@ Plan optimized by CBO. Vertex dependency in root stage Reducer 2 <- Map 1 (SIMPLE_EDGE) -Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) +Reducer 3 <- Map 1 (SIMPLE_EDGE) Stage-3 Stats Work{} @@ -1509,7 +1523,7 @@ Stage-3 Select Operator [SEL_16] Output:["_col0","_col1","_col2","_col2","iceberg_day(_col1)"] <-Map 1 [SIMPLE_EDGE] - PARTITION_ONLY_SHUFFLE [RS_13] + SHUFFLE [RS_13] PartitionCols:_col2, iceberg_day(_col1) Select Operator [SEL_3] (rows=1 width=224) Output:["_col0","_col1","_col2"] @@ -1521,14 +1535,15 @@ Stage-3 _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE Reducer 3 vectorized File Output Operator [FS_20] - Select Operator [SEL_19] (rows=1 width=863) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] - Group By Operator [GBY_18] (rows=1 width=564) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] - <-Map 1 [CUSTOM_SIMPLE_EDGE] - PARTITION_ONLY_SHUFFLE [RS_9] - Group By Operator [GBY_8] (rows=1 width=632) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_timestamp)","max(date_time_timestamp)","count(date_time_timestamp)","compute_bit_vector_hll(date_time_timestamp)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"] + Select Operator [SEL_19] (rows=1 width=919) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"] + Group By Operator [GBY_18] (rows=1 width=572) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"],keys:KEY._col0, KEY._col1 + <-Map 1 [SIMPLE_EDGE] + SHUFFLE [RS_9] + PartitionCols:_col0, _col1 + Group By Operator [GBY_8] (rows=1 width=640) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_timestamp)","max(date_time_timestamp)","count(date_time_timestamp)","compute_bit_vector_hll(date_time_timestamp)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"],keys:year_partition, iceberg_day(date_time_timestamp) Select Operator [SEL_7] (rows=1 width=224) Output:["id","date_time_timestamp","year_partition"] Please refer to the previous Select Operator [SEL_3] @@ -1578,7 +1593,7 @@ Plan optimized by CBO. Vertex dependency in root stage Reducer 2 <- Map 1 (SIMPLE_EDGE) -Reducer 3 <- Map 1 (CUSTOM_SIMPLE_EDGE) +Reducer 3 <- Map 1 (SIMPLE_EDGE) Stage-3 Stats Work{} @@ -1594,7 +1609,7 @@ Stage-3 Select Operator [SEL_16] Output:["_col0","_col1","_col2","_col2","iceberg_hour(_col1)"] <-Map 1 [SIMPLE_EDGE] - PARTITION_ONLY_SHUFFLE [RS_13] + SHUFFLE [RS_13] PartitionCols:_col2, iceberg_hour(_col1) Select Operator [SEL_3] (rows=1 width=224) Output:["_col0","_col1","_col2"] @@ -1606,14 +1621,15 @@ Stage-3 _dummy_database@_dummy_table,_dummy_table,Tbl:COMPLETE,Col:COMPLETE Reducer 3 vectorized File Output Operator [FS_20] - Select Operator [SEL_19] (rows=1 width=863) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"] - Group By Operator [GBY_18] (rows=1 width=564) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"] - <-Map 1 [CUSTOM_SIMPLE_EDGE] - PARTITION_ONLY_SHUFFLE [RS_9] - Group By Operator [GBY_8] (rows=1 width=632) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_timestamp)","max(date_time_timestamp)","count(date_time_timestamp)","compute_bit_vector_hll(date_time_timestamp)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"] + Select Operator [SEL_19] (rows=1 width=919) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"] + Group By Operator [GBY_18] (rows=1 width=572) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["max(VALUE._col0)","avg(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)"],keys:KEY._col0, KEY._col1 + <-Map 1 [SIMPLE_EDGE] + SHUFFLE [RS_9] + PartitionCols:_col0, _col1 + Group By Operator [GBY_8] (rows=1 width=640) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14"],aggregations:["max(length(id))","avg(COALESCE(length(id),0))","count(1)","count(id)","compute_bit_vector_hll(id)","min(date_time_timestamp)","max(date_time_timestamp)","count(date_time_timestamp)","compute_bit_vector_hll(date_time_timestamp)","min(year_partition)","max(year_partition)","count(year_partition)","compute_bit_vector_hll(year_partition)"],keys:year_partition, iceberg_hour(date_time_timestamp) Select Operator [SEL_7] (rows=1 width=224) Output:["id","date_time_timestamp","year_partition"] Please refer to the previous Select Operator [SEL_3] diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_stats_with_ppr.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_stats_with_ppr.q.out index 57bdbac57c57..c6365d049e94 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_stats_with_ppr.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_stats_with_ppr.q.out @@ -53,7 +53,7 @@ STAGE PLANS: Map Operator Tree: TableScan alias: ice01 - filterExpr: ((day = 3) and (year = 2023) and (month = 10)) (type: boolean) + filterExpr: ((year = 2023) and (month = 10) and (day = 3)) (type: boolean) Statistics: Num rows: 3 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: i (type: int) 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 7dd13fe4a088..9218f556ab19 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 @@ -85,14 +85,14 @@ STAGE PLANS: TableScan alias: test_ice_int filterExpr: (a = 22) (type: boolean) - Statistics: Num rows: 1 Data size: 91 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: PARTIAL Select Operator 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: 1 Data size: 483 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 480 Basic stats: COMPLETE Column stats: PARTIAL File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 480 Basic stats: COMPLETE Column stats: PARTIAL table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -275,14 +275,14 @@ STAGE PLANS: TableScan alias: test_ice_bigint filterExpr: (a = 226784902765739L) (type: boolean) - Statistics: Num rows: 1 Data size: 91 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: PARTIAL Select Operator 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: 1 Data size: 487 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 484 Basic stats: COMPLETE Column stats: PARTIAL File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 487 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 484 Basic stats: COMPLETE Column stats: PARTIAL table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -491,14 +491,14 @@ STAGE PLANS: TableScan alias: test_ice_str filterExpr: (b = 'ddd') (type: boolean) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL 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), 'ddd' (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: PARTIAL File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: PARTIAL table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -716,17 +716,17 @@ STAGE PLANS: TableScan alias: test_ice_date filterExpr: (b = DATE'2022-02-07') (type: boolean) - Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: PARTIAL Filter Operator predicate: (b = DATE'2022-02-07') (type: boolean) - Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: PARTIAL 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), DATE'2022-02-07' (type: date) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 444 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 444 Basic stats: COMPLETE Column stats: PARTIAL File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 444 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 444 Basic stats: COMPLETE Column stats: PARTIAL table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -904,14 +904,14 @@ STAGE PLANS: TableScan alias: test_ice_double filterExpr: (a = 1.156748927566759E11D) (type: boolean) - Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: PARTIAL Select Operator 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: 1 Data size: 452 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 452 Basic stats: COMPLETE Column stats: PARTIAL File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 452 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 452 Basic stats: COMPLETE Column stats: PARTIAL table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1081,17 +1081,17 @@ STAGE PLANS: TableScan alias: test_ice_double_date filterExpr: ((a = 1.156748927566759E11D) and (b = DATE'2022-02-07')) (type: boolean) - Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: PARTIAL Filter Operator predicate: ((a = 1.156748927566759E11D) and (b = DATE'2022-02-07')) (type: boolean) - Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: PARTIAL Select Operator 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 + Statistics: Num rows: 1 Data size: 452 Basic stats: COMPLETE Column stats: PARTIAL File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 452 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 452 Basic stats: COMPLETE Column stats: PARTIAL table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_1.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_1.q.out index 7a6af1c20fd0..aedc7fc43de6 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_1.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_1.q.out @@ -120,9 +120,9 @@ Stage-0 PartitionCols:_col1 Select Operator [SEL_5] (rows=3 width=168) Output:["_col0","_col1"] - Filter Operator [FIL_14] (rows=3 width=260) + Filter Operator [FIL_14] (rows=3 width=168) predicate:(if(decimal_col is not null, (CAST( decimal_col AS STRING) = '50000000000000000005905545593'), false) and date_col is not null and decimal_col is not null) - TableScan [TS_3] (rows=7 width=5077) + TableScan [TS_3] (rows=7 width=168) default@source_table,source_table,Tbl:COMPLETE,Col:COMPLETE,Output:["date_col","decimal_col"] <-Select Operator [SEL_2] (rows=10 width=168) Output:["_col0","_col1"] @@ -189,11 +189,11 @@ Stage-0 PartitionCols:_col0, _col1 Group By Operator [GBY_6] (rows=1 width=168) Output:["_col0","_col1"],keys:date_col, decimal_col - Select Operator [SEL_5] (rows=3 width=260) + Select Operator [SEL_5] (rows=3 width=168) Output:["date_col","decimal_col"] - Filter Operator [FIL_18] (rows=3 width=260) + Filter Operator [FIL_18] (rows=3 width=168) predicate:(if(decimal_col is not null, (CAST( decimal_col AS STRING) = '50000000000000000005905545593'), false) and date_col is not null and decimal_col is not null) - TableScan [TS_3] (rows=7 width=5077) + TableScan [TS_3] (rows=7 width=168) default@source_table,source_table,Tbl:COMPLETE,Col:COMPLETE,Output:["date_col","decimal_col"] <-Select Operator [SEL_2] (rows=10 width=168) Output:["_col0","_col1"] @@ -254,9 +254,9 @@ Stage-0 PartitionCols:_col0, _col1 Select Operator [SEL_5] (rows=3 width=168) Output:["_col0","_col1"] - Filter Operator [FIL_14] (rows=3 width=260) + Filter Operator [FIL_14] (rows=3 width=168) predicate:(if(decimal_col is not null, (CAST( decimal_col AS STRING) = '50000000000000000005905545593'), false) and date_col is not null and decimal_col is not null) - TableScan [TS_3] (rows=7 width=5077) + TableScan [TS_3] (rows=7 width=168) default@source_table,source_table,Tbl:COMPLETE,Col:COMPLETE,Output:["date_col","decimal_col"] <-Select Operator [SEL_2] (rows=10 width=168) Output:["_col0","_col1"] @@ -323,11 +323,11 @@ Stage-0 PartitionCols:_col0, _col1 Group By Operator [GBY_6] (rows=1 width=168) Output:["_col0","_col1"],keys:date_col, decimal_col - Select Operator [SEL_5] (rows=3 width=260) + Select Operator [SEL_5] (rows=3 width=168) Output:["date_col","decimal_col"] - Filter Operator [FIL_18] (rows=3 width=260) + Filter Operator [FIL_18] (rows=3 width=168) predicate:(if(decimal_col is not null, (CAST( decimal_col AS STRING) = '50000000000000000005905545593'), false) and date_col is not null and decimal_col is not null) - TableScan [TS_3] (rows=7 width=5077) + TableScan [TS_3] (rows=7 width=168) default@source_table,source_table,Tbl:COMPLETE,Col:COMPLETE,Output:["date_col","decimal_col"] <-Select Operator [SEL_2] (rows=10 width=168) Output:["_col0","_col1"] @@ -388,9 +388,9 @@ Stage-0 PartitionCols:_col1 Select Operator [SEL_48] (rows=3 width=168) Output:["_col0","_col1"] - Filter Operator [FIL_47] (rows=3 width=260) + Filter Operator [FIL_47] (rows=3 width=168) predicate:(if(decimal_col is not null, (CAST( decimal_col AS STRING) = '50000000000000000005905545593'), false) and date_col is not null and decimal_col is not null) - TableScan [TS_3] (rows=7 width=5077) + TableScan [TS_3] (rows=7 width=168) default@source_table,source_table,Tbl:COMPLETE,Col:COMPLETE,Output:["date_col","decimal_col"] <-Select Operator [SEL_51] (rows=10 width=168) Output:["_col0","_col1"] @@ -457,11 +457,11 @@ Stage-0 PartitionCols:_col0, _col1 Group By Operator [GBY_53] (rows=1 width=168) Output:["_col0","_col1"],keys:date_col, decimal_col - Select Operator [SEL_52] (rows=3 width=260) + Select Operator [SEL_52] (rows=3 width=168) Output:["date_col","decimal_col"] - Filter Operator [FIL_51] (rows=3 width=260) + Filter Operator [FIL_51] (rows=3 width=168) predicate:(if(decimal_col is not null, (CAST( decimal_col AS STRING) = '50000000000000000005905545593'), false) and date_col is not null and decimal_col is not null) - TableScan [TS_3] (rows=7 width=5077) + TableScan [TS_3] (rows=7 width=168) default@source_table,source_table,Tbl:COMPLETE,Col:COMPLETE,Output:["date_col","decimal_col"] <-Select Operator [SEL_58] (rows=10 width=168) Output:["_col0","_col1"] @@ -522,9 +522,9 @@ Stage-0 PartitionCols:_col0, _col1 Select Operator [SEL_48] (rows=3 width=168) Output:["_col0","_col1"] - Filter Operator [FIL_47] (rows=3 width=260) + Filter Operator [FIL_47] (rows=3 width=168) predicate:(if(decimal_col is not null, (CAST( decimal_col AS STRING) = '50000000000000000005905545593'), false) and date_col is not null and decimal_col is not null) - TableScan [TS_3] (rows=7 width=5077) + TableScan [TS_3] (rows=7 width=168) default@source_table,source_table,Tbl:COMPLETE,Col:COMPLETE,Output:["date_col","decimal_col"] <-Select Operator [SEL_51] (rows=10 width=168) Output:["_col0","_col1"] @@ -591,11 +591,11 @@ Stage-0 PartitionCols:_col0, _col1 Group By Operator [GBY_53] (rows=1 width=168) Output:["_col0","_col1"],keys:date_col, decimal_col - Select Operator [SEL_52] (rows=3 width=260) + Select Operator [SEL_52] (rows=3 width=168) Output:["date_col","decimal_col"] - Filter Operator [FIL_51] (rows=3 width=260) + Filter Operator [FIL_51] (rows=3 width=168) predicate:(if(decimal_col is not null, (CAST( decimal_col AS STRING) = '50000000000000000005905545593'), false) and date_col is not null and decimal_col is not null) - TableScan [TS_3] (rows=7 width=5077) + TableScan [TS_3] (rows=7 width=168) default@source_table,source_table,Tbl:COMPLETE,Col:COMPLETE,Output:["date_col","decimal_col"] <-Select Operator [SEL_58] (rows=10 width=168) Output:["_col0","_col1"] diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_3.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_3.q.out index 37927fecface..bd82280c9c07 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_3.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_3.q.out @@ -129,16 +129,16 @@ Stage-0 PARTITION_ONLY_SHUFFLE [RS_38] Group By Operator [GBY_37] (rows=1 width=8) Output:["_col0"],aggregations:["count()"] - Map Join Operator [MAPJOIN_36] (rows=261 width=4) + Map Join Operator [MAPJOIN_36] (rows=372 width=8) BucketMapJoin:true,Conds:SEL_35._col0=RS_33._col0(Inner) <-Map 3 [CUSTOM_EDGE] vectorized, llap MULTICAST [RS_33] PartitionCols:_col0 - Select Operator [SEL_32] (rows=1 width=0) + Select Operator [SEL_32] (rows=238 width=4) Output:["_col0"] - Filter Operator [FIL_31] (rows=1 width=84) + Filter Operator [FIL_31] (rows=238 width=89) predicate:((part = '1') and key is not null) - TableScan [TS_3] (rows=1 width=84) + TableScan [TS_3] (rows=238 width=89) default@srcbucket_mapjoin_part_2_n4,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","part"] <-Select Operator [SEL_35] (rows=238 width=4) Output:["_col0"] @@ -203,16 +203,16 @@ Stage-0 PARTITION_ONLY_SHUFFLE [RS_38] Group By Operator [GBY_37] (rows=1 width=8) Output:["_col0"],aggregations:["count()"] - Map Join Operator [MAPJOIN_36] (rows=261 width=4) + Map Join Operator [MAPJOIN_36] (rows=372 width=8) BucketMapJoin:true,Conds:SEL_35._col0=RS_33._col0(Inner) <-Map 3 [CUSTOM_EDGE] vectorized, llap MULTICAST [RS_33] PartitionCols:_col0 - Select Operator [SEL_32] (rows=1 width=0) + Select Operator [SEL_32] (rows=238 width=4) Output:["_col0"] - Filter Operator [FIL_31] (rows=1 width=84) + Filter Operator [FIL_31] (rows=238 width=89) predicate:((part = '1') and key is not null) - TableScan [TS_3] (rows=1 width=84) + TableScan [TS_3] (rows=238 width=89) default@srcbucket_mapjoin_part_2_n4,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","part"] <-Select Operator [SEL_35] (rows=238 width=4) Output:["_col0"] diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_4.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_4.q.out index 6aadd2c86fbb..4ddd445ecbfb 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_4.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_4.q.out @@ -242,22 +242,22 @@ Stage-0 PARTITION_ONLY_SHUFFLE [RS_38] Group By Operator [GBY_37] (rows=1 width=8) Output:["_col0"],aggregations:["count()"] - Map Join Operator [MAPJOIN_36] (rows=500 width=8) + Map Join Operator [MAPJOIN_36] (rows=1797 width=8) Conds:SEL_35._col0=RS_33._col0(Inner) <-Map 3 [BROADCAST_EDGE] vectorized, llap BROADCAST [RS_33] PartitionCols:_col0 - Select Operator [SEL_32] (rows=238 width=4) + Select Operator [SEL_32] (rows=738 width=4) Output:["_col0"] - Filter Operator [FIL_31] (rows=238 width=89) + Filter Operator [FIL_31] (rows=738 width=89) predicate:(part is not null and key is not null) - TableScan [TS_3] (rows=238 width=89) + TableScan [TS_3] (rows=738 width=89) default@srcbucket_mapjoin_part_2_n6,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","part"] - <-Select Operator [SEL_35] (rows=500 width=4) + <-Select Operator [SEL_35] (rows=738 width=4) Output:["_col0"] - Filter Operator [FIL_34] (rows=500 width=89) + Filter Operator [FIL_34] (rows=738 width=89) predicate:(part is not null and key is not null) - TableScan [TS_0] (rows=500 width=89) + TableScan [TS_0] (rows=738 width=89) default@srcbucket_mapjoin_part_1_n2,a,Tbl:COMPLETE,Col:COMPLETE,Output:["key","part"] PREHOOK: query: SELECT /*+ MAPJOIN(b) */ count(*) @@ -309,28 +309,28 @@ Stage-0 PARTITION_ONLY_SHUFFLE [RS_47] Group By Operator [GBY_46] (rows=1 width=8) Output:["_col0"],aggregations:["count()"] - Map Join Operator [MAPJOIN_45] (rows=392 width=8) + Map Join Operator [MAPJOIN_45] (rows=1797 width=8) Conds:SEL_44._col0, _col1=RS_39._col0, _col1(Inner) <-Map 3 [BROADCAST_EDGE] vectorized, llap BROADCAST [RS_39] PartitionCols:_col0, _col1 - Select Operator [SEL_38] (rows=238 width=89) + Select Operator [SEL_38] (rows=738 width=89) Output:["_col0","_col1"] - Filter Operator [FIL_37] (rows=238 width=89) + Filter Operator [FIL_37] (rows=738 width=89) predicate:(part is not null and key is not null) - TableScan [TS_3] (rows=238 width=89) + TableScan [TS_3] (rows=738 width=89) default@srcbucket_mapjoin_part_2_n6,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","part"] Dynamic Partitioning Event Operator [EVENT_42] (rows=2 width=85) Group By Operator [GBY_41] (rows=2 width=85) Output:["_col0"],keys:_col0 - Select Operator [SEL_40] (rows=238 width=85) + Select Operator [SEL_40] (rows=738 width=85) Output:["_col0"] Please refer to the previous Select Operator [SEL_38] - <-Select Operator [SEL_44] (rows=500 width=89) + <-Select Operator [SEL_44] (rows=738 width=89) Output:["_col0","_col1"] - Filter Operator [FIL_43] (rows=500 width=89) + Filter Operator [FIL_43] (rows=738 width=89) predicate:(part is not null and key is not null) - TableScan [TS_0] (rows=500 width=89) + TableScan [TS_0] (rows=738 width=89) default@srcbucket_mapjoin_part_1_n2,a,Tbl:COMPLETE,Col:COMPLETE,Output:["key","part"] PREHOOK: query: SELECT /*+ MAPJOIN(b) */ count(*) diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_5.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_5.q.out index 4334e53c03c4..4b92f571e020 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_5.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_5.q.out @@ -181,16 +181,16 @@ Stage-0 PARTITION_ONLY_SHUFFLE [RS_38] Group By Operator [GBY_37] (rows=1 width=8) Output:["_col0"],aggregations:["count()"] - Map Join Operator [MAPJOIN_36] (rows=261 width=4) + Map Join Operator [MAPJOIN_36] (rows=372 width=8) BucketMapJoin:true,Conds:SEL_35._col0=RS_33._col0(Inner) <-Map 3 [CUSTOM_EDGE] vectorized, llap MULTICAST [RS_33] PartitionCols:_col0 - Select Operator [SEL_32] (rows=1 width=0) + Select Operator [SEL_32] (rows=238 width=4) Output:["_col0"] - Filter Operator [FIL_31] (rows=1 width=84) + Filter Operator [FIL_31] (rows=238 width=89) predicate:((part = '1') and key is not null) - TableScan [TS_3] (rows=1 width=84) + TableScan [TS_3] (rows=238 width=89) default@srcbucket_mapjoin_part_2_n0,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","part"] <-Select Operator [SEL_35] (rows=238 width=4) Output:["_col0"] @@ -248,16 +248,16 @@ Stage-0 PARTITION_ONLY_SHUFFLE [RS_38] Group By Operator [GBY_37] (rows=1 width=8) Output:["_col0"],aggregations:["count()"] - Map Join Operator [MAPJOIN_36] (rows=261 width=4) + Map Join Operator [MAPJOIN_36] (rows=372 width=8) BucketMapJoin:true,Conds:SEL_35._col0=RS_33._col0(Inner) <-Map 3 [CUSTOM_EDGE] vectorized, llap MULTICAST [RS_33] PartitionCols:_col0 - Select Operator [SEL_32] (rows=1 width=0) + Select Operator [SEL_32] (rows=238 width=4) Output:["_col0"] - Filter Operator [FIL_31] (rows=1 width=84) + Filter Operator [FIL_31] (rows=238 width=89) predicate:((part = '1') and key is not null) - TableScan [TS_3] (rows=1 width=84) + TableScan [TS_3] (rows=238 width=89) default@srcbucket_mapjoin_part_3,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","part"] <-Select Operator [SEL_35] (rows=238 width=4) Output:["_col0"] diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_8.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_8.q.out index f4bd02b739cb..20b49faf6f6d 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_8.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_bucket_map_join_8.q.out @@ -102,20 +102,20 @@ Stage-0 Stage-1 Reducer 3 vectorized, llap File Output Operator [FS_37] - Select Operator [SEL_36] (rows=2 width=190) + Select Operator [SEL_36] (rows=4 width=192) Output:["_col0","_col1","_col2","_col3","_col4"] <-Map 2 [SIMPLE_EDGE] vectorized, llap SHUFFLE [RS_35] - Map Join Operator [MAPJOIN_34] (rows=2 width=190) + Map Join Operator [MAPJOIN_34] (rows=4 width=192) Conds:RS_31._col0=SEL_33._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4"] <-Map 1 [BROADCAST_EDGE] vectorized, llap BROADCAST [RS_31] PartitionCols:_col0 - Select Operator [SEL_30] (rows=2 width=97) + Select Operator [SEL_30] (rows=4 width=99) Output:["_col0","_col1","_col2"] - Filter Operator [FIL_29] (rows=2 width=97) + Filter Operator [FIL_29] (rows=4 width=99) predicate:key is not null - TableScan [TS_0] (rows=4 width=73) + TableScan [TS_0] (rows=4 width=99) default@srcbucket_big,a,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value","id"] <-Select Operator [SEL_33] (rows=4 width=93) Output:["_col0","_col1"] @@ -174,11 +174,11 @@ Stage-0 Stage-1 Reducer 2 vectorized, llap File Output Operator [FS_37] - Select Operator [SEL_36] (rows=4 width=190) + Select Operator [SEL_36] (rows=5 width=192) Output:["_col0","_col1","_col2","_col3","_col4"] <-Map 1 [SIMPLE_EDGE] vectorized, llap SHUFFLE [RS_35] - Map Join Operator [MAPJOIN_34] (rows=4 width=190) + Map Join Operator [MAPJOIN_34] (rows=5 width=192) Conds:SEL_33._col1=RS_31._col1(Inner),Output:["_col0","_col1","_col2","_col3","_col4"] <-Map 3 [BROADCAST_EDGE] vectorized, llap BROADCAST [RS_31] @@ -189,12 +189,12 @@ Stage-0 predicate:value is not null TableScan [TS_3] (rows=6 width=77) default@src_small,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"] - <-Select Operator [SEL_33] (rows=4 width=97) + <-Select Operator [SEL_33] (rows=5 width=99) Output:["_col0","_col1","_col2"] - Filter Operator [FIL_32] (rows=4 width=97) + Filter Operator [FIL_32] (rows=5 width=99) predicate:value is not null - TableScan [TS_0] (rows=6 width=81) - default@srcbucket_big,a,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value","id"] + TableScan [TS_0] (rows=6 width=99) + default@srcbucket_big,a,Tbl:COMPLETE,Col:PARTIAL,Output:["key","value","id"] PREHOOK: query: SELECT * FROM srcbucket_big a diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution.q.out index 1497963d94ab..51ec9f5c79a3 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution.q.out @@ -404,7 +404,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: EXTERNAL_TABLE Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"company_id\":\"true\",\"dept_id\":\"true\",\"first_name\":\"true\",\"last_name\":\"true\",\"team_id\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} EXTERNAL TRUE bucketing_version 2 compactor.threshold.target.size 1500 @@ -418,6 +418,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 4 + numPartitions 3 numRows 9 parquet.compression zstd #### A masked pattern was here #### diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution2.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution2.q.out index ae4078e85ce9..1d57fc249965 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution2.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution2.q.out @@ -246,6 +246,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 2 + numPartitions 2 numRows 4 parquet.compression zstd #### A masked pattern was here #### diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution_ordered.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution_ordered.q.out index 43577ea86040..c1e7ef38431d 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution_ordered.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution_ordered.q.out @@ -158,6 +158,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 4 + numPartitions 3 numRows 12 parquet.compression zstd #### A masked pattern was here #### @@ -303,6 +304,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 3 + numPartitions 3 numRows 12 parquet.compression zstd #### A masked pattern was here #### diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution_w_dyn_spec_w_filter.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution_w_dyn_spec_w_filter.q.out index a109d0571613..587bd56bd788 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution_w_dyn_spec_w_filter.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution_w_dyn_spec_w_filter.q.out @@ -253,6 +253,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 13 + numPartitions 8 numRows 26 parquet.compression zstd #### A masked pattern was here #### @@ -357,6 +358,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 8 + numPartitions 8 numRows 26 parquet.compression zstd #### A masked pattern was here #### diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution_w_id_spec_w_filter.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution_w_id_spec_w_filter.q.out index 89c159204387..206e7b223a93 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution_w_id_spec_w_filter.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partition_evolution_w_id_spec_w_filter.q.out @@ -205,6 +205,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 8 + numPartitions 5 numRows 10 parquet.compression zstd #### A masked pattern was here #### @@ -298,7 +299,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: EXTERNAL_TABLE Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"company_id\":\"true\",\"dept_id\":\"true\",\"first_name\":\"true\",\"last_name\":\"true\",\"team_id\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} EXTERNAL TRUE bucketing_version 2 compactor.threshold.target.size 1500 @@ -311,6 +312,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 8 + numPartitions 4 numRows 10 parquet.compression zstd #### A masked pattern was here #### diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partitioned.q.out index b5681f8f45b5..d4c81a698621 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_partitioned.q.out @@ -284,7 +284,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: EXTERNAL_TABLE Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"dept_id\":\"true\",\"first_name\":\"true\",\"last_name\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} EXTERNAL TRUE bucketing_version 2 compactor.threshold.target.size 1500 @@ -297,6 +297,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 2 + numPartitions 2 numRows 4 parquet.compression zstd #### A masked pattern was here #### @@ -618,7 +619,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: EXTERNAL_TABLE Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"dept_id\":\"true\",\"first_name\":\"true\",\"last_name\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} EXTERNAL TRUE bucketing_version 2 compactor.threshold.target.size 1500 @@ -631,6 +632,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 2 + numPartitions 2 numRows 8 parquet.compression zstd #### A masked pattern was here #### diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_schema_evolution.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_schema_evolution.q.out index 1e36ae752139..8ccaabaab9f5 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_schema_evolution.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_schema_evolution.q.out @@ -321,7 +321,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: EXTERNAL_TABLE Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"address\":\"true\",\"dept_id\":\"true\",\"fname\":\"true\",\"last_name\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} EXTERNAL TRUE bucketing_version 2 compactor.threshold.target.size 1500 @@ -334,6 +334,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 2 + numPartitions 2 numRows 5 parquet.compression zstd #### A masked pattern was here #### diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_single_partition.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_single_partition.q.out index 12e1cc05373f..711a0a3ada89 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_single_partition.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_single_partition.q.out @@ -299,7 +299,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: EXTERNAL_TABLE Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"city\":\"true\",\"dept_id\":\"true\",\"first_name\":\"true\",\"last_name\":\"true\",\"registration_date\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} EXTERNAL TRUE bucketing_version 2 compactor.threshold.target.size 1500 @@ -312,6 +312,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 6 + numPartitions 2 numRows 4 parquet.compression zstd #### A masked pattern was here #### @@ -410,7 +411,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: EXTERNAL_TABLE Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"city\":\"true\",\"dept_id\":\"true\",\"first_name\":\"true\",\"last_name\":\"true\",\"registration_date\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} EXTERNAL TRUE bucketing_version 2 compactor.threshold.target.size 1500 @@ -423,6 +424,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 2 + numPartitions 2 numRows 4 parquet.compression zstd #### A masked pattern was here #### diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_single_partition_with_evolution.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_single_partition_with_evolution.q.out index 3908b608bb8e..6ddbc10a80e0 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_single_partition_with_evolution.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_single_partition_with_evolution.q.out @@ -188,6 +188,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 8 + numPartitions 4 numRows 8 parquet.compression zstd #### A masked pattern was here #### @@ -284,7 +285,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: EXTERNAL_TABLE Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"city\":\"true\",\"dept_id\":\"true\",\"first_name\":\"true\",\"last_name\":\"true\",\"registration_date\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} EXTERNAL TRUE bucketing_version 2 compactor.threshold.target.size 1500 @@ -297,6 +298,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 7 + numPartitions 4 numRows 8 parquet.compression zstd #### A masked pattern was here #### @@ -393,7 +395,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: EXTERNAL_TABLE Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"city\":\"true\",\"dept_id\":\"true\",\"first_name\":\"true\",\"last_name\":\"true\",\"registration_date\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} EXTERNAL TRUE bucketing_version 2 compactor.threshold.target.size 1500 @@ -406,6 +408,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 6 + numPartitions 4 numRows 8 parquet.compression zstd #### A masked pattern was here #### @@ -501,7 +504,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: EXTERNAL_TABLE Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"city\":\"true\",\"dept_id\":\"true\",\"first_name\":\"true\",\"last_name\":\"true\",\"registration_date\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} EXTERNAL TRUE bucketing_version 2 compactor.threshold.target.size 1500 @@ -514,6 +517,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 5 + numPartitions 3 numRows 8 parquet.compression zstd #### A masked pattern was here #### @@ -609,7 +613,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: EXTERNAL_TABLE Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"city\":\"true\",\"dept_id\":\"true\",\"first_name\":\"true\",\"last_name\":\"true\",\"registration_date\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} EXTERNAL TRUE bucketing_version 2 compactor.threshold.target.size 1500 @@ -622,6 +626,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 4 + numPartitions 2 numRows 8 parquet.compression zstd #### A masked pattern was here #### diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_single_partition_with_evolution2.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_single_partition_with_evolution2.q.out index acd5e8e861df..abeeaa5dadc1 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_single_partition_with_evolution2.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_single_partition_with_evolution2.q.out @@ -133,6 +133,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 4 + numPartitions 2 numRows 12 parquet.compression zstd #### A masked pattern was here #### @@ -221,6 +222,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 4 + numPartitions 2 numRows 8 parquet.compression zstd #### A masked pattern was here #### @@ -310,7 +312,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: EXTERNAL_TABLE Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"a\":\"true\",\"b\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} EXTERNAL TRUE bucketing_version 2 compactor.threshold.target.size 1500 @@ -323,6 +325,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 3 + numPartitions 2 numRows 8 parquet.compression zstd #### A masked pattern was here #### @@ -412,7 +415,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: EXTERNAL_TABLE Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"a\":\"true\",\"b\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} EXTERNAL TRUE bucketing_version 2 compactor.threshold.target.size 1500 @@ -425,6 +428,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 5 + numPartitions 5 numRows 8 parquet.compression zstd #### A masked pattern was here #### diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_minor_compaction_partition_evolution.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_minor_compaction_partition_evolution.q.out index 7eecfb31129e..2687fd3fb2a2 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_minor_compaction_partition_evolution.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_minor_compaction_partition_evolution.q.out @@ -119,6 +119,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 4 + numPartitions 1 numRows 6 parquet.compression zstd #### A masked pattern was here #### @@ -198,7 +199,8 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 4 - numRows 8 + numPartitions 1 + numRows 6 parquet.compression zstd #### A masked pattern was here #### rawDataSize 0 @@ -287,7 +289,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: EXTERNAL_TABLE Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"dept_id\":\"true\",\"first_name\":\"true\",\"last_name\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} EXTERNAL TRUE bucketing_version 2 compactor.threshold.min.input.files 4 @@ -301,6 +303,7 @@ Table Parameters: iceberg.orc.files.only true #### A masked pattern was here #### numFiles 2 + numPartitions 2 numRows 6 parquet.compression zstd #### A masked pattern was here #### diff --git a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out index 875b6e3ca303..b6e253358ee6 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_partitioned.q.out @@ -132,68 +132,68 @@ STAGE PLANS: TableScan alias: target_ice filterExpr: (a <= 100) (type: boolean) - Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: PARTIAL Filter Operator predicate: (a <= 100) (type: boolean) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: PARTIAL 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: int), c (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 792 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: _col5 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: int) - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 792 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col6 (type: int) Execution mode: vectorized Map 12 Map Operator Tree: TableScan alias: target_ice - Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: PARTIAL Filter Operator predicate: FILE__PATH is not null (type: boolean) - Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: PARTIAL 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: int), b (type: string), c (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 4 Data size: 1932 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 1932 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: _col5 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: int) - Statistics: Num rows: 4 Data size: 1932 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 1932 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col6 (type: string), _col7 (type: int) Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), PARTITION__PROJECTION (type: string), a (type: int), b (type: string), c (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 4 Data size: 1900 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 1900 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: _col4 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col4 (type: int) - Statistics: Num rows: 4 Data size: 1900 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 1900 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: string), _col5 (type: string), _col6 (type: int) Filter Operator predicate: (a is not null and FILE__PATH is not null) (type: boolean) - Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: FILE__PATH (type: string), a (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 4 Data size: 752 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 752 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: _col1 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col1 (type: int) - Statistics: Num rows: 4 Data size: 752 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 752 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: string) Execution mode: vectorized Reducer 10 @@ -202,7 +202,7 @@ STAGE PLANS: Select Operator expressions: VALUE._col1 (type: int), VALUE._col2 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: int) outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 4 Data size: 1900 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 1900 Basic stats: COMPLETE Column stats: PARTIAL PTF Operator Function definitions: Input definition @@ -221,20 +221,20 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 4 Data size: 1900 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 1900 Basic stats: COMPLETE Column stats: PARTIAL Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 2 Data size: 950 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 950 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: _col1 (type: int), _col2 (type: bigint), _col3 (type: string), -1L (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 2 Data size: 966 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 966 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 7 Data size: 3384 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 8 Data size: 3870 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 2 Reduce Operator Tree: @@ -245,17 +245,17 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col5 (type: int) outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 792 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: _col1 (type: int), _col2 (type: bigint), _col3 (type: string), _col4 (type: bigint), _col5 (type: string), _col6 (type: int), 'Merged' (type: string), (_col7 + 10) (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 486 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 972 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 7 Data size: 3384 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 8 Data size: 3870 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 4 Execution mode: vectorized @@ -266,7 +266,7 @@ STAGE PLANS: File Output Operator compressed: false Dp Sort State: PARTITION_SORTED - Statistics: Num rows: 7 Data size: 3384 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 8 Data size: 3870 Basic stats: COMPLETE Column stats: PARTIAL table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -281,20 +281,20 @@ STAGE PLANS: 0 _col5 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 8 Data size: 3884 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 8 Data size: 3884 Basic stats: COMPLETE Column stats: PARTIAL Filter Operator predicate: ((_col5 <> _col8) or _col5 is null or _col8 is null) (type: boolean) - Statistics: Num rows: 8 Data size: 3884 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 8 Data size: 3884 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 8 Data size: 3864 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 8 Data size: 3864 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: _col2 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 8 Data size: 3864 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 8 Data size: 3864 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: bigint), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 6 Reduce Operator Tree: @@ -305,13 +305,13 @@ STAGE PLANS: 0 _col2 (type: string) 1 _col0 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 4 Data size: 1932 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 1932 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 7 Data size: 3384 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 8 Data size: 3870 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 7 Reduce Operator Tree: @@ -322,20 +322,20 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col1 (type: int) outputColumnNames: _col1 - Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: _col1 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: PARTIAL Reducer 8 Execution mode: vectorized Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: string) outputColumnNames: _col1 - Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: PARTIAL PTF Operator Function definitions: Input definition @@ -355,26 +355,26 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: PARTIAL Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: _col1 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: PARTIAL Group By Operator keys: _col0 (type: string) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: _col0 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: PARTIAL Reducer 9 Reduce Operator Tree: Merge Join Operator @@ -384,13 +384,13 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col4 (type: int) outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 4 Data size: 1900 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 1900 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: _col3 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col3 (type: string) - Statistics: Num rows: 4 Data size: 1900 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 1900 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col1 (type: int), _col2 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Union 3 Vertex: Union 3 @@ -470,17 +470,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: target_ice - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE 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: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 3 Data size: 1176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: int) - Statistics: Num rows: 3 Data size: 1176 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string) Execution mode: vectorized Map 4 @@ -509,20 +509,20 @@ STAGE PLANS: 0 _col5 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 12 Data size: 1952 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 633 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: _col5 is null (type: boolean) - Statistics: Num rows: 9 Data size: 1464 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 316 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col6 (type: int), _col7 (type: string), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 9 Data size: 1452 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 316 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 9 Data size: 1452 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 316 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 3 Execution mode: vectorized @@ -533,7 +533,7 @@ STAGE PLANS: File Output Operator compressed: false Dp Sort State: PARTITION_SORTED - Statistics: Num rows: 9 Data size: 1452 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 316 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat diff --git a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out index 24a47f458777..0584c5fc95e2 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out @@ -860,59 +860,59 @@ STAGE PLANS: Map Operator Tree: TableScan alias: target_ice - Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1152 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: FILE__PATH is not null (type: boolean) - Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1152 Basic stats: COMPLETE Column stats: NONE 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: int), b (type: string), c (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 6 Data size: 2898 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1152 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: int) - Statistics: Num rows: 6 Data size: 2898 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1152 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col6 (type: string), _col7 (type: int) Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1152 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), PARTITION__PROJECTION (type: string), a (type: int), b (type: string), c (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 6 Data size: 2850 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1152 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col4 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col4 (type: int) - Statistics: Num rows: 6 Data size: 2850 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1152 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: string), _col5 (type: string), _col6 (type: int) 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: int), c (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 6 Data size: 2376 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: int) - Statistics: Num rows: 6 Data size: 2376 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col6 (type: int) Filter Operator predicate: (a is not null and FILE__PATH is not null) (type: boolean) - Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: FILE__PATH (type: string), a (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 6 Data size: 1128 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col1 (type: int) - Statistics: Num rows: 6 Data size: 1128 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Execution mode: vectorized Map 10 @@ -962,20 +962,20 @@ STAGE PLANS: 0 _col5 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 12 Data size: 5824 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1267 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: ((_col5 <> _col8) or _col5 is null or _col8 is null) (type: boolean) - Statistics: Num rows: 12 Data size: 5824 Basic stats: COMPLETE Column stats: COMPLETE + predicate: (_col5 is null or (_col5 <> _col8) or _col8 is null) (type: boolean) + Statistics: Num rows: 6 Data size: 1267 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 12 Data size: 5796 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1267 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 12 Data size: 5796 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1267 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 3 Reduce Operator Tree: @@ -986,10 +986,10 @@ STAGE PLANS: 0 _col2 (type: string) 1 _col0 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 6 Data size: 2898 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1393 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 15 Data size: 7263 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 15 Data size: 2078 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1004,13 +1004,13 @@ STAGE PLANS: 0 _col4 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 6 Data size: 2850 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1267 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 6 Data size: 2850 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1267 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col3 (type: string), _col4 (type: int), _col5 (type: string), _col6 (type: int) Reducer 6 Execution mode: vectorized @@ -1018,7 +1018,7 @@ STAGE PLANS: Select Operator expressions: VALUE._col0 (type: int), VALUE._col1 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 6 Data size: 2850 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1267 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -1037,17 +1037,17 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 6 Data size: 2850 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 1267 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 3 Data size: 1425 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 633 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), -1L (type: bigint), _col3 (type: string), _col4 (type: int), _col5 (type: string), _col6 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 3 Data size: 1449 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 633 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 15 Data size: 7263 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 15 Data size: 2078 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1062,14 +1062,14 @@ STAGE PLANS: 0 _col5 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 6 Data size: 2376 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 52 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), 'Merged' (type: string), (_col6 - 10) (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 6 Data size: 2916 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 52 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 15 Data size: 7263 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 15 Data size: 2078 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1084,20 +1084,20 @@ STAGE PLANS: 0 _col1 (type: int) 1 _col0 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 26 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 26 Basic stats: COMPLETE Column stats: NONE Reducer 9 Execution mode: vectorized Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 26 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -1117,26 +1117,26 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 26 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: string) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE Union 4 Vertex: Union 4 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 6a4c03088686..693513fa2dfc 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 @@ -96,20 +96,20 @@ STAGE PLANS: TableScan alias: target_ice filterExpr: a is not null (type: boolean) - Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: PARTIAL Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: PARTIAL 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: int), b (type: string), c (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 4 Data size: 1932 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 1932 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: _col5 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: int) - Statistics: Num rows: 4 Data size: 1932 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 1932 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col6 (type: string), _col7 (type: int) Execution mode: vectorized Reducer 2 @@ -121,21 +121,21 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col5 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 - Statistics: Num rows: 10 Data size: 3375 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 10 Data size: 3375 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: _col1 (type: string), _col0 (type: int), _col5 (type: string), _col7 (type: string), _col2 (type: int), _col6 (type: bigint), _col4 (type: bigint), _col3 (type: int), _col10 (type: int), _col9 (type: string), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 - Statistics: Num rows: 10 Data size: 3375 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 10 Data size: 3375 Basic stats: COMPLETE Column stats: PARTIAL Filter Operator predicate: ((_col10 = _col1) and (_col10 > 100)) (type: boolean) - Statistics: Num rows: 4 Data size: 1833 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 1833 Basic stats: COMPLETE Column stats: PARTIAL Select Operator 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 + Statistics: Num rows: 4 Data size: 1449 Basic stats: COMPLETE Column stats: PARTIAL File Output Operator compressed: false - Statistics: Num rows: 4 Data size: 1449 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 1449 Basic stats: COMPLETE Column stats: PARTIAL table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -143,14 +143,14 @@ STAGE PLANS: 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 + Statistics: Num rows: 1 Data size: 579 Basic stats: COMPLETE Column stats: PARTIAL Select Operator 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 + Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: PARTIAL File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: PARTIAL table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -158,52 +158,52 @@ STAGE PLANS: 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 + Statistics: Num rows: 1 Data size: 579 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: _col10 (type: int), 'Merged' (type: string), (_col8 + 10) (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 98 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 98 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: iceberg_bucket(_col0, 16) (type: int), iceberg_truncate(_col1, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col0, 16) (type: int), iceberg_truncate(_col1, 3) (type: string) - Statistics: Num rows: 1 Data size: 98 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 98 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int) Filter Operator predicate: _col10 is null (type: boolean) - Statistics: Num rows: 6 Data size: 2025 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 2025 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: _col1 (type: int), _col0 (type: string), _col4 (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: iceberg_bucket(_col0, 16) (type: int), iceberg_truncate(_col1, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col0, 16) (type: int), iceberg_truncate(_col1, 3) (type: string) - Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int) Filter Operator predicate: (_col10 = _col1) (type: boolean) - Statistics: Num rows: 5 Data size: 1929 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 1929 Basic stats: COMPLETE Column stats: PARTIAL Select Operator 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 + Statistics: Num rows: 5 Data size: 1929 Basic stats: COMPLETE Column stats: PARTIAL Group By Operator aggregations: count() keys: _col7 (type: int), _col6 (type: bigint), _col2 (type: string), _col5 (type: bigint) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator 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 + Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col4 (type: bigint) Reducer 3 Execution mode: vectorized @@ -214,7 +214,7 @@ STAGE PLANS: File Output Operator compressed: false Dp Sort State: PARTITION_SORTED - Statistics: Num rows: 1 Data size: 98 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 98 Basic stats: COMPLETE Column stats: PARTIAL table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -229,7 +229,7 @@ STAGE PLANS: File Output Operator compressed: false Dp Sort State: PARTITION_SORTED - Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: PARTIAL table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -243,17 +243,17 @@ STAGE PLANS: 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 - Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: PARTIAL Filter Operator predicate: (_col4 > 1L) (type: boolean) - Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: cardinality_violation(_col0,_col1,_col2,_col3) (type: int) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: PARTIAL File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: PARTIAL table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat diff --git a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc.q.out b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc.q.out index 6b15c0cd2d78..7c274a85a948 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc.q.out @@ -61,7 +61,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: MATERIALIZED_VIEW Table Parameters: - COLUMN_STATS_ACCURATE {\"COLUMN_STATS\":{\"b\":\"true\",\"c\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"c\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"}]} current-snapshot-id #SnapshotId# @@ -72,9 +72,11 @@ Table Parameters: iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### numFiles 2 + numPartitions 2 numRows 2 parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### + rawDataSize 0 snapshot-count 1 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler table_type ICEBERG @@ -146,7 +148,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: MATERIALIZED_VIEW Table Parameters: - COLUMN_STATS_ACCURATE {\"COLUMN_STATS\":{\"b\":\"true\",\"c\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"c\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"b\",\"required\":false,\"type\":\"string\"}]} current-snapshot-id #SnapshotId# @@ -157,9 +159,11 @@ Table Parameters: iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### numFiles 2 + numPartitions 2 numRows 2 parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### + rawDataSize 0 snapshot-count 1 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler table_type ICEBERG diff --git a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc2.q.out b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc2.q.out index b7a90b13b2d1..94b707fe73f7 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc2.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/mv_iceberg_partitioned_orc2.q.out @@ -62,7 +62,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: MATERIALIZED_VIEW Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"b\":\"true\",\"c\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"b\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"c\",\"required\":false,\"type\":\"int\"}]} current-snapshot-id #SnapshotId# @@ -73,9 +73,11 @@ Table Parameters: iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### numFiles 2 + numPartitions 2 numRows 2 parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### + rawDataSize 0 snapshot-count 1 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler table_type ICEBERG @@ -148,7 +150,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: MATERIALIZED_VIEW Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"b\":\"true\",\"c\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"b\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"c\",\"required\":false,\"type\":\"int\"}]} current-snapshot-id #SnapshotId# @@ -159,9 +161,11 @@ Table Parameters: iceberg.orc.files.only true metadata_location hdfs://### HDFS PATH ### numFiles 2 + numPartitions 2 numRows 2 parquet.compression zstd previous_metadata_location hdfs://### HDFS PATH ### + rawDataSize 0 snapshot-count 1 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler table_type ICEBERG diff --git a/iceberg/iceberg-handler/src/test/results/positive/show_iceberg_materialized_views.q.out b/iceberg/iceberg-handler/src/test/results/positive/show_iceberg_materialized_views.q.out index b2f9fa0fef3d..dd9c7561b7fb 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/show_iceberg_materialized_views.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/show_iceberg_materialized_views.q.out @@ -362,7 +362,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: MATERIALIZED_VIEW Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"ds\":\"true\",\"key\":\"true\",\"value\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"key\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"value\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ds\",\"required\":false,\"type\":\"string\"}]} format-version 1 @@ -414,7 +414,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: MATERIALIZED_VIEW Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"ds\":\"true\",\"key\":\"true\",\"value\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"key\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"value\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ds\",\"required\":false,\"type\":\"string\"}]} format-version 1 @@ -465,7 +465,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: MATERIALIZED_VIEW Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"ds\":\"true\",\"key\":\"true\",\"value\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"key\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"value\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"ds\",\"required\":false,\"type\":\"string\"}]} format-version 1 @@ -570,7 +570,7 @@ Retention: 0 #### A masked pattern was here #### Table Type: MATERIALIZED_VIEW Table Parameters: - COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"a\":\"true\"}} + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"a\",\"required\":false,\"type\":\"int\"}]} format-version 1 diff --git a/iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out b/iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out index 5505f6151c31..c3c309b656aa 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/show_partitions_test.q.out @@ -6,6 +6,51 @@ POSTHOOK: query: create table hiveT1 (a string, b int, c int) PARTITIONED BY (d_ POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hiveT1 +PREHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@ice1 +POSTHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2') +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@ice1 +PREHOOK: query: show partitions hiveT1 +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@hivet1 +POSTHOOK: query: show partitions hiveT1 +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@hivet1 +PREHOOK: query: describe default.ice1.partitions +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@ice1 +POSTHOOK: query: describe default.ice1.partitions +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@ice1 +partition struct +spec_id int +record_count bigint Count of records in data files +file_count int Count of data files +total_data_file_size_in_bytes bigint Total size in bytes of data files +position_delete_record_count bigint Count of records in position delete files +position_delete_file_count int Count of position delete files +equality_delete_record_count bigint Count of records in equality delete files +equality_delete_file_count int Count of equality delete files +last_updated_at timestamp with local time zone Commit time of snapshot that last updated this partition +last_updated_snapshot_id bigint Id of snapshot that last updated this partition +PREHOOK: query: select * from default.ice1.partitions +PREHOOK: type: QUERY +PREHOOK: Input: default@ice1 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from default.ice1.partitions +POSTHOOK: type: QUERY +POSTHOOK: Input: default@ice1 +POSTHOOK: Output: hdfs://### HDFS PATH ### +PREHOOK: query: show partitions ice1 +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@ice1 +POSTHOOK: query: show partitions ice1 +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@ice1 PREHOOK: query: insert into hiveT1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5) PREHOOK: type: QUERY PREHOOK: Input: _dummy_database@_dummy_table @@ -26,14 +71,6 @@ POSTHOOK: Lineage: hivet1 PARTITION(d_part=2,e_part=5).c SCRIPT [] POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).a SCRIPT [] POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).b SCRIPT [] POSTHOOK: Lineage: hivet1 PARTITION(d_part=3,e_part=4).c SCRIPT [] -PREHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2') -PREHOOK: type: CREATETABLE -PREHOOK: Output: database:default -PREHOOK: Output: default@ice1 -POSTHOOK: query: create table ice1 (a string, b int, c int) PARTITIONED BY (d_part int, e_part int) stored by iceberg stored as orc TBLPROPERTIES("format-version"='2') -POSTHOOK: type: CREATETABLE -POSTHOOK: Output: database:default -POSTHOOK: Output: default@ice1 PREHOOK: query: insert into ice1 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 2, 2, 5), ('aa', 1, 2, 10, 5), ('aa', 1, 2, 10, 5) PREHOOK: type: QUERY PREHOOK: Input: _dummy_database@_dummy_table @@ -51,28 +88,11 @@ POSTHOOK: Input: default@hivet1 d_part=10/e_part=5 d_part=2/e_part=5 d_part=3/e_part=4 -PREHOOK: query: describe default.ice1.partitions -PREHOOK: type: DESCTABLE -PREHOOK: Input: default@ice1 -POSTHOOK: query: describe default.ice1.partitions -POSTHOOK: type: DESCTABLE -POSTHOOK: Input: default@ice1 -partition struct -spec_id int -record_count bigint Count of records in data files -file_count int Count of data files -total_data_file_size_in_bytes bigint Total size in bytes of data files -position_delete_record_count bigint Count of records in position delete files -position_delete_file_count int Count of position delete files -equality_delete_record_count bigint Count of records in equality delete files -equality_delete_file_count int Count of equality delete files -last_updated_at timestamp with local time zone Commit time of snapshot that last updated this partition -last_updated_snapshot_id bigint Id of snapshot that last updated this partition -PREHOOK: query: select `partition` from default.ice1.partitions order by `partition` +PREHOOK: query: select `partition` from default.ice1.partitions PREHOOK: type: QUERY PREHOOK: Input: default@ice1 PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select `partition` from default.ice1.partitions order by `partition` +POSTHOOK: query: select `partition` from default.ice1.partitions POSTHOOK: type: QUERY POSTHOOK: Input: default@ice1 POSTHOOK: Output: hdfs://### HDFS PATH ### @@ -85,9 +105,9 @@ PREHOOK: Input: default@ice1 POSTHOOK: query: show partitions ice1 POSTHOOK: type: SHOWPARTITIONS POSTHOOK: Input: default@ice1 -current-spec-id=0/d_part=10/e_part=5 -current-spec-id=0/d_part=2/e_part=5 -current-spec-id=0/d_part=3/e_part=4 +d_part=10/e_part=5 +d_part=2/e_part=5 +d_part=3/e_part=4 PREHOOK: query: explain show partitions hiveT1 PREHOOK: type: SHOWPARTITIONS PREHOOK: Input: default@hivet1 @@ -148,11 +168,11 @@ POSTHOOK: query: insert into ice2 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4) POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@ice2 -PREHOOK: query: select `partition` from default.ice2.partitions order by `partition` +PREHOOK: query: select `partition` from default.ice2.partitions PREHOOK: type: QUERY PREHOOK: Input: default@ice2 PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select `partition` from default.ice2.partitions order by `partition` +POSTHOOK: query: select `partition` from default.ice2.partitions POSTHOOK: type: QUERY POSTHOOK: Input: default@ice2 POSTHOOK: Output: hdfs://### HDFS PATH ### @@ -165,9 +185,9 @@ PREHOOK: Input: default@ice2 POSTHOOK: query: show partitions ice2 POSTHOOK: type: SHOWPARTITIONS POSTHOOK: Input: default@ice2 -current-spec-id=0/d_part=10/e_part=5 -current-spec-id=0/d_part=2/e_part=5 -current-spec-id=0/d_part=3/e_part=4 +d_part=10/e_part=5 +d_part=2/e_part=5 +d_part=3/e_part=4 PREHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c) PREHOOK: type: ALTERTABLE_SETPARTSPEC PREHOOK: Input: default@ice2 @@ -175,11 +195,11 @@ POSTHOOK: query: ALTER TABLE ice2 SET PARTITION SPEC (c) POSTHOOK: type: ALTERTABLE_SETPARTSPEC POSTHOOK: Input: default@ice2 POSTHOOK: Output: default@ice2 -PREHOOK: query: select `partition` from default.ice2.partitions order by `partition` +PREHOOK: query: select `partition` from default.ice2.partitions PREHOOK: type: QUERY PREHOOK: Input: default@ice2 PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select `partition` from default.ice2.partitions order by `partition` +POSTHOOK: query: select `partition` from default.ice2.partitions POSTHOOK: type: QUERY POSTHOOK: Input: default@ice2 POSTHOOK: Output: hdfs://### HDFS PATH ### @@ -192,9 +212,9 @@ PREHOOK: Input: default@ice2 POSTHOOK: query: show partitions ice2 POSTHOOK: type: SHOWPARTITIONS POSTHOOK: Input: default@ice2 -spec-id=0/d_part=10/e_part=5 -spec-id=0/d_part=2/e_part=5 -spec-id=0/d_part=3/e_part=4 +d_part=10/e_part=5 +d_part=2/e_part=5 +d_part=3/e_part=4 PREHOOK: query: insert into ice2 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4), ('aa', 1, 3, 2, 5), ('aa', 1, 4, 10, 5), ('aa', 1, 5, 10, 5) PREHOOK: type: QUERY @@ -205,11 +225,11 @@ POSTHOOK: query: insert into ice2 values ('aa', 1, 2, 3, 4), ('aa', 1, 2, 3, 4) POSTHOOK: type: QUERY POSTHOOK: Input: _dummy_database@_dummy_table POSTHOOK: Output: default@ice2 -PREHOOK: query: select `partition` from default.ice2.partitions order by `partition` +PREHOOK: query: select `partition` from default.ice2.partitions PREHOOK: type: QUERY PREHOOK: Input: default@ice2 PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: select `partition` from default.ice2.partitions order by `partition` +POSTHOOK: query: select `partition` from default.ice2.partitions POSTHOOK: type: QUERY POSTHOOK: Input: default@ice2 POSTHOOK: Output: hdfs://### HDFS PATH ### @@ -226,10 +246,10 @@ PREHOOK: Input: default@ice2 POSTHOOK: query: show partitions ice2 POSTHOOK: type: SHOWPARTITIONS POSTHOOK: Input: default@ice2 -current-spec-id=1/c=2 -current-spec-id=1/c=3 -current-spec-id=1/c=4 -current-spec-id=1/c=5 -spec-id=0/d_part=10/e_part=5 -spec-id=0/d_part=2/e_part=5 -spec-id=0/d_part=3/e_part=4 +c=2 +c=3 +c=4 +c=5 +d_part=10/e_part=5 +d_part=2/e_part=5 +d_part=3/e_part=4 diff --git a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out index 3254ae32eaf5..e1d97e48b33e 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_partitioned.q.out @@ -51,20 +51,20 @@ STAGE PLANS: TableScan alias: tbl_ice filterExpr: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) - Statistics: Num rows: 3 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 288 Basic stats: COMPLETE Column stats: PARTIAL Filter Operator predicate: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) - Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 288 Basic stats: COMPLETE Column stats: PARTIAL 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: int), 'Changed' (type: string), c (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 487 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 1461 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 3 Data size: 1443 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 2417 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Execution mode: vectorized Map 4 @@ -72,20 +72,20 @@ STAGE PLANS: TableScan alias: tbl_ice filterExpr: (((b <> 'one') and (b <> 'four') and (a <> 22)) or ((a = 22) or (b) IN ('one', 'four')) is null) (type: boolean) - Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: PARTIAL Filter Operator predicate: ((((b <> 'one') and (b <> 'four') and (a <> 22)) or ((a = 22) or (b) IN ('one', 'four')) is null) and FILE__PATH is not null) (type: boolean) - Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: a (type: int), b (type: string), c (type: int), 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, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: _col5 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint), _col7 (type: string) Execution mode: vectorized Map 6 @@ -93,26 +93,26 @@ STAGE PLANS: TableScan alias: tbl_ice filterExpr: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) - Statistics: Num rows: 3 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 288 Basic stats: COMPLETE Column stats: PARTIAL Filter Operator predicate: ((a = 22) or (b) IN ('one', 'four')) (type: boolean) - Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 288 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: FILE__PATH (type: string) null sort order: a sort order: + Map-reduce partition columns: FILE__PATH (type: string) - Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 288 Basic stats: COMPLETE Column stats: PARTIAL value expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), PARTITION__PROJECTION (type: string) Filter Operator predicate: (((b) IN ('one', 'four') or (a = 22)) and FILE__PATH is not null) (type: boolean) - Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 276 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: FILE__PATH (type: string) null sort order: a sort order: + Map-reduce partition columns: FILE__PATH (type: string) - Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 276 Basic stats: COMPLETE Column stats: PARTIAL Execution mode: vectorized Reducer 3 Execution mode: vectorized @@ -123,7 +123,7 @@ STAGE PLANS: File Output Operator compressed: false Dp Sort State: PARTITION_SORTED - Statistics: Num rows: 3 Data size: 1443 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 2417 Basic stats: COMPLETE Column stats: PARTIAL table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -138,17 +138,17 @@ STAGE PLANS: 0 _col5 (type: string) 1 _col0 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col0 (type: int), _col1 (type: string), _col2 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 3 Data size: 1443 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 2417 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 7 Execution mode: vectorized @@ -156,7 +156,7 @@ STAGE PLANS: Select Operator expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col6 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col7 - Statistics: Num rows: 1 Data size: 476 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 1428 Basic stats: COMPLETE Column stats: PARTIAL PTF Operator Function definitions: Input definition @@ -175,20 +175,20 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 1 Data size: 476 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 1428 Basic stats: COMPLETE Column stats: PARTIAL Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 476 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 476 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col7 (type: string), _col0 (type: int), _col1 (type: string), _col2 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 484 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 484 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 3 Data size: 1443 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 2417 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 8 Execution mode: vectorized @@ -196,7 +196,7 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: string) outputColumnNames: _col5 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: PARTIAL PTF Operator Function definitions: Input definition @@ -216,26 +216,26 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: PARTIAL Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: _col5 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: PARTIAL Group By Operator keys: _col0 (type: string) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: _col0 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: PARTIAL Union 2 Vertex: Union 2 @@ -347,135 +347,135 @@ STAGE PLANS: TableScan alias: tbl_ice filterExpr: (a <= 5) (type: boolean) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (a <= 5) (type: boolean) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int) outputColumnNames: a - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: a (type: int) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(), count(a) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Select Operator - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Execution mode: vectorized Map 26 Map Operator Tree: TableScan alias: tbl_ice - Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 891 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), PARTITION__PROJECTION (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 9 Data size: 4293 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4311 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 9 Data size: 4293 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4311 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: string) Filter Operator predicate: FILE__PATH is not null (type: boolean) - Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 891 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: a (type: int), b (type: string), c (type: int), 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, _col5, _col6, _col7 - Statistics: Num rows: 9 Data size: 4365 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4383 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 4365 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4383 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) Select Operator expressions: a (type: int), c (type: int), FILE__PATH (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) Select Operator expressions: a (type: int), c (type: int), 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, _col5, _col6 - Statistics: Num rows: 9 Data size: 3564 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 3564 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 9 Data size: 3564 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 3564 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: string) Filter Operator predicate: (c > 800) (type: boolean) - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: PARTIAL Select Operator - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: PARTIAL Group By Operator aggregations: count() - minReductionHashAggr: 0.8888889 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: bigint) Group By Operator keys: c (type: int) - minReductionHashAggr: 0.7777778 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: PARTIAL Group By Operator aggregations: count(), count(c) - minReductionHashAggr: 0.8888889 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: bigint), _col1 (type: bigint) Execution mode: vectorized Reducer 10 @@ -487,11 +487,11 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9 - Statistics: Num rows: 9 Data size: 3660 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4049 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 9 Data size: 3660 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4049 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: string), _col7 (type: bigint), _col9 (type: boolean) Reducer 11 Reduce Operator Tree: @@ -502,13 +502,13 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9, _col10 - Statistics: Num rows: 9 Data size: 3732 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4130 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col1 (type: int) - Statistics: Num rows: 9 Data size: 3732 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4130 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: string), _col7 (type: bigint), _col9 (type: boolean), _col10 (type: bigint) Reducer 12 Reduce Operator Tree: @@ -519,20 +519,20 @@ STAGE PLANS: 0 _col1 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9, _col10, _col12 - Statistics: Num rows: 9 Data size: 3768 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4543 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((_col7 <> 0L) and _col9 is not null) or ((_col10 <> 0L) and _col12 is not null)) (type: boolean) - Statistics: Num rows: 9 Data size: 3768 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4543 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: string), _col0 (type: int), 'Changed again' (type: string), _col1 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 9 Data size: 4437 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4543 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 19 Data size: 9287 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 19 Data size: 11116 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 13 Reduce Operator Tree: @@ -543,11 +543,11 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9 - Statistics: Num rows: 9 Data size: 4389 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4821 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 9 Data size: 4389 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4821 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col9 (type: boolean) Reducer 14 Reduce Operator Tree: @@ -558,17 +558,17 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9, _col10, _col11 - Statistics: Num rows: 9 Data size: 4533 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 5010 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col10 (type: bigint), _col11 (type: bigint), _col9 (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col11 - Statistics: Num rows: 9 Data size: 4533 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 5010 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col2 (type: int) - Statistics: Num rows: 9 Data size: 4533 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 5010 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col8 (type: bigint), _col9 (type: bigint), _col11 (type: boolean) Reducer 15 Reduce Operator Tree: @@ -579,11 +579,11 @@ STAGE PLANS: 0 _col2 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col11, _col13 - Statistics: Num rows: 9 Data size: 4569 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 5511 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 9 Data size: 4569 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 5511 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col8 (type: bigint), _col9 (type: bigint), _col11 (type: boolean), _col13 (type: boolean) Reducer 16 Reduce Operator Tree: @@ -594,24 +594,24 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col11, _col13, _col14, _col15 - Statistics: Num rows: 9 Data size: 4713 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 5664 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col8 (type: bigint), _col9 (type: bigint), _col11 (type: boolean), _col14 (type: bigint), _col15 (type: bigint), _col13 (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col11, _col12, _col13, _col15 - Statistics: Num rows: 9 Data size: 4713 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 5664 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((_col11 is not null and (_col8 <> 0L)) or ((_col0 is null or (_col9 < _col8)) and null and (_col8 <> 0L) and _col11 is null) or (_col15 is not null and (_col12 <> 0L)) or ((_col2 is null or (_col13 < _col12)) and null and (_col12 <> 0L) and _col15 is null)) is null or (((_col8 = 0L) or (_col11 is null and (_col9 >= _col8) and _col0 is not null)) and ((_col12 = 0L) or (_col15 is null and (_col13 >= _col12) and _col2 is not null)))) (type: boolean) - Statistics: Num rows: 9 Data size: 4713 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3776 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 9 Data size: 4365 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3776 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 9 Data size: 4365 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 3776 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint), _col7 (type: string) Reducer 17 Reduce Operator Tree: @@ -622,11 +622,11 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9 - Statistics: Num rows: 9 Data size: 4389 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4870 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 9 Data size: 4389 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4870 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: string), _col7 (type: bigint), _col9 (type: boolean) Reducer 18 Reduce Operator Tree: @@ -637,13 +637,13 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9, _col10 - Statistics: Num rows: 9 Data size: 4461 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4951 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col2 (type: int) - Statistics: Num rows: 9 Data size: 4461 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4951 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: string), _col7 (type: bigint), _col9 (type: boolean), _col10 (type: bigint) Reducer 19 Reduce Operator Tree: @@ -654,16 +654,16 @@ STAGE PLANS: 0 _col2 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9, _col10, _col12 - Statistics: Num rows: 9 Data size: 4497 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 5446 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((_col7 <> 0L) and _col9 is not null) or ((_col10 <> 0L) and _col12 is not null)) (type: boolean) - Statistics: Num rows: 9 Data size: 4497 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 5446 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 9 Data size: 4497 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 5446 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: string) Reducer 2 Execution mode: vectorized @@ -672,38 +672,38 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), true (type: boolean) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean) Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean) Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean) Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean) Reducer 20 Execution mode: vectorized @@ -711,7 +711,7 @@ STAGE PLANS: Select Operator expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col5 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 9 Data size: 4293 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 5446 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -730,20 +730,20 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 9 Data size: 4293 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 5446 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 4 Data size: 1908 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 2420 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col6 (type: string), _col0 (type: int), _col1 (type: string), _col2 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 4 Data size: 1940 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 2420 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 19 Data size: 9287 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 19 Data size: 11116 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 21 Execution mode: vectorized @@ -752,11 +752,11 @@ STAGE PLANS: aggregations: count(VALUE._col0), count(VALUE._col1) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 22 Execution mode: vectorized @@ -765,21 +765,21 @@ STAGE PLANS: aggregations: count(VALUE._col0) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint) Reducer 23 Reduce Operator Tree: @@ -790,13 +790,13 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 9 Data size: 1800 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1845 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 1800 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1845 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint) Reducer 24 Reduce Operator Tree: @@ -807,13 +807,13 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 9 Data size: 4365 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4428 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 4365 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 4428 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: string), _col7 (type: bigint) Reducer 25 Reduce Operator Tree: @@ -824,13 +824,13 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 9 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 3681 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 3681 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: string), _col7 (type: bigint) Reducer 27 Execution mode: vectorized @@ -839,21 +839,21 @@ STAGE PLANS: aggregations: count(VALUE._col0) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: bigint) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: bigint) Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: bigint) Reducer 28 Execution mode: vectorized @@ -862,38 +862,38 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: PARTIAL Select Operator expressions: _col0 (type: int), true (type: boolean) outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col1 (type: boolean) Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col1 (type: boolean) Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col1 (type: boolean) Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col1 (type: boolean) Reducer 29 Execution mode: vectorized @@ -902,11 +902,11 @@ STAGE PLANS: aggregations: count(VALUE._col0), count(VALUE._col1) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: PARTIAL Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: PARTIAL value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 3 Reduce Operator Tree: @@ -917,11 +917,11 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col1, _col2, _col3, _col5 - Statistics: Num rows: 13 Data size: 2568 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2029 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 13 Data size: 2568 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2029 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean) Reducer 4 Reduce Operator Tree: @@ -932,13 +932,13 @@ STAGE PLANS: 0 1 outputColumnNames: _col1, _col2, _col3, _col5, _col6 - Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2110 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col1 (type: int) - Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2110 Basic stats: COMPLETE Column stats: NONE value expressions: _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean), _col6 (type: bigint) Reducer 5 Reduce Operator Tree: @@ -949,23 +949,23 @@ STAGE PLANS: 0 _col1 (type: int) 1 _col0 (type: int) outputColumnNames: _col2, _col3, _col5, _col6, _col8 - Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2321 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((_col3 <> 0L) and _col5 is not null) or ((_col6 <> 0L) and _col8 is not null)) (type: boolean) - Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2321 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2321 Basic stats: COMPLETE Column stats: NONE Reducer 6 Execution mode: vectorized Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: string) outputColumnNames: _col2 - Statistics: Num rows: 13 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2321 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -985,26 +985,26 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 13 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2321 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 1031 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 1031 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: string) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 1031 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 1031 Basic stats: COMPLETE Column stats: NONE Reducer 7 Reduce Operator Tree: Merge Join Operator @@ -1014,17 +1014,17 @@ STAGE PLANS: 0 _col5 (type: string) 1 _col0 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 6 Data size: 2910 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 4153 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col0 (type: int), _col1 (type: string), _col2 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 6 Data size: 2910 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 4153 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 19 Data size: 9287 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 19 Data size: 11116 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 9 Execution mode: vectorized @@ -1035,7 +1035,7 @@ STAGE PLANS: File Output Operator compressed: false Dp Sort State: PARTITION_SORTED - Statistics: Num rows: 19 Data size: 9287 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 19 Data size: 11116 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1156,69 +1156,69 @@ STAGE PLANS: Map Operator Tree: TableScan alias: tbl_ice - Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: FILE__PATH is not null (type: boolean) - Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), b (type: string), c (type: int), 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, _col5, _col6, _col7 - Statistics: Num rows: 9 Data size: 4365 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 4365 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), c (type: int), 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, _col5, _col6 - Statistics: Num rows: 9 Data size: 3564 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 3564 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: string) Select Operator expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), PARTITION__PROJECTION (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 9 Data size: 4293 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 4293 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: string) Select Operator expressions: a (type: int) outputColumnNames: _col0 - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (a is not null and FILE__PATH is not null) (type: boolean) - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), FILE__PATH (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 9 Data size: 1692 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 1692 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Execution mode: vectorized Map 15 @@ -1250,47 +1250,47 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: int) - minReductionHashAggr: 0.5 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(), count(_col0) - minReductionHashAggr: 0.5 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 11 Execution mode: vectorized @@ -1299,17 +1299,17 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), true (type: boolean) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean) Reducer 12 Reduce Operator Tree: @@ -1320,20 +1320,20 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col1 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reducer 13 Execution mode: vectorized Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: string) outputColumnNames: _col1 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -1353,26 +1353,26 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: string) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Reducer 14 Execution mode: vectorized Reduce Operator Tree: @@ -1380,11 +1380,11 @@ STAGE PLANS: aggregations: count(VALUE._col0), count(VALUE._col1) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 2 Reduce Operator Tree: @@ -1395,11 +1395,11 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9 - Statistics: Num rows: 10 Data size: 4858 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 10 Data size: 4858 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col9 (type: boolean) Reducer 3 Reduce Operator Tree: @@ -1410,24 +1410,24 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9, _col10, _col11 - Statistics: Num rows: 10 Data size: 5018 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col10 (type: bigint), _col11 (type: bigint), _col9 (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col11 - Statistics: Num rows: 10 Data size: 5018 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: ((_col8 = 0L) or (_col11 is null and (_col9 >= _col8) and _col0 is not null) or ((_col11 is not null and (_col8 <> 0L)) or ((_col0 is null or (_col9 < _col8)) and null and (_col8 <> 0L) and _col11 is null)) is null) (type: boolean) - Statistics: Num rows: 10 Data size: 5018 Basic stats: COMPLETE Column stats: COMPLETE + predicate: ((_col8 = 0L) or ((_col11 is not null and (_col8 <> 0L)) or ((_col0 is null or (_col9 < _col8)) and null and (_col8 <> 0L) and _col11 is null)) is null or (_col11 is null and (_col9 >= _col8) and _col0 is not null)) (type: boolean) + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 10 Data size: 4850 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 10 Data size: 4850 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint), _col7 (type: string) Reducer 4 Reduce Operator Tree: @@ -1438,17 +1438,17 @@ STAGE PLANS: 0 _col5 (type: string) 1 _col0 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2258 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col0 (type: int), _col1 (type: string), _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 + Statistics: Num rows: 9 Data size: 2258 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 3 Data size: 1465 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 22 Data size: 3181 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 6 Execution mode: vectorized @@ -1459,7 +1459,7 @@ STAGE PLANS: File Output Operator compressed: false Dp Sort State: PARTITION_SORTED - Statistics: Num rows: 3 Data size: 1465 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 22 Data size: 3181 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1474,17 +1474,17 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 79 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: string), _col0 (type: int), 'Changed forever' (type: string), _col1 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 79 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 3 Data size: 1465 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 22 Data size: 3181 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 8 Reduce Operator Tree: @@ -1495,13 +1495,13 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: string) Reducer 9 Execution mode: vectorized @@ -1509,7 +1509,7 @@ STAGE PLANS: Select Operator expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col5 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -1528,20 +1528,20 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 844 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col6 (type: string), _col0 (type: int), _col1 (type: string), _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 + Statistics: Num rows: 4 Data size: 844 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 3 Data size: 1465 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 22 Data size: 3181 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Union 5 Vertex: Union 5 @@ -1675,69 +1675,69 @@ STAGE PLANS: Map Operator Tree: TableScan alias: tbl_ice - Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: FILE__PATH is not null (type: boolean) - Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), b (type: string), c (type: int), 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, _col5, _col6, _col7 - Statistics: Num rows: 9 Data size: 4365 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 4365 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), c (type: int), 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, _col5, _col6 - Statistics: Num rows: 9 Data size: 3564 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 3564 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: string) Select Operator expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), PARTITION__PROJECTION (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 9 Data size: 4293 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 4293 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: string) Select Operator expressions: a (type: int) outputColumnNames: _col0 - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (a is not null and FILE__PATH is not null) (type: boolean) - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), FILE__PATH (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 9 Data size: 1692 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 1692 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Execution mode: vectorized Reducer 10 @@ -1747,11 +1747,11 @@ STAGE PLANS: aggregations: count(VALUE._col0), count(VALUE._col1) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 11 Reduce Operator Tree: @@ -1762,20 +1762,20 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col1 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reducer 12 Execution mode: vectorized Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: string) outputColumnNames: _col1 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -1795,26 +1795,26 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: string) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Reducer 13 Reduce Operator Tree: Merge Join Operator @@ -1824,13 +1824,13 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: string) Reducer 14 Execution mode: vectorized @@ -1838,7 +1838,7 @@ STAGE PLANS: Select Operator expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col5 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -1857,20 +1857,20 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 844 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col6 (type: string), _col0 (type: int), _col1 (type: string), _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 + Statistics: Num rows: 4 Data size: 844 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 3 Data size: 1462 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 22 Data size: 3181 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 2 Reduce Operator Tree: @@ -1881,60 +1881,60 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: int) - minReductionHashAggr: 0.5 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(), count(_col0) - minReductionHashAggr: 0.5 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Group By Operator keys: _col0 (type: int) - minReductionHashAggr: 0.5 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: int) - minReductionHashAggr: 0.5 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reducer 3 Reduce Operator Tree: Merge Join Operator @@ -1944,17 +1944,17 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 79 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: string), _col0 (type: int), 'The last one' (type: string), _col1 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 492 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 79 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 3 Data size: 1462 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 22 Data size: 3181 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Reducer 5 Execution mode: vectorized @@ -1965,7 +1965,7 @@ STAGE PLANS: File Output Operator compressed: false Dp Sort State: PARTITION_SORTED - Statistics: Num rows: 3 Data size: 1462 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 22 Data size: 3181 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1978,17 +1978,17 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), true (type: boolean) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean) Reducer 7 Reduce Operator Tree: @@ -1999,11 +1999,11 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9 - Statistics: Num rows: 10 Data size: 4858 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 10 Data size: 4858 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col9 (type: boolean) Reducer 8 Reduce Operator Tree: @@ -2014,24 +2014,24 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9, _col10, _col11 - Statistics: Num rows: 10 Data size: 5018 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col10 (type: bigint), _col11 (type: bigint), _col9 (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col11 - Statistics: Num rows: 10 Data size: 5018 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: ((_col8 = 0L) or (_col11 is null and (_col9 >= _col8) and _col0 is not null) or ((_col11 is not null and (_col8 <> 0L)) or ((_col0 is null or (_col9 < _col8)) and null and (_col8 <> 0L) and _col11 is null)) is null) (type: boolean) - Statistics: Num rows: 10 Data size: 5018 Basic stats: COMPLETE Column stats: COMPLETE + predicate: ((_col8 = 0L) or ((_col11 is not null and (_col8 <> 0L)) or ((_col0 is null or (_col9 < _col8)) and null and (_col8 <> 0L) and _col11 is null)) is null or (_col11 is null and (_col9 >= _col8) and _col0 is not null)) (type: boolean) + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 10 Data size: 4850 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 10 Data size: 4850 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint), _col7 (type: string) Reducer 9 Reduce Operator Tree: @@ -2042,17 +2042,17 @@ STAGE PLANS: 0 _col5 (type: string) 1 _col0 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2258 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col0 (type: int), _col1 (type: string), _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 + Statistics: Num rows: 9 Data size: 2258 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) null sort order: zz sort order: ++ Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string) - Statistics: Num rows: 3 Data size: 1462 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 22 Data size: 3181 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int) Union 4 Vertex: Union 4 diff --git a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out index 21837dae7d6f..627959eee2ee 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/update_iceberg_copy_on_write_unpartitioned.q.out @@ -1155,69 +1155,69 @@ STAGE PLANS: Map Operator Tree: TableScan alias: tbl_ice - Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: FILE__PATH is not null (type: boolean) - Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), b (type: string), c (type: int), 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, _col5, _col6, _col7 - Statistics: Num rows: 9 Data size: 4365 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 4365 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), c (type: int), 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, _col5, _col6 - Statistics: Num rows: 9 Data size: 3564 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 3564 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: string) Select Operator expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), PARTITION__PROJECTION (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 9 Data size: 4293 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 4293 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: string) Select Operator expressions: a (type: int) outputColumnNames: _col0 - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (a is not null and FILE__PATH is not null) (type: boolean) - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), FILE__PATH (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 9 Data size: 1692 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 1692 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Execution mode: vectorized Map 14 @@ -1247,17 +1247,17 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), true (type: boolean) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean) Reducer 11 Reduce Operator Tree: @@ -1268,20 +1268,20 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col1 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reducer 12 Execution mode: vectorized Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: string) outputColumnNames: _col1 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -1301,26 +1301,26 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: string) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Reducer 13 Execution mode: vectorized Reduce Operator Tree: @@ -1328,11 +1328,11 @@ STAGE PLANS: aggregations: count(VALUE._col0), count(VALUE._col1) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Reducer 2 Reduce Operator Tree: @@ -1343,11 +1343,11 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9 - Statistics: Num rows: 10 Data size: 4858 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 10 Data size: 4858 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col9 (type: boolean) Reducer 3 Reduce Operator Tree: @@ -1358,24 +1358,24 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9, _col10, _col11 - Statistics: Num rows: 10 Data size: 5018 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col10 (type: bigint), _col11 (type: bigint), _col9 (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col11 - Statistics: Num rows: 10 Data size: 5018 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: ((_col8 = 0L) or (_col11 is null and (_col9 >= _col8) and _col0 is not null) or ((_col11 is not null and (_col8 <> 0L)) or ((_col0 is null or (_col9 < _col8)) and null and (_col8 <> 0L) and _col11 is null)) is null) (type: boolean) - Statistics: Num rows: 10 Data size: 5018 Basic stats: COMPLETE Column stats: COMPLETE + predicate: ((_col8 = 0L) or ((_col11 is not null and (_col8 <> 0L)) or ((_col0 is null or (_col9 < _col8)) and null and (_col8 <> 0L) and _col11 is null)) is null or (_col11 is null and (_col9 >= _col8) and _col0 is not null)) (type: boolean) + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 10 Data size: 4850 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 10 Data size: 4850 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint), _col7 (type: string) Reducer 4 Reduce Operator Tree: @@ -1386,14 +1386,14 @@ STAGE PLANS: 0 _col5 (type: string) 1 _col0 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2258 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col0 (type: int), _col1 (type: string), _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 + Statistics: Num rows: 9 Data size: 2258 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 1465 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 22 Data size: 3181 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1408,14 +1408,14 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 79 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: string), _col0 (type: int), 'Changed forever' (type: string), _col1 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 79 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 1465 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 22 Data size: 3181 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1430,13 +1430,13 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: string) Reducer 8 Execution mode: vectorized @@ -1444,7 +1444,7 @@ STAGE PLANS: Select Operator expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col5 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -1463,17 +1463,17 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 844 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col6 (type: string), _col0 (type: int), _col1 (type: string), _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 + Statistics: Num rows: 4 Data size: 844 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 1465 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 22 Data size: 3181 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1488,47 +1488,47 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: int) - minReductionHashAggr: 0.5 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(), count(_col0) - minReductionHashAggr: 0.5 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Union 5 Vertex: Union 5 @@ -1661,69 +1661,69 @@ STAGE PLANS: Map Operator Tree: TableScan alias: tbl_ice - Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: FILE__PATH is not null (type: boolean) - Statistics: Num rows: 9 Data size: 873 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), b (type: string), c (type: int), 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, _col5, _col6, _col7 - Statistics: Num rows: 9 Data size: 4365 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 4365 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), c (type: int), 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, _col5, _col6 - Statistics: Num rows: 9 Data size: 3564 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 3564 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: string) Select Operator expressions: a (type: int), b (type: string), c (type: int), PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), PARTITION__PROJECTION (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 9 Data size: 4293 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 4293 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1728 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: string) Select Operator expressions: a (type: int) outputColumnNames: _col0 - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (a is not null and FILE__PATH is not null) (type: boolean) - Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), FILE__PATH (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 9 Data size: 1692 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 9 Data size: 1692 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Execution mode: vectorized Reducer 10 @@ -1735,20 +1735,20 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col1 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reducer 11 Execution mode: vectorized Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: string) outputColumnNames: _col1 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -1768,26 +1768,26 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: string) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Reducer 12 Reduce Operator Tree: Merge Join Operator @@ -1797,13 +1797,13 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: a sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: string) Reducer 13 Execution mode: vectorized @@ -1811,7 +1811,7 @@ STAGE PLANS: Select Operator expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col5 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE PTF Operator Function definitions: Input definition @@ -1830,17 +1830,17 @@ STAGE PLANS: window function: GenericUDAFRowNumberEvaluator window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX) isPivotResult: true - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (row_number_window_0 = 1) (type: boolean) - Statistics: Num rows: 1 Data size: 477 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 844 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), -1L (type: bigint), _col6 (type: string), _col0 (type: int), _col1 (type: string), _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 + Statistics: Num rows: 4 Data size: 844 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 1462 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 22 Data size: 3181 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1855,60 +1855,60 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: int) - minReductionHashAggr: 0.5 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(), count(_col0) - minReductionHashAggr: 0.5 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Group By Operator keys: _col0 (type: int) - minReductionHashAggr: 0.5 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: int) - minReductionHashAggr: 0.5 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE Reducer 3 Reduce Operator Tree: Merge Join Operator @@ -1918,14 +1918,14 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 1 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 79 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: string), _col0 (type: int), 'The last one' (type: string), _col1 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 492 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 79 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 1462 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 22 Data size: 3181 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -1938,17 +1938,17 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), true (type: boolean) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean) Reducer 6 Reduce Operator Tree: @@ -1959,11 +1959,11 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9 - Statistics: Num rows: 10 Data size: 4858 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 10 Data size: 4858 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 1900 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col9 (type: boolean) Reducer 7 Reduce Operator Tree: @@ -1974,24 +1974,24 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9, _col10, _col11 - Statistics: Num rows: 10 Data size: 5018 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col10 (type: bigint), _col11 (type: bigint), _col9 (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col11 - Statistics: Num rows: 10 Data size: 5018 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: ((_col8 = 0L) or (_col11 is null and (_col9 >= _col8) and _col0 is not null) or ((_col11 is not null and (_col8 <> 0L)) or ((_col0 is null or (_col9 < _col8)) and null and (_col8 <> 0L) and _col11 is null)) is null) (type: boolean) - Statistics: Num rows: 10 Data size: 5018 Basic stats: COMPLETE Column stats: COMPLETE + predicate: ((_col8 = 0L) or ((_col11 is not null and (_col8 <> 0L)) or ((_col0 is null or (_col9 < _col8)) and null and (_col8 <> 0L) and _col11 is null)) is null or (_col11 is null and (_col9 >= _col8) and _col0 is not null)) (type: boolean) + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 10 Data size: 4850 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: string) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: string) - Statistics: Num rows: 10 Data size: 4850 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2053 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint), _col7 (type: string) Reducer 8 Reduce Operator Tree: @@ -2002,14 +2002,14 @@ STAGE PLANS: 0 _col5 (type: string) 1 _col0 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 2258 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: bigint), _col7 (type: string), _col0 (type: int), _col1 (type: string), _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 + Statistics: Num rows: 9 Data size: 2258 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 1462 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 22 Data size: 3181 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -2022,11 +2022,11 @@ STAGE PLANS: aggregations: count(VALUE._col0), count(VALUE._col1) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator null sort order: sort order: - Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: bigint), _col1 (type: bigint) Union 4 Vertex: Union 4 @@ -2099,14 +2099,14 @@ STAGE PLANS: Map Operator Tree: TableScan alias: tbl_ice - Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), 'All' (type: string), c (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 9 Data size: 855 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 9 Data size: 855 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 9 Data size: 72 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat 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 feb7f1f41821..31676dd56a8c 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 @@ -335,7 +335,7 @@ STAGE PLANS: Map Operator Tree: TableScan alias: s - Statistics: Num rows: 5 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE TableScan Vectorization: native: true vectorizationSchemaColumns: [0:ss_sold_date_sk:int, 1:ss_sold_time_sk:int, 2:ss_item_sk2:int, 3:ss_customer_sk2:int, 4:ss_cdemo_sk:int, 5:ss_hdemo_sk:int, 6:ss_addr_sk:int, 7:ss_store_sk:int, 8:ss_promo_sk:int, 9:ss_ticket_number:int, 10:ss_quantity:int, 11:ss_wholesale_cost:decimal(7,2), 12:ss_list_price:decimal(7,2), 13:ss_sales_price:decimal(7,2), 14:ss_ext_discount_amt:decimal(7,2), 15:ss_ext_sales_price:decimal(7,2), 16:ss_ext_wholesale_cost:decimal(7,2), 17:ss_ext_list_price:decimal(7,2), 18:ss_ext_tax:decimal(7,2), 19:ss_coupon_amt:decimal(7,2), 20:ss_net_paid:decimal(7,2), 21:ss_net_paid_inc_tax:decimal(7,2), 22:ss_net_profit:decimal(7,2), 23:PARTITION__SPEC__ID:int, 24:PARTITION__HASH:bigint, 25:FILE__PATH:string, 26:ROW__POSITION:bigint, 27:PARTITION__PROJECTION:string] @@ -347,7 +347,7 @@ STAGE PLANS: native: true projectedOutputColumnNums: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 32, 33] selectExpressions: LongColumnBetween(col 31:bigint, left 1000, right 2000)(children: LongColMultiplyLongScalar(col 30:bigint, val 1000)(children: FuncFloorDoubleToLong(col 29:double)(children: DoubleColDivideDoubleScalar(col 28:double, val 1000.0)(children: CastLongToDouble(col 2:int) -> 28:double) -> 29:double) -> 30:bigint) -> 31:bigint) -> 32:boolean, DecimalColLessDecimalScalar(col 14:decimal(7,2), val 0) -> 33:boolean - Statistics: Num rows: 5 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE Map Join Operator condition map: Left Outer Join 0 to 1 @@ -372,7 +372,7 @@ STAGE PLANS: 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 5 - Statistics: Num rows: 7 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE 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)) 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, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38, _col39, _col40, _col41, _col42, _col43, _col44, _col45, _col46, _col47, _col48, _col49 @@ -380,14 +380,14 @@ STAGE PLANS: className: VectorSelectOperator native: true projectedOutputColumnNums: [50, 20, 43, 49, 34, 22, 36, 8, 4, 11, 59, 48, 38, 7, 60, 12, 44, 18, 15, 55, 57, 21, 9, 45, 41, 51, 46, 5, 58, 6, 14, 13, 54, 53, 2, 17, 39, 3, 16, 19, 35, 10, 40, 52, 1, 37, 61, 42, 47, 56] - Statistics: Num rows: 7 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE Filter Operator Filter Vectorization: className: VectorFilterOperator native: true predicateExpression: FilterExprAndExpr(children: FilterLongColEqualLongColumn(col 41:int, col 2:int), FilterLongColEqualLongColumn(col 42:int, col 3:int), FilterLongColEqualLongScalar(col 39:int, val 2451181), FilterLongColumnBetween(col 31:bigint, left 1000, right 2000)(children: LongColMultiplyLongScalar(col 30:bigint, val 1000)(children: FuncFloorDoubleToLong(col 28:double)(children: LongColDivideLongScalar(col 2:int, val 1000) -> 28:double) -> 30:bigint) -> 31:bigint), FilterDecimalColLessDecimalScalar(col 14:decimal(7,2), val 0), SelectColumnIsNull(col 53:decimal(7,2))) 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: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE Select Operator 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 @@ -396,13 +396,13 @@ STAGE PLANS: native: true 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: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false File Sink Vectorization: className: VectorFileSinkOperator native: false - Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -414,7 +414,7 @@ STAGE PLANS: native: true predicateExpression: FilterExprAndExpr(children: FilterLongColEqualLongColumn(col 41:int, col 2:int), FilterLongColEqualLongColumn(col 42:int, col 3:int), FilterLongColEqualLongScalar(col 39:int, val 2451181), FilterLongColumnBetween(col 31:bigint, left 1000, right 2000)(children: LongColMultiplyLongScalar(col 30:bigint, val 1000)(children: FuncFloorDoubleToLong(col 28:double)(children: LongColDivideLongScalar(col 2:int, val 1000) -> 28:double) -> 30:bigint) -> 31:bigint), FilterDecimalColLessDecimalScalar(col 14:decimal(7,2), val 0), SelectColumnIsNull(col 53:decimal(7,2))) 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: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE Select Operator expressions: 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)), 0 (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 @@ -423,7 +423,7 @@ STAGE PLANS: native: true 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: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col3 (type: int), iceberg_bucket(_col2, 3) (type: int) null sort order: zz @@ -436,7 +436,7 @@ STAGE PLANS: 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, 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: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE 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 Filter Vectorization: @@ -444,7 +444,7 @@ STAGE PLANS: native: true predicateExpression: FilterExprAndExpr(children: SelectColumnIsNull(col 41:int), SelectColumnIsNull(col 42:int), SelectColumnIsNull(col 39:int)) predicate: (_col24 is null and _col47 is null and _col36 is null) (type: boolean) - Statistics: Num rows: 3 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE Select Operator expressions: 2451181 (type: int), _col44 (type: int), _col34 (type: int), _col37 (type: int), _col8 (type: int), _col27 (type: int), _col29 (type: int), _col13 (type: int), _col7 (type: int), _col22 (type: int), _col41 (type: int), _col9 (type: decimal(7,2)), _col15 (type: decimal(7,2)), _col31 (type: decimal(7,2)), _col30 (type: decimal(7,2)), _col18 (type: decimal(7,2)), _col38 (type: decimal(7,2)), _col35 (type: decimal(7,2)), _col17 (type: decimal(7,2)), _col39 (type: decimal(7,2)), _col1 (type: decimal(7,2)), _col21 (type: decimal(7,2)), _col5 (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 @@ -453,7 +453,7 @@ STAGE PLANS: native: true 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: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col3 (type: int), iceberg_bucket(_col2, 3) (type: int) null sort order: zz @@ -466,7 +466,7 @@ STAGE PLANS: 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: 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: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE 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 Filter Vectorization: @@ -474,7 +474,7 @@ STAGE PLANS: native: true predicateExpression: FilterExprAndExpr(children: FilterLongColEqualLongColumn(col 41:int, col 2:int), FilterLongColEqualLongColumn(col 42:int, col 3:int), FilterLongColEqualLongScalar(col 39:int, val 2451181), FilterLongColumnBetween(col 31:bigint, left 1000, right 2000)(children: LongColMultiplyLongScalar(col 30:bigint, val 1000)(children: FuncFloorDoubleToLong(col 28:double)(children: LongColDivideLongScalar(col 2:int, val 1000) -> 28:double) -> 30:bigint) -> 31:bigint), FilterDecimalColLessDecimalScalar(col 14:decimal(7,2), val 0)) 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: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col4 (type: int), _col6 (type: string), _col40 (type: bigint), _col45 (type: bigint) outputColumnNames: _col4, _col6, _col40, _col45 @@ -482,7 +482,7 @@ STAGE PLANS: className: VectorSelectOperator native: true projectedOutputColumnNums: [34, 36, 35, 37] - Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() Group By Vectorization: @@ -494,10 +494,10 @@ STAGE PLANS: vectorProcessingMode: HASH projectedOutputColumnNums: [0] keys: _col4 (type: int), _col40 (type: bigint), _col6 (type: string), _col45 (type: bigint) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint) null sort order: zzzz @@ -509,7 +509,7 @@ STAGE PLANS: 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: 4:bigint - Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE value expressions: _col4 (type: bigint) Execution mode: vectorized Map Vectorization: @@ -532,7 +532,7 @@ STAGE PLANS: TableScan alias: store_sales filterExpr: ((ss_sold_date_sk = 2451181) and ss_item_sk is not null and ss_customer_sk is not null) (type: boolean) - Statistics: Num rows: 2 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE TableScan Vectorization: native: true vectorizationSchemaColumns: [0:ss_sold_date_sk:int, 1:ss_sold_time_sk:int, 2:ss_item_sk:int, 3:ss_customer_sk:int, 4:ss_cdemo_sk:int, 5:ss_hdemo_sk:int, 6:ss_addr_sk:int, 7:ss_store_sk:int, 8:ss_promo_sk:int, 9:ss_ticket_number:int, 10:ss_quantity:int, 11:ss_wholesale_cost:decimal(7,2), 12:ss_list_price:decimal(7,2), 13:ss_sales_price:decimal(7,2), 14:ss_ext_discount_amt:decimal(7,2), 15:ss_ext_sales_price:decimal(7,2), 16:ss_ext_wholesale_cost:decimal(7,2), 17:ss_ext_list_price:decimal(7,2), 18:ss_ext_tax:decimal(7,2), 19:ss_coupon_amt:decimal(7,2), 20:ss_net_paid:decimal(7,2), 21:ss_net_paid_inc_tax:decimal(7,2), 22:ss_net_profit:decimal(7,2), 23:PARTITION__SPEC__ID:int, 24:PARTITION__HASH:bigint, 25:FILE__PATH:string, 26:ROW__POSITION:bigint, 27:PARTITION__PROJECTION:string] @@ -542,7 +542,7 @@ STAGE PLANS: native: true predicateExpression: FilterExprAndExpr(children: FilterLongColEqualLongScalar(col 0:int, val 2451181), SelectColumnIsNotNull(col 2:int), SelectColumnIsNotNull(col 3:int)) predicate: ((ss_sold_date_sk = 2451181) and ss_item_sk is not null and ss_customer_sk is not null) (type: boolean) - Statistics: Num rows: 2 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE Select Operator expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), 2451181 (type: int), ss_sold_time_sk (type: int), ss_item_sk (type: int), ss_customer_sk (type: int), ss_cdemo_sk (type: int), ss_hdemo_sk (type: int), ss_addr_sk (type: int), ss_store_sk (type: int), ss_promo_sk (type: int), ss_ticket_number (type: int), ss_quantity (type: int), ss_wholesale_cost (type: decimal(7,2)), ss_list_price (type: decimal(7,2)), ss_sales_price (type: decimal(7,2)), ss_ext_discount_amt (type: decimal(7,2)), ss_ext_sales_price (type: decimal(7,2)), ss_ext_wholesale_cost (type: decimal(7,2)), ss_ext_list_price (type: decimal(7,2)), ss_ext_tax (type: decimal(7,2)), ss_coupon_amt (type: decimal(7,2)), ss_net_paid (type: decimal(7,2)), ss_net_paid_inc_tax (type: decimal(7,2)), ss_net_profit (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 @@ -551,7 +551,7 @@ STAGE PLANS: native: true projectedOutputColumnNums: [23, 24, 25, 26, 27, 28, 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) -> 28:int - Statistics: Num rows: 2 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col8 (type: int), _col7 (type: int) null sort order: zz @@ -564,7 +564,7 @@ STAGE PLANS: 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 partitionColumns: 2:int valueColumns: 23:int, 24:bigint, 25:string, 26:bigint, 27:string, 28:int, 1: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: 2 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 2 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (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)), _col19 (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)) Execution mode: vectorized Map Vectorization: @@ -612,7 +612,7 @@ STAGE PLANS: File Sink Vectorization: className: VectorFileSinkOperator native: false - Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -647,7 +647,7 @@ STAGE PLANS: File Sink Vectorization: className: VectorFileSinkOperator native: false - Statistics: Num rows: 3 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -682,14 +682,14 @@ STAGE PLANS: 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 - Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE Filter Operator Filter Vectorization: className: VectorFilterOperator native: true predicateExpression: FilterLongColGreaterLongScalar(col 4:bigint, val 1) predicate: (_col4 > 1L) (type: boolean) - Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE Select Operator expressions: cardinality_violation(_col0,_col1,_col2,_col3) (type: int) outputColumnNames: _col0 @@ -698,13 +698,13 @@ STAGE PLANS: native: true projectedOutputColumnNums: [5] selectExpressions: VectorUDFAdaptor(cardinality_violation(_col0,_col1,_col2,_col3)) -> 5:int - Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false File Sink Vectorization: className: VectorFileSinkOperator native: false - Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: #Masked# Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -910,9 +910,9 @@ Stage-6 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=#Masked#) 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_49] (rows=7 width=#Masked#) + Select Operator [SEL_49] (rows=5 width=#Masked#) 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_48] (rows=7 width=#Masked#) + Map Join Operator [MAPJOIN_48] (rows=5 width=#Masked#) BucketMapJoin:true,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 [CUSTOM_EDGE] vectorized MULTICAST [RS_46] @@ -922,11 +922,11 @@ Stage-6 Filter Operator [FIL_44] (rows=2 width=#Masked#) 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=#Masked#) - 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"] + default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,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_47] (rows=5 width=#Masked#) 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=#Masked#) - default@ssv,s,Tbl:COMPLETE,Col:COMPLETE,Grouping Num Buckets:3,Grouping Partition Columns:["ss_item_sk2"],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"] + default@ssv,s,Tbl:COMPLETE,Col:NONE,Grouping Num Buckets:3,Grouping Partition Columns:["ss_item_sk2"],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_66] table:{"name:":"default.store_sales"} @@ -935,9 +935,9 @@ Stage-6 <-Map 1 [SIMPLE_EDGE] vectorized SHUFFLE [RS_60] PartitionCols:_col3, iceberg_bucket(_col2, 3) - Select Operator [SEL_56] (rows=3 width=#Masked#) + Select Operator [SEL_56] (rows=1 width=#Masked#) 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_52] (rows=3 width=#Masked#) + Filter Operator [FIL_52] (rows=1 width=#Masked#) predicate:(_col24 is null and _col47 is null and _col36 is null) Please refer to the previous Select Operator [SEL_49] Reducer 4 vectorized 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 ea0f7a5ed528..56a158528ded 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 @@ -234,17 +234,17 @@ STAGE PLANS: alias: ice01 filterExpr: (a = 22) (type: boolean) Snapshot ref: branch_test1 - Statistics: Num rows: 5 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 960 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (a = 22) (type: boolean) - Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 960 Basic stats: COMPLETE Column stats: NONE Select Operator 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: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 960 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 5 Data size: 960 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -316,21 +316,21 @@ STAGE PLANS: alias: ice01 filterExpr: (c = 66) (type: boolean) Snapshot ref: branch_test1 - Statistics: Num rows: 4 Data size: 388 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 768 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (c = 66) (type: boolean) - Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 768 Basic stats: COMPLETE Column stats: NONE 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: int), b (type: string), b (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col9 - Statistics: Num rows: 1 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 768 Basic stats: COMPLETE Column stats: NONE Select Operator 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: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 768 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 768 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -339,10 +339,10 @@ STAGE PLANS: Select Operator expressions: 33 (type: int), _col9 (type: string), 66 (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 768 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 768 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -451,20 +451,20 @@ STAGE PLANS: alias: ice01 filterExpr: a is not null (type: boolean) Snapshot ref: branch_test1 - Statistics: Num rows: 4 Data size: 388 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 768 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: a is not null (type: boolean) - Statistics: Num rows: 4 Data size: 388 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 768 Basic stats: COMPLETE Column stats: NONE 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: int), b (type: string), c (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 4 Data size: 1940 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 768 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col5 (type: int) null sort order: z sort order: + Map-reduce partition columns: _col5 (type: int) - Statistics: Num rows: 4 Data size: 1940 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 4 Data size: 768 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: string), _col6 (type: string), _col7 (type: int) Execution mode: vectorized Reducer 2 @@ -476,21 +476,21 @@ STAGE PLANS: 0 _col0 (type: int) 1 _col5 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 - Statistics: Num rows: 10 Data size: 3385 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 633 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string), _col0 (type: int), _col5 (type: string), _col7 (type: string), _col2 (type: int), _col6 (type: bigint), _col4 (type: bigint), _col3 (type: int), _col10 (type: int), _col9 (type: string), _col8 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 - Statistics: Num rows: 10 Data size: 3385 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 633 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((_col10 = _col1) and (_col10 > 100)) (type: boolean) - Statistics: Num rows: 1 Data size: 581 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: NONE Select Operator 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 + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 485 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -498,14 +498,14 @@ STAGE PLANS: name: default.ice01 Filter Operator predicate: ((_col10 = _col1) and (_col10 <= 100)) (type: boolean) - Statistics: Num rows: 5 Data size: 1935 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: NONE Select Operator 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: 5 Data size: 1455 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 5 Data size: 1455 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -513,14 +513,14 @@ STAGE PLANS: name: default.ice01 Filter Operator predicate: ((_col10 = _col1) and (_col10 <= 100)) (type: boolean) - Statistics: Num rows: 5 Data size: 1935 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col10 (type: int), 'Merged' (type: string), (_col8 + 10) (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 5 Data size: 482 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 5 Data size: 482 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -528,14 +528,14 @@ STAGE PLANS: name: default.ice01 Filter Operator predicate: _col10 is null (type: boolean) - Statistics: Num rows: 6 Data size: 2031 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 316 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: int), _col0 (type: string), _col4 (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 316 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 316 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat @@ -543,24 +543,24 @@ STAGE PLANS: name: default.ice01 Filter Operator predicate: (_col10 = _col1) (type: boolean) - Statistics: Num rows: 5 Data size: 1935 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 316 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: int) outputColumnNames: _col2, _col5, _col6, _col7 - Statistics: Num rows: 5 Data size: 1935 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 316 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() keys: _col7 (type: int), _col6 (type: bigint), _col2 (type: string), _col5 (type: bigint) - minReductionHashAggr: 0.4 + minReductionHashAggr: 0.99 mode: hash outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 3 Data size: 316 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator 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 + Statistics: Num rows: 3 Data size: 316 Basic stats: COMPLETE Column stats: NONE value expressions: _col4 (type: bigint) Reducer 3 Execution mode: vectorized @@ -570,17 +570,17 @@ STAGE PLANS: 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 - Statistics: Num rows: 4 Data size: 644 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (_col4 > 1L) (type: boolean) - Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: cardinality_violation(_col0,_col1,_col2,_col3) (type: int) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 105 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -795,14 +795,14 @@ STAGE PLANS: TableScan alias: ice01 Snapshot ref: branch_test1 - Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: a (type: int), b (type: string), c (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/desc/DescTableOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/desc/DescTableOperation.java index 5dcd8cae1ed4..3fa15fdc651e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/desc/DescTableOperation.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/desc/DescTableOperation.java @@ -25,10 +25,10 @@ import java.util.List; import java.util.Map; +import com.google.common.primitives.Longs; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.StatsSetupConst; -import org.apache.hadoop.hive.common.TableName; import org.apache.hadoop.hive.common.ValidTxnList; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.conf.HiveConf; @@ -48,13 +48,14 @@ import org.apache.hadoop.hive.ql.ddl.DDLOperation; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.PartitionIterable; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.metadata.TableConstraintsInfo; -import org.apache.hadoop.hive.ql.parse.HiveTableName; import org.apache.hadoop.hive.ql.plan.ColStatistics; import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.ql.stats.Partish; import org.apache.hadoop.hive.ql.stats.StatsUtils; import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.serde2.Deserializer; @@ -67,6 +68,7 @@ * Operation process of describing a table. */ public class DescTableOperation extends DDLOperation { + public DescTableOperation(DDLOperationContext context, DescTableDesc desc) { super(context, desc); } @@ -136,8 +138,7 @@ private Partition getPartition(Table table) throws HiveException { } private Deserializer getDeserializer(Table table) throws SQLException { - Deserializer deserializer = table.getDeserializer(true); - return deserializer; + return table.getDeserializer(true); } private void getColumnsNoColumnPath(Table table, Partition partition, List cols) throws HiveException { @@ -149,13 +150,13 @@ private void getColumnsNoColumnPath(Table table, Partition partition, List tblProps = table.getParameters() == null ? - new HashMap<>() : table.getParameters(); + new HashMap<>() : table.getParameters(); Map valueMap = new HashMap<>(); Map stateMap = new HashMap<>(); @@ -163,20 +164,30 @@ private void getColumnsNoColumnPath(Table table, Partition partition, List partitionProps = p.getParameters(); - Boolean state = StatsSetupConst.areBasicStatsUptoDate(partitionProps); - for (String stat : StatsSetupConst.SUPPORTED_STATS) { - stateMap.put(stat, stateMap.get(stat) && state); - if (partitionProps != null && partitionProps.get(stat) != null) { - valueMap.put(stat, valueMap.get(stat) + Long.parseLong(partitionProps.get(stat))); + + if (table.isNonNative()) { + HiveStorageHandler sh = table.getStorageHandler(); + + sh.getBasicStatistics(Partish.buildFor(table)).forEach((k, v) -> + valueMap.put(k, Longs.tryParse(v))); + numParts = sh.getPartitionNames(table).size(); + + } else { + PartitionIterable partitions = new PartitionIterable(context.getDb(), table, null, + MetastoreConf.getIntVar(context.getConf(), MetastoreConf.ConfVars.BATCH_RETRIEVE_MAX)); + + for (Partition p : partitions) { + Map partitionProps = p.getParameters(); + Boolean state = StatsSetupConst.areBasicStatsUptoDate(partitionProps); + for (String stat : StatsSetupConst.SUPPORTED_STATS) { + stateMap.put(stat, stateMap.get(stat) && state); + if (partitionProps != null && partitionProps.get(stat) != null) { + valueMap.put(stat, valueMap.get(stat) + Long.parseLong(partitionProps.get(stat))); + } } + numParts++; } - numParts++; } tblProps.put(StatsSetupConst.NUM_PARTITIONS, Integer.toString(numParts)); @@ -196,33 +207,27 @@ private void getColumnDataColPathSpecified(Table table, Partition part, List colNames = Lists.newArrayList(colName.toLowerCase()); - TableName tableName = HiveTableName.of(desc.getDbTableName()); - if (null == part) { - if (table.isPartitioned() && !table.hasNonNativePartitionSupport()) { + if (part == null) { + if (table.isPartitioned() && StatsUtils.checkCanProvidePartitionStats(table)) { Map tableProps = table.getParameters() == null ? new HashMap<>() : table.getParameters(); if (table.isPartitionKey(colNames.get(0))) { getColumnDataForPartitionKeyColumn(table, cols, colStats, colNames, tableProps); } else { - getColumnsForNotPartitionKeyColumn(cols, colStats, deserializer, colNames, tableName, tableProps); + getColumnsForNotPartitionKeyColumn(table, cols, colStats, deserializer, colNames, tableProps); } table.setParameters(tableProps); } else { cols.addAll(Hive.getFieldsFromDeserializer(desc.getColumnPath(), deserializer, context.getConf())); - if (table.isNonNative() && table.getStorageHandler().canProvideColStatistics(table)) { - colStats.addAll(table.getStorageHandler().getColStatistics(table)); - } else { - colStats.addAll(context.getDb().getTableColumnStatistics(tableName.getDb().toLowerCase(), - tableName.getTable().toLowerCase(), colNames, false)); - } + colStats.addAll(context.getDb().getTableColumnStatistics(table, colNames, false)); } } else { - List partitions = new ArrayList(); + List partitions = new ArrayList<>(); String partName = part.getName(); partitions.add(partName); cols.addAll(Hive.getFieldsFromDeserializer(desc.getColumnPath(), deserializer, context.getConf())); Map> partitionColumnStatistics = context.getDb().getPartitionColumnStatistics( - tableName.getDb().toLowerCase(), tableName.getTable().toLowerCase(), partitions, colNames, false); + table.getDbName(), table.getTableName(), partitions, colNames, false); List partitionColStat = partitionColumnStatistics.get(partName); if (partitionColStat != null) { colStats.addAll(partitionColStat); @@ -252,15 +257,15 @@ private void getColumnDataForPartitionKeyColumn(Table table, List c StatsSetupConst.setColumnStatsState(tableProps, colNames); } - private void getColumnsForNotPartitionKeyColumn(List cols, List colStats, - Deserializer deserializer, List colNames, TableName tableName, Map tableProps) + private void getColumnsForNotPartitionKeyColumn(Table table, List cols, List colStats, + Deserializer deserializer, List colNames, Map tableProps) throws HiveException { cols.addAll(Hive.getFieldsFromDeserializer(desc.getColumnPath(), deserializer, context.getConf())); - List parts = context.getDb().getPartitionNames(tableName.getDb().toLowerCase(), - tableName.getTable().toLowerCase(), (short) -1); - AggrStats aggrStats = context.getDb().getAggrColStatsFor( - tableName.getDb().toLowerCase(), tableName.getTable().toLowerCase(), colNames, parts, false); + List parts = context.getDb().getPartitionNames(table, (short) -1); + + AggrStats aggrStats = context.getDb().getAggrColStatsFor(table, colNames, parts, false); colStats.addAll(aggrStats.getColStats()); + if (parts.size() == aggrStats.getPartsFound()) { StatsSetupConst.setColumnStatsState(tableProps, colNames); } else { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/alter/AlterTableAlterPartitionOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/alter/AlterTableAlterPartitionOperation.java index 24deedf63b0c..761e89c2169a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/alter/AlterTableAlterPartitionOperation.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/alter/AlterTableAlterPartitionOperation.java @@ -91,8 +91,7 @@ private void checkPartitionValues(Table tbl, int colIndex) throws HiveException Converter converter = ObjectInspectorConverters.getConverter( PrimitiveObjectInspectorFactory.javaStringObjectInspector, outputOI); - List partNames = context.getDb().getPartitionNames(tbl.getDbName(), - tbl.getTableName(), (short) -1); + List partNames = context.getDb().getPartitionNames(tbl, (short) -1); for (String partName : partNames) { try { List values = Warehouse.getPartValuesFromPartName(partName); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java index e0f7b7bdf695..74067f5317ee 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/show/ShowPartitionsOperation.java @@ -55,18 +55,15 @@ public ShowPartitionsOperation(DDLOperationContext context, ShowPartitionsDesc d public int execute() throws HiveException { Table tbl = context.getDb().getTable(desc.getTabName()); List parts; - if (tbl.isNonNative() && tbl.getStorageHandler().supportsPartitionTransform()) { - parts = tbl.getStorageHandler().showPartitions(context, tbl); - } else if (!tbl.isPartitioned()) { + if (!tbl.isPartitioned()) { context.getTask().setException(new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, desc.getTabName())); return ErrorMsg.TABLE_NOT_PARTITIONED.getErrorCode(); } else if (desc.getCond() != null || desc.getOrder() != null) { parts = getPartitionNames(tbl); } else if (desc.getPartSpec() != null) { - parts = context.getDb().getPartitionNames(tbl.getDbName(), tbl.getTableName(), - desc.getPartSpec(), desc.getLimit()); + parts = context.getDb().getPartitionNames(tbl, desc.getPartSpec(), desc.getLimit()); } else { - parts = context.getDb().getPartitionNames(tbl.getDbName(), tbl.getTableName(), desc.getLimit()); + parts = context.getDb().getPartitionNames(tbl, desc.getLimit()); } // write the results in the file diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/compact/AlterTableCompactOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/compact/AlterTableCompactOperation.java index 953df3a3d831..571c8e196700 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/compact/AlterTableCompactOperation.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/compact/AlterTableCompactOperation.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hive.ql.ddl.table.storage.compact; +import com.google.common.collect.Maps; import org.apache.commons.lang3.ObjectUtils; import org.apache.hadoop.hive.common.ServerUtils; import org.apache.hadoop.hive.conf.HiveConf; @@ -37,6 +38,7 @@ import org.apache.hadoop.hive.ql.ddl.DDLUtils; import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil; @@ -86,7 +88,7 @@ else if (desc.getPartitionSpec() != null) { } Map partitionMap = - convertPartitionsFromThriftToDB(getPartitions(table, desc, context)); + convertPartitionsFromThriftToDB(getPartitions(table)); TxnStore txnHandler = TxnUtils.getTxnStore(context.getConf()); @@ -113,8 +115,7 @@ else if (desc.getPartitionSpec() != null) { } //Will directly initiate compaction if an un-partitioned table/a partition is specified in the request - if (desc.getPartitionSpec() != null || !(table.isPartitioned() || - (DDLUtils.isIcebergTable(table) && table.getStorageHandler().isPartitioned(table)))) { + if (desc.getPartitionSpec() != null || !table.isPartitioned()) { if (desc.getPartitionSpec() != null) { Optional partitionName = partitionMap.keySet().stream().findFirst(); partitionName.ifPresent(compactionRequest::setPartitionname); @@ -164,18 +165,20 @@ private void parseCompactionResponse(CompactionResponse compactionResponse, Tabl } } - private List getPartitions(Table table, AlterTableCompactDesc desc, DDLOperationContext context) - throws HiveException { + private List getPartitions(Table table) throws HiveException { List partitions = new ArrayList<>(); - if (desc.getPartitionSpec() == null) { - if (desc.getFilterExpr() != null && DDLUtils.isIcebergTable(table) && table.getStorageHandler().isPartitioned(table)) { - partitions = table.getStorageHandler().getPartitionsByExpr(table, desc.getFilterExpr(), true); - } else if (table.isPartitioned() || DDLUtils.isIcebergTable(table) && table.getStorageHandler().isPartitioned(table)) { - // Compaction will get initiated for all the potential partitions that meets the criteria + if (desc.getPartitionSpec() == null && table.isPartitioned()) { + if (DDLUtils.isIcebergTable(table)) { + HiveStorageHandler sh = table.getStorageHandler(); + + partitions = (desc.getFilterExpr() != null) ? + sh.getPartitionsByExpr(table, desc.getFilterExpr(), true) : + sh.getPartitions(table, Maps.newHashMap(), true); + } else { partitions = context.getDb().getPartitions(table); } - } else { + } else if (desc.getPartitionSpec() != null) { Map partitionSpec = desc.getPartitionSpec(); partitions = context.getDb().getPartitions(table, partitionSpec); if (partitions.isEmpty()) { @@ -183,7 +186,8 @@ private List getPartitions(Table table, AlterTableCompactDesc desc, D } // This validates that the partition spec given in the compaction command matches exactly one partition // in the table, not a partial partition spec. - partitions = partitions.stream().filter(part -> part.getSpec().size() == partitionSpec.size()).collect(Collectors.toList()); + partitions = partitions.stream().filter(part -> part.getSpec().size() == partitionSpec.size()) + .collect(Collectors.toList()); if (partitions.size() != 1) { throw new HiveException(ErrorMsg.TOO_MANY_COMPACTION_PARTITIONS); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLPlanUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLPlanUtils.java index e64b2a99892a..51cab2d035e7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLPlanUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLPlanUtils.java @@ -522,10 +522,9 @@ public List getAlterTableStmtTableStatsColsAll(Table tbl) throws HiveException { List alterTblStmt = new ArrayList<>(); List accessedColumns = getTableColumnNames(tbl); - List tableColumnStatistics = Hive.get().getTableColumnStatistics(tbl.getDbName(), - tbl.getTableName(), - accessedColumns, - true); + List tableColumnStatistics = Hive.get().getTableColumnStatistics( + tbl, accessedColumns, true); + ColumnStatisticsObj[] columnStatisticsObj = tableColumnStatistics.toArray(new ColumnStatisticsObj[0]); for (ColumnStatisticsObj statisticsObj : columnStatisticsObj) { alterTblStmt.add(getAlterTableStmtCol( diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java index 818a1c7af5ba..3debb1ee4e41 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java @@ -1537,8 +1537,7 @@ public void truncateTable(String dbDotTableName, Map partSpec, L } // TODO: APIs with catalog names - List partNames = ((null == partSpec) - ? null : getPartitionNames(table.getDbName(), table.getTableName(), partSpec, (short) -1)); + List partNames = (partSpec != null) ? getPartitionNames(table, partSpec, (short) -1) : null; if (snapshot == null) { getMSC().truncateTable(table.getFullTableName(), partNames); } else { @@ -1800,17 +1799,11 @@ public Table getTable(final String dbName, final String tableName, String tableM * @throws LockException */ private ValidWriteIdList getValidWriteIdList(String dbName, String tableName) throws LockException { - ValidWriteIdList validWriteIdList = null; - SessionState sessionState = SessionState.get(); - HiveTxnManager txnMgr = sessionState != null? sessionState.getTxnMgr() : null; - long txnId = txnMgr != null ? txnMgr.getCurrentTxnId() : 0; - if (txnId > 0) { - validWriteIdList = AcidUtils.getTableValidWriteIdListWithTxnList(conf, dbName, tableName); - } else { - String fullTableName = getFullTableName(dbName, tableName); - validWriteIdList = new ValidReaderWriteIdList(fullTableName, new long[0], new BitSet(), Long.MAX_VALUE); - } - return validWriteIdList; + long txnId = Optional.ofNullable(SessionState.get()) + .map(ss -> ss.getTxnMgr().getCurrentTxnId()).orElse(0L); + + return (txnId > 0) ? AcidUtils.getTableValidWriteIdListWithTxnList(conf, dbName, tableName) : + new ValidReaderWriteIdList(); } /** @@ -4041,9 +4034,18 @@ public List dropPartitions(String dbName, String tableName, public List getPartitionNames(String dbName, String tblName, short max) throws HiveException { + Table tbl = new Table(dbName, tblName); + return getPartitionNames(tbl, max); + } + + public List getPartitionNames(Table tbl, short max) + throws HiveException { List names = null; try { - names = getMSC().listPartitionNames(dbName, tblName, max); + if (tbl.hasNonNativePartitionSupport()) { + return tbl.getStorageHandler().getPartitionNames(tbl); + } + names = getMSC().listPartitionNames(tbl.getDbName(), tbl.getTableName(), max); } catch (NoSuchObjectException nsoe) { // this means no partition exists for the given dbName and tblName // key value pairs - thrift cannot handle null return values, hence @@ -4056,34 +4058,36 @@ public List getPartitionNames(String dbName, String tblName, short max) return names; } - public List getPartitionNames(String dbName, String tblName, - Map partSpec, short max) throws HiveException { - Table t = getTable(dbName, tblName); - if (t.hasNonNativePartitionSupport()) { - return t.getStorageHandler().getPartitionNames(t, partSpec); + public List getPartitionNames(String dbName, String tblName, Map partSpec, short max) + throws HiveException { + Table tbl = getTable(dbName, tblName); + return getPartitionNames(tbl, partSpec, max); + } + + public List getPartitionNames(Table tbl, Map partSpec, short max) + throws HiveException { + if (tbl.hasNonNativePartitionSupport()) { + return tbl.getStorageHandler().getPartitionNames(tbl, partSpec); } - - List pvals = MetaStoreUtils.getPvals(t.getPartCols(), partSpec); - - return getPartitionNamesByPartitionVals(dbName, tblName, pvals, max); + List pvals = MetaStoreUtils.getPvals(tbl.getPartCols(), partSpec); + return getPartitionNamesByPartitionVals(tbl, pvals, max); } // get partition names from provided partition values - public List getPartitionNamesByPartitionVals(String dbName, String tblName, - List pVals, short max) throws HiveException { + public List getPartitionNamesByPartitionVals(Table tbl, List pVals, short max) + throws HiveException { List names = null; - Table t = getTable(dbName, tblName); try { GetPartitionNamesPsRequest req = new GetPartitionNamesPsRequest(); - req.setTblName(tblName); - req.setDbName(dbName); + req.setDbName(tbl.getDbName()); + req.setTblName(tbl.getTableName()); req.setPartValues(pVals); req.setMaxParts(max); - if (AcidUtils.isTransactionalTable(t)) { - ValidWriteIdList validWriteIdList = getValidWriteIdList(dbName, tblName); + if (AcidUtils.isTransactionalTable(tbl)) { + ValidWriteIdList validWriteIdList = getValidWriteIdList(tbl.getDbName(), tbl.getTableName()); req.setValidWriteIdList(validWriteIdList != null ? validWriteIdList.toString() : null); - req.setId(t.getTTable().getId()); + req.setId(tbl.getTTable().getId()); } GetPartitionNamesPsResponse res = getMSC().listPartitionNamesRequest(req); names = res.getNames(); @@ -4337,11 +4341,11 @@ private List getPartitionsWithAuth(Table tbl, Map par * @throws HiveException */ public List getPartitions(Table tbl, Map partialPartSpec) - throws HiveException { + throws HiveException { if (tbl.hasNonNativePartitionSupport()) { - return tbl.getStorageHandler().getPartitions(tbl, partialPartSpec, false); + return tbl.getStorageHandler().getPartitions(tbl, partialPartSpec); } else { - return getPartitions(tbl, partialPartSpec, (short)-1); + return getPartitions(tbl, partialPartSpec, (short) -1); } } @@ -4357,19 +4361,15 @@ public List getPartitions(Table tbl, Map partialPartS * @return list of partition objects * @throws HiveException */ - public List getPartitionsByNames(Table tbl, - Map partialPartSpec) + public List getPartitionsByNames(Table tbl, Map partialPartSpec) throws HiveException { if (!tbl.isPartitioned()) { throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, tbl.getTableName()); } + List names = getPartitionNames(tbl, partialPartSpec, (short) -1); - List names = getPartitionNames(tbl.getDbName(), tbl.getTableName(), - partialPartSpec, (short)-1); - - List partitions = getPartitionsByNames(tbl, names); - return partitions; + return getPartitionsByNames(tbl, names); } /** @@ -6105,22 +6105,33 @@ public boolean setPartitionColumnStatistics( } public List getTableColumnStatistics( - String dbName, String tableName, List colNames, boolean checkTransactional) + String dbName, String tblName, List colNames, boolean checkTransactional) + throws HiveException { + Table tbl = getTable(dbName, tblName); + return getTableColumnStatistics(tbl, colNames, checkTransactional); + } + + public List getTableColumnStatistics( + Table tbl, List colNames, boolean checkTransactional) throws HiveException { PerfLogger perfLogger = SessionState.getPerfLogger(); perfLogger.perfLogBegin(CLASS_NAME, PerfLogger.HIVE_GET_TABLE_COLUMN_STATS); + List retv = null; try { + if (tbl.isNonNative() && tbl.getStorageHandler().canProvideColStatistics(tbl)) { + return tbl.getStorageHandler().getColStatistics(tbl); + } if (checkTransactional) { - Table tbl = getTable(dbName, tableName); AcidUtils.TableSnapshot tableSnapshot = AcidUtils.getTableSnapshot(conf, tbl); - retv = getMSC().getTableColumnStatistics(dbName, tableName, colNames, Constants.HIVE_ENGINE, + retv = getMSC().getTableColumnStatistics(tbl.getDbName(), tbl.getTableName(), colNames, + Constants.HIVE_ENGINE, tableSnapshot != null ? tableSnapshot.getValidWriteIdList() : null); } else { - retv = getMSC().getTableColumnStatistics(dbName, tableName, colNames, Constants.HIVE_ENGINE); + retv = getMSC().getTableColumnStatistics(tbl.getDbName(), tbl.getTableName(), colNames, + Constants.HIVE_ENGINE); } - return retv; } catch (Exception e) { LOG.debug("Failed getTableColumnStatistics", e); @@ -6149,25 +6160,32 @@ public Map> getPartitionColumnStatistics( throw new HiveException(e); } } - public AggrStats getAggrColStatsFor(String dbName, String tblName, - List colNames, List partName, boolean checkTransactional) { + List colNames, List partName, boolean checkTransactional) throws HiveException { + Table tbl = getTable(dbName, tblName); + return getAggrColStatsFor(tbl, colNames, partName, checkTransactional); + } + + public AggrStats getAggrColStatsFor(Table tbl, + List colNames, List partName, boolean checkTransactional) { + PerfLogger perfLogger = SessionState.getPerfLogger(); perfLogger.perfLogBegin(CLASS_NAME, PerfLogger.HIVE_GET_AGGR_COL_STATS); + String writeIdList = null; try { + if (tbl.isNonNative() && tbl.getStorageHandler().canProvideColStatistics(tbl)) { + return tbl.getStorageHandler().getAggrColStatsFor(tbl, colNames, partName); + } if (checkTransactional) { - Table tbl = getTable(dbName, tblName); AcidUtils.TableSnapshot tableSnapshot = AcidUtils.getTableSnapshot(conf, tbl); writeIdList = tableSnapshot != null ? tableSnapshot.getValidWriteIdList() : null; } - AggrStats result = getMSC().getAggrColStatsFor(dbName, tblName, colNames, partName, Constants.HIVE_ENGINE, - writeIdList); - - return result; + return getMSC().getAggrColStatsFor(tbl.getDbName(), tbl.getTableName(), colNames, partName, + Constants.HIVE_ENGINE, writeIdList); } catch (Exception e) { LOG.debug("Failed getAggrColStatsFor", e); - return new AggrStats(new ArrayList(),0); + return new AggrStats(new ArrayList<>(),0); } finally { perfLogger.perfLogEnd(CLASS_NAME, PerfLogger.HIVE_GET_AGGR_COL_STATS, "HS2-cache"); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java index 0ee681a26a65..5f29f1bb2442 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java @@ -25,6 +25,8 @@ import java.net.URISyntaxException; import java.util.Collections; import java.util.concurrent.ExecutorService; + +import com.google.common.collect.Maps; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.hive.common.classification.InterfaceAudience; @@ -32,6 +34,7 @@ import org.apache.hadoop.hive.common.type.SnapshotContext; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.HiveMetaHook; +import org.apache.hadoop.hive.metastore.api.AggrStats; import org.apache.hadoop.hive.metastore.api.ColumnStatistics; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; import org.apache.hadoop.hive.metastore.api.EnvironmentContext; @@ -246,10 +249,10 @@ default Map getOperatorDescProperties(OperatorDesc operatorDesc, } /** - * Return some basic statistics (numRows, numFiles, totalSize) calculated by the underlying storage handler + * Returns basic statistics (numRows, numFiles, totalSize) calculated by the underlying storage handler * implementation. - * @param partish a partish wrapper class - * @return map of basic statistics, can be null + * @param partish table/partition wrapper object + * @return map of basic statistics */ default Map getBasicStatistics(Partish partish) { return null; @@ -257,8 +260,8 @@ default Map getBasicStatistics(Partish partish) { /** * Compute basic statistics (numRows, numFiles, totalSize) for the given table/partition. - * @param partish a partish wrapper class - * @return map of basic statistics, can be null + * @param partish table/partition wrapper object + * @return map of basic statistics */ default Map computeBasicStatistics(Partish partish) { return null; @@ -271,50 +274,67 @@ default Map computeBasicStatistics(Partish partish) { default boolean canProvideBasicStatistics() { return false; } - - default boolean canProvidePartitionStatistics(org.apache.hadoop.hive.ql.metadata.Table hmsTable) { + + /** + * Check if the storage handler can provide partition statistics. + * @return true if the storage handler can supply the partition statistics + */ + default boolean canProvidePartitionStatistics(org.apache.hadoop.hive.ql.metadata.Table table) { return false; } /** - * Return some col statistics (Lower bounds, Upper bounds, Null value counts, NaN, total counts) calculated by - * the underlying storage handler implementation. - * @param table - * @return A List of Column Statistics Objects, can be null + * Returns column statistics (upper/lower bounds, number of Null/NaN values, NDVs, histogram). + * @param table table object + * @return list of ColumnStatisticsObj objects */ default List getColStatistics(org.apache.hadoop.hive.ql.metadata.Table table) { return null; } + /** + * Returns an aggregated column statistics for the supplied partition list + * @param table table object + * @param colNames list of column names + * @param partNames list of partition names + * @return AggrStats object + */ + default AggrStats getAggrColStatsFor(org.apache.hadoop.hive.ql.metadata.Table table, List colNames, + List partNames) throws MetaException { + return null; + } + /** * Set column stats for non-native tables - * @param table - * @param colStats - * @return boolean + * @param table table object + * @param colStats list of ColumnStatistics objects + * @return true if operation is successful */ default boolean setColStatistics(org.apache.hadoop.hive.ql.metadata.Table table, List colStats) { return false; } /** - * Check if the storage handler can provide col statistics. - * @param tbl - * @return true if the storage handler can supply the col statistics + * Check if the storage handler can provide column statistics. + * @param table table object + * @return true if the storage handler can supply the column statistics */ - default boolean canProvideColStatistics(org.apache.hadoop.hive.ql.metadata.Table tbl) { + default boolean canProvideColStatistics(org.apache.hadoop.hive.ql.metadata.Table table) { return false; } /** * Check if the storage handler can set col statistics. + * @param table table object * @return true if the storage handler can set the col statistics */ - default boolean canSetColStatistics(org.apache.hadoop.hive.ql.metadata.Table tbl) { + default boolean canSetColStatistics(org.apache.hadoop.hive.ql.metadata.Table table) { return false; } /** - * Check if the storage handler answer a few queries like count(1) purely using stats. + * Check if the storage handler can answer a few queries like count(1) purely using statistics. + * @param partish table/partition wrapper object * @return true if the storage handler can answer query using statistics */ default boolean canComputeQueryUsingStats(Partish partish) { @@ -322,8 +342,8 @@ default boolean canComputeQueryUsingStats(Partish partish) { } @Deprecated - default boolean canComputeQueryUsingStats(org.apache.hadoop.hive.ql.metadata.Table tbl) { - return canComputeQueryUsingStats(Partish.buildFor(tbl)); + default boolean canComputeQueryUsingStats(org.apache.hadoop.hive.ql.metadata.Table table) { + return canComputeQueryUsingStats(Partish.buildFor(table)); } /** @@ -553,6 +573,10 @@ default List getPartitionTransformSpec(org.apache.hadoop.hive.ql. return null; } + default Map> getPartitionTransformSpecs(org.apache.hadoop.hive.ql.metadata.Table table) { + return null; + } + /** * Creates a DynamicPartitionCtx instance that will be set up by the storage handler itself. Useful for non-native * tables where partitions are not handled by Hive, and sorting is required in a custom way before writing the table. @@ -791,15 +815,10 @@ default Boolean hasAppendsOnly(org.apache.hadoop.hive.ql.metadata.Table hmsTable return null; } - /** - * Checks if storage handler supports Show Partitions and returns a list of partitions - * @return List of partitions - * @throws UnsupportedOperationException - * @throws HiveException - */ + @Deprecated default List showPartitions(DDLOperationContext context, org.apache.hadoop.hive.ql.metadata.Table tbl) throws UnsupportedOperationException, HiveException { - throw new UnsupportedOperationException("Storage handler does not support show partitions command"); + return getPartitionNames(tbl); } /** @@ -830,15 +849,19 @@ default boolean canUseTruncate(org.apache.hadoop.hive.ql.metadata.Table hmsTable /** * Returns partitions names for the current table spec that correspond to the provided partition spec. - * @param hmsTable {@link org.apache.hadoop.hive.ql.metadata.Table} table metadata stored in Hive Metastore + * @param table {@link org.apache.hadoop.hive.ql.metadata.Table} table metadata stored in Hive Metastore * @param partitionSpec Map of Strings {@link java.util.Map} partition specification * @return List of partition names */ - default List getPartitionNames(org.apache.hadoop.hive.ql.metadata.Table hmsTable, + default List getPartitionNames(org.apache.hadoop.hive.ql.metadata.Table table, Map partitionSpec) throws SemanticException { throw new UnsupportedOperationException("Storage handler does not support getting partition names"); } + default List getPartitionNames(org.apache.hadoop.hive.ql.metadata.Table table) throws SemanticException { + return getPartitionNames(table, Maps.newHashMap()); + } + default ColumnInfo getColumnInfo(org.apache.hadoop.hive.ql.metadata.Table hmsTable, String colName) throws SemanticException { throw new UnsupportedOperationException("Storage handler does not support getting column type " + @@ -854,32 +877,26 @@ default List getPartitionKeys(org.apache.hadoop.hive.ql.metadata.Ta throw new UnsupportedOperationException("Storage handler does not support getting partition keys for a table."); } - /** - * Returns a list of partitions with the latest partition spec which contain any files whose content falls under - * the provided filter condition. - * @param hmsTable {@link org.apache.hadoop.hive.ql.metadata.Table} table metadata stored in Hive Metastore - * @param filter Iceberg filter expression - * @return List of Partitions {@link org.apache.hadoop.hive.ql.metadata.Partition} - */ - default List getPartitionsByExpr(org.apache.hadoop.hive.ql.metadata.Table hmsTable, ExprNodeDesc filter) - throws SemanticException { - throw new UnsupportedOperationException("Storage handler does not support getting partitions by expression " + - "for a table."); - } - /** * Returns a list of partitions which contain any files whose content falls under the provided filter condition. - * @param hmsTable {@link org.apache.hadoop.hive.ql.metadata.Table} table metadata stored in Hive Metastore + * @param table {@link org.apache.hadoop.hive.ql.metadata.Table} table metadata stored in Hive Metastore * @param filter Iceberg filter expression - * @param latestSpecOnly When true, returns partitions with the latest partition spec, else with the older specs only. + * @param latestSpecOnly when True, returns partitions with the current spec only; + * False - older specs only; + * Null - any spec * @return List of Partitions {@link org.apache.hadoop.hive.ql.metadata.Partition} */ - default List getPartitionsByExpr(org.apache.hadoop.hive.ql.metadata.Table hmsTable, - ExprNodeDesc filter, boolean latestSpecOnly) throws SemanticException { + default List getPartitionsByExpr(org.apache.hadoop.hive.ql.metadata.Table table, + ExprNodeDesc filter, Boolean latestSpecOnly) throws SemanticException { throw new UnsupportedOperationException("Storage handler does not support getting partitions " + "by generic expressions"); } + default List getPartitionsByExpr(org.apache.hadoop.hive.ql.metadata.Table table, ExprNodeDesc filter) + throws SemanticException { + return getPartitionsByExpr(table, filter, null); + } + /** * Returns partition based on table and partition specification. * @param table {@link org.apache.hadoop.hive.ql.metadata.Table} table metadata stored in Hive Metastore @@ -888,7 +905,7 @@ default List getPartitionsByExpr(org.apache.hadoop.hive.ql.metadata.T * @throws SemanticException {@link org.apache.hadoop.hive.ql.parse.SemanticException} */ default Partition getPartition(org.apache.hadoop.hive.ql.metadata.Table table, Map partitionSpec) - throws SemanticException { + throws SemanticException { return getPartition(table, partitionSpec, Context.RewritePolicy.DEFAULT); } @@ -901,37 +918,30 @@ default Partition getPartition(org.apache.hadoop.hive.ql.metadata.Table table, M * @throws SemanticException {@link org.apache.hadoop.hive.ql.parse.SemanticException} */ default Partition getPartition(org.apache.hadoop.hive.ql.metadata.Table table, Map partitionSpec, - Context.RewritePolicy policy) throws SemanticException { + Context.RewritePolicy policy) throws SemanticException { throw new UnsupportedOperationException("Storage handler does not support getting partition for a table."); } /** - * Returns a list of partitions with the latest partition spec based on table and partial partition specification. + * Returns a list of partitions based on table and partial partition specification. * @param table {@link org.apache.hadoop.hive.ql.metadata.Table} table metadata stored in Hive Metastore * @param partitionSpec Map of Strings {@link java.util.Map} partition specification + * @param latestSpecOnly when True, returns partitions with the current spec only, else - any spec * @return List of Partitions {@link org.apache.hadoop.hive.ql.metadata.Partition} * @throws SemanticException {@link org.apache.hadoop.hive.ql.parse.SemanticException} */ - default List getPartitions(org.apache.hadoop.hive.ql.metadata.Table table, - Map partitionSpec) throws SemanticException { - return getPartitions(table, partitionSpec, true); + default List getPartitions(org.apache.hadoop.hive.ql.metadata.Table table, + Map partitionSpec, boolean latestSpecOnly) throws SemanticException { + throw new UnsupportedOperationException("Storage handler does not support getting partitions for a table."); } - default List getPartitions(org.apache.hadoop.hive.ql.metadata.Table table) throws SemanticException { - return getPartitions(table, Collections.emptyMap()); + default List getPartitions(org.apache.hadoop.hive.ql.metadata.Table table, + Map partitionSpec) throws SemanticException { + return getPartitions(table, partitionSpec, false); } - /** - * Returns a list of partitions based on table and partial partition specification. - * @param table {@link org.apache.hadoop.hive.ql.metadata.Table} table metadata stored in Hive Metastore - * @param partitionSpec Map of Strings {@link java.util.Map} partition specification - * @param latestSpecOnly When true, returns partitions with the latest partition spec, else with older specs - * @return List of Partitions {@link org.apache.hadoop.hive.ql.metadata.Partition} - * @throws SemanticException {@link org.apache.hadoop.hive.ql.parse.SemanticException} - */ - default List getPartitions(org.apache.hadoop.hive.ql.metadata.Table table, - Map partitionSpec, boolean latestSpecOnly) throws SemanticException { - throw new UnsupportedOperationException("Storage handler does not support getting partitions for a table."); + default List getPartitions(org.apache.hadoop.hive.ql.metadata.Table table) throws SemanticException { + return getPartitions(table, Collections.emptyMap()); } default boolean isPartitioned(org.apache.hadoop.hive.ql.metadata.Table table) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java index aa8e0d0bb91c..e71823be474c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java @@ -197,8 +197,7 @@ public PartitionIterable(Hive db, GetPartitionsRequest getPartitionsRequest, int pVals = this.getPartitionsRequest.getFilterSpec().getFilters(); } if (pVals == null) { - partitionNames = db.getPartitionNames( - table.getDbName(),table.getTableName(), (short) -1); + partitionNames = db.getPartitionNames(table, (short) -1); } else { PartitionFilterMode filterMode = getPartitionsRequest.getFilterSpec().getFilterMode(); switch (filterMode) { @@ -206,8 +205,7 @@ public PartitionIterable(Hive db, GetPartitionsRequest getPartitionsRequest, int partitionNames = pVals; break; case BY_VALUES: - partitionNames = db.getPartitionNamesByPartitionVals( - table.getDbName(),table.getTableName(),pVals,(short)-1); + partitionNames = db.getPartitionNamesByPartitionVals(table, pVals, (short) -1); break; case BY_EXPR: // TO-DO: this can be dealt with in a seperate PR. The current changes does not have a particular use case for this. @@ -235,11 +233,9 @@ private PartitionIterable(Hive db, Table table, Map partialParti this.groupNames = groupNames; if (this.partialPartitionSpec == null){ - partitionNames = db.getPartitionNames( - table.getDbName(),table.getTableName(), (short) -1); + partitionNames = db.getPartitionNames(table, (short) -1); } else { - partitionNames = db.getPartitionNames( - table.getDbName(),table.getTableName(),partialPartitionSpec,(short)-1); + partitionNames = db.getPartitionNames(table, partialPartitionSpec, (short) -1); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java index 175d25b835e3..3ed16bae7ee5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java @@ -302,8 +302,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, "Skip StatsOptimizer."); return null; } - if (MetaStoreUtils.isNonNativeTable(tbl.getTTable()) - && !tbl.getStorageHandler().canProvideBasicStatistics()) { + if (tbl.isNonNative() && !tbl.getStorageHandler().canProvideBasicStatistics()) { Logger.info("Table " + tbl.getTableName() + " is non Native table. Skip StatsOptimizer."); return null; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java index 4fbdbea6c382..f90783ff8743 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java @@ -212,7 +212,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, public static ExprNodeGenericFuncDesc getOutExpr( ExprNodeGenericFuncDesc funcExpr, Object[] nodeOutputs) { - ArrayList children = new ArrayList(); + List children = new ArrayList<>(); if (nodeOutputs != null) { for (Object child : nodeOutputs) { NodeInfoWrapper wrapper = (NodeInfoWrapper) child; @@ -289,7 +289,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, private Object handleDeterministicUdf(PcrExprProcCtx ctx, ExprNodeGenericFuncDesc fd, Object... nodeOutputs) throws SemanticException { - Boolean has_part_col = checkForPartColsAndUnknown(fd, nodeOutputs); + Boolean has_part_col = checkForPartColsAndUnknown(nodeOutputs); if (has_part_col == null) { return new NodeInfoWrapper(WalkState.UNKNOWN, null, getOutExpr(fd, nodeOutputs)); } @@ -322,15 +322,17 @@ private Object handleDeterministicUdf(PcrExprProcCtx ctx, // constant values. return new NodeInfoWrapper(WalkState.UNKNOWN, null, getOutExpr(fd, nodeOutputs)); } - return new NodeInfoWrapper(WalkState.CONSTANT, null, - new ExprNodeConstantDesc(fd.getTypeInfo(), result)); + return new NodeInfoWrapper( + WalkState.CONSTANT, null, new ExprNodeConstantDesc(fd.getTypeInfo(), result)); } // Try to fold, otherwise return the expression itself final ExprNodeGenericFuncDesc desc = getOutExpr(fd, nodeOutputs); final ExprNodeDesc foldedDesc = ConstantPropagateProcFactory.foldExpr(desc); + if (foldedDesc instanceof ExprNodeConstantDesc) { ExprNodeConstantDesc constant = (ExprNodeConstantDesc) foldedDesc; + if (Boolean.TRUE.equals(constant.getValue())) { return new NodeInfoWrapper(WalkState.TRUE, null, constant); } else if (Boolean.FALSE.equals(constant.getValue())) { @@ -342,8 +344,7 @@ private Object handleDeterministicUdf(PcrExprProcCtx ctx, return new NodeInfoWrapper(WalkState.CONSTANT, null, desc); } - private Boolean checkForPartColsAndUnknown(ExprNodeGenericFuncDesc fd, - Object... nodeOutputs) { + private Boolean checkForPartColsAndUnknown(Object... nodeOutputs) { boolean has_part_col = false; for (Object child : nodeOutputs) { NodeInfoWrapper wrapper = (NodeInfoWrapper) child; @@ -361,8 +362,8 @@ private Object handleUdfOr(PcrExprProcCtx ctx, ExprNodeGenericFuncDesc fd, Object... nodeOutputs) { boolean anyUnknown = false; // Whether any of the node outputs is unknown boolean allDivided = true; // Whether all of the node outputs are divided - List newNodeOutputsList = - new ArrayList(nodeOutputs.length); + + List newNodeOutputsList = new ArrayList<>(nodeOutputs.length); for (int i = 0; i< nodeOutputs.length; i++) { NodeInfoWrapper c = (NodeInfoWrapper)nodeOutputs[i]; if (c.state == WalkState.TRUE) { @@ -409,8 +410,8 @@ private Object handleUdfAnd(PcrExprProcCtx ctx, ExprNodeGenericFuncDesc fd, Object... nodeOutputs) { boolean anyUnknown = false; // Whether any of the node outputs is unknown boolean allDivided = true; // Whether all of the node outputs are divided - List newNodeOutputsList = - new ArrayList(nodeOutputs.length); + + List newNodeOutputsList = new ArrayList<>(nodeOutputs.length); for (int i = 0; i < nodeOutputs.length; i++) { NodeInfoWrapper c = (NodeInfoWrapper)nodeOutputs[i]; if (c.state == WalkState.FALSE) { @@ -477,7 +478,7 @@ private Object handleUdfNot(PcrExprProcCtx ctx, ExprNodeGenericFuncDesc fd, getOutExpr(fd, nodeOutputs)); } } - }; + } /** * FieldExprProcessor. @@ -552,15 +553,14 @@ public static SemanticNodeProcessor getColumnProcessor() { * @param pred * expression tree of the target filter operator * @return the node information of the root expression - * @throws SemanticException */ public static NodeInfoWrapper walkExprTree( - String tabAlias, ArrayList parts, List vcs, ExprNodeDesc pred) + String tabAlias, List parts, List vcs, ExprNodeDesc pred) throws SemanticException { // Create the walker, the rules dispatcher and the context. PcrExprProcCtx pprCtx = new PcrExprProcCtx(tabAlias, parts, vcs); - Map exprRules = new LinkedHashMap(); + Map exprRules = new LinkedHashMap<>(); exprRules.put( new RuleRegExp("R1", ExprNodeColumnDesc.class.getName() + "%"), getColumnProcessor()); @@ -576,10 +576,10 @@ public static NodeInfoWrapper walkExprTree( exprRules, pprCtx); SemanticGraphWalker egw = new ExpressionWalker(disp); - List startNodes = new ArrayList(); + List startNodes = new ArrayList<>(); startNodes.add(pred); - HashMap outputMap = new HashMap(); + HashMap outputMap = new HashMap<>(); egw.startWalking(startNodes, outputMap); // Return the wrapper of the root node diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java index 48c56de6d1de..91340b1b76ef 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java @@ -55,7 +55,10 @@ static public Object evalExprWithPart(ExprNodeDesc expr, Partition p) throws Hiv Properties partProps = p.getSchema(); String[] partKeyTypes; - if (DDLUtils.isIcebergTable(p.getTable())) { + if (p.getTable().hasNonNativePartitionSupport()) { + if (!partSpec.keySet().containsAll(expr.getCols())) { + return null; + } partKeyTypes = p.getTable().getStorageHandler().getPartitionKeys(p.getTable()).stream() .map(FieldSchema::getType).toArray(String[]::new); } else { @@ -64,8 +67,11 @@ static public Object evalExprWithPart(ExprNodeDesc expr, Partition p) throws Hiv } if (partSpec.size() != partKeyTypes.length) { - throw new HiveException("Internal error : Partition Spec size, " + partSpec.size() + - " doesn't match partition key definition size, " + partKeyTypes.length); + if (DDLUtils.isIcebergTable(p.getTable())) { + return null; + } + throw new HiveException("Internal error : Partition Spec size, " + partSpec.size() + + " doesn't match partition key definition size, " + partKeyTypes.length); } // Create the row object List partNames = new ArrayList<>(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java index bf4d10d55902..b1bc9eaf0a75 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java @@ -27,7 +27,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hive.conf.HiveConf; @@ -170,7 +169,6 @@ public static PrunedPartitionList prune(TableScanOperator ts, ParseContext parse * cached result for the table * @return the partition list for the table that satisfies the partition * pruner condition. - * @throws SemanticException */ public static PrunedPartitionList prune(Table tab, ExprNodeDesc prunerExpr, HiveConf conf, String alias, Map prunedPartitionsMap) @@ -209,7 +207,7 @@ public static PrunedPartitionList prune(Table tab, ExprNodeDesc prunerExpr, return getAllPartsFromCacheOrServer(tab, key, false, prunedPartitionsMap); } - Set partColsUsedInFilter = new LinkedHashSet(); + Set partColsUsedInFilter = new LinkedHashSet<>(); // Replace virtual columns with nulls. See javadoc for details. prunerExpr = removeNonPartCols(prunerExpr, tab.getPartColNames(), partColsUsedInFilter); // Remove all parts that are not partition columns. See javadoc for details. @@ -233,7 +231,7 @@ public static PrunedPartitionList prune(Table tab, ExprNodeDesc prunerExpr, } ppList = getPartitionsFromServer(tab, key, compactExpr, - conf, alias, partColsUsedInFilter, oldFilter.equals(compactExprString)); + conf, partColsUsedInFilter, oldFilter.equals(compactExprString)); prunedPartitionsMap.put(key, ppList); return ppList; } @@ -257,21 +255,20 @@ private static PrunedPartitionList getAllPartsFromCacheOrServer(Table tab, Strin return ppList; } - static private boolean isBooleanExpr(ExprNodeDesc expr) { - return expr != null && expr instanceof ExprNodeConstantDesc && - ((ExprNodeConstantDesc)expr).getTypeInfo() instanceof PrimitiveTypeInfo && - ((PrimitiveTypeInfo)(((ExprNodeConstantDesc)expr).getTypeInfo())). - getTypeName().equals(serdeConstants.BOOLEAN_TYPE_NAME); + private static boolean isBooleanExpr(ExprNodeDesc expr) { + return expr instanceof ExprNodeConstantDesc && + expr.getTypeInfo() instanceof PrimitiveTypeInfo && + expr.getTypeInfo().getTypeName().equals(serdeConstants.BOOLEAN_TYPE_NAME); } - static private boolean isTrueExpr(ExprNodeDesc expr) { - return isBooleanExpr(expr) && - ((ExprNodeConstantDesc)expr).getValue() != null && - ((ExprNodeConstantDesc)expr).getValue().equals(Boolean.TRUE); + + private static boolean isTrueExpr(ExprNodeDesc expr) { + return isBooleanExpr(expr) && + Boolean.TRUE.equals(((ExprNodeConstantDesc) expr).getValue()); } - static private boolean isFalseExpr(ExprNodeDesc expr) { - return isBooleanExpr(expr) && - ((ExprNodeConstantDesc)expr).getValue() != null && - ((ExprNodeConstantDesc)expr).getValue().equals(Boolean.FALSE); + + private static boolean isFalseExpr(ExprNodeDesc expr) { + return isBooleanExpr(expr) && + Boolean.FALSE.equals(((ExprNodeConstantDesc) expr).getValue()); } /** @@ -306,7 +303,7 @@ static ExprNodeDesc compactExpr(ExprNodeDesc expr) { if (isAnd) { // Non-partition expressions are converted to nulls. - List newChildren = new ArrayList(); + List newChildren = new ArrayList<>(); boolean allTrue = true; for (ExprNodeDesc child : children) { ExprNodeDesc compactChild = compactExpr(child); @@ -326,7 +323,7 @@ static ExprNodeDesc compactExpr(ExprNodeDesc expr) { if (allTrue) { return new ExprNodeConstantDesc(Boolean.TRUE); } - if (newChildren.size() == 0) { + if (newChildren.isEmpty()) { return null; } if (newChildren.size() == 1) { @@ -337,7 +334,7 @@ static ExprNodeDesc compactExpr(ExprNodeDesc expr) { ((ExprNodeGenericFuncDesc) expr).setChildren(newChildren); } else if (isOr) { // Non-partition expressions are converted to nulls. - List newChildren = new ArrayList(); + List newChildren = new ArrayList<>(); boolean allFalse = true; boolean isNull = false; for (ExprNodeDesc child : children) { @@ -444,8 +441,9 @@ static private boolean hasUserFunctions(ExprNodeDesc expr) { return false; } - private static PrunedPartitionList getPartitionsFromServer(Table tab, final String key, final ExprNodeDesc compactExpr, - HiveConf conf, String alias, Set partColsUsedInFilter, boolean isPruningByExactFilter) throws SemanticException { + private static PrunedPartitionList getPartitionsFromServer(Table tab, String key, ExprNodeDesc compactExpr, + HiveConf conf, Set partColsUsedInFilter, boolean isPruningByExactFilter) + throws SemanticException { try { // Finally, check the filter for non-built-in UDFs. If these are present, we cannot @@ -453,7 +451,7 @@ private static PrunedPartitionList getPartitionsFromServer(Table tab, final Stri boolean doEvalClientSide = hasUserFunctions(compactExpr); // Now filter. - List partitions = new ArrayList(); + List partitions = new ArrayList<>(); boolean hasUnknownPartitions = false; PerfLogger perfLogger = SessionState.getPerfLogger(); if (!doEvalClientSide) { @@ -477,8 +475,7 @@ private static PrunedPartitionList getPartitionsFromServer(Table tab, final Stri // evaluator returning null for a partition, or if we sent a partial expression to // metastore and so some partitions may have no data based on other filters. return new PrunedPartitionList(tab, key, - new LinkedHashSet(partitions), - new ArrayList(partColsUsedInFilter), + new LinkedHashSet<>(partitions), new ArrayList<>(partColsUsedInFilter), hasUnknownPartitions || !isPruningByExactFilter); } catch (SemanticException e) { throw e; @@ -509,8 +506,7 @@ static private boolean pruneBySequentialScan(Table tab, List partitio PerfLogger perfLogger = SessionState.getPerfLogger(); perfLogger.perfLogBegin(CLASS_NAME, PerfLogger.PRUNE_LISTING); - List partNames = Hive.get().getPartitionNames( - tab.getDbName(), tab.getTableName(), (short) -1); + List partNames = Hive.get().getPartitionNames(tab, (short) -1); String defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULT_PARTITION_NAME); List partCols = tab.getPartColNames(); @@ -530,7 +526,7 @@ static private boolean pruneBySequentialScan(Table tab, List partitio private static List extractPartColTypes(Table tab) { List pCols = tab.getPartCols(); - List partColTypeInfos = new ArrayList(pCols.size()); + List partColTypeInfos = new ArrayList<>(pCols.size()); for (FieldSchema pCol : pCols) { partColTypeInfos.add(TypeInfoFactory.getPrimitiveTypeInfo(pCol.getType())); } @@ -556,10 +552,10 @@ public static boolean prunePartitionNames(List partColumnNames, // Filter the name list. Removing elements one by one can be slow on e.g. ArrayList, // so let's create a new list and copy it if we don't have a linked list boolean inPlace = partNames instanceof AbstractSequentialList; - List partNamesSeq = inPlace ? partNames : new LinkedList(partNames); + List partNamesSeq = inPlace ? partNames : new LinkedList<>(partNames); // Array for the values to pass to evaluator. - ArrayList values = new ArrayList(partColumnNames.size()); + ArrayList values = new ArrayList<>(partColumnNames.size()); for (int i = 0; i < partColumnNames.size(); ++i) { values.add(null); } @@ -570,7 +566,7 @@ public static boolean prunePartitionNames(List partColumnNames, String partName = partIter.next(); Warehouse.makeValsFromName(partName, values); - ArrayList convertedValues = new ArrayList(values.size()); + List convertedValues = new ArrayList<>(values.size()); for(int i=0; i partitionColumns; private boolean isInsertInto; private Table tbl; + private List partTransformSpec; private Map partSpec; private Context origCtx; @@ -122,16 +123,18 @@ public void insertAnalyzePipeline() throws SemanticException { */ public void insertTableValuesAnalyzePipeline() throws SemanticException { // Instead of starting from analyze statement, we just generate the Select plan - boolean isPartitionStats = conf.getBoolVar(ConfVars.HIVE_STATS_COLLECT_PART_LEVEL_STATS) && tbl.isPartitioned(); + boolean isPartitionStats = StatsUtils.isPartitionStats(tbl, conf); if (isPartitionStats) { partSpec = new HashMap<>(); List partKeys = Utilities.getColumnNamesFromFieldSchema(tbl.getPartitionKeys()); - for (String partKey : partKeys) { - partSpec.put(partKey, null); + partKeys.forEach(k -> partSpec.put(k, null)); + + if (tbl.hasNonNativePartitionSupport()) { + partTransformSpec = tbl.getStorageHandler().getPartitionTransformSpec(tbl); } } String command = ColumnStatsSemanticAnalyzer.genRewrittenQuery( - tbl, conf, partSpec, isPartitionStats, true); + tbl, conf, partTransformSpec, partSpec, isPartitionStats); insertAnalyzePipeline(command, true); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java index a52f3f5073f6..023934d9eb24 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java @@ -19,12 +19,17 @@ package org.apache.hadoop.hive.ql.parse; import static org.apache.hadoop.hive.ql.metadata.HiveUtils.unparseIdentifier; +import static org.apache.hadoop.hive.ql.metadata.VirtualColumn.PARTITION_SPEC_ID; import com.google.common.base.Preconditions; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; +import com.google.common.collect.Maps; import org.apache.hadoop.hive.common.HiveStatsUtils; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; @@ -43,6 +48,7 @@ import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; import org.apache.hadoop.hive.ql.stats.ColStatsProcessor.ColumnStatsField; import org.apache.hadoop.hive.ql.stats.ColStatsProcessor.ColumnStatsType; +import org.apache.hadoop.hive.ql.stats.StatsUtils; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category; import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; @@ -104,7 +110,7 @@ private List getColumnName(ASTNode tree) throws SemanticException { return Utilities.getColumnNamesFromFieldSchema(tbl.getCols()); case 3: int numCols = tree.getChild(2).getChildCount(); - List colName = new ArrayList(numCols); + List colName = new ArrayList<>(numCols); for (int i = 0; i < numCols; i++) { colName.add(getUnescapedName((ASTNode) tree.getChild(2).getChild(i))); } @@ -126,7 +132,7 @@ private void handlePartialPartitionSpec(Map partSpec, ColumnStat try { // for static partition, it may not exist when HIVE_STATS_COL_AUTOGATHER is // set to true - if (context == null) { + if (context == null && partValsSpecified > 0) { if ((partValsSpecified == tbl.getPartitionKeys().size()) && (db.getPartition(tbl, partSpec, false, null, false) == null)) { throw new SemanticException(ErrorMsg.COLUMNSTATSCOLLECTOR_INVALID_PARTITION.getMsg() @@ -154,34 +160,31 @@ private void handlePartialPartitionSpec(Map partSpec, ColumnStat } } - private static StringBuilder genPartitionClause(Table tbl, Map partSpec, HiveConf conf) - throws SemanticException { - StringBuilder whereClause = new StringBuilder(" where "); - boolean predPresent = false; - StringBuilder groupByClause = new StringBuilder(" group by "); - boolean aggPresent = false; - - for (Map.Entry part : partSpec.entrySet()) { - String value = part.getValue(); - if (value != null) { - if (!predPresent) { - predPresent = true; - } else { - whereClause.append(" and "); - } - whereClause.append(unparseIdentifier(part.getKey(), conf)).append(" = ") - .append(genPartValueString(getColTypeOf(tbl, part.getKey()), value)); - } + private static CharSequence genPartitionClause(Table tbl, List partTransformSpec, int specId, + Map partSpec, HiveConf conf) { + boolean predPresent = partSpec.values().stream().anyMatch(Objects::nonNull); + + StringBuilder whereClause = new StringBuilder(" where ").append( + partSpec.entrySet().stream() + .filter(part -> part.getValue() != null) + .map(part -> unparseIdentifier(part.getKey(), conf) + " = " + + genPartValueString(getColTypeOf(tbl, part.getKey()), part.getValue())) + .collect(Collectors.joining(" and ")) + ); + + if (specId >= 0) { + whereClause.append((predPresent) ? " and " : "") + .append(unparseIdentifier(PARTITION_SPEC_ID.getName(), conf) + "=" + specId); + predPresent = true; } - for (FieldSchema fs : tbl.getPartitionKeys()) { - if (!aggPresent) { - aggPresent = true; - } else { - groupByClause.append(','); - } - groupByClause.append(unparseIdentifier(fs.getName(), conf)); - } + StringBuilder groupByClause = new StringBuilder(" group by ").append(( + (partTransformSpec != null) ? + partTransformSpec.stream().map(spec -> spec.toHiveExpr(conf)) : + tbl.getPartColNames().stream().map(col -> unparseIdentifier(col, conf)) + ) + .collect(Collectors.joining(", ")) + ); // attach the predicate and group by to the return clause return predPresent ? whereClause.append(groupByClause) : groupByClause; @@ -189,17 +192,18 @@ private static StringBuilder genPartitionClause(Table tbl, Map p - private static String getColTypeOf(Table tbl, String partKey) throws SemanticException{ - for (FieldSchema fs : tbl.getPartitionKeys()) { + private static String getColTypeOf(Table tbl, String partKey) { + for (FieldSchema fs : tbl.hasNonNativePartitionSupport() ? + tbl.getStorageHandler().getPartitionKeys(tbl) : tbl.getPartitionKeys()) { if (partKey.equalsIgnoreCase(fs.getName())) { return fs.getType().toLowerCase(); } } - throw new SemanticException("Unknown partition key : " + partKey); + throw new RuntimeException("Unknown partition key : " + partKey); } protected static List getColumnTypes(Table tbl, List colNames) { - List colTypes = new ArrayList(); + List colTypes = new ArrayList<>(); List cols = tbl.getCols(); List copyColNames = new ArrayList<>(colNames); @@ -222,8 +226,10 @@ protected static List getColumnTypes(Table tbl, List colNames) { } private String genRewrittenQuery(List colNames, List colTypes, HiveConf conf, - Map partSpec, boolean isPartitionStats) throws SemanticException { - String rewritten = genRewrittenQuery(tbl, colNames, colTypes, conf, partSpec, isPartitionStats, false); + List partTransformSpec, int specId, Map partSpec, + boolean isPartitionStats) { + String rewritten = genRewrittenQuery(tbl, colNames, colTypes, conf, partTransformSpec, specId, partSpec, + isPartitionStats, false); isRewritten = true; return rewritten; } @@ -233,71 +239,87 @@ private String genRewrittenQuery(List colNames, List colTypes, H * included in the input table. */ protected static String genRewrittenQuery(Table tbl, - HiveConf conf, Map partSpec, boolean isPartitionStats, - boolean useTableValues) throws SemanticException { + HiveConf conf, List partTransformSpec, Map partSpec, + boolean isPartitionStats) { List colNames = Utilities.getColumnNamesFromFieldSchema(tbl.getCols()); List colTypes = ColumnStatsSemanticAnalyzer.getColumnTypes(tbl, colNames); return ColumnStatsSemanticAnalyzer.genRewrittenQuery( - tbl, colNames, colTypes, conf, partSpec, isPartitionStats, useTableValues); + tbl, colNames, colTypes, conf, partTransformSpec, -1, partSpec, isPartitionStats, true); } private static String genRewrittenQuery(Table tbl, List colNames, List colTypes, - HiveConf conf, Map partSpec, boolean isPartitionStats, - boolean useTableValues) throws SemanticException { + HiveConf conf, List partTransformSpec, int specId, Map partSpec, + boolean isPartitionStats, boolean useTableValues) { StringBuilder rewrittenQueryBuilder = new StringBuilder("select "); StringBuilder columnNamesBuilder = new StringBuilder(); StringBuilder columnDummyValuesBuilder = new StringBuilder(); for (int i = 0; i < colNames.size(); i++) { if (i > 0) { - rewrittenQueryBuilder.append(" , "); - columnNamesBuilder.append(" , "); - columnDummyValuesBuilder.append(" , "); + rewrittenQueryBuilder.append(", "); + columnNamesBuilder.append(", "); + columnDummyValuesBuilder.append(", "); } final String columnName = unparseIdentifier(colNames.get(i), conf); final TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(colTypes.get(i)); - genComputeStats(rewrittenQueryBuilder, conf, i, columnName, typeInfo); + + try { + genComputeStats(rewrittenQueryBuilder, conf, i, columnName, typeInfo); + } catch (SemanticException e) { + throw new RuntimeException(e); + } columnNamesBuilder.append(columnName); columnDummyValuesBuilder.append( - "cast(null as " + typeInfo.toString() + ")"); + "cast(null as " + typeInfo + ")"); } if (isPartitionStats) { - for (FieldSchema fs : tbl.getPartCols()) { - String identifier = unparseIdentifier(fs.getName(), conf); - rewrittenQueryBuilder.append(" , ").append(identifier); - columnNamesBuilder.append(" , ").append(identifier); - - columnDummyValuesBuilder.append(" , cast(null as ") - .append(TypeInfoUtils.getTypeInfoFromTypeString(fs.getType()).toString()).append(")"); + if (partTransformSpec == null) { + for (FieldSchema fs : tbl.getPartCols()) { + String identifier = unparseIdentifier(fs.getName(), conf); + rewrittenQueryBuilder.append(", ").append(identifier); + columnNamesBuilder.append(", ").append(identifier); + + columnDummyValuesBuilder.append(", cast(null as ") + .append(TypeInfoUtils.getTypeInfoFromTypeString(fs.getType()).toString()) + .append(")"); + } + } else { + rewrittenQueryBuilder.append(", ") + .append(TransformSpec.toNamedStruct(partTransformSpec, conf)); } } rewrittenQueryBuilder.append(" from "); if (useTableValues) { //TABLE(VALUES(cast(null as int),cast(null as string))) AS tablename(col1,col2) - rewrittenQueryBuilder.append("table(values("); - // Values - rewrittenQueryBuilder.append(columnDummyValuesBuilder.toString()); - rewrittenQueryBuilder.append(")) as "); - rewrittenQueryBuilder.append(unparseIdentifier(tbl.getTableName() ,conf)); - rewrittenQueryBuilder.append("("); - // Columns - rewrittenQueryBuilder.append(columnNamesBuilder.toString()); - rewrittenQueryBuilder.append(")"); + rewrittenQueryBuilder.append("table(values(") + // Values + .append(columnDummyValuesBuilder) + .append(")) as ") + .append(unparseIdentifier(tbl.getTableName() ,conf)) + .append("(") + // Columns + .append(columnNamesBuilder) + .append(")"); } else { - rewrittenQueryBuilder.append(unparseIdentifier(tbl.getDbName(), conf)); - rewrittenQueryBuilder.append("."); - rewrittenQueryBuilder.append(unparseIdentifier(tbl.getTableName(), conf)); + rewrittenQueryBuilder.append(unparseIdentifier(tbl.getDbName(), conf)) + .append(".") + .append(unparseIdentifier(tbl.getTableName(), conf)); + + if (tbl.getMetaTable() != null) { + rewrittenQueryBuilder.append(".") + .append(unparseIdentifier(tbl.getMetaTable(), conf)); + } } // If partition level statistics is requested, add predicate and group by as needed to rewritten // query if (isPartitionStats) { - rewrittenQueryBuilder.append(genPartitionClause(tbl, partSpec, conf)); + rewrittenQueryBuilder.append(genPartitionClause(tbl, partTransformSpec, specId, partSpec, conf)); } String rewrittenQuery = rewrittenQueryBuilder.toString(); @@ -554,7 +576,7 @@ private void validateSpecifiedColumnNames(List specifiedCols) List tableCols = Utilities.getColumnNamesFromFieldSchema(tbl.getCols()); for (String sc : specifiedCols) { if (!tableCols.contains(sc.toLowerCase())) { - String msg = "'" + sc + "' (possible columns are " + tableCols.toString() + ")"; + String msg = "'" + sc + "' (possible columns are " + tableCols + ")"; throw new SemanticException(ErrorMsg.INVALID_COLUMN.getMsg(msg)); } } @@ -599,25 +621,30 @@ public void analyze(ASTNode ast, Context origCtx) throws SemanticException { colNames = getColumnName(ast); // Save away the original AST originalTree = ast; - boolean isPartitionStats = AnalyzeCommandUtils.isPartitionLevelStats(ast); - Map partSpec = null; + boolean isPartitionStats = AnalyzeCommandUtils.isPartitionLevelStats(ast) + || StatsUtils.isPartitionStats(tbl, conf); + + Map> partTransformSpecs = Collections.singletonMap(-1, null); + Map partSpec = (isPartitionStats) ? + AnalyzeCommandUtils.getPartKeyValuePairsFromAST(tbl, ast, conf) : null; checkForPartitionColumns( colNames, Utilities.getColumnNamesFromFieldSchema(tbl.getPartitionKeys())); validateSpecifiedColumnNames(colNames); - if (conf.getBoolVar(ConfVars.HIVE_STATS_COLLECT_PART_LEVEL_STATS) && tbl.isPartitioned() - && !tbl.hasNonNativePartitionSupport()) { - isPartitionStats = true; - } if (isPartitionStats) { - isTableLevel = false; - partSpec = AnalyzeCommandUtils.getPartKeyValuePairsFromAST(tbl, ast, conf); handlePartialPartitionSpec(partSpec, null); - } else { - isTableLevel = true; + if (tbl.hasNonNativePartitionSupport()) { + partTransformSpecs = tbl.getStorageHandler().getPartitionTransformSpecs(tbl); + } } colType = getColumnTypes(tbl, colNames); - rewrittenQuery = genRewrittenQuery(colNames, colType, conf, partSpec, isPartitionStats); + isTableLevel = !isPartitionStats; + + rewrittenQuery = String.join(" union all ", + Maps.transformEntries(partTransformSpecs, (specId, partTransformSpec) -> + genRewrittenQuery(colNames, colType, conf, partTransformSpec, specId, partSpec, isPartitionStats)) + .values()); + rewrittenTree = genRewrittenTree(rewrittenQuery); } else { // Not an analyze table column compute statistics statement - don't do any rewrites @@ -657,8 +684,7 @@ public void analyze(ASTNode ast, Context origCtx) throws SemanticException { * is the original analyze ast * @param context * the column stats auto gather context - * @return - * @throws SemanticException + * @return the rewritten AST */ public ASTNode rewriteAST(ASTNode ast, ColumnStatsAutoGatherContext context) throws SemanticException { @@ -668,26 +694,27 @@ public ASTNode rewriteAST(ASTNode ast, ColumnStatsAutoGatherContext context) tbl = AnalyzeCommandUtils.getTable(ast, this); colNames = getColumnName(ast); - boolean isPartitionStats = AnalyzeCommandUtils.isPartitionLevelStats(ast); + boolean isPartitionStats = AnalyzeCommandUtils.isPartitionLevelStats(ast) + || StatsUtils.isPartitionStats(tbl, conf); + + List partTransformSpec = null; Map partSpec = null; checkForPartitionColumns(colNames, Utilities.getColumnNamesFromFieldSchema(tbl.getPartitionKeys())); validateSpecifiedColumnNames(colNames); - if (conf.getBoolVar(ConfVars.HIVE_STATS_COLLECT_PART_LEVEL_STATS) && tbl.isPartitioned() - && !tbl.hasNonNativePartitionSupport()) { - isPartitionStats = true; - } if (isPartitionStats) { partSpec = AnalyzeCommandUtils.getPartKeyValuePairsFromAST(tbl, ast, conf); handlePartialPartitionSpec(partSpec, context); + if (tbl.hasNonNativePartitionSupport()) { + partTransformSpec = tbl.getStorageHandler().getPartitionTransformSpec(tbl); + } } - colType = getColumnTypes(tbl, colNames); - isTableLevel = !isPartitionStats; - rewrittenQuery = genRewrittenQuery(colNames, colType, conf, partSpec, isPartitionStats); + rewrittenQuery = genRewrittenQuery(colNames, colType, conf, partTransformSpec, -1, + partSpec, isPartitionStats); rewrittenTree = genRewrittenTree(rewrittenQuery); return rewrittenTree; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TransformSpec.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TransformSpec.java index 625924dfb2aa..e50be89450c6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TransformSpec.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TransformSpec.java @@ -17,10 +17,16 @@ */ package org.apache.hadoop.hive.ql.parse; +import org.apache.hadoop.conf.Configuration; + +import java.util.List; import java.util.Locale; import java.util.Optional; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static org.apache.hadoop.hive.ql.metadata.HiveUtils.unparseIdentifier; public class TransformSpec { @@ -33,6 +39,8 @@ public enum TransformType { private TransformType transformType; private Optional transformParam; + private String fieldName; + public TransformSpec() { } @@ -66,6 +74,14 @@ public void setTransformParam(Optional transformParam) { this.transformParam = transformParam; } + public void setFieldName(String fieldName) { + this.fieldName = fieldName; + } + + public String getFieldName() { + return fieldName; + } + public String transformTypeString() { if (transformType == null) { return null; @@ -75,6 +91,31 @@ public String transformTypeString() { } return transformType.name(); } + + public static String toNamedStruct(List partTransformSpec, Configuration conf) { + StringBuilder builder = new StringBuilder("named_struct("); + builder.append( + partTransformSpec.stream().map(spec -> + "'" + spec.getColumnName() + "', " + spec.toHiveExpr(conf)) + .collect(Collectors.joining(", ")) + ).append(")"); + return builder.toString(); + } + + public String toHiveExpr(Configuration conf) { + String identifier = unparseIdentifier(columnName, conf); + if (transformType == TransformSpec.TransformType.IDENTITY) { + return identifier; + } + String fn = "iceberg_" + transformType.name().toLowerCase() + "(" + identifier; + switch (transformType) { + case BUCKET: + case TRUNCATE: + fn += ", " + transformParam.get(); + break; + } + return fn + ")"; + } public static TransformType fromString(String transformString) { Matcher widthMatcher = HAS_WIDTH.matcher(transformString); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStats.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStats.java index 4e7b1027eb34..4e43dad5a350 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStats.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStats.java @@ -231,8 +231,8 @@ private long getFileSizeForPath(Path path) throws IOException { public BasicStats(Partish p) { partish = p; + tryGetBasicStatsFromStorageHandler(); - checkForBasicStatsFromStorageHandler(); rowCount = parseLong(StatsSetupConst.ROW_COUNT); rawDataSize = parseLong(StatsSetupConst.RAW_DATA_SIZE); totalSize = parseLong(StatsSetupConst.TOTAL_SIZE); @@ -272,10 +272,11 @@ public BasicStats(List partStats) { } - private void checkForBasicStatsFromStorageHandler() { + private void tryGetBasicStatsFromStorageHandler() { if (partish.getTable() != null && partish.getTable().isNonNative() && partish.getTable().getStorageHandler().canProvideBasicStatistics()) { - partish.getPartParameters().putAll(partish.getTable().getStorageHandler().getBasicStatistics(partish)); + partish.getPartParameters().putAll( + partish.getTable().getStorageHandler().getBasicStatistics(partish)); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStatsTask.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStatsTask.java index 52e163246e71..fbeeb57c77d0 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStatsTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStatsTask.java @@ -126,15 +126,17 @@ private static class BasicStatsProcessor { private BasicStatsWork work; private boolean followedColStats1; private Map providedBasicStats; + private boolean skipUpdate = false; public BasicStatsProcessor(Partish partish, BasicStatsWork work, boolean followedColStats2) { this.partish = partish; this.work = work; - followedColStats1 = followedColStats2; + this.followedColStats1 = followedColStats2; Table table = partish.getTable(); if (table.isNonNative() && table.getStorageHandler().canProvideBasicStatistics()) { - providedBasicStats = table.getStorageHandler().computeBasicStatistics(partish); + this.providedBasicStats = table.getStorageHandler().computeBasicStatistics(partish); + this.skipUpdate = providedBasicStats.containsKey(StatsSetupConst.ROW_COUNT); } } @@ -175,19 +177,19 @@ public Object process(StatsAggregator statsAggregator) throws HiveException, Met if (providedBasicStats == null) { MetaStoreServerUtils.populateQuickStats(partfileStatus, parameters); - - if (statsAggregator != null) { - // Update stats for transactional tables (MM, or full ACID with overwrite), even - // though we are marking stats as not being accurate. - if (StatsSetupConst.areBasicStatsUptoDate(parameters) || p.isTransactionalTable()) { - String prefix = getAggregationPrefix(p.getTable(), p.getPartition()); - updateStats(statsAggregator, parameters, prefix); - } - } } else { parameters.putAll(providedBasicStats); } + if (statsAggregator != null && !skipUpdate) { + // Update stats for transactional tables (MM, or full ACID with overwrite), even + // though we are marking stats as not being accurate. + if (StatsSetupConst.areBasicStatsUptoDate(parameters) || p.isTransactionalTable()) { + String prefix = getAggregationPrefix(p.getTable(), p.getPartition()); + updateStats(statsAggregator, parameters, prefix); + } + } + return p.getOutput(); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/ColStatsProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/ColStatsProcessor.java index 285175414d62..2dd7c347a592 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/stats/ColStatsProcessor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/ColStatsProcessor.java @@ -141,14 +141,29 @@ private boolean constructColumnStatsFromPackedRows(Table tbl, List partColSchema = tbl.getPartCols(); + List partColSchema = new ArrayList<>(); List partVals = new ArrayList<>(); - // Iterate over partition columns to figure out partition name - for (int i = pos; i < pos + partColSchema.size(); i++) { - Object partVal = ((PrimitiveObjectInspector) fields.get(i).getFieldObjectInspector()) + + if (tbl.hasNonNativePartitionSupport()) { + ObjectInspector inspector = fields.get(pos).getFieldObjectInspector(); + if (inspector.getCategory() == ObjectInspector.Category.STRUCT) { + Object obj = values.get(pos); + StructObjectInspector oi = (StructObjectInspector) inspector; + + for (StructField field : oi.getAllStructFieldRefs()) { + partColSchema.add(new FieldSchema(field.getFieldName(), null, "")); + partVals.add(String.valueOf(oi.getStructFieldData(obj, field))); + } + } + } else { + partColSchema.addAll(tbl.getPartCols()); + // Iterate over partition columns to figure out partition name + for (int i = pos; i < pos + partColSchema.size(); i++) { + Object partVal = ((PrimitiveObjectInspector) fields.get(i).getFieldObjectInspector()) .getPrimitiveJavaObject(values.get(i)); - partVals.add(partVal == null ? // could be null for default partition - this.conf.getVar(ConfVars.DEFAULT_PARTITION_NAME) : partVal.toString()); + partVals.add(partVal == null ? // could be null for default partition + this.conf.getVar(ConfVars.DEFAULT_PARTITION_NAME) : partVal.toString()); + } } partName = Warehouse.makePartName(partColSchema, partVals); } @@ -221,14 +236,14 @@ public int persistColumnStats(Hive db, Table tbl) throws HiveException, MetaExce } start = System. currentTimeMillis(); - if (tbl != null && tbl.isNonNative() && tbl.getStorageHandler().canSetColStatistics(tbl)) { + if (tbl.isNonNative() && tbl.getStorageHandler().canSetColStatistics(tbl)) { boolean success = tbl.getStorageHandler().setColStatistics(tbl, colStats); if (!(tbl.isMaterializedView() || tbl.isView() || tbl.isTemporary())) { setOrRemoveColumnStatsAccurateProperty(db, tbl, colStatDesc.getColName(), success); } + } else { + db.setPartitionColumnStatistics(request); } - // TODO: Write stats for native tables only (See HIVE-27421) - db.setPartitionColumnStatistics(request); end = System.currentTimeMillis(); LOG.info("Time taken to update " + colStats.size() + " stats : " + ((end - start)/1000F) + " seconds."); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java index 794edbec1dab..c3bb1a64de00 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java @@ -50,7 +50,6 @@ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; import org.apache.hadoop.hive.metastore.api.Decimal; import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; -import org.apache.hadoop.hive.ql.ddl.DDLUtils; import org.apache.hadoop.hive.ql.exec.ColumnInfo; import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.exec.Operator; @@ -358,31 +357,6 @@ private static Statistics collectStatistics(HiveConf conf, PrunedPartitionList p } if (needColStats) { - - if (DDLUtils.isIcebergTable(table)) { - // TODO: replace with partition column stats once implemented - List colStats = getTableColumnStats(table, neededColumns, colStatsCache, fetchColStats); - if (estimateStats) { - estimateStatsForMissingCols(neededColumns, colStats, conf, nr, schema); - // we should have stats for all columns (estimated or actual) - if (neededColumns.size() == colStats.size()) { - long betterDS = getDataSizeFromColumnStats(nr, colStats); - stats.setDataSize((betterDS < 1 || colStats.isEmpty()) ? ds : betterDS); - } - } - // infer if any column can be primary key based on column statistics - inferAndSetPrimaryKey(stats.getNumRows(), colStats); - - stats.setColumnStatsState(deriveStatType(colStats, neededColumns)); - stats.addToColumnStats(colStats); - - if (partStats.isEmpty()) { - // all partitions are filtered by partition pruning - stats.setBasicStatsState(State.COMPLETE); - } - return stats; - } - List partitionCols = getPartitionColumns(schema, neededColumns, referencedColumns); // We will retrieve stats from the metastore only for columns that are not cached @@ -401,8 +375,7 @@ private static Statistics collectStatistics(HiveConf conf, PrunedPartitionList p // size is 0, aggrStats is null after several retries. Thus, we can // skip the step to connect to the metastore. if (fetchColStats && !neededColsToRetrieve.isEmpty() && !partNames.isEmpty()) { - aggrStats = Hive.get().getAggrColStatsFor(table.getDbName(), table.getTableName(), - neededColsToRetrieve, partNames, false); + aggrStats = Hive.get().getAggrColStatsFor(table, neededColsToRetrieve, partNames, false); } boolean statsRetrieved = aggrStats != null && @@ -1006,7 +979,7 @@ private static List estimateStats(List schema, } /** - * Get table level column statistics from metastore for needed columns + * Get table level column statistics for needed columns * @param table * - table * @param neededColumns @@ -1043,12 +1016,8 @@ public static List getTableColumnStats( } if (fetchColStats && !colStatsToRetrieve.isEmpty()) { try { - List colStat; - if (table.isNonNative() && table.getStorageHandler().canProvideColStatistics(table)) { - colStat = table.getStorageHandler().getColStatistics(table); - } else { - colStat = Hive.get().getTableColumnStatistics(dbName, tabName, colStatsToRetrieve, false); - } + List colStat = Hive.get().getTableColumnStatistics( + table, colStatsToRetrieve, false); stats = convertColStats(colStat); } catch (HiveException e) { LOG.error("Failed to retrieve table statistics: ", e); @@ -1978,6 +1947,11 @@ public static Range combineRange(Range range1, Range range2) { return null; } + public static boolean isPartitionStats(Table table, HiveConf conf) { + return conf.getBoolVar(ConfVars.HIVE_STATS_COLLECT_PART_LEVEL_STATS) && table.isPartitioned() + && (!table.isNonNative() || table.getStorageHandler().canSetColStatistics(table)); + } + public static boolean checkCanProvideStats(Table table) { return !MetaStoreUtils.isExternalTable(table.getTTable()) || table.isNonNative() && table.getStorageHandler().canProvideBasicStatistics(); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionInBatches.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionInBatches.java index 01768f3bc927..70d2389b0c84 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionInBatches.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionInBatches.java @@ -66,6 +66,7 @@ public class TestGetPartitionInBatches { @BeforeClass public static void setupClass() throws HiveException { hiveConf = new HiveConfForTest(TestGetPartitionInBatches.class); + hiveConf.set("hive.security.authorization.manager","org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider"); SessionState ss = SessionState.start(hiveConf); hive = ss.getHiveDb(); try { @@ -88,7 +89,7 @@ public void after() throws Exception { } @Test - public void TestNumberOfPartitionsRetrieved() throws HiveException { + public void testNumberOfPartitionsRetrieved() throws HiveException { List numParts = hive.getPartitionNames(dbName, tableName, (short)-1); Assert.assertEquals(numParts.size(), NUM_PARTITIONS); List partitions = hive.getPartitionsByNames(new org.apache.hadoop.hive.ql.metadata.Table(table), diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionsWithSpecsInBatches.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionsWithSpecsInBatches.java index 62d9f75110d6..f6a3d1c985cd 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionsWithSpecsInBatches.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionsWithSpecsInBatches.java @@ -51,6 +51,7 @@ public class TestGetPartitionsWithSpecsInBatches { @BeforeClass public static void setupClass() throws HiveException { hiveConf = new HiveConfForTest(TestGetPartitionsWithSpecsInBatches.class); + hiveConf.set("hive.security.authorization.manager","org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider"); SessionState ss = SessionState.start(hiveConf); hive = ss.getHiveDb(); try { diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreClient.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreClient.java index b220ceb127b4..b40f6753c3d7 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreClient.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreClient.java @@ -107,6 +107,8 @@ public Table getTable(String dbName, String tableName, boolean getColumnStats, S public Table getTable(GetTableRequest getTableRequest) throws TException { Table tTable = new Table(); + tTable.setDbName(getTableRequest.getDbName()); + tTable.setTableName(getTableRequest.getTblName()); tTable.setId(Long.MAX_VALUE); Map parameters = new HashMap<>(); parameters.put(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, "true"); diff --git a/ql/src/test/results/clientnegative/fileformat_void_input.q.out b/ql/src/test/results/clientnegative/fileformat_void_input.q.out index b629a5ebed25..477e19059e79 100644 --- a/ql/src/test/results/clientnegative/fileformat_void_input.q.out +++ b/ql/src/test/results/clientnegative/fileformat_void_input.q.out @@ -10,4 +10,4 @@ POSTHOOK: query: CREATE TABLE dest1(key INT, value STRING) STORED AS POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@dest1 -FAILED: SemanticException 1:693 Input format must implement InputFormat. Error encountered near token 'dest1' +FAILED: SemanticException 1:692 Input format must implement InputFormat. Error encountered near token 'dest1'