diff --git a/iceberg/checkstyle/checkstyle.xml b/iceberg/checkstyle/checkstyle.xml
index aef7d6fff6f8..c4179072ff4f 100644
--- a/iceberg/checkstyle/checkstyle.xml
+++ b/iceberg/checkstyle/checkstyle.xml
@@ -127,6 +127,7 @@
org.apache.iceberg.MetadataTableType.*,
org.apache.iceberg.SortDirection.*,
org.apache.iceberg.TableProperties.*,
+ org.apache.iceberg.SnapshotSummary.*,
org.apache.iceberg.types.Type.*,
org.apache.iceberg.types.Types.NestedField.*,
org.apache.parquet.schema.OriginalType.*,
diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/data/PartitionStatsHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/data/PartitionStatsHandler.java
new file mode 100644
index 000000000000..9cc1ffd3e915
--- /dev/null
+++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/data/PartitionStatsHandler.java
@@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ImmutableGenericPartitionStatisticsFile;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PartitionStatisticsFile;
+import org.apache.iceberg.PartitionStats;
+import org.apache.iceberg.PartitionStatsUtil;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.avro.InternalReader;
+import org.apache.iceberg.avro.InternalWriter;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Types.IntegerType;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.SnapshotUtil;
+
+// TODO: remove class once Iceberg PR #11216 is merged and released
+
+/**
+ * Computes, writes and reads the {@link PartitionStatisticsFile}. Uses generic readers and writers
+ * to support writing and reading of the stats in table default format.
+ */
+public final class PartitionStatsHandler {
+
+ private PartitionStatsHandler() {
+ }
+
+ public enum Column {
+ PARTITION(0),
+ SPEC_ID(1),
+ DATA_RECORD_COUNT(2),
+ DATA_FILE_COUNT(3),
+ TOTAL_DATA_FILE_SIZE_IN_BYTES(4),
+ POSITION_DELETE_RECORD_COUNT(5),
+ POSITION_DELETE_FILE_COUNT(6),
+ EQUALITY_DELETE_RECORD_COUNT(7),
+ EQUALITY_DELETE_FILE_COUNT(8),
+ TOTAL_RECORD_COUNT(9),
+ LAST_UPDATED_AT(10),
+ LAST_UPDATED_SNAPSHOT_ID(11);
+
+ private final int id;
+
+ Column(int id) {
+ this.id = id;
+ }
+
+ public int id() {
+ return id;
+ }
+ }
+
+ /**
+ * Generates the partition stats file schema based on a given partition type.
+ *
+ *
Note: Provide the unified partition schema type as mentioned in the spec.
+ *
+ * @param partitionType unified partition schema type.
+ * @return a schema that corresponds to the provided unified partition type.
+ */
+ public static Schema schema(StructType partitionType) {
+ Preconditions.checkState(!partitionType.fields().isEmpty(), "table must be partitioned");
+ return new Schema(
+ NestedField.required(1, Column.PARTITION.name(), partitionType),
+ NestedField.required(2, Column.SPEC_ID.name(), IntegerType.get()),
+ NestedField.required(3, Column.DATA_RECORD_COUNT.name(), LongType.get()),
+ NestedField.required(4, Column.DATA_FILE_COUNT.name(), IntegerType.get()),
+ NestedField.required(5, Column.TOTAL_DATA_FILE_SIZE_IN_BYTES.name(), LongType.get()),
+ NestedField.optional(6, Column.POSITION_DELETE_RECORD_COUNT.name(), LongType.get()),
+ NestedField.optional(7, Column.POSITION_DELETE_FILE_COUNT.name(), IntegerType.get()),
+ NestedField.optional(8, Column.EQUALITY_DELETE_RECORD_COUNT.name(), LongType.get()),
+ NestedField.optional(9, Column.EQUALITY_DELETE_FILE_COUNT.name(), IntegerType.get()),
+ NestedField.optional(10, Column.TOTAL_RECORD_COUNT.name(), LongType.get()),
+ NestedField.optional(11, Column.LAST_UPDATED_AT.name(), LongType.get()),
+ NestedField.optional(12, Column.LAST_UPDATED_SNAPSHOT_ID.name(), LongType.get()));
+ }
+
+ /**
+ * Computes and writes the {@link PartitionStatisticsFile} for a given table's current snapshot.
+ *
+ * @param table The {@link Table} for which the partition statistics is computed.
+ * @return {@link PartitionStatisticsFile} for the current snapshot.
+ */
+ public static PartitionStatisticsFile computeAndWriteStatsFile(Table table) {
+ return computeAndWriteStatsFile(table, null);
+ }
+
+ /**
+ * Computes and writes the {@link PartitionStatisticsFile} for a given table and branch.
+ *
+ * @param table The {@link Table} for which the partition statistics is computed.
+ * @param branch A branch information to select the required snapshot.
+ * @return {@link PartitionStatisticsFile} for the given branch.
+ */
+ public static PartitionStatisticsFile computeAndWriteStatsFile(Table table, String branch) {
+ Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch);
+ if (currentSnapshot == null) {
+ Preconditions.checkArgument(
+ branch == null, "Couldn't find the snapshot for the branch %s", branch);
+ return null;
+ }
+
+ StructType partitionType = Partitioning.partitionType(table);
+ Collection stats = PartitionStatsUtil.computeStats(table, currentSnapshot);
+ List sortedStats = PartitionStatsUtil.sortStats(stats, partitionType);
+ return writePartitionStatsFile(
+ table, currentSnapshot.snapshotId(), schema(partitionType), sortedStats.iterator());
+ }
+
+ @VisibleForTesting
+ static PartitionStatisticsFile writePartitionStatsFile(
+ Table table, long snapshotId, Schema dataSchema, Iterator records) {
+ OutputFile outputFile = newPartitionStatsFile(table, snapshotId);
+
+ try (DataWriter writer = dataWriter(dataSchema, outputFile)) {
+ records.forEachRemaining(writer::write);
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+
+ return ImmutableGenericPartitionStatisticsFile.builder()
+ .snapshotId(snapshotId)
+ .path(outputFile.location())
+ .fileSizeInBytes(outputFile.toInputFile().getLength())
+ .build();
+ }
+
+ /**
+ * Reads partition statistics from the specified {@link InputFile} using given schema.
+ *
+ * @param schema The {@link Schema} of the partition statistics file.
+ * @param inputFile An {@link InputFile} pointing to the partition stats file.
+ */
+ public static CloseableIterable readPartitionStatsFile(
+ Schema schema, InputFile inputFile) {
+ CloseableIterable records = dataReader(schema, inputFile);
+ return CloseableIterable.transform(records, PartitionStatsHandler::recordToPartitionStats);
+ }
+
+ private static FileFormat fileFormat(String fileLocation) {
+ return FileFormat.fromString(fileLocation.substring(fileLocation.lastIndexOf(".") + 1));
+ }
+
+ private static OutputFile newPartitionStatsFile(Table table, long snapshotId) {
+ FileFormat fileFormat = FileFormat.AVRO;
+ return table
+ .io()
+ .newOutputFile(
+ ((HasTableOperations) table)
+ .operations()
+ .metadataFileLocation(
+ fileFormat.addExtension(
+ String.format(Locale.ROOT, "partition-stats-%d", snapshotId))));
+ }
+
+ private static DataWriter dataWriter(Schema dataSchema, OutputFile outputFile)
+ throws IOException {
+ FileFormat fileFormat = fileFormat(outputFile.location());
+ switch (fileFormat) {
+ case AVRO:
+ return Avro.writeData(outputFile)
+ .schema(dataSchema)
+ .createWriterFunc(InternalWriter::create)
+ .overwrite()
+ .withSpec(PartitionSpec.unpartitioned())
+ .build();
+ case PARQUET:
+ case ORC:
+ // Internal writers are not supported for PARQUET & ORC yet.
+ default:
+ throw new UnsupportedOperationException("Unsupported file format:" + fileFormat.name());
+ }
+ }
+
+ private static CloseableIterable dataReader(Schema schema, InputFile inputFile) {
+ FileFormat fileFormat = fileFormat(inputFile.location());
+ switch (fileFormat) {
+ case AVRO:
+ return Avro.read(inputFile)
+ .project(schema)
+ .createReaderFunc(fileSchema -> InternalReader.create(schema))
+ .build();
+ case PARQUET:
+ case ORC:
+ // Internal readers are not supported for PARQUET & ORC yet.
+ default:
+ throw new UnsupportedOperationException("Unsupported file format:" + fileFormat.name());
+ }
+ }
+
+ private static PartitionStats recordToPartitionStats(StructLike record) {
+ PartitionStats stats =
+ new PartitionStats(
+ record.get(Column.PARTITION.id(), StructLike.class),
+ record.get(Column.SPEC_ID.id(), Integer.class));
+ stats.set(Column.DATA_RECORD_COUNT.id(), record.get(Column.DATA_RECORD_COUNT.id(), Long.class));
+ stats.set(Column.DATA_FILE_COUNT.id(), record.get(Column.DATA_FILE_COUNT.id(), Integer.class));
+ stats.set(
+ Column.TOTAL_DATA_FILE_SIZE_IN_BYTES.id(),
+ record.get(Column.TOTAL_DATA_FILE_SIZE_IN_BYTES.id(), Long.class));
+ stats.set(
+ Column.POSITION_DELETE_RECORD_COUNT.id(),
+ record.get(Column.POSITION_DELETE_RECORD_COUNT.id(), Long.class));
+ stats.set(
+ Column.POSITION_DELETE_FILE_COUNT.id(),
+ record.get(Column.POSITION_DELETE_FILE_COUNT.id(), Integer.class));
+ stats.set(
+ Column.EQUALITY_DELETE_RECORD_COUNT.id(),
+ record.get(Column.EQUALITY_DELETE_RECORD_COUNT.id(), Long.class));
+ stats.set(
+ Column.EQUALITY_DELETE_FILE_COUNT.id(),
+ record.get(Column.EQUALITY_DELETE_FILE_COUNT.id(), Integer.class));
+ stats.set(
+ Column.TOTAL_RECORD_COUNT.id(), record.get(Column.TOTAL_RECORD_COUNT.id(), Long.class));
+ stats.set(Column.LAST_UPDATED_AT.id(), record.get(Column.LAST_UPDATED_AT.id(), Long.class));
+ stats.set(
+ Column.LAST_UPDATED_SNAPSHOT_ID.id(),
+ record.get(Column.LAST_UPDATED_SNAPSHOT_ID.id(), Long.class));
+ return stats;
+ }
+}
diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
index 533037669b0e..2186106a5236 100644
--- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
+++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
@@ -68,7 +68,6 @@
import org.apache.iceberg.RewriteFiles;
import org.apache.iceberg.RowDelta;
import org.apache.iceberg.Snapshot;
-import org.apache.iceberg.SnapshotRef;
import org.apache.iceberg.Table;
import org.apache.iceberg.exceptions.NotFoundException;
import org.apache.iceberg.expressions.Expression;
@@ -514,12 +513,8 @@ private void commitTable(FileIO io, ExecutorService executor, OutputTable output
}
private Long getSnapshotId(Table table, String branchName) {
- Optional snapshotId = Optional.ofNullable(table.currentSnapshot()).map(Snapshot::snapshotId);
- if (StringUtils.isNotEmpty(branchName)) {
- String ref = HiveUtils.getTableSnapshotRef(branchName);
- snapshotId = Optional.ofNullable(table.refs().get(ref)).map(SnapshotRef::snapshotId);
- }
- return snapshotId.orElse(null);
+ Snapshot snapshot = IcebergTableUtil.getTableSnapshot(table, branchName);
+ return (snapshot != null) ? snapshot.snapshotId() : null;
}
/**
@@ -861,10 +856,9 @@ public List getOutputFiles(List jobContexts) throws IOEx
tableExecutor.shutdown();
}
}
- return Stream.concat(
- parentDirToDataFile.values().stream(),
- parentDirToDeleteFile.values().stream())
- .flatMap(List::stream)
+ return Stream.of(parentDirToDataFile, parentDirToDeleteFile)
+ .flatMap(files ->
+ files.values().stream().flatMap(List::stream))
.collect(Collectors.toList());
}
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 9ecd964c0c12..4652df6bf480 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
@@ -20,8 +20,10 @@
package org.apache.iceberg.mr.hive;
import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.Closeable;
import java.io.IOException;
import java.io.Serializable;
+import java.io.UncheckedIOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.ByteBuffer;
@@ -121,6 +123,7 @@
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.serde2.AbstractSerDe;
import org.apache.hadoop.hive.serde2.DefaultFetchFormatter;
import org.apache.hadoop.hive.serde2.Deserializer;
@@ -155,11 +158,13 @@
import org.apache.iceberg.PartitionField;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.PartitionSpecParser;
+import org.apache.iceberg.PartitionStatisticsFile;
+import org.apache.iceberg.PartitionStats;
+import org.apache.iceberg.Partitioning;
import org.apache.iceberg.Schema;
import org.apache.iceberg.SchemaParser;
import org.apache.iceberg.SerializableTable;
import org.apache.iceberg.Snapshot;
-import org.apache.iceberg.SnapshotSummary;
import org.apache.iceberg.SortDirection;
import org.apache.iceberg.SortField;
import org.apache.iceberg.SortOrder;
@@ -169,6 +174,8 @@
import org.apache.iceberg.TableProperties;
import org.apache.iceberg.TableScan;
import org.apache.iceberg.actions.DeleteOrphanFiles;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.PartitionStatsHandler;
import org.apache.iceberg.exceptions.NoSuchTableException;
import org.apache.iceberg.exceptions.ValidationException;
import org.apache.iceberg.expressions.Evaluator;
@@ -216,6 +223,13 @@
import static org.apache.hadoop.hive.ql.metadata.VirtualColumn.PARTITION_PROJECTION;
import static org.apache.hadoop.hive.ql.metadata.VirtualColumn.PARTITION_SPEC_ID;
import static org.apache.hadoop.hive.ql.metadata.VirtualColumn.ROW_POSITION;
+import static org.apache.iceberg.SnapshotSummary.ADDED_RECORDS_PROP;
+import static org.apache.iceberg.SnapshotSummary.DELETED_RECORDS_PROP;
+import static org.apache.iceberg.SnapshotSummary.TOTAL_DATA_FILES_PROP;
+import static org.apache.iceberg.SnapshotSummary.TOTAL_EQ_DELETES_PROP;
+import static org.apache.iceberg.SnapshotSummary.TOTAL_FILE_SIZE_PROP;
+import static org.apache.iceberg.SnapshotSummary.TOTAL_POS_DELETES_PROP;
+import static org.apache.iceberg.SnapshotSummary.TOTAL_RECORDS_PROP;
public class HiveIcebergStorageHandler implements HiveStoragePredicateHandler, HiveStorageHandler {
private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergStorageHandler.class);
@@ -369,7 +383,7 @@ public boolean directInsert() {
}
@Override
- public boolean alwaysUnpartitioned() {
+ public boolean supportsPartitioning() {
return true;
}
@@ -427,6 +441,19 @@ public boolean canProvideBasicStatistics() {
return true;
}
+ @Override
+ public boolean canProvidePartitionStatistics(org.apache.hadoop.hive.ql.metadata.Table hmsTable) {
+ if (!getStatsSource().equals(HiveMetaHook.ICEBERG)) {
+ return false;
+ }
+ Table table = IcebergTableUtil.getTable(conf, hmsTable.getTTable());
+ Snapshot snapshot = IcebergTableUtil.getTableSnapshot(table, hmsTable);
+ if (snapshot != null) {
+ return IcebergTableUtil.getPartitionStatsFile(table, snapshot.snapshotId()) != null;
+ }
+ return false;
+ }
+
@Override
public StorageFormatDescriptor getStorageFormatDescriptor(org.apache.hadoop.hive.metastore.api.Table table)
throws SemanticException {
@@ -464,30 +491,29 @@ public void appendFiles(org.apache.hadoop.hive.metastore.api.Table table, URI fr
@Override
public Map getBasicStatistics(Partish partish) {
- 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.
Map stats = Maps.newHashMap();
+
if (!getStatsSource().equals(HiveMetaHook.ICEBERG)) {
- return hmsTable.getParameters();
+ 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(hmsTable, table);
+ Snapshot snapshot = IcebergTableUtil.getTableSnapshot(table, hmsTable);
if (snapshot != null) {
- Map summary = snapshot.summary();
+ Map summary = getPartishSummary(partish, table, snapshot);
if (summary != null) {
-
- if (summary.containsKey(SnapshotSummary.TOTAL_DATA_FILES_PROP)) {
- stats.put(StatsSetupConst.NUM_FILES, summary.get(SnapshotSummary.TOTAL_DATA_FILES_PROP));
+ if (summary.containsKey(TOTAL_DATA_FILES_PROP)) {
+ stats.put(StatsSetupConst.NUM_FILES, summary.get(TOTAL_DATA_FILES_PROP));
}
+ if (summary.containsKey(TOTAL_RECORDS_PROP)) {
+ long totalRecords = Long.parseLong(summary.get(TOTAL_RECORDS_PROP));
+ if (summary.containsKey(TOTAL_EQ_DELETES_PROP) &&
+ summary.containsKey(TOTAL_POS_DELETES_PROP)) {
- if (summary.containsKey(SnapshotSummary.TOTAL_RECORDS_PROP)) {
- long totalRecords = Long.parseLong(summary.get(SnapshotSummary.TOTAL_RECORDS_PROP));
- if (summary.containsKey(SnapshotSummary.TOTAL_EQ_DELETES_PROP) &&
- summary.containsKey(SnapshotSummary.TOTAL_POS_DELETES_PROP)) {
-
- long totalEqDeletes = Long.parseLong(summary.get(SnapshotSummary.TOTAL_EQ_DELETES_PROP));
- long totalPosDeletes = Long.parseLong(summary.get(SnapshotSummary.TOTAL_POS_DELETES_PROP));
+ long totalEqDeletes = Long.parseLong(summary.get(TOTAL_EQ_DELETES_PROP));
+ long totalPosDeletes = Long.parseLong(summary.get(TOTAL_POS_DELETES_PROP));
long actualRecords = totalRecords - (totalEqDeletes > 0 ? 0 : totalPosDeletes);
totalRecords = actualRecords > 0 ? actualRecords : totalRecords;
@@ -495,9 +521,8 @@ public Map getBasicStatistics(Partish partish) {
}
stats.put(StatsSetupConst.ROW_COUNT, String.valueOf(totalRecords));
}
-
- if (summary.containsKey(SnapshotSummary.TOTAL_FILE_SIZE_PROP)) {
- stats.put(StatsSetupConst.TOTAL_SIZE, summary.get(SnapshotSummary.TOTAL_FILE_SIZE_PROP));
+ if (summary.containsKey(TOTAL_FILE_SIZE_PROP)) {
+ stats.put(StatsSetupConst.TOTAL_SIZE, summary.get(TOTAL_FILE_SIZE_PROP));
}
}
} else {
@@ -508,6 +533,80 @@ public Map getBasicStatistics(Partish partish) {
return stats;
}
+ @Override
+ public Map computeBasicStatistics(Partish partish) {
+ 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 && table.spec().isPartitioned()) {
+ PartitionStatisticsFile statsFile = IcebergTableUtil.getPartitionStatsFile(table, snapshot.snapshotId());
+ if (statsFile == null) {
+ statsFile = PartitionStatsHandler.computeAndWriteStatsFile(table);
+
+ table.updatePartitionStatistics()
+ .setPartitionStatistics(statsFile)
+ .commit();
+ }
+ }
+ return getBasicStatistics(partish);
+ }
+
+ private static Map getPartishSummary(Partish partish, Table table, Snapshot snapshot) {
+ if (partish.getPartition() != null) {
+ PartitionStatisticsFile statsFile = IcebergTableUtil.getPartitionStatsFile(table, snapshot.snapshotId());
+ if (statsFile != null) {
+ Types.StructType partitionType = Partitioning.partitionType(table);
+ Schema schema = PartitionStatsHandler.schema(partitionType);
+
+ CloseableIterable partitionStatsRecords = PartitionStatsHandler.readPartitionStatsFile(
+ schema, table.io().newInputFile(statsFile.path()));
+
+ try (Closeable toClose = partitionStatsRecords) {
+ PartitionStats partitionStats = Iterables.tryFind(partitionStatsRecords, stats -> {
+ PartitionSpec spec = table.specs().get(stats.specId());
+ Schema readSchema = spec.partitionType().asSchema();
+ GenericRecord record = GenericRecord.create(readSchema);
+
+ List fields = partitionType.fields();
+ for (int index = 0, pos = 0; index < fields.size(); index++) {
+ if (readSchema.findField(fields.get(index).fieldId()) != null) {
+ record.set(pos++, stats.partition().get(index, Object.class));
+ }
+ }
+ return spec.partitionToPath(record).equals(partish.getPartition().getName());
+
+ }).orNull();
+
+ if (partitionStats != null) {
+ Map stats = ImmutableMap.of(
+ TOTAL_DATA_FILES_PROP, String.valueOf(partitionStats.dataFileCount()),
+ TOTAL_RECORDS_PROP, String.valueOf(partitionStats.dataRecordCount()),
+ TOTAL_EQ_DELETES_PROP, String.valueOf(partitionStats.equalityDeleteRecordCount()),
+ TOTAL_POS_DELETES_PROP, String.valueOf(partitionStats.positionDeleteRecordCount()),
+ TOTAL_FILE_SIZE_PROP, String.valueOf(partitionStats.totalDataFileSizeInBytes())
+ );
+ return stats;
+ } else {
+ LOG.warn("Partition {} not found in stats file: {}",
+ partish.getPartition().getName(), statsFile.path());
+ return null;
+ }
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ } else {
+ LOG.warn("Partition stats file not found for snapshot: {}", snapshot.snapshotId());
+ return null;
+ }
+ }
+ return snapshot.summary();
+ }
+
private Table getTable(org.apache.hadoop.hive.ql.metadata.Table hmsTable) {
Table table;
final Optional queryState = SessionStateUtil.getQueryState(conf);
@@ -585,11 +684,11 @@ private boolean writeColStats(ColumnStatistics tableColStats, Table tbl) {
@Override
public boolean canProvideColStatistics(org.apache.hadoop.hive.ql.metadata.Table hmsTable) {
Table table = IcebergTableUtil.getTable(conf, hmsTable.getTTable());
- Snapshot snapshot = IcebergTableUtil.getTableSnapshot(hmsTable, table);
- if (snapshot == null) {
- return false;
+ Snapshot snapshot = IcebergTableUtil.getTableSnapshot(table, hmsTable);
+ if (snapshot != null) {
+ return canSetColStatistics(hmsTable) && canProvideColStats(table, snapshot.snapshotId());
}
- return canSetColStatistics(hmsTable) && canProvideColStats(table, snapshot.snapshotId());
+ return false;
}
private boolean canProvideColStats(Table table, long snapshotId) {
@@ -599,14 +698,15 @@ private boolean canProvideColStats(Table table, long snapshotId) {
@Override
public List getColStatistics(org.apache.hadoop.hive.ql.metadata.Table hmsTable) {
Table table = IcebergTableUtil.getTable(conf, hmsTable.getTTable());
- Snapshot snapshot = IcebergTableUtil.getTableSnapshot(hmsTable, table);
+ Snapshot snapshot = IcebergTableUtil.getTableSnapshot(table, hmsTable);
+
ColumnStatistics emptyStats = new ColumnStatistics();
- if (snapshot == null) {
- return emptyStats.getStatsObj();
+ if (snapshot != null) {
+ return IcebergTableUtil.getColStatsPath(table, snapshot.snapshotId())
+ .map(statsPath -> readColStats(table, statsPath))
+ .orElse(emptyStats).getStatsObj();
}
- long snapshotId = IcebergTableUtil.getTableSnapshot(hmsTable, table).snapshotId();
- return IcebergTableUtil.getColStatsPath(table, snapshotId).map(statsPath -> readColStats(table, statsPath))
- .orElse(emptyStats).getStatsObj();
+ return emptyStats.getStatsObj();
}
private ColumnStatistics readColStats(Table table, Path statsPath) {
@@ -625,19 +725,25 @@ private ColumnStatistics readColStats(Table table, Path statsPath) {
}
@Override
- public boolean canComputeQueryUsingStats(org.apache.hadoop.hive.ql.metadata.Table hmsTable) {
- if (getStatsSource().equals(HiveMetaHook.ICEBERG) && hmsTable.getMetaTable() == null) {
- Table table = getTable(hmsTable);
- Snapshot snapshot = IcebergTableUtil.getTableSnapshot(hmsTable, table);
- if (snapshot != null) {
- Map summary = snapshot.summary();
- if (summary != null && summary.containsKey(SnapshotSummary.TOTAL_EQ_DELETES_PROP) &&
- summary.containsKey(SnapshotSummary.TOTAL_POS_DELETES_PROP)) {
+ public boolean canComputeQueryUsingStats(Partish partish) {
+ org.apache.hadoop.hive.ql.metadata.Table hmsTable = partish.getTable();
+ if (hmsTable.getMetaTable() != null) {
+ return false;
+ }
+ if (!getStatsSource().equals(HiveMetaHook.ICEBERG) &&
+ StatsUtils.areBasicStatsUptoDateForQueryAnswering(partish.getTable(), partish.getPartParameters())) {
+ return true;
+ }
+ Table table = getTable(hmsTable);
+ Snapshot snapshot = IcebergTableUtil.getTableSnapshot(table, hmsTable);
+ if (snapshot != null) {
+ Map summary = getPartishSummary(partish, table, snapshot);
+ if (summary != null && summary.containsKey(TOTAL_EQ_DELETES_PROP) &&
+ summary.containsKey(TOTAL_POS_DELETES_PROP)) {
- long totalEqDeletes = Long.parseLong(summary.get(SnapshotSummary.TOTAL_EQ_DELETES_PROP));
- long totalPosDeletes = Long.parseLong(summary.get(SnapshotSummary.TOTAL_POS_DELETES_PROP));
- return totalEqDeletes + totalPosDeletes == 0;
- }
+ long totalEqDeletes = Long.parseLong(summary.get(TOTAL_EQ_DELETES_PROP));
+ long totalPosDeletes = Long.parseLong(summary.get(TOTAL_POS_DELETES_PROP));
+ return totalEqDeletes + totalPosDeletes == 0;
}
}
return false;
@@ -1211,7 +1317,7 @@ public void validateSinkDesc(FileSinkDesc sinkDesc) throws SemanticException {
if (sinkDesc.getInsertOverwrite()) {
Table table = IcebergTableUtil.getTable(conf, sinkDesc.getTableInfo().getProperties());
if (table.currentSnapshot() != null &&
- Long.parseLong(table.currentSnapshot().summary().get(SnapshotSummary.TOTAL_RECORDS_PROP)) == 0) {
+ Long.parseLong(table.currentSnapshot().summary().get(TOTAL_RECORDS_PROP)) == 0) {
// If the table is empty we don't have any danger that some data can get lost.
return;
}
@@ -1700,8 +1806,8 @@ public SnapshotContext getCurrentSnapshotContext(org.apache.hadoop.hive.ql.metad
private SnapshotContext toSnapshotContext(Snapshot snapshot) {
Map summaryMap = snapshot.summary();
- long addedRecords = getLongSummary(summaryMap, SnapshotSummary.ADDED_RECORDS_PROP);
- long deletedRecords = getLongSummary(summaryMap, SnapshotSummary.DELETED_RECORDS_PROP);
+ long addedRecords = getLongSummary(summaryMap, ADDED_RECORDS_PROP);
+ long deletedRecords = getLongSummary(summaryMap, DELETED_RECORDS_PROP);
return new SnapshotContext(
snapshot.snapshotId(), toWriteOperationType(snapshot.operation()), addedRecords, deletedRecords);
}
@@ -1916,8 +2022,8 @@ private List getPartitions(DDLOperationContext context, Configuration jo
public void validatePartSpec(org.apache.hadoop.hive.ql.metadata.Table hmsTable, Map partitionSpec,
Context.RewritePolicy policy) throws SemanticException {
Table table = IcebergTableUtil.getTable(conf, hmsTable.getTTable());
- List partitionFields = (policy == Context.RewritePolicy.PARTITION) ?
- IcebergTableUtil.getPartitionFields(table) : table.spec().fields();
+ List partitionFields = IcebergTableUtil.getPartitionFields(table,
+ policy != Context.RewritePolicy.PARTITION);
validatePartSpecImpl(hmsTable, partitionSpec, partitionFields);
}
@@ -1977,7 +2083,7 @@ public boolean canUseTruncate(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
return false;
}
- Expression finalExp = IcebergTableUtil.generateExpressionFromPartitionSpec(table, partitionSpec);
+ Expression finalExp = IcebergTableUtil.generateExpressionFromPartitionSpec(table, partitionSpec, true);
FindFiles.Builder builder = new FindFiles.Builder(table).withRecordsMatching(finalExp);
Set dataFiles = Sets.newHashSet(builder.collect());
boolean result = true;
@@ -2002,20 +2108,6 @@ private boolean hasUndergonePartitionEvolution(Table table) {
.anyMatch(id -> id != table.spec().specId());
}
- private boolean isIdentityPartitionTable(org.apache.hadoop.hive.ql.metadata.Table table) {
- return getPartitionTransformSpec(table).stream().map(TransformSpec::getTransformType)
- .allMatch(type -> type == TransformSpec.TransformType.IDENTITY);
- }
-
- @Override
- public Optional isEligibleForCompaction(
- org.apache.hadoop.hive.ql.metadata.Table table, Map partitionSpec) {
- if (partitionSpec != null && !isIdentityPartitionTable(table)) {
- return Optional.of(ErrorMsg.COMPACTION_NON_IDENTITY_PARTITION_SPEC);
- }
- return Optional.empty();
- }
-
@Override
public List getPartitions(org.apache.hadoop.hive.ql.metadata.Table table,
Map partitionSpec, boolean latestSpecOnly) throws SemanticException {
@@ -2029,6 +2121,11 @@ public List getPartitions(org.apache.hadoop.hive.ql.metadata.Table ta
}
public boolean isPartitioned(org.apache.hadoop.hive.ql.metadata.Table hmsTable) {
+ if ((hmsTable.getAsOfVersion() != null || hmsTable.getAsOfTimestamp() != null) &&
+ hasUndergonePartitionEvolution(hmsTable) ||
+ !hmsTable.getTTable().isSetId()) {
+ return false;
+ }
Table table = IcebergTableUtil.getTable(conf, hmsTable.getTTable());
return table.spec().isPartitioned();
}
@@ -2143,9 +2240,12 @@ public boolean canPerformMetadataDelete(org.apache.hadoop.hive.ql.metadata.Table
}
@Override
- public List getPartitionKeys(org.apache.hadoop.hive.ql.metadata.Table hmsTable, boolean latestSpecOnly) {
+ public List getPartitionKeys(org.apache.hadoop.hive.ql.metadata.Table hmsTable) {
+ if (!hmsTable.getTTable().isSetId()) {
+ return Collections.emptyList();
+ }
Table icebergTable = IcebergTableUtil.getTable(conf, hmsTable.getTTable());
- return IcebergTableUtil.getPartitionKeys(icebergTable, latestSpecOnly);
+ return IcebergTableUtil.getPartitionKeys(icebergTable, icebergTable.spec().specId());
}
@Override
@@ -2155,10 +2255,12 @@ public List getPartitionsByExpr(org.apache.hadoop.hive.ql.metadata.Ta
}
@Override
- public List getPartitionsByExpr(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
- ExprNodeDesc filter, boolean latestSpecOnly) throws SemanticException {
- SearchArgument sarg = ConvertAstToSearchArg.create(conf, (ExprNodeGenericFuncDesc) filter);
- Expression exp = HiveIcebergFilterFactory.generateFilterExpression(sarg);
+ public List getPartitionsByExpr(org.apache.hadoop.hive.ql.metadata.Table hmsTable, ExprNodeDesc filter,
+ boolean latestSpecOnly) throws SemanticException {
+ Expression exp = HiveIcebergInputFormat.getFilterExpr(conf, (ExprNodeGenericFuncDesc) filter);
+ if (exp == null) {
+ return ImmutableList.of();
+ }
Table table = IcebergTableUtil.getTable(conf, hmsTable.getTTable());
int tableSpecId = table.spec().specId();
Set partitions = Sets.newHashSet();
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 8f876a727815..3230363ad7fd 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
@@ -57,6 +57,7 @@
import org.apache.iceberg.PartitionData;
import org.apache.iceberg.PartitionField;
import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PartitionStatisticsFile;
import org.apache.iceberg.Partitioning;
import org.apache.iceberg.PartitionsTable;
import org.apache.iceberg.RowLevelOperationMode;
@@ -132,19 +133,18 @@ public static Table getTable(Configuration configuration, org.apache.hadoop.hive
*/
static Table getTable(Configuration configuration, Properties properties, boolean skipCache) {
String metaTable = properties.getProperty(IcebergAcidUtil.META_TABLE_PROPERTY);
- String tableName = properties.getProperty(Catalogs.NAME);
- String location = properties.getProperty(Catalogs.LOCATION);
+
+ Properties props = new Properties(properties); // use input properties as default
if (metaTable != null) {
// HiveCatalog, HadoopCatalog uses NAME to identify the metadata table
- properties.setProperty(Catalogs.NAME, tableName + "." + metaTable);
+ props.put(Catalogs.NAME, properties.get(Catalogs.NAME) + "." + metaTable);
// HadoopTable uses LOCATION to identify the metadata table
- properties.setProperty(Catalogs.LOCATION, location + "#" + metaTable);
+ props.put(Catalogs.LOCATION, properties.get(Catalogs.LOCATION) + "#" + metaTable);
}
-
- String tableIdentifier = properties.getProperty(Catalogs.NAME);
+ String tableIdentifier = props.getProperty(Catalogs.NAME);
Function tableLoadFunc =
unused -> {
- Table tab = Catalogs.loadTable(configuration, properties);
+ Table tab = Catalogs.loadTable(configuration, props);
SessionStateUtil.addResource(configuration, tableIdentifier, tab);
return tab;
};
@@ -165,6 +165,41 @@ static Table getTable(Configuration configuration, Properties properties) {
return getTable(configuration, properties, false);
}
+ static Snapshot getTableSnapshot(Table table, org.apache.hadoop.hive.ql.metadata.Table hmsTable) {
+ long snapshotId = -1;
+
+ if (hmsTable.getAsOfTimestamp() != null) {
+ ZoneId timeZone = SessionState.get() == null ?
+ new HiveConf().getLocalTimeZone() : SessionState.get().getConf().getLocalTimeZone();
+ TimestampTZ time = TimestampTZUtil.parse(hmsTable.getAsOfTimestamp(), timeZone);
+ snapshotId = SnapshotUtil.snapshotIdAsOfTime(table, time.toEpochMilli());
+
+ } else if (hmsTable.getAsOfVersion() != null) {
+ try {
+ snapshotId = Long.parseLong(hmsTable.getAsOfVersion());
+ } catch (NumberFormatException e) {
+ SnapshotRef ref = table.refs().get(hmsTable.getAsOfVersion());
+ if (ref == null) {
+ throw new RuntimeException("Cannot find matching snapshot ID or reference name for version " +
+ hmsTable.getAsOfVersion());
+ }
+ snapshotId = ref.snapshotId();
+ }
+ }
+ if (snapshotId > 0) {
+ return table.snapshot(snapshotId);
+ }
+ return getTableSnapshot(table, hmsTable.getSnapshotRef());
+ }
+
+ static Snapshot getTableSnapshot(Table table, String snapshotRef) {
+ if (snapshotRef != null) {
+ String ref = HiveUtils.getTableSnapshotRef(snapshotRef);
+ return table.snapshot(ref);
+ }
+ return table.currentSnapshot();
+ }
+
static Optional getColStatsPath(Table table) {
return getColStatsPath(table, table.currentSnapshot().snapshotId());
}
@@ -179,6 +214,12 @@ static Optional getColStatsPath(Table table, long snapshotId) {
.findAny();
}
+ static PartitionStatisticsFile getPartitionStatsFile(Table table, long snapshotId) {
+ return table.partitionStatisticsFiles().stream()
+ .filter(stats -> stats.snapshotId() == snapshotId)
+ .findAny().orElse(null);
+ }
+
/**
* Create {@link PartitionSpec} based on the partition information stored in
* {@link TransformSpec}.
@@ -188,14 +229,13 @@ static Optional getColStatsPath(Table table, long snapshotId) {
*/
public static PartitionSpec spec(Configuration configuration, Schema schema) {
List partitionTransformSpecList = SessionStateUtil
- .getResource(configuration, hive_metastoreConstants.PARTITION_TRANSFORM_SPEC)
- .map(o -> (List) o).orElseGet(() -> null);
+ .getResource(configuration, hive_metastoreConstants.PARTITION_TRANSFORM_SPEC)
+ .map(o -> (List) o).orElse(null);
if (partitionTransformSpecList == null) {
- LOG.debug("Iceberg partition transform spec is not found in QueryState.");
+ LOG.warn("Iceberg partition transform spec is not found in QueryState.");
return null;
}
-
PartitionSpec.Builder builder = PartitionSpec.builderFor(schema);
partitionTransformSpecList.forEach(spec -> {
switch (spec.getTransformType()) {
@@ -229,7 +269,7 @@ public static void updateSpec(Configuration configuration, Table table) {
// get the new partition transform spec
PartitionSpec newPartitionSpec = spec(configuration, table.schema());
if (newPartitionSpec == null) {
- LOG.debug("Iceberg Partition spec is not updated due to empty partition spec definition.");
+ LOG.warn("Iceberg partition spec is not updated due to empty partition spec definition.");
return;
}
@@ -239,8 +279,12 @@ public static void updateSpec(Configuration configuration, Table table) {
List partitionTransformSpecList = SessionStateUtil
.getResource(configuration, hive_metastoreConstants.PARTITION_TRANSFORM_SPEC)
- .map(o -> (List) o).orElseGet(() -> null);
+ .map(o -> (List) o).orElse(null);
+ if (partitionTransformSpecList == null) {
+ LOG.warn("Iceberg partition transform spec is not found in QueryState.");
+ return;
+ }
partitionTransformSpecList.forEach(spec -> {
switch (spec.getTransformType()) {
case IDENTITY:
@@ -388,28 +432,27 @@ public static PartitionData toPartitionData(StructLike sourceKey, Types.StructTy
Types.StructType targetKeyType) {
PartitionData data = new PartitionData(targetKeyType);
for (int i = 0; i < targetKeyType.fields().size(); i++) {
-
- int fi = i;
- String fieldName = targetKeyType.fields().get(fi).name();
- Object val = sourceKeyType.fields().stream()
- .filter(f -> f.name().equals(fieldName)).findFirst()
- .map(sourceKeyElem -> sourceKey.get(sourceKeyType.fields().indexOf(sourceKeyElem),
- targetKeyType.fields().get(fi).type().typeId().javaClass()))
- .orElseThrow(() -> new RuntimeException(
- String.format("Error retrieving value of partition field %s", fieldName)));
-
- if (val != null) {
- data.set(fi, val);
- } else {
- throw new RuntimeException(String.format("Partition field's %s value is null", fieldName));
+ Types.NestedField targetKey = targetKeyType.fields().get(i);
+
+ Optional val = sourceKeyType.fields().stream()
+ .filter(f -> f.name().equals(targetKey.name()))
+ .findFirst()
+ .map(sourceKeyElem ->
+ sourceKey.get(
+ sourceKeyType.fields().indexOf(sourceKeyElem),
+ targetKey.type().typeId().javaClass()
+ )
+ );
+ if (val.isPresent()) {
+ data.set(i, val.get());
}
}
return data;
}
- public static Expression generateExpressionFromPartitionSpec(Table table, Map partitionSpec)
- throws SemanticException {
- Map partitionFieldMap = getPartitionFields(table).stream()
+ public static Expression generateExpressionFromPartitionSpec(Table table, Map partitionSpec,
+ boolean latestSpecOnly) throws SemanticException {
+ Map partitionFieldMap = getPartitionFields(table, latestSpecOnly).stream()
.collect(Collectors.toMap(PartitionField::name, Function.identity()));
Expression finalExp = Expressions.alwaysTrue();
for (Map.Entry entry : partitionSpec.entrySet()) {
@@ -440,23 +483,20 @@ public static List getPartitionKeys(Table table, int specId) {
List hiveSchema = HiveSchemaUtil.convert(schema);
Map colNameToColType = hiveSchema.stream()
.collect(Collectors.toMap(FieldSchema::getName, FieldSchema::getType));
- return table.specs().get(specId).fields().stream().map(partField ->
- new FieldSchema(schema.findColumnName(partField.sourceId()),
+ return table.specs().get(specId).fields().stream()
+ .map(partField -> new FieldSchema(
+ schema.findColumnName(partField.sourceId()),
colNameToColType.get(schema.findColumnName(partField.sourceId())),
- String.format("Transform: %s", partField.transform().toString()))).collect(Collectors.toList());
+ String.format("Transform: %s", partField.transform().toString()))
+ )
+ .collect(Collectors.toList());
}
- public static List getPartitionKeys(Table table, boolean latestSpecOnly) {
- if (latestSpecOnly) {
- return getPartitionKeys(table, table.spec().specId());
- } else {
- return table.specs().keySet().stream().flatMap(id -> getPartitionKeys(table, id).stream())
- .distinct().collect(Collectors.toList());
- }
- }
- public static List getPartitionFields(Table table) {
- return table.specs().values().stream().flatMap(spec -> spec.fields()
- .stream()).distinct().collect(Collectors.toList());
+ public static List getPartitionFields(Table table, boolean latestSpecOnly) {
+ return latestSpecOnly ? table.spec().fields() :
+ table.specs().values().stream()
+ .flatMap(spec -> spec.fields().stream()).distinct()
+ .collect(Collectors.toList());
}
/**
@@ -469,9 +509,10 @@ public static List getPartitionFields(Table table) {
*/
public static Map getPartitionInfo(Table icebergTable, Map partSpecMap,
boolean allowPartialSpec, boolean latestSpecOnly) throws SemanticException, IOException {
- Expression expression = IcebergTableUtil.generateExpressionFromPartitionSpec(icebergTable, partSpecMap);
- PartitionsTable partitionsTable = (PartitionsTable) MetadataTableUtils
- .createMetadataTableInstance(icebergTable, MetadataTableType.PARTITIONS);
+ Expression expression = IcebergTableUtil.generateExpressionFromPartitionSpec(
+ icebergTable, partSpecMap, latestSpecOnly);
+ PartitionsTable partitionsTable = (PartitionsTable) MetadataTableUtils.createMetadataTableInstance(
+ icebergTable, MetadataTableType.PARTITIONS);
Map result = Maps.newLinkedHashMap();
try (CloseableIterable fileScanTasks = partitionsTable.newScan().planFiles()) {
@@ -530,31 +571,4 @@ public static List getPartitionNames(Table icebergTable, Map recordIterator = PartitionStatsHandler.readPartitionStatsFile(
+ dataSchema, Files.localInput(statisticsFile.path()))) {
+ assertThat(Iterables.size(recordIterator)).isEqualTo(0L);
+ }
+ assertThat(statisticsFile.snapshotId())
+ .isEqualTo(testTable.refs().get("b1").snapshotId());
+ }
+
+ @Test
+ public void testPartitionStatsOnInvalidSnapshot() throws Exception {
+ Table testTable =
+ TestTables.create(tempDir("invalid_snapshot"), "invalid_snapshot", SCHEMA, SPEC, 2);
+ assertThatThrownBy(
+ () -> PartitionStatsHandler.computeAndWriteStatsFile(testTable, "INVALID_BRANCH"))
+ .isInstanceOf(IllegalArgumentException.class)
+ .hasMessage("Couldn't find the snapshot for the branch INVALID_BRANCH");
+ }
+
+ @Test
+ public void testPartitionStatsOnUnPartitionedTable() throws Exception {
+ Table testTable =
+ TestTables.create(
+ tempDir("unpartitioned_table"),
+ "unpartitioned_table",
+ SCHEMA,
+ PartitionSpec.unpartitioned(),
+ 2);
+
+ List records = prepareRecords(testTable.schema());
+ DataFile dataFile = FileHelpers.writeDataFile(testTable, outputFile(), records);
+ testTable.newAppend().appendFile(dataFile).commit();
+
+ assertThatThrownBy(() -> PartitionStatsHandler.computeAndWriteStatsFile(testTable))
+ .isInstanceOf(IllegalArgumentException.class)
+ .hasMessage("table must be partitioned");
+ }
+
+ @Test
+ public void testAllDatatypePartitionWriting() throws Exception {
+ Schema schema =
+ new Schema(
+ required(100, "id", Types.LongType.get()),
+ optional(101, "data", Types.StringType.get()),
+ required(102, "b", Types.BooleanType.get()),
+ optional(103, "i", Types.IntegerType.get()),
+ required(104, "l", Types.LongType.get()),
+ optional(105, "f", Types.FloatType.get()),
+ required(106, "d", Types.DoubleType.get()),
+ optional(107, "date", Types.DateType.get()),
+ required(108, "ts", Types.TimestampType.withoutZone()),
+ required(110, "s", Types.StringType.get()),
+ required(111, "uuid", Types.UUIDType.get()),
+ required(112, "fixed", Types.FixedType.ofLength(7)),
+ optional(113, "bytes", Types.BinaryType.get()),
+ required(114, "dec_9_0", Types.DecimalType.of(9, 0)),
+ required(115, "dec_11_2", Types.DecimalType.of(11, 2)),
+ required(116, "dec_38_10", Types.DecimalType.of(38, 10)), // maximum precision
+ required(117, "time", Types.TimeType.get()));
+
+ PartitionSpec spec =
+ PartitionSpec.builderFor(schema)
+ .identity("b")
+ .identity("i")
+ .identity("l")
+ .identity("f")
+ .identity("d")
+ .identity("date")
+ .identity("ts")
+ .identity("s")
+ .identity("uuid")
+ .identity("fixed")
+ .identity("bytes")
+ .identity("dec_9_0")
+ .identity("dec_11_2")
+ .identity("dec_38_10")
+ .identity("time")
+ .build();
+
+ Table testTable =
+ TestTables.create(
+ tempDir("test_all_type"), "test_all_type", schema, spec, SortOrder.unsorted(), 2);
+
+ Types.StructType partitionSchema = Partitioning.partitionType(testTable);
+ Schema dataSchema = PartitionStatsHandler.schema(partitionSchema);
+
+ PartitionData partitionData =
+ new PartitionData(dataSchema.findField(Column.PARTITION.name()).type().asStructType());
+ partitionData.set(0, true);
+ partitionData.set(1, 42);
+ partitionData.set(2, 42L);
+ partitionData.set(3, 3.14f);
+ partitionData.set(4, 3.141592653589793);
+ partitionData.set(5, Literal.of("2022-01-01").to(Types.DateType.get()).value());
+ partitionData.set(
+ 6, Literal.of("2017-12-01T10:12:55.038194").to(Types.TimestampType.withoutZone()).value());
+ partitionData.set(7, "string");
+ partitionData.set(8, UUID.randomUUID());
+ partitionData.set(9, ByteBuffer.wrap(new byte[] {0, 1, 2, 3, 4, 5, 6}));
+ partitionData.set(10, ByteBuffer.wrap(new byte[] {1, 2, 3}));
+ partitionData.set(11, new BigDecimal("123456789"));
+ partitionData.set(12, new BigDecimal("1234567.89"));
+ partitionData.set(13, new BigDecimal("12345678901234567890.1234567890"));
+ partitionData.set(14, Literal.of("10:10:10").to(Types.TimeType.get()).value());
+
+ PartitionStats partitionStats = new PartitionStats(partitionData, RANDOM.nextInt(10));
+ partitionStats.set(Column.DATA_RECORD_COUNT.id(), RANDOM.nextLong());
+ partitionStats.set(Column.DATA_FILE_COUNT.id(), RANDOM.nextInt());
+ partitionStats.set(Column.TOTAL_DATA_FILE_SIZE_IN_BYTES.id(), 1024L * RANDOM.nextInt(20));
+ List expected = Collections.singletonList(partitionStats);
+ PartitionStatisticsFile statisticsFile =
+ PartitionStatsHandler.writePartitionStatsFile(
+ testTable, 42L, dataSchema, expected.iterator());
+
+ List written;
+ try (CloseableIterable recordIterator =
+ PartitionStatsHandler.readPartitionStatsFile(
+ dataSchema, Files.localInput(statisticsFile.path()))) {
+ written = Lists.newArrayList(recordIterator);
+ }
+
+ assertThat(written).hasSize(expected.size());
+ Comparator comparator = Comparators.forType(partitionSchema);
+ for (int i = 0; i < written.size(); i++) {
+ assertThat(PartitionStatsUtil.isEqual(comparator, written.get(i), expected.get(i))).isTrue();
+ }
+ }
+
+ @Test
+ public void testOptionalFieldsWriting() throws Exception {
+ PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+ Table testTable =
+ TestTables.create(
+ tempDir("test_partition_stats_optional"),
+ "test_partition_stats_optional",
+ SCHEMA,
+ spec,
+ SortOrder.unsorted(),
+ 2);
+
+ Types.StructType partitionSchema = Partitioning.partitionType(testTable);
+ Schema dataSchema = PartitionStatsHandler.schema(partitionSchema);
+
+ ImmutableList.Builder partitionListBuilder = ImmutableList.builder();
+ for (int i = 0; i < 5; i++) {
+ PartitionData partitionData =
+ new PartitionData(dataSchema.findField(Column.PARTITION.name()).type().asStructType());
+ partitionData.set(0, RANDOM.nextInt());
+
+ PartitionStats stats = new PartitionStats(partitionData, RANDOM.nextInt(10));
+ stats.set(Column.PARTITION.ordinal(), partitionData);
+ stats.set(Column.DATA_RECORD_COUNT.ordinal(), RANDOM.nextLong());
+ stats.set(Column.DATA_FILE_COUNT.ordinal(), RANDOM.nextInt());
+ stats.set(Column.TOTAL_DATA_FILE_SIZE_IN_BYTES.ordinal(), 1024L * RANDOM.nextInt(20));
+ stats.set(Column.POSITION_DELETE_RECORD_COUNT.ordinal(), null);
+ stats.set(Column.POSITION_DELETE_FILE_COUNT.ordinal(), null);
+ stats.set(Column.EQUALITY_DELETE_RECORD_COUNT.ordinal(), null);
+ stats.set(Column.EQUALITY_DELETE_FILE_COUNT.ordinal(), null);
+ stats.set(Column.TOTAL_RECORD_COUNT.ordinal(), null);
+ stats.set(Column.LAST_UPDATED_AT.ordinal(), null);
+ stats.set(Column.LAST_UPDATED_SNAPSHOT_ID.ordinal(), null);
+
+ partitionListBuilder.add(stats);
+ }
+
+ List expected = partitionListBuilder.build();
+
+ assertThat(expected.get(0))
+ .extracting(
+ PartitionStats::positionDeleteRecordCount,
+ PartitionStats::positionDeleteFileCount,
+ PartitionStats::equalityDeleteRecordCount,
+ PartitionStats::equalityDeleteFileCount,
+ PartitionStats::totalRecordCount,
+ PartitionStats::lastUpdatedAt,
+ PartitionStats::lastUpdatedSnapshotId)
+ .isEqualTo(
+ Arrays.asList(
+ 0L, 0, 0L, 0, 0L, null, null)); // null counters must be initialized to zero.
+
+ PartitionStatisticsFile statisticsFile =
+ PartitionStatsHandler.writePartitionStatsFile(
+ testTable, 42L, dataSchema, expected.iterator());
+
+ List written;
+ try (CloseableIterable recordIterator =
+ PartitionStatsHandler.readPartitionStatsFile(
+ dataSchema, Files.localInput(statisticsFile.path()))) {
+ written = Lists.newArrayList(recordIterator);
+ }
+
+ assertThat(written).hasSize(expected.size());
+ Comparator comparator = Comparators.forType(partitionSchema);
+ for (int i = 0; i < written.size(); i++) {
+ assertThat(PartitionStatsUtil.isEqual(comparator, written.get(i), expected.get(i))).isTrue();
+ }
+ }
+
+ @SuppressWarnings("checkstyle:MethodLength")
+ @Test
+ public void testPartitionStats() throws Exception {
+ Table testTable =
+ TestTables.create(
+ tempDir("partition_stats_" + format.name()),
+ "partition_stats_compute_" + format.name(),
+ SCHEMA,
+ SPEC,
+ 2);
+
+ List records = prepareRecords(testTable.schema());
+ DataFile dataFile1 =
+ FileHelpers.writeDataFile(
+ testTable, outputFile(), TestHelpers.Row.of("foo", "A"), records.subList(0, 3));
+ DataFile dataFile2 =
+ FileHelpers.writeDataFile(
+ testTable, outputFile(), TestHelpers.Row.of("foo", "B"), records.subList(3, 4));
+ DataFile dataFile3 =
+ FileHelpers.writeDataFile(
+ testTable, outputFile(), TestHelpers.Row.of("bar", "A"), records.subList(4, 5));
+ DataFile dataFile4 =
+ FileHelpers.writeDataFile(
+ testTable, outputFile(), TestHelpers.Row.of("bar", "B"), records.subList(5, 7));
+
+ for (int i = 0; i < 3; i++) {
+ // insert same set of seven records thrice to have a new manifest files
+ testTable
+ .newAppend()
+ .appendFile(dataFile1)
+ .appendFile(dataFile2)
+ .appendFile(dataFile3)
+ .appendFile(dataFile4)
+ .commit();
+ }
+
+ Snapshot snapshot1 = testTable.currentSnapshot();
+ Schema recordSchema = PartitionStatsHandler.schema(Partitioning.partitionType(testTable));
+ Types.StructType partitionType =
+ recordSchema.findField(Column.PARTITION.name()).type().asStructType();
+ computeAndValidatePartitionStats(
+ testTable,
+ recordSchema,
+ Tuple.tuple(
+ partitionRecord(partitionType, "foo", "A"),
+ 0,
+ 9L,
+ 3,
+ 3 * dataFile1.fileSizeInBytes(),
+ 0L,
+ 0,
+ 0L,
+ 0,
+ 0L,
+ snapshot1.timestampMillis(),
+ snapshot1.snapshotId()),
+ Tuple.tuple(
+ partitionRecord(partitionType, "foo", "B"),
+ 0,
+ 3L,
+ 3,
+ 3 * dataFile2.fileSizeInBytes(),
+ 0L,
+ 0,
+ 0L,
+ 0,
+ 0L,
+ snapshot1.timestampMillis(),
+ snapshot1.snapshotId()),
+ Tuple.tuple(
+ partitionRecord(partitionType, "bar", "A"),
+ 0,
+ 3L,
+ 3,
+ 3 * dataFile3.fileSizeInBytes(),
+ 0L,
+ 0,
+ 0L,
+ 0,
+ 0L,
+ snapshot1.timestampMillis(),
+ snapshot1.snapshotId()),
+ Tuple.tuple(
+ partitionRecord(partitionType, "bar", "B"),
+ 0,
+ 6L,
+ 3,
+ 3 * dataFile4.fileSizeInBytes(),
+ 0L,
+ 0,
+ 0L,
+ 0,
+ 0L,
+ snapshot1.timestampMillis(),
+ snapshot1.snapshotId()));
+
+ DeleteFile posDeletes = commitPositionDeletes(testTable, dataFile1);
+ Snapshot snapshot2 = testTable.currentSnapshot();
+
+ DeleteFile eqDeletes = commitEqualityDeletes(testTable);
+ Snapshot snapshot3 = testTable.currentSnapshot();
+
+ recordSchema = PartitionStatsHandler.schema(Partitioning.partitionType(testTable));
+ partitionType = recordSchema.findField(Column.PARTITION.name()).type().asStructType();
+ computeAndValidatePartitionStats(
+ testTable,
+ recordSchema,
+ Tuple.tuple(
+ partitionRecord(partitionType, "foo", "A"),
+ 0,
+ 9L,
+ 3,
+ 3 * dataFile1.fileSizeInBytes(),
+ 0L,
+ 0,
+ eqDeletes.recordCount(),
+ 1,
+ 0L,
+ snapshot3.timestampMillis(),
+ snapshot3.snapshotId()),
+ Tuple.tuple(
+ partitionRecord(partitionType, "foo", "B"),
+ 0,
+ 3L,
+ 3,
+ 3 * dataFile2.fileSizeInBytes(),
+ 0L,
+ 0,
+ 0L,
+ 0,
+ 0L,
+ snapshot1.timestampMillis(),
+ snapshot1.snapshotId()),
+ Tuple.tuple(
+ partitionRecord(partitionType, "bar", "A"),
+ 0,
+ 3L,
+ 3,
+ 3 * dataFile3.fileSizeInBytes(),
+ posDeletes.recordCount(),
+ 1,
+ 0L,
+ 0,
+ 0L,
+ snapshot2.timestampMillis(),
+ snapshot2.snapshotId()),
+ Tuple.tuple(
+ partitionRecord(partitionType, "bar", "B"),
+ 0,
+ 6L,
+ 3,
+ 3 * dataFile4.fileSizeInBytes(),
+ 0L,
+ 0,
+ 0L,
+ 0,
+ 0L,
+ snapshot1.timestampMillis(),
+ snapshot1.snapshotId()));
+ }
+
+ private OutputFile outputFile() throws IOException {
+ return Files.localOutput(File.createTempFile("data", null, tempDir("stats")));
+ }
+
+ private static StructLike partitionRecord(
+ Types.StructType partitionType, String val1, String val2) {
+ GenericRecord record = GenericRecord.create(partitionType);
+ record.set(0, val1);
+ record.set(1, val2);
+ return record;
+ }
+
+ private static List prepareRecords(Schema schema) {
+ GenericRecord record = GenericRecord.create(schema);
+ List records = Lists.newArrayList();
+ // foo 4 records, bar 3 records
+ // foo, A -> 3 records
+ records.add(record.copy("c1", 0, "c2", "foo", "c3", "A"));
+ records.add(record.copy("c1", 1, "c2", "foo", "c3", "A"));
+ records.add(record.copy("c1", 2, "c2", "foo", "c3", "A"));
+ // foo, B -> 1 record
+ records.add(record.copy("c1", 3, "c2", "foo", "c3", "B"));
+ // bar, A -> 1 record
+ records.add(record.copy("c1", 4, "c2", "bar", "c3", "A"));
+ // bar, B -> 2 records
+ records.add(record.copy("c1", 5, "c2", "bar", "c3", "B"));
+ records.add(record.copy("c1", 6, "c2", "bar", "c3", "B"));
+ return records;
+ }
+
+ private static void computeAndValidatePartitionStats(
+ Table testTable, Schema recordSchema, Tuple... expectedValues) throws IOException {
+ // compute and commit partition stats file
+ Snapshot currentSnapshot = testTable.currentSnapshot();
+ PartitionStatisticsFile result = PartitionStatsHandler.computeAndWriteStatsFile(testTable);
+ testTable.updatePartitionStatistics().setPartitionStatistics(result).commit();
+ assertThat(result.snapshotId()).isEqualTo(currentSnapshot.snapshotId());
+
+ // read the partition entries from the stats file
+ List partitionStats;
+ try (CloseableIterable recordIterator =
+ PartitionStatsHandler.readPartitionStatsFile(
+ recordSchema, Files.localInput(result.path()))) {
+ partitionStats = Lists.newArrayList(recordIterator);
+ }
+
+ assertThat(partitionStats)
+ .extracting(
+ PartitionStats::partition,
+ PartitionStats::specId,
+ PartitionStats::dataRecordCount,
+ PartitionStats::dataFileCount,
+ PartitionStats::totalDataFileSizeInBytes,
+ PartitionStats::positionDeleteRecordCount,
+ PartitionStats::positionDeleteFileCount,
+ PartitionStats::equalityDeleteRecordCount,
+ PartitionStats::equalityDeleteFileCount,
+ PartitionStats::totalRecordCount,
+ PartitionStats::lastUpdatedAt,
+ PartitionStats::lastUpdatedSnapshotId)
+ .containsExactlyInAnyOrder(expectedValues);
+ }
+
+ private DeleteFile commitEqualityDeletes(Table testTable) throws IOException {
+ Schema deleteRowSchema = testTable.schema().select("c1");
+ Record dataDelete = GenericRecord.create(deleteRowSchema);
+ List dataDeletes =
+ Lists.newArrayList(dataDelete.copy("c1", 1), dataDelete.copy("c1", 2));
+
+ DeleteFile eqDeletes =
+ FileHelpers.writeDeleteFile(
+ testTable,
+ Files.localOutput(File.createTempFile("junit", null, tempDir("eq_delete"))),
+ TestHelpers.Row.of("foo", "A"),
+ dataDeletes,
+ deleteRowSchema);
+ testTable.newRowDelta().addDeletes(eqDeletes).commit();
+ return eqDeletes;
+ }
+
+ private DeleteFile commitPositionDeletes(Table testTable, DataFile dataFile1) throws IOException {
+ List> deletes = Lists.newArrayList();
+ for (long i = 0; i < 2; i++) {
+ deletes.add(Pair.of(dataFile1.path(), i));
+ }
+
+ DeleteFile posDeletes =
+ FileHelpers.writeDeleteFile(
+ testTable,
+ Files.localOutput(File.createTempFile("junit", null, tempDir("pos_delete"))),
+ TestHelpers.Row.of("bar", "A"),
+ deletes)
+ .first();
+ testTable.newRowDelta().addDeletes(posDeletes).commit();
+ return posDeletes;
+ }
+
+ private static PositionDelete positionDelete(
+ Schema tableSchema, CharSequence path, Long position, Object... values) {
+ PositionDelete posDelete = PositionDelete.create();
+ GenericRecord nested = GenericRecord.create(tableSchema);
+ for (int i = 0; i < values.length; i++) {
+ nested.set(i, values[i]);
+ }
+
+ posDelete.set(path, position, nested);
+ return posDelete;
+ }
+
+ private File tempDir(String folderName) throws IOException {
+ return java.nio.file.Files.createTempDirectory(temp.getRoot().toPath(), folderName).toFile();
+ }
+}
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_partition_evolution_w_id_spec_w_filter.q b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_partition_evolution_w_id_spec_w_filter.q
index d256b200bad7..19202ae4407a 100644
--- a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_partition_evolution_w_id_spec_w_filter.q
+++ b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_partition_evolution_w_id_spec_w_filter.q
@@ -66,21 +66,6 @@ delete from ice_orc where last_name in ('ln1', 'ln9');
delete from ice_orc where last_name in ('ln3', 'ln11');
delete from ice_orc where last_name in ('ln5', 'ln13');
-alter table ice_orc set partition spec(team_id);
-insert into ice_orc VALUES
- ('fn17', 'ln17', 1, 10, 100),
- ('fn18','ln18', 1, 10, 100);
-insert into ice_orc VALUES
- ('fn19','ln19', 2, 11, 100),
- ('fn20','ln20', 2, 11, 100);
-insert into ice_orc VALUES
- ('fn21','ln21', 3, 12, 100),
- ('fn22','ln22', 3, 12, 100);
-insert into ice_orc VALUES
- ('fn23','ln23', 4, 13, 100),
- ('fn24','ln24', 4, 13, 100);
-
-
select * from ice_orc;
describe formatted ice_orc;
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_stats_with_ppr.q b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_stats_with_ppr.q
new file mode 100644
index 000000000000..144a14a0814f
--- /dev/null
+++ b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_stats_with_ppr.q
@@ -0,0 +1,29 @@
+--! qt:replace:/(\s+Statistics\: Num rows\: \d+ Data size\:\s+)\S+(\s+Basic stats\: \S+ Column stats\: \S+)/$1#Masked#$2/
+
+set hive.explain.user=false;
+set hive.compute.query.using.stats=true;
+set hive.fetch.task.conversion=none;
+
+create external table ice01 (`i` int, `t` timestamp)
+ partitioned by (year int, month int, day int)
+stored by iceberg tblproperties ('format-version'='2');
+
+insert into ice01 (i, year, month, day) values
+(1, 2023, 10, 3),
+(2, 2023, 10, 3),
+(2, 2023, 10, 3),
+(3, 2023, 10, 4),
+(4, 2023, 10, 4);
+
+explain
+select i from ice01
+ where year=2023 and month = 10 and day = 3;
+
+explain
+select count(*) from ice01
+ where year=2023 and month = 10 and day = 3;
+
+select count(1) from ice01
+ where year=2023 and month = 10 and day = 3;
+
+drop table ice01;
\ No newline at end of file
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/row_count.q b/iceberg/iceberg-handler/src/test/queries/positive/row_count.q
index b2dd62328ea4..727084d892f3 100644
--- a/iceberg/iceberg-handler/src/test/queries/positive/row_count.q
+++ b/iceberg/iceberg-handler/src/test/queries/positive/row_count.q
@@ -9,7 +9,7 @@
-- Mask total file size
--! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
-- Mask width
---! qt:replace:/(width=1)\d+/$1####/
+--! qt:replace:/(width=[15])\d+/$1###/
-- Mask iceberg version
--! qt:replace:/(\S\"iceberg-version\\\":\\\")(\w+\s\w+\s\d+\.\d+\.\d+\s\(\w+\s\w+\))(\\\")/$1#Masked#$3/
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 3b594187f2a2..16e924f62f75 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
@@ -92,29 +92,29 @@ STAGE PLANS:
Tez
#### A masked pattern was here ####
Edges:
- Reducer 10 <- Reducer 19 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
- Reducer 11 <- Reducer 10 (XPROD_EDGE), Reducer 24 (XPROD_EDGE)
- Reducer 12 <- Reducer 11 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE)
+ Reducer 10 <- Reducer 24 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
+ Reducer 11 <- Reducer 10 (XPROD_EDGE), Reducer 19 (XPROD_EDGE)
+ Reducer 12 <- Reducer 11 (SIMPLE_EDGE), Reducer 20 (SIMPLE_EDGE)
Reducer 13 <- Reducer 12 (SIMPLE_EDGE)
- Reducer 14 <- Map 1 (XPROD_EDGE), Reducer 20 (XPROD_EDGE)
- Reducer 15 <- Reducer 14 (SIMPLE_EDGE), Reducer 19 (SIMPLE_EDGE)
- Reducer 16 <- Reducer 15 (XPROD_EDGE), Reducer 24 (XPROD_EDGE)
- Reducer 17 <- Reducer 16 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE)
+ Reducer 14 <- Map 1 (XPROD_EDGE), Reducer 23 (XPROD_EDGE)
+ Reducer 15 <- Reducer 14 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE)
+ Reducer 16 <- Reducer 15 (XPROD_EDGE), Reducer 19 (XPROD_EDGE)
+ Reducer 17 <- Reducer 16 (SIMPLE_EDGE), Reducer 20 (SIMPLE_EDGE)
Reducer 18 <- Reducer 17 (SIMPLE_EDGE), Union 7 (CONTAINS)
- Reducer 19 <- Map 1 (SIMPLE_EDGE)
- Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 19 (SIMPLE_EDGE)
- Reducer 20 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+ Reducer 19 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE)
+ Reducer 20 <- Map 1 (SIMPLE_EDGE)
Reducer 21 <- Map 1 (CUSTOM_SIMPLE_EDGE)
- Reducer 22 <- Map 1 (CUSTOM_SIMPLE_EDGE)
- Reducer 23 <- Map 1 (SIMPLE_EDGE)
- Reducer 24 <- Map 1 (CUSTOM_SIMPLE_EDGE)
- Reducer 25 <- Map 1 (CUSTOM_SIMPLE_EDGE)
- Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 21 (XPROD_EDGE)
- Reducer 4 <- Reducer 23 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
- Reducer 5 <- Reducer 25 (XPROD_EDGE), Reducer 4 (XPROD_EDGE)
+ Reducer 23 <- Map 22 (CUSTOM_SIMPLE_EDGE)
+ Reducer 24 <- Map 22 (SIMPLE_EDGE)
+ Reducer 25 <- Map 22 (CUSTOM_SIMPLE_EDGE)
+ Reducer 26 <- Map 22 (CUSTOM_SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (XPROD_EDGE), Reducer 25 (XPROD_EDGE)
+ Reducer 4 <- Reducer 20 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+ Reducer 5 <- Reducer 21 (XPROD_EDGE), Reducer 4 (XPROD_EDGE)
Reducer 6 <- Reducer 13 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE), Union 7 (CONTAINS)
Reducer 8 <- Union 7 (SIMPLE_EDGE)
- Reducer 9 <- Map 1 (XPROD_EDGE), Reducer 22 (XPROD_EDGE)
+ Reducer 9 <- Map 1 (XPROD_EDGE), Reducer 26 (XPROD_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -155,25 +155,13 @@ STAGE PLANS:
Statistics: Num rows: 6 Data size: 2862 Basic stats: COMPLETE Column stats: COMPLETE
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: (a <= 5) (type: boolean)
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- keys: a (type: int)
- minReductionHashAggr: 0.4
- mode: hash
- outputColumnNames: _col0
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
- 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
+ predicate: (c > 800) (type: boolean)
+ Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
Group By Operator
aggregations: count()
- minReductionHashAggr: 0.4
+ minReductionHashAggr: 0.8333333
mode: hash
outputColumnNames: _col0
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -183,8 +171,20 @@ STAGE PLANS:
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
value expressions: _col0 (type: bigint)
Group By Operator
- aggregations: count(), count(a)
- minReductionHashAggr: 0.4
+ keys: c (type: int)
+ minReductionHashAggr: 0.6666666
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ 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
+ Group By Operator
+ aggregations: count(), count(c)
+ minReductionHashAggr: 0.8333333
mode: hash
outputColumnNames: _col0, _col1
Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
@@ -193,6 +193,16 @@ STAGE PLANS:
sort order:
Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
value expressions: _col0 (type: bigint), _col1 (type: bigint)
+ Execution mode: vectorized
+ Map 22
+ Map Operator Tree:
+ TableScan
+ alias: tbl_ice
+ filterExpr: (a <= 5) (type: boolean)
+ Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (a <= 5) (type: boolean)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
Group By Operator
@@ -206,26 +216,38 @@ STAGE PLANS:
sort order:
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
value expressions: _col0 (type: bigint)
- Filter Operator
- predicate: (c > 800) (type: boolean)
- Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- keys: c (type: int)
- minReductionHashAggr: 0.6666666
- mode: hash
- outputColumnNames: _col0
- Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- 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
Select Operator
- Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+ expressions: a (type: int)
+ outputColumnNames: a
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: a (type: int)
+ minReductionHashAggr: 0.4
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ 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
+ Group By Operator
+ aggregations: count(), count(a)
+ minReductionHashAggr: 0.4
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: bigint), _col1 (type: bigint)
+ Select Operator
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
Group By Operator
aggregations: count()
- minReductionHashAggr: 0.8333333
+ minReductionHashAggr: 0.4
mode: hash
outputColumnNames: _col0
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -234,17 +256,6 @@ STAGE PLANS:
sort order:
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
value expressions: _col0 (type: bigint)
- Group By Operator
- aggregations: count(), count(c)
- minReductionHashAggr: 0.8333333
- mode: hash
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint), _col1 (type: bigint)
Execution mode: vectorized
Reducer 10
Reduce Operator Tree:
@@ -456,35 +467,20 @@ STAGE PLANS:
Execution mode: vectorized
Reduce Operator Tree:
Group By Operator
- keys: KEY._col0 (type: int)
+ aggregations: count(VALUE._col0)
mode: mergepartial
outputColumnNames: _col0
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
- 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
+ Reduce Output Operator
+ null sort order:
+ sort order:
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- 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
- 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
- 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
- value expressions: _col1 (type: boolean)
+ 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
+ value expressions: _col0 (type: bigint)
Reducer 2
Reduce Operator Tree:
Merge Join Operator
@@ -504,15 +500,35 @@ STAGE PLANS:
Execution mode: vectorized
Reduce Operator Tree:
Group By Operator
- aggregations: count(VALUE._col0)
+ keys: KEY._col0 (type: int)
mode: mergepartial
outputColumnNames: _col0
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint)
+ Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ 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
+ 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
+ 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
+ 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
+ value expressions: _col1 (type: boolean)
Reducer 21
Execution mode: vectorized
Reduce Operator Tree:
@@ -526,7 +542,7 @@ STAGE PLANS:
sort order:
Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
value expressions: _col0 (type: bigint), _col1 (type: bigint)
- Reducer 22
+ Reducer 23
Execution mode: vectorized
Reduce Operator Tree:
Group By Operator
@@ -539,70 +555,65 @@ STAGE PLANS:
sort order:
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
value expressions: _col0 (type: bigint)
- Reducer 23
+ Reducer 24
Execution mode: vectorized
Reduce Operator Tree:
Group By Operator
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: COMPLETE
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: COMPLETE
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: COMPLETE
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: COMPLETE
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: COMPLETE
value expressions: _col1 (type: boolean)
- Reducer 24
+ Reducer 25
Execution mode: vectorized
Reduce Operator Tree:
Group By Operator
- aggregations: count(VALUE._col0)
+ aggregations: count(VALUE._col0), count(VALUE._col1)
mode: mergepartial
- outputColumnNames: _col0
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
null sort order:
sort order:
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint)
- Reducer 25
+ Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: bigint), _col1 (type: bigint)
+ Reducer 26
Execution mode: vectorized
Reduce Operator Tree:
Group By Operator
- aggregations: count(VALUE._col0), count(VALUE._col1)
+ aggregations: count(VALUE._col0)
mode: mergepartial
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
null sort order:
sort order:
- Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint), _col1 (type: bigint)
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: bigint)
Reducer 3
Reduce Operator Tree:
Merge Join Operator
diff --git a/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_pruning.q.out b/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_pruning.q.out
index 2607c84e1fab..a1f9aef00823 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_pruning.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/dynamic_partition_pruning.q.out
@@ -307,19 +307,16 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: ds is not null (type: boolean)
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: ds is not null (type: boolean)
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: ds (type: string)
- outputColumnNames: _col0
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: string)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized
Map 4
Map Operator Tree:
@@ -439,19 +436,16 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: ds is not null (type: boolean)
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: ds is not null (type: boolean)
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: ds (type: string)
- outputColumnNames: _col0
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: string)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized
Map 4
Map Operator Tree:
@@ -573,20 +567,17 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: (ds is not null and hr is not null) (type: boolean)
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (ds is not null and hr is not null) (type: boolean)
+ Select Operator
+ expressions: ds (type: string), hr (type: string)
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: ds (type: string), hr (type: string)
- outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: string)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: string)
+ value expressions: _col1 (type: string)
Execution mode: vectorized
Map 5
Map Operator Tree:
@@ -767,20 +758,17 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: (ds is not null and hr is not null) (type: boolean)
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (ds is not null and hr is not null) (type: boolean)
+ Select Operator
+ expressions: ds (type: string), hr (type: string)
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: ds (type: string), hr (type: string)
- outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: string)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: string)
+ value expressions: _col1 (type: string)
Execution mode: vectorized
Map 5
Map Operator Tree:
@@ -935,19 +923,16 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: (ds is not null and hr is not null) (type: boolean)
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (ds is not null and hr is not null) (type: boolean)
+ Select Operator
+ expressions: ds (type: string), hr (type: string)
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: ds (type: string), hr (type: string)
- outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: string)
+ null sort order: zz
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: string), _col1 (type: string)
- null sort order: zz
- sort order: ++
- Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
- Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized
Map 4
Map Operator Tree:
@@ -1083,19 +1068,16 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: (ds is not null and hr is not null) (type: boolean)
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (ds is not null and hr is not null) (type: boolean)
+ Select Operator
+ expressions: ds (type: string), hr (type: string)
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: ds (type: string), hr (type: string)
- outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: string)
+ null sort order: zz
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: string), _col1 (type: string)
- null sort order: zz
- sort order: ++
- Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
- Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized
Map 4
Map Operator Tree:
@@ -1210,19 +1192,16 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: ds is not null (type: boolean)
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: ds is not null (type: boolean)
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: ds (type: string)
- outputColumnNames: _col0
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: string)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized
Map 4
Map Operator Tree:
@@ -1342,19 +1321,16 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: ds is not null (type: boolean)
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: ds is not null (type: boolean)
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: ds (type: string)
- outputColumnNames: _col0
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: string)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized
Map 4
Map Operator Tree:
@@ -1467,19 +1443,16 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: hr is not null (type: boolean)
Statistics: Num rows: 2000 Data size: 172000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: hr is not null (type: boolean)
+ Select Operator
+ expressions: hr (type: string)
+ outputColumnNames: _col0
Statistics: Num rows: 2000 Data size: 172000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: hr (type: string)
- outputColumnNames: _col0
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 2000 Data size: 172000 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: string)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 2000 Data size: 172000 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized
Map 4
Map Operator Tree:
@@ -1599,19 +1572,16 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: hr is not null (type: boolean)
Statistics: Num rows: 2000 Data size: 172000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: hr is not null (type: boolean)
+ Select Operator
+ expressions: hr (type: string)
+ outputColumnNames: _col0
Statistics: Num rows: 2000 Data size: 172000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: hr (type: string)
- outputColumnNames: _col0
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 2000 Data size: 172000 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: string)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 2000 Data size: 172000 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized
Map 4
Map Operator Tree:
@@ -1715,19 +1685,16 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: (ds is not null and hr is not null) (type: boolean)
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (ds is not null and hr is not null) (type: boolean)
+ Select Operator
+ expressions: ds (type: string), hr (type: string)
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: ds (type: string), hr (type: string)
- outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: string)
+ null sort order: zz
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: string), _col1 (type: string)
- null sort order: zz
- sort order: ++
- Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
- Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized
Map 4
Map Operator Tree:
@@ -1869,7 +1836,7 @@ STAGE PLANS:
filterExpr: ((UDFToDouble(hr) = 11.0D) and ds is not null) (type: boolean)
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
- predicate: ((UDFToDouble(hr) = 11.0D) and ds is not null) (type: boolean)
+ predicate: (UDFToDouble(hr) = 11.0D) (type: boolean)
Statistics: Num rows: 1000 Data size: 180000 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: ds (type: string), hr (type: string)
@@ -2063,7 +2030,7 @@ STAGE PLANS:
filterExpr: ((UDFToDouble(hr) = 13.0D) and ds is not null) (type: boolean)
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
- predicate: ((UDFToDouble(hr) = 13.0D) and ds is not null) (type: boolean)
+ predicate: (UDFToDouble(hr) = 13.0D) (type: boolean)
Statistics: Num rows: 1000 Data size: 180000 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: ds (type: string), hr (type: string)
@@ -2251,19 +2218,16 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: ds is not null (type: boolean)
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: ds is not null (type: boolean)
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: ds (type: string)
- outputColumnNames: _col0
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: string)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized
Map 4
Map Operator Tree:
@@ -2467,19 +2431,16 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: ds is not null (type: boolean)
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: ds is not null (type: boolean)
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: ds (type: string)
- outputColumnNames: _col0
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: string)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized
Map 4
Map Operator Tree:
@@ -2687,8 +2648,9 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: ds is not null (type: boolean)
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: ds is not null (type: boolean)
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: ds
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
Group By Operator
keys: ds (type: string)
@@ -2730,8 +2692,9 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: ds is not null (type: boolean)
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: ds is not null (type: boolean)
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: ds
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
Group By Operator
keys: ds (type: string)
@@ -2974,33 +2937,30 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: ds is not null (type: boolean)
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: ds is not null (type: boolean)
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: ds (type: string)
- outputColumnNames: _col0
- Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Map Join Operator
- condition map:
- Inner Join 0 to 1
- keys:
- 0 _col0 (type: string)
- 1 _col0 (type: string)
- input vertices:
- 1 Map 3
- Statistics: Num rows: 1000 Data size: 8000 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- aggregations: count()
- minReductionHashAggr: 0.99
- mode: hash
- outputColumnNames: _col0
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0 _col0 (type: string)
+ 1 _col0 (type: string)
+ input vertices:
+ 1 Map 3
+ Statistics: Num rows: 1000 Data size: 8000 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ aggregations: count()
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint)
+ value expressions: _col0 (type: bigint)
Execution mode: vectorized
Map 3
Map Operator Tree:
@@ -3117,43 +3077,40 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: (ds is not null and hr is not null) (type: boolean)
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (ds is not null and hr is not null) (type: boolean)
+ Select Operator
+ expressions: ds (type: string), hr (type: string)
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: ds (type: string), hr (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0 _col0 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col1
+ input vertices:
+ 1 Map 3
+ Statistics: Num rows: 1000 Data size: 86000 Basic stats: COMPLETE Column stats: COMPLETE
Map Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
+ 0 _col1 (type: string)
1 _col0 (type: string)
- outputColumnNames: _col1
input vertices:
- 1 Map 3
- Statistics: Num rows: 1000 Data size: 86000 Basic stats: COMPLETE Column stats: COMPLETE
- Map Join Operator
- condition map:
- Inner Join 0 to 1
- keys:
- 0 _col1 (type: string)
- 1 _col0 (type: string)
- input vertices:
- 1 Map 4
- Statistics: Num rows: 1000 Data size: 8000 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- aggregations: count()
- minReductionHashAggr: 0.99
- mode: hash
- outputColumnNames: _col0
+ 1 Map 4
+ Statistics: Num rows: 1000 Data size: 8000 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ aggregations: count()
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint)
+ value expressions: _col0 (type: bigint)
Execution mode: vectorized
Map 3
Map Operator Tree:
@@ -3304,33 +3261,30 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: (ds is not null and hr is not null) (type: boolean)
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (ds is not null and hr is not null) (type: boolean)
+ Select Operator
+ expressions: ds (type: string), hr (type: string)
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: ds (type: string), hr (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Map Join Operator
- condition map:
- Inner Join 0 to 1
- keys:
- 0 _col0 (type: string), _col1 (type: string)
- 1 _col0 (type: string), _col1 (type: string)
- input vertices:
- 1 Map 3
- Statistics: Num rows: 1000 Data size: 8000 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- aggregations: count()
- minReductionHashAggr: 0.99
- mode: hash
- outputColumnNames: _col0
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0 _col0 (type: string), _col1 (type: string)
+ 1 _col0 (type: string), _col1 (type: string)
+ input vertices:
+ 1 Map 3
+ Statistics: Num rows: 1000 Data size: 8000 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ aggregations: count()
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint)
+ value expressions: _col0 (type: bigint)
Execution mode: vectorized
Map 3
Map Operator Tree:
@@ -3457,33 +3411,30 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: ds is not null (type: boolean)
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: ds is not null (type: boolean)
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: ds (type: string)
- outputColumnNames: _col0
- Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Map Join Operator
- condition map:
- Inner Join 0 to 1
- keys:
- 0 _col0 (type: string)
- 1 _col0 (type: string)
- input vertices:
- 1 Map 3
- Statistics: Num rows: 1000 Data size: 8000 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- aggregations: count()
- minReductionHashAggr: 0.99
- mode: hash
- outputColumnNames: _col0
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0 _col0 (type: string)
+ 1 _col0 (type: string)
+ input vertices:
+ 1 Map 3
+ Statistics: Num rows: 1000 Data size: 8000 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ aggregations: count()
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint)
+ value expressions: _col0 (type: bigint)
Execution mode: vectorized
Map 3
Map Operator Tree:
@@ -3583,33 +3534,30 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: hr is not null (type: boolean)
Statistics: Num rows: 2000 Data size: 172000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: hr is not null (type: boolean)
+ Select Operator
+ expressions: hr (type: string)
+ outputColumnNames: _col0
Statistics: Num rows: 2000 Data size: 172000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: hr (type: string)
- outputColumnNames: _col0
- Statistics: Num rows: 2000 Data size: 172000 Basic stats: COMPLETE Column stats: COMPLETE
- Map Join Operator
- condition map:
- Inner Join 0 to 1
- keys:
- 0 _col0 (type: string)
- 1 _col0 (type: string)
- input vertices:
- 1 Map 3
- Statistics: Num rows: 1000 Data size: 8000 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- aggregations: count()
- minReductionHashAggr: 0.99
- mode: hash
- outputColumnNames: _col0
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0 _col0 (type: string)
+ 1 _col0 (type: string)
+ input vertices:
+ 1 Map 3
+ Statistics: Num rows: 1000 Data size: 8000 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ aggregations: count()
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint)
+ value expressions: _col0 (type: bigint)
Execution mode: vectorized
Map 3
Map Operator Tree:
@@ -3727,7 +3675,7 @@ STAGE PLANS:
filterExpr: ((UDFToDouble(hr) = 11.0D) and ds is not null) (type: boolean)
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
- predicate: ((UDFToDouble(hr) = 11.0D) and ds is not null) (type: boolean)
+ predicate: (UDFToDouble(hr) = 11.0D) (type: boolean)
Statistics: Num rows: 1000 Data size: 180000 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: ds (type: string), hr (type: string)
@@ -3907,7 +3855,7 @@ STAGE PLANS:
filterExpr: ((UDFToDouble(hr) = 13.0D) and ds is not null) (type: boolean)
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
- predicate: ((UDFToDouble(hr) = 13.0D) and ds is not null) (type: boolean)
+ predicate: (UDFToDouble(hr) = 13.0D) (type: boolean)
Statistics: Num rows: 1000 Data size: 180000 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: ds (type: string), hr (type: string)
@@ -4082,35 +4030,32 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: ds is not null (type: boolean)
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: ds is not null (type: boolean)
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: ds (type: string)
+ Map Join Operator
+ condition map:
+ Left Semi Join 0 to 1
+ keys:
+ 0 _col0 (type: string)
+ 1 _col0 (type: string)
outputColumnNames: _col0
- Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: COMPLETE
- Map Join Operator
- condition map:
- Left Semi Join 0 to 1
- keys:
- 0 _col0 (type: string)
- 1 _col0 (type: string)
+ input vertices:
+ 1 Union 5
+ Statistics: Num rows: 1000 Data size: 94000 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: string)
+ minReductionHashAggr: 0.99
+ mode: hash
outputColumnNames: _col0
- input vertices:
- 1 Union 5
- Statistics: Num rows: 1000 Data size: 94000 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- keys: _col0 (type: string)
- minReductionHashAggr: 0.99
- mode: hash
- outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE
+ 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: 94 Basic stats: COMPLETE Column stats: COMPLETE
- 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: 94 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized
Map 3
Map Operator Tree:
@@ -4321,33 +4266,30 @@ STAGE PLANS:
alias: srcpart_iceberg
filterExpr: (ds is not null and hr is not null) (type: boolean)
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (ds is not null and hr is not null) (type: boolean)
+ Select Operator
+ expressions: ds (type: string), hr (type: string)
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: ds (type: string), hr (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 2000 Data size: 360000 Basic stats: COMPLETE Column stats: COMPLETE
- Map Join Operator
- condition map:
- Inner Join 0 to 1
- keys:
- 0 _col0 (type: string), _col1 (type: string)
- 1 _col0 (type: string), _col1 (type: string)
- input vertices:
- 1 Map 3
- Statistics: Num rows: 2000 Data size: 16000 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- aggregations: count()
- minReductionHashAggr: 0.99
- mode: hash
- outputColumnNames: _col0
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0 _col0 (type: string), _col1 (type: string)
+ 1 _col0 (type: string), _col1 (type: string)
+ input vertices:
+ 1 Map 3
+ Statistics: Num rows: 2000 Data size: 16000 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ aggregations: count()
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint)
+ value expressions: _col0 (type: bigint)
Map 3
Map Operator Tree:
TableScan
diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition.q.out
index ba6436dc836a..bde257309862 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition.q.out
@@ -102,6 +102,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -213,6 +215,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 2
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -263,37 +267,34 @@ STAGE PLANS:
TableScan
alias: ice_parquet_int
filterExpr: (pcol = 2) (type: boolean)
- Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (pcol = 2) (type: boolean)
+ Statistics: Num rows: 2 Data size: 182 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: strcol (type: string), intcol (type: int), 3 (type: int)
+ outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.ice_parquet_int
Select Operator
- expressions: strcol (type: string), intcol (type: int), 3 (type: int)
- outputColumnNames: _col0, _col1, _col2
+ expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int)
+ outputColumnNames: strcol, intcol, pcol
Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
- File Output Operator
- compressed: false
- Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
- table:
- input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
- output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
- serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
- name: default.ice_parquet_int
- Select Operator
- expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int)
- outputColumnNames: strcol, intcol, pcol
- Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- aggregations: max(length(strcol)), avg(COALESCE(length(strcol),0)), count(1), count(strcol), compute_bit_vector_hll(strcol), min(intcol), max(intcol), count(intcol), compute_bit_vector_hll(intcol), min(pcol), max(pcol), count(pcol), compute_bit_vector_hll(pcol)
- minReductionHashAggr: 0.5
- mode: hash
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
+ Group By Operator
+ aggregations: max(length(strcol)), avg(COALESCE(length(strcol),0)), count(1), count(strcol), compute_bit_vector_hll(strcol), min(intcol), max(intcol), count(intcol), compute_bit_vector_hll(intcol), min(pcol), max(pcol), count(pcol), compute_bit_vector_hll(pcol)
+ minReductionHashAggr: 0.5
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
+ Statistics: Num rows: 1 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
Statistics: Num rows: 1 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: int), _col1 (type: struct), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: int), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: int), _col11 (type: bigint), _col12 (type: binary)
+ value expressions: _col0 (type: int), _col1 (type: struct), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: int), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: int), _col11 (type: bigint), _col12 (type: binary)
Execution mode: vectorized
Reducer 2
Execution mode: vectorized
@@ -321,6 +322,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 3
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -371,37 +374,34 @@ STAGE PLANS:
TableScan
alias: ice_parquet_int
filterExpr: (pcol = 2) (type: boolean)
- Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (pcol = 2) (type: boolean)
- Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 2 Data size: 182 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: strcol (type: string), intcol (type: int), 4 (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.ice_parquet_int
Select Operator
- expressions: strcol (type: string), intcol (type: int), 4 (type: int)
- outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
- File Output Operator
- compressed: false
- Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
- table:
- input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
- output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
- serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
- name: default.ice_parquet_int
- Select Operator
- expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int)
- outputColumnNames: strcol, intcol, pcol
- Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- aggregations: max(length(strcol)), avg(COALESCE(length(strcol),0)), count(1), count(strcol), compute_bit_vector_hll(strcol), min(intcol), max(intcol), count(intcol), compute_bit_vector_hll(intcol), min(pcol), max(pcol), count(pcol), compute_bit_vector_hll(pcol)
- minReductionHashAggr: 0.6666666
- mode: hash
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
+ expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int)
+ outputColumnNames: strcol, intcol, pcol
+ Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ aggregations: max(length(strcol)), avg(COALESCE(length(strcol),0)), count(1), count(strcol), compute_bit_vector_hll(strcol), min(intcol), max(intcol), count(intcol), compute_bit_vector_hll(intcol), min(pcol), max(pcol), count(pcol), compute_bit_vector_hll(pcol)
+ minReductionHashAggr: 0.5
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
+ Statistics: Num rows: 1 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
Statistics: Num rows: 1 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: int), _col1 (type: struct), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: int), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: int), _col11 (type: bigint), _col12 (type: binary)
+ value expressions: _col0 (type: int), _col1 (type: struct), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: int), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: int), _col11 (type: bigint), _col12 (type: binary)
Execution mode: vectorized
Reducer 2
Execution mode: vectorized
@@ -429,6 +429,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 04
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -619,6 +621,9 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ country USA
+ state CA
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -730,6 +735,9 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ country USA
+ state CA
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -841,6 +849,9 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ country USA
+ state TX
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1331,6 +1342,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ state TX
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1450,6 +1463,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ country India
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1569,6 +1584,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ country India
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2145,6 +2162,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-31
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2256,6 +2275,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-26
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2367,6 +2388,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-12
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2485,6 +2508,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-13
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2589,6 +2614,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-02
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2805,6 +2832,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 34567890123456787
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2916,6 +2945,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 12346577399277578
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -3027,6 +3058,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 45637829068876994
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -3145,6 +3178,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 45637829068876994
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -3249,6 +3284,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 12346577399277578
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -3463,6 +3500,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 3.14786
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -3574,6 +3613,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 3.189
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -3685,6 +3726,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 45.789
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -3803,6 +3846,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 3.14786
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -3907,6 +3952,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 3.189
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -4121,6 +4168,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 3.14786
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -4232,6 +4281,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 3.189
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -4343,6 +4394,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 45.789
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -4461,6 +4514,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 3.14786
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -4565,6 +4620,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 3.189
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_transforms.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_transforms.q.out
index 477788d736ed..1fb6a3255c21 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_transforms.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_transforms.q.out
@@ -104,6 +104,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-05
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -215,6 +217,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-26
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -326,6 +330,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-12
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -444,6 +450,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-13
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -548,6 +556,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-02
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -764,6 +774,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-31
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -875,6 +887,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-26
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -986,6 +1000,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-12
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1104,6 +1120,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-13
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1208,6 +1226,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-02
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1424,6 +1444,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-31
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1535,6 +1557,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-26
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1646,6 +1670,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-12
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1764,6 +1790,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-13
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1868,6 +1896,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-02
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2084,6 +2114,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol gfhutjkgkd
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2195,6 +2227,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol gfhyuitogh
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2299,6 +2333,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol gfhuiyoprj
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2499,6 +2535,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol gfhutjkgkd
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2610,6 +2648,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol gfhutjkgkd
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2714,6 +2754,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol gfhutjkgkd
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_with_evolution.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_with_evolution.q.out
index c87dd5c6b4b4..9fce0ec2f235 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_with_evolution.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_into_partition_with_evolution.q.out
@@ -125,6 +125,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ b rtyuiy
replace: false
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition.q.out
index 673be32525e7..fdbc4cf4b08e 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition.q.out
@@ -57,37 +57,34 @@ STAGE PLANS:
TableScan
alias: ice_parquet_int
filterExpr: (pcol = 2) (type: boolean)
- Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (pcol = 2) (type: boolean)
+ Statistics: Num rows: 2 Data size: 182 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: strcol (type: string), intcol (type: int), 1 (type: int)
+ outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.ice_parquet_int
Select Operator
- expressions: strcol (type: string), intcol (type: int), 1 (type: int)
- outputColumnNames: _col0, _col1, _col2
+ expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int)
+ outputColumnNames: strcol, intcol, pcol
Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
- File Output Operator
- compressed: false
- Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
- table:
- input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
- output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
- serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
- name: default.ice_parquet_int
- Select Operator
- expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int)
- outputColumnNames: strcol, intcol, pcol
- Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- aggregations: max(length(strcol)), avg(COALESCE(length(strcol),0)), count(1), count(strcol), compute_bit_vector_hll(strcol), min(intcol), max(intcol), count(intcol), compute_bit_vector_hll(intcol), min(pcol), max(pcol), count(pcol), compute_bit_vector_hll(pcol)
- minReductionHashAggr: 0.5
- mode: hash
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
+ Group By Operator
+ aggregations: max(length(strcol)), avg(COALESCE(length(strcol),0)), count(1), count(strcol), compute_bit_vector_hll(strcol), min(intcol), max(intcol), count(intcol), compute_bit_vector_hll(intcol), min(pcol), max(pcol), count(pcol), compute_bit_vector_hll(pcol)
+ minReductionHashAggr: 0.5
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
+ Statistics: Num rows: 1 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
Statistics: Num rows: 1 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: int), _col1 (type: struct), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: int), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: int), _col11 (type: bigint), _col12 (type: binary)
+ value expressions: _col0 (type: int), _col1 (type: struct), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: int), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: int), _col11 (type: bigint), _col12 (type: binary)
Execution mode: vectorized
Reducer 2
Execution mode: vectorized
@@ -115,6 +112,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -165,37 +164,34 @@ STAGE PLANS:
TableScan
alias: ice_parquet_int
filterExpr: (pcol = 2) (type: boolean)
- Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (pcol = 2) (type: boolean)
+ Statistics: Num rows: 2 Data size: 182 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: strcol (type: string), intcol (type: int), 1 (type: int)
+ outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.ice_parquet_int
Select Operator
- expressions: strcol (type: string), intcol (type: int), 1 (type: int)
- outputColumnNames: _col0, _col1, _col2
+ expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int)
+ outputColumnNames: strcol, intcol, pcol
Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
- File Output Operator
- compressed: false
- Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
- table:
- input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
- output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
- serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
- name: default.ice_parquet_int
- Select Operator
- expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int)
- outputColumnNames: strcol, intcol, pcol
- Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- aggregations: max(length(strcol)), avg(COALESCE(length(strcol),0)), count(1), count(strcol), compute_bit_vector_hll(strcol), min(intcol), max(intcol), count(intcol), compute_bit_vector_hll(intcol), min(pcol), max(pcol), count(pcol), compute_bit_vector_hll(pcol)
- minReductionHashAggr: 0.5
- mode: hash
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
+ Group By Operator
+ aggregations: max(length(strcol)), avg(COALESCE(length(strcol),0)), count(1), count(strcol), compute_bit_vector_hll(strcol), min(intcol), max(intcol), count(intcol), compute_bit_vector_hll(intcol), min(pcol), max(pcol), count(pcol), compute_bit_vector_hll(pcol)
+ minReductionHashAggr: 0.5
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
+ Statistics: Num rows: 1 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
Statistics: Num rows: 1 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 560 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: int), _col1 (type: struct), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: int), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: int), _col11 (type: bigint), _col12 (type: binary)
+ value expressions: _col0 (type: int), _col1 (type: struct), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: int), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: int), _col11 (type: bigint), _col12 (type: binary)
Execution mode: vectorized
Reducer 2
Execution mode: vectorized
@@ -223,6 +219,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -417,6 +415,9 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ country USA
+ state CA
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -913,6 +914,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ state TX
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1032,6 +1035,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ country India
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1151,6 +1156,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ country India
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1389,6 +1396,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-31
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1505,6 +1514,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-31
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1609,6 +1620,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-26
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1833,6 +1846,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 34567890123456787
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1949,6 +1964,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 34567890123456787
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2053,6 +2070,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 12346577399277578
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2277,6 +2296,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 3.14786
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2393,6 +2414,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 3.14786
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2497,6 +2520,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 3.189
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2721,6 +2746,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 3.14786
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2837,6 +2864,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 3.14786
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2941,6 +2970,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 3.189
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
diff --git a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition_transforms.q.out b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition_transforms.q.out
index 380a76a0cc41..33a342eac6d6 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition_transforms.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_insert_overwrite_partition_transforms.q.out
@@ -104,6 +104,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-05
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -215,6 +217,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-26
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -326,6 +330,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-12
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -440,6 +446,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-13
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -544,6 +552,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-02
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -738,6 +748,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-31
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -849,6 +861,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-26
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -960,6 +974,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-12
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1074,6 +1090,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-13
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1178,6 +1196,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-02
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1372,6 +1392,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-31
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1483,6 +1505,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-26
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1594,6 +1618,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-12
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1712,6 +1738,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-13
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -1816,6 +1844,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol 1999-12-02
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2032,6 +2062,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol gfhutjkgkd
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2143,6 +2175,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol gfhyuitogh
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
@@ -2247,6 +2281,8 @@ STAGE PLANS:
Stage: Stage-0
Move Operator
tables:
+ partition:
+ pcol gfhuiyoprj
replace: true
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
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
new file mode 100644
index 000000000000..57bdbac57c57
--- /dev/null
+++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_stats_with_ppr.q.out
@@ -0,0 +1,119 @@
+PREHOOK: query: create external table ice01 (`i` int, `t` timestamp)
+ partitioned by (year int, month int, day int)
+stored by iceberg tblproperties ('format-version'='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice01
+POSTHOOK: query: create external table ice01 (`i` int, `t` timestamp)
+ partitioned by (year int, month int, day int)
+stored by iceberg tblproperties ('format-version'='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice01
+PREHOOK: query: insert into ice01 (i, year, month, day) values
+(1, 2023, 10, 3),
+(2, 2023, 10, 3),
+(2, 2023, 10, 3),
+(3, 2023, 10, 4),
+(4, 2023, 10, 4)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ice01
+POSTHOOK: query: insert into ice01 (i, year, month, day) values
+(1, 2023, 10, 3),
+(2, 2023, 10, 3),
+(2, 2023, 10, 3),
+(3, 2023, 10, 4),
+(4, 2023, 10, 4)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ice01
+PREHOOK: query: explain
+select i from ice01
+ where year=2023 and month = 10 and day = 3
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice01
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: explain
+select i from ice01
+ where year=2023 and month = 10 and day = 3
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice01
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: ice01
+ filterExpr: ((day = 3) and (year = 2023) and (month = 10)) (type: boolean)
+ Statistics: Num rows: 3 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: i (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 3 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 3 Data size: #Masked# Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ Execution mode: vectorized
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: explain
+select count(*) from ice01
+ where year=2023 and month = 10 and day = 3
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice01
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: explain
+select count(*) from ice01
+ where year=2023 and month = 10 and day = 3
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice01
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: 1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(1) from ice01
+ where year=2023 and month = 10 and day = 3
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice01
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(1) from ice01
+ where year=2023 and month = 10 and day = 3
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice01
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+3
+PREHOOK: query: drop table ice01
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@ice01
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice01
+POSTHOOK: query: drop table ice01
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@ice01
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice01
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 eee345606329..7dd13fe4a088 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,22 +85,19 @@ STAGE PLANS:
TableScan
alias: test_ice_int
filterExpr: (a = 22) (type: boolean)
- Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (a = 22) (type: boolean)
- Statistics: Num rows: 2 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), 22 (type: int), b (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
- Statistics: Num rows: 2 Data size: 966 Basic stats: COMPLETE Column stats: COMPLETE
- File Output Operator
- compressed: false
- Statistics: Num rows: 2 Data size: 966 Basic stats: COMPLETE Column stats: COMPLETE
- table:
- input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
- output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
- serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
- name: default.test_ice_int
+ Statistics: Num rows: 1 Data size: 91 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), 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
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.test_ice_int
Execution mode: vectorized
Stage: Stage-3
@@ -135,7 +132,6 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@test_ice_int
POSTHOOK: Output: hdfs://### HDFS PATH ###
11 ddd
-11 ddd
33 rrfdfdf
33 rrfdfdf
PREHOOK: query: select `partition` from default.test_ice_int.partitions
@@ -181,7 +177,6 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@test_ice_int
POSTHOOK: Output: hdfs://### HDFS PATH ###
11 ddd
-11 ddd
PREHOOK: query: select `partition` from default.test_ice_int.partitions
PREHOOK: type: QUERY
PREHOOK: Input: default@test_ice_int
@@ -280,22 +275,19 @@ STAGE PLANS:
TableScan
alias: test_ice_bigint
filterExpr: (a = 226784902765739L) (type: boolean)
- Statistics: Num rows: 6 Data size: 594 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (a = 226784902765739L) (type: boolean)
- Statistics: Num rows: 2 Data size: 198 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), 226784902765739L (type: bigint), b (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
- Statistics: Num rows: 2 Data size: 974 Basic stats: COMPLETE Column stats: COMPLETE
- File Output Operator
- compressed: false
- Statistics: Num rows: 2 Data size: 974 Basic stats: COMPLETE Column stats: COMPLETE
- table:
- input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
- output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
- serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
- name: default.test_ice_bigint
+ Statistics: Num rows: 1 Data size: 91 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), 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
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 487 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.test_ice_bigint
Execution mode: vectorized
Stage: Stage-3
@@ -330,7 +322,6 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@test_ice_bigint
POSTHOOK: Output: hdfs://### HDFS PATH ###
115674892756 ddd
-115674892756 ddd
3367849937755673 rrfdfdf
3367849937755673 rrfdfdf
PREHOOK: query: select `partition` from default.test_ice_bigint.partitions
@@ -376,7 +367,6 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@test_ice_bigint
POSTHOOK: Output: hdfs://### HDFS PATH ###
115674892756 ddd
-115674892756 ddd
PREHOOK: query: select `partition` from default.test_ice_bigint.partitions
PREHOOK: type: QUERY
PREHOOK: Input: default@test_ice_bigint
@@ -501,22 +491,19 @@ STAGE PLANS:
TableScan
alias: test_ice_str
filterExpr: (b = 'ddd') (type: boolean)
- Statistics: Num rows: 14 Data size: 1386 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (b = 'ddd') (type: boolean)
- Statistics: Num rows: 3 Data size: 297 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), a (type: bigint), 'ddd' (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
- Statistics: Num rows: 3 Data size: 1449 Basic stats: COMPLETE Column stats: COMPLETE
- File Output Operator
- compressed: false
- Statistics: Num rows: 3 Data size: 1449 Basic stats: COMPLETE Column stats: COMPLETE
- table:
- input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
- output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
- serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
- name: default.test_ice_str
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), a (type: bigint), 'ddd' (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+ Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 483 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.test_ice_str
Execution mode: vectorized
Stage: Stage-3
@@ -551,15 +538,10 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@test_ice_str
POSTHOOK: Output: hdfs://### HDFS PATH ###
115674892756 a"ab
-115674892756 a"ab
-115674892756 a'ab
115674892756 a'ab
2267849027657399057 eefe
2267849027657399057 eefe
2267849027657399057 eefe
-2267849027657399057 eefe
-2267849027657399057 eefe
-2267849027657399057 eefe
33678499377556738 rrfdfdf
33678499377556738 rrfdfdf
PREHOOK: query: select `partition` from default.test_ice_str.partitions
@@ -607,12 +589,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@test_ice_str
POSTHOOK: Output: hdfs://### HDFS PATH ###
115674892756 a"ab
-115674892756 a"ab
115674892756 a'ab
-115674892756 a'ab
-2267849027657399057 eefe
-2267849027657399057 eefe
-2267849027657399057 eefe
2267849027657399057 eefe
2267849027657399057 eefe
2267849027657399057 eefe
@@ -652,9 +629,6 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
2267849027657399057 eefe
2267849027657399057 eefe
2267849027657399057 eefe
-2267849027657399057 eefe
-2267849027657399057 eefe
-2267849027657399057 eefe
PREHOOK: query: select `partition` from default.test_ice_str.partitions
PREHOOK: type: QUERY
PREHOOK: Input: default@test_ice_str
@@ -742,17 +716,17 @@ STAGE PLANS:
TableScan
alias: test_ice_date
filterExpr: (b = DATE'2022-02-07') (type: boolean)
- Statistics: Num rows: 3 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
predicate: (b = DATE'2022-02-07') (type: boolean)
- Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), a (type: bigint), 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: 444 Basic stats: COMPLETE Column stats: COMPLETE
File Output Operator
compressed: false
- Statistics: Num rows: 1 Data size: 452 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 444 Basic stats: COMPLETE Column stats: COMPLETE
table:
input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
@@ -930,22 +904,19 @@ STAGE PLANS:
TableScan
alias: test_ice_double
filterExpr: (a = 1.156748927566759E11D) (type: boolean)
- Statistics: Num rows: 6 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (a = 1.156748927566759E11D) (type: boolean)
- Statistics: Num rows: 2 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), 1.156748927566759E11D (type: double), b (type: date)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
- Statistics: Num rows: 2 Data size: 904 Basic stats: COMPLETE Column stats: COMPLETE
- File Output Operator
- compressed: false
- Statistics: Num rows: 2 Data size: 904 Basic stats: COMPLETE Column stats: COMPLETE
- table:
- input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
- output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
- serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
- name: default.test_ice_double
+ Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), 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
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 452 Basic stats: COMPLETE Column stats: COMPLETE
+ table:
+ input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+ serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+ name: default.test_ice_double
Execution mode: vectorized
Stage: Stage-3
@@ -980,7 +951,6 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@test_ice_double
POSTHOOK: Output: hdfs://### HDFS PATH ###
2.267849027657399E9 2022-08-07
-2.267849027657399E9 2022-08-07
3.367849937755674E7 2022-08-09
3.367849937755674E7 2022-08-09
PREHOOK: query: select `partition` from default.test_ice_double.partitions
@@ -1026,7 +996,6 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@test_ice_double
POSTHOOK: Output: hdfs://### HDFS PATH ###
2.267849027657399E9 2022-08-07
-2.267849027657399E9 2022-08-07
PREHOOK: query: select `partition` from default.test_ice_double.partitions
PREHOOK: type: QUERY
PREHOOK: Input: default@test_ice_double
@@ -1112,10 +1081,10 @@ STAGE PLANS:
TableScan
alias: test_ice_double_date
filterExpr: ((a = 1.156748927566759E11D) and (b = DATE'2022-02-07')) (type: boolean)
- Statistics: Num rows: 3 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
predicate: ((a = 1.156748927566759E11D) and (b = DATE'2022-02-07')) (type: boolean)
- Statistics: Num rows: 1 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), 1.156748927566759E11D (type: double), DATE'2022-02-07' (type: date)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
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 81d630178667..7a6af1c20fd0 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=168)
+ Filter Operator [FIL_14] (rows=3 width=260)
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=168)
+ TableScan [TS_3] (rows=7 width=5077)
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,10 +189,12 @@ Stage-0
PartitionCols:_col0, _col1
Group By Operator [GBY_6] (rows=1 width=168)
Output:["_col0","_col1"],keys:date_col, decimal_col
- 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=168)
- default@source_table,source_table,Tbl:COMPLETE,Col:COMPLETE,Output:["date_col","decimal_col"]
+ Select Operator [SEL_5] (rows=3 width=260)
+ Output:["date_col","decimal_col"]
+ Filter Operator [FIL_18] (rows=3 width=260)
+ 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)
+ 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"]
Filter Operator [FIL_17] (rows=10 width=260)
@@ -252,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=168)
+ Filter Operator [FIL_14] (rows=3 width=260)
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=168)
+ TableScan [TS_3] (rows=7 width=5077)
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"]
@@ -321,10 +323,12 @@ Stage-0
PartitionCols:_col0, _col1
Group By Operator [GBY_6] (rows=1 width=168)
Output:["_col0","_col1"],keys:date_col, decimal_col
- 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=168)
- default@source_table,source_table,Tbl:COMPLETE,Col:COMPLETE,Output:["date_col","decimal_col"]
+ Select Operator [SEL_5] (rows=3 width=260)
+ Output:["date_col","decimal_col"]
+ Filter Operator [FIL_18] (rows=3 width=260)
+ 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)
+ 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"]
Filter Operator [FIL_17] (rows=10 width=260)
@@ -384,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=168)
+ Filter Operator [FIL_47] (rows=3 width=260)
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=168)
+ TableScan [TS_3] (rows=7 width=5077)
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"]
@@ -438,28 +442,30 @@ Stage-0
limit:-1
Stage-1
Map 1 vectorized, llap
- File Output Operator [FS_60]
- Select Operator [SEL_59] (rows=10 width=520)
+ File Output Operator [FS_61]
+ Select Operator [SEL_60] (rows=10 width=520)
Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
- Map Join Operator [MAPJOIN_58] (rows=10 width=336)
- BucketMapJoin:true,Conds:SEL_57._col0, _col1=RS_55._col0, _col1(Inner),Output:["_col0","_col1","_col2","_col3"]
+ Map Join Operator [MAPJOIN_59] (rows=10 width=336)
+ BucketMapJoin:true,Conds:SEL_58._col0, _col1=RS_56._col0, _col1(Inner),Output:["_col0","_col1","_col2","_col3"]
<-Reducer 3 [CUSTOM_EDGE] vectorized, llap
- MULTICAST [RS_55]
+ MULTICAST [RS_56]
PartitionCols:_col1
- Group By Operator [GBY_54] (rows=1 width=168)
+ Group By Operator [GBY_55] (rows=1 width=168)
Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
<-Map 2 [SIMPLE_EDGE] vectorized, llap
- SHUFFLE [RS_53]
+ SHUFFLE [RS_54]
PartitionCols:_col0, _col1
- Group By Operator [GBY_52] (rows=1 width=168)
+ Group By Operator [GBY_53] (rows=1 width=168)
Output:["_col0","_col1"],keys:date_col, decimal_col
- 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=168)
- default@source_table,source_table,Tbl:COMPLETE,Col:COMPLETE,Output:["date_col","decimal_col"]
- <-Select Operator [SEL_57] (rows=10 width=168)
+ Select Operator [SEL_52] (rows=3 width=260)
+ Output:["date_col","decimal_col"]
+ Filter Operator [FIL_51] (rows=3 width=260)
+ 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)
+ 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"]
- Filter Operator [FIL_56] (rows=10 width=260)
+ Filter Operator [FIL_57] (rows=10 width=260)
predicate:((string_col = 'pipeline') and 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_0] (rows=20 width=260)
default@target_table,target_table,Tbl:COMPLETE,Col:COMPLETE,Grouping Num Buckets:7,Grouping Partition Columns:["decimal_col"],Output:["date_col","string_col","decimal_col"]
@@ -516,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=168)
+ Filter Operator [FIL_47] (rows=3 width=260)
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=168)
+ TableScan [TS_3] (rows=7 width=5077)
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"]
@@ -570,28 +576,30 @@ Stage-0
limit:-1
Stage-1
Map 1 vectorized, llap
- File Output Operator [FS_60]
- Select Operator [SEL_59] (rows=10 width=520)
+ File Output Operator [FS_61]
+ Select Operator [SEL_60] (rows=10 width=520)
Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
- Map Join Operator [MAPJOIN_58] (rows=10 width=336)
- Conds:SEL_57._col0, _col1=RS_55._col0, _col1(Inner),Output:["_col0","_col1","_col2","_col3"]
+ Map Join Operator [MAPJOIN_59] (rows=10 width=336)
+ Conds:SEL_58._col0, _col1=RS_56._col0, _col1(Inner),Output:["_col0","_col1","_col2","_col3"]
<-Reducer 3 [BROADCAST_EDGE] vectorized, llap
- BROADCAST [RS_55]
+ BROADCAST [RS_56]
PartitionCols:_col0, _col1
- Group By Operator [GBY_54] (rows=1 width=168)
+ Group By Operator [GBY_55] (rows=1 width=168)
Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
<-Map 2 [SIMPLE_EDGE] vectorized, llap
- SHUFFLE [RS_53]
+ SHUFFLE [RS_54]
PartitionCols:_col0, _col1
- Group By Operator [GBY_52] (rows=1 width=168)
+ Group By Operator [GBY_53] (rows=1 width=168)
Output:["_col0","_col1"],keys:date_col, decimal_col
- 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=168)
- default@source_table,source_table,Tbl:COMPLETE,Col:COMPLETE,Output:["date_col","decimal_col"]
- <-Select Operator [SEL_57] (rows=10 width=168)
+ Select Operator [SEL_52] (rows=3 width=260)
+ Output:["date_col","decimal_col"]
+ Filter Operator [FIL_51] (rows=3 width=260)
+ 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)
+ 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"]
- Filter Operator [FIL_56] (rows=10 width=260)
+ Filter Operator [FIL_57] (rows=10 width=260)
predicate:((string_col = 'pipeline') and 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_0] (rows=20 width=260)
default@target_table,target_table,Tbl:COMPLETE,Col:COMPLETE,Output:["date_col","string_col","decimal_col"]
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 bd82280c9c07..37927fecface 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=372 width=8)
+ Map Join Operator [MAPJOIN_36] (rows=261 width=4)
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=238 width=4)
+ Select Operator [SEL_32] (rows=1 width=0)
Output:["_col0"]
- Filter Operator [FIL_31] (rows=238 width=89)
+ Filter Operator [FIL_31] (rows=1 width=84)
predicate:((part = '1') and key is not null)
- TableScan [TS_3] (rows=238 width=89)
+ TableScan [TS_3] (rows=1 width=84)
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=372 width=8)
+ Map Join Operator [MAPJOIN_36] (rows=261 width=4)
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=238 width=4)
+ Select Operator [SEL_32] (rows=1 width=0)
Output:["_col0"]
- Filter Operator [FIL_31] (rows=238 width=89)
+ Filter Operator [FIL_31] (rows=1 width=84)
predicate:((part = '1') and key is not null)
- TableScan [TS_3] (rows=238 width=89)
+ TableScan [TS_3] (rows=1 width=84)
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 4ddd445ecbfb..6aadd2c86fbb 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=1797 width=8)
+ Map Join Operator [MAPJOIN_36] (rows=500 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=738 width=4)
+ Select Operator [SEL_32] (rows=238 width=4)
Output:["_col0"]
- Filter Operator [FIL_31] (rows=738 width=89)
+ Filter Operator [FIL_31] (rows=238 width=89)
predicate:(part is not null and key is not null)
- TableScan [TS_3] (rows=738 width=89)
+ TableScan [TS_3] (rows=238 width=89)
default@srcbucket_mapjoin_part_2_n6,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","part"]
- <-Select Operator [SEL_35] (rows=738 width=4)
+ <-Select Operator [SEL_35] (rows=500 width=4)
Output:["_col0"]
- Filter Operator [FIL_34] (rows=738 width=89)
+ Filter Operator [FIL_34] (rows=500 width=89)
predicate:(part is not null and key is not null)
- TableScan [TS_0] (rows=738 width=89)
+ TableScan [TS_0] (rows=500 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=1797 width=8)
+ Map Join Operator [MAPJOIN_45] (rows=392 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=738 width=89)
+ Select Operator [SEL_38] (rows=238 width=89)
Output:["_col0","_col1"]
- Filter Operator [FIL_37] (rows=738 width=89)
+ Filter Operator [FIL_37] (rows=238 width=89)
predicate:(part is not null and key is not null)
- TableScan [TS_3] (rows=738 width=89)
+ TableScan [TS_3] (rows=238 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=738 width=85)
+ Select Operator [SEL_40] (rows=238 width=85)
Output:["_col0"]
Please refer to the previous Select Operator [SEL_38]
- <-Select Operator [SEL_44] (rows=738 width=89)
+ <-Select Operator [SEL_44] (rows=500 width=89)
Output:["_col0","_col1"]
- Filter Operator [FIL_43] (rows=738 width=89)
+ Filter Operator [FIL_43] (rows=500 width=89)
predicate:(part is not null and key is not null)
- TableScan [TS_0] (rows=738 width=89)
+ TableScan [TS_0] (rows=500 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 4b92f571e020..4334e53c03c4 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=372 width=8)
+ Map Join Operator [MAPJOIN_36] (rows=261 width=4)
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=238 width=4)
+ Select Operator [SEL_32] (rows=1 width=0)
Output:["_col0"]
- Filter Operator [FIL_31] (rows=238 width=89)
+ Filter Operator [FIL_31] (rows=1 width=84)
predicate:((part = '1') and key is not null)
- TableScan [TS_3] (rows=238 width=89)
+ TableScan [TS_3] (rows=1 width=84)
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=372 width=8)
+ Map Join Operator [MAPJOIN_36] (rows=261 width=4)
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=238 width=4)
+ Select Operator [SEL_32] (rows=1 width=0)
Output:["_col0"]
- Filter Operator [FIL_31] (rows=238 width=89)
+ Filter Operator [FIL_31] (rows=1 width=84)
predicate:((part = '1') and key is not null)
- TableScan [TS_3] (rows=238 width=89)
+ TableScan [TS_3] (rows=1 width=84)
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 b3e5082c3020..f4bd02b739cb 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
@@ -93,36 +93,36 @@ POSTHOOK: Input: default@srcbucket_big
Plan optimized by CBO.
Vertex dependency in root stage
-Map 1 <- Map 3 (CUSTOM_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Map 2 <- Map 1 (BROADCAST_EDGE)
+Reducer 3 <- Map 2 (SIMPLE_EDGE)
Stage-0
Fetch Operator
limit:-1
Stage-1
- Reducer 2 vectorized, llap
+ Reducer 3 vectorized, llap
File Output Operator [FS_37]
- Select Operator [SEL_36] (rows=4 width=190)
+ Select Operator [SEL_36] (rows=2 width=190)
Output:["_col0","_col1","_col2","_col3","_col4"]
- <-Map 1 [SIMPLE_EDGE] vectorized, llap
+ <-Map 2 [SIMPLE_EDGE] vectorized, llap
SHUFFLE [RS_35]
- Map Join Operator [MAPJOIN_34] (rows=4 width=190)
- BucketMapJoin:true,Conds:SEL_33._col0=RS_31._col0(Inner),Output:["_col0","_col1","_col2","_col3","_col4"]
- <-Map 3 [CUSTOM_EDGE] vectorized, llap
- MULTICAST [RS_31]
+ Map Join Operator [MAPJOIN_34] (rows=2 width=190)
+ 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=4 width=93)
- Output:["_col0","_col1"]
- Filter Operator [FIL_29] (rows=4 width=93)
+ Select Operator [SEL_30] (rows=2 width=97)
+ Output:["_col0","_col1","_col2"]
+ Filter Operator [FIL_29] (rows=2 width=97)
predicate:key 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)
- Output:["_col0","_col1","_col2"]
- Filter Operator [FIL_32] (rows=4 width=97)
+ TableScan [TS_0] (rows=4 width=73)
+ default@srcbucket_big,a,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value","id"]
+ <-Select Operator [SEL_33] (rows=4 width=93)
+ Output:["_col0","_col1"]
+ Filter Operator [FIL_32] (rows=4 width=93)
predicate:key is not null
- TableScan [TS_0] (rows=6 width=81)
- default@srcbucket_big,a,Tbl:COMPLETE,Col:COMPLETE,Grouping Num Buckets:4,Grouping Partition Columns:["key"],Output:["key","value","id"]
+ TableScan [TS_3] (rows=6 width=77)
+ default@src_small,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
PREHOOK: query: SELECT *
FROM srcbucket_big a
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 99aa09be2ed0..1d43ba579090 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
@@ -149,61 +149,6 @@ POSTHOOK: query: delete from ice_orc where last_name in ('ln5', 'ln13')
POSTHOOK: type: QUERY
POSTHOOK: Input: default@ice_orc
POSTHOOK: Output: default@ice_orc
-PREHOOK: query: alter table ice_orc set partition spec(team_id)
-PREHOOK: type: ALTERTABLE_SETPARTSPEC
-PREHOOK: Input: default@ice_orc
-POSTHOOK: query: alter table ice_orc set partition spec(team_id)
-POSTHOOK: type: ALTERTABLE_SETPARTSPEC
-POSTHOOK: Input: default@ice_orc
-POSTHOOK: Output: default@ice_orc
-PREHOOK: query: insert into ice_orc VALUES
- ('fn17', 'ln17', 1, 10, 100),
- ('fn18','ln18', 1, 10, 100)
-PREHOOK: type: QUERY
-PREHOOK: Input: _dummy_database@_dummy_table
-PREHOOK: Output: default@ice_orc
-POSTHOOK: query: insert into ice_orc VALUES
- ('fn17', 'ln17', 1, 10, 100),
- ('fn18','ln18', 1, 10, 100)
-POSTHOOK: type: QUERY
-POSTHOOK: Input: _dummy_database@_dummy_table
-POSTHOOK: Output: default@ice_orc
-PREHOOK: query: insert into ice_orc VALUES
- ('fn19','ln19', 2, 11, 100),
- ('fn20','ln20', 2, 11, 100)
-PREHOOK: type: QUERY
-PREHOOK: Input: _dummy_database@_dummy_table
-PREHOOK: Output: default@ice_orc
-POSTHOOK: query: insert into ice_orc VALUES
- ('fn19','ln19', 2, 11, 100),
- ('fn20','ln20', 2, 11, 100)
-POSTHOOK: type: QUERY
-POSTHOOK: Input: _dummy_database@_dummy_table
-POSTHOOK: Output: default@ice_orc
-PREHOOK: query: insert into ice_orc VALUES
- ('fn21','ln21', 3, 12, 100),
- ('fn22','ln22', 3, 12, 100)
-PREHOOK: type: QUERY
-PREHOOK: Input: _dummy_database@_dummy_table
-PREHOOK: Output: default@ice_orc
-POSTHOOK: query: insert into ice_orc VALUES
- ('fn21','ln21', 3, 12, 100),
- ('fn22','ln22', 3, 12, 100)
-POSTHOOK: type: QUERY
-POSTHOOK: Input: _dummy_database@_dummy_table
-POSTHOOK: Output: default@ice_orc
-PREHOOK: query: insert into ice_orc VALUES
- ('fn23','ln23', 4, 13, 100),
- ('fn24','ln24', 4, 13, 100)
-PREHOOK: type: QUERY
-PREHOOK: Input: _dummy_database@_dummy_table
-PREHOOK: Output: default@ice_orc
-POSTHOOK: query: insert into ice_orc VALUES
- ('fn23','ln23', 4, 13, 100),
- ('fn24','ln24', 4, 13, 100)
-POSTHOOK: type: QUERY
-POSTHOOK: Input: _dummy_database@_dummy_table
-POSTHOOK: Output: default@ice_orc
PREHOOK: query: select * from ice_orc
PREHOOK: type: QUERY
PREHOOK: Input: default@ice_orc
@@ -217,15 +162,7 @@ fn12 ln12 2 11 100
fn14 ln14 3 12 100
fn15 ln15 4 13 100
fn16 ln16 4 13 100
-fn17 ln17 1 10 100
-fn18 ln18 1 10 100
-fn19 ln19 2 11 100
fn2 ln2 1 10 100
-fn20 ln20 2 11 100
-fn21 ln21 3 12 100
-fn22 ln22 3 12 100
-fn23 ln23 4 13 100
-fn24 ln24 4 13 100
fn4 ln4 2 11 100
fn6 ln6 3 12 100
fn7 ln7 4 13 100
@@ -245,7 +182,8 @@ company_id bigint
# Partition Transform Information
# col_name transform_type
-team_id IDENTITY
+company_id IDENTITY
+dept_id IDENTITY
# Detailed Table Information
Database: default
@@ -254,24 +192,24 @@ 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
current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"first_name\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"dept_id\",\"required\":false,\"type\":\"long\"},{\"id\":4,\"name\":\"team_id\",\"required\":false,\"type\":\"long\"},{\"id\":5,\"name\":\"company_id\",\"required\":false,\"type\":\"long\"}]}
current-snapshot-id #Masked#
- current-snapshot-summary {\"added-data-files\":\"1\",\"added-records\":\"2\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"24\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"12\",\"total-delete-files\":\"6\",\"total-position-deletes\":\"6\",\"total-equality-deletes\":\"0\",\"iceberg-version\":\"#Masked#\"}
+ current-snapshot-summary {\"added-position-delete-files\":\"2\",\"added-delete-files\":\"2\",\"added-files-size\":\"#Masked#\",\"added-position-deletes\":\"2\",\"changed-partition-count\":\"2\",\"total-records\":\"16\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"8\",\"total-delete-files\":\"6\",\"total-position-deletes\":\"6\",\"total-equality-deletes\":\"0\",\"iceberg-version\":\"#Masked#\"}
current-snapshot-timestamp-ms #Masked#
- default-partition-spec {\"spec-id\":2,\"fields\":[{\"name\":\"team_id\",\"transform\":\"identity\",\"source-id\":4,\"field-id\":1002}]}
+ default-partition-spec {\"spec-id\":1,\"fields\":[{\"name\":\"company_id\",\"transform\":\"identity\",\"source-id\":5,\"field-id\":1000},{\"name\":\"dept_id\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1001}]}
format-version 2
iceberg.orc.files.only true
#### A masked pattern was here ####
- numFiles 12
- numRows 18
+ numFiles 8
+ numRows 10
parquet.compression zstd
#### A masked pattern was here ####
rawDataSize 0
serialization.format 1
- snapshot-count 15
+ snapshot-count 11
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -329,15 +267,7 @@ fn12 ln12 2 11 100
fn14 ln14 3 12 100
fn15 ln15 4 13 100
fn16 ln16 4 13 100
-fn17 ln17 1 10 100
-fn18 ln18 1 10 100
-fn19 ln19 2 11 100
fn2 ln2 1 10 100
-fn20 ln20 2 11 100
-fn21 ln21 3 12 100
-fn22 ln22 3 12 100
-fn23 ln23 4 13 100
-fn24 ln24 4 13 100
fn4 ln4 2 11 100
fn6 ln6 3 12 100
fn7 ln7 4 13 100
@@ -357,7 +287,8 @@ company_id bigint
# Partition Transform Information
# col_name transform_type
-team_id IDENTITY
+company_id IDENTITY
+dept_id IDENTITY
# Detailed Table Information
Database: default
@@ -371,19 +302,19 @@ Table Parameters:
bucketing_version 2
current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"first_name\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"},{\"id\":3,\"name\":\"dept_id\",\"required\":false,\"type\":\"long\"},{\"id\":4,\"name\":\"team_id\",\"required\":false,\"type\":\"long\"},{\"id\":5,\"name\":\"company_id\",\"required\":false,\"type\":\"long\"}]}
current-snapshot-id #Masked#
- current-snapshot-summary {\"added-data-files\":\"4\",\"deleted-data-files\":\"8\",\"removed-position-delete-files\":\"6\",\"removed-delete-files\":\"6\",\"added-records\":\"10\",\"deleted-records\":\"16\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"removed-position-deletes\":\"6\",\"changed-partition-count\":\"9\",\"total-records\":\"18\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"8\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\",\"iceberg-version\":\"#Masked#\"}
+ current-snapshot-summary {\"added-data-files\":\"4\",\"deleted-data-files\":\"4\",\"removed-position-delete-files\":\"3\",\"removed-delete-files\":\"3\",\"added-records\":\"5\",\"deleted-records\":\"8\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"removed-position-deletes\":\"3\",\"changed-partition-count\":\"5\",\"total-records\":\"10\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"8\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\",\"iceberg-version\":\"#Masked#\"}
current-snapshot-timestamp-ms #Masked#
- default-partition-spec {\"spec-id\":2,\"fields\":[{\"name\":\"team_id\",\"transform\":\"identity\",\"source-id\":4,\"field-id\":1002}]}
+ default-partition-spec {\"spec-id\":1,\"fields\":[{\"name\":\"company_id\",\"transform\":\"identity\",\"source-id\":5,\"field-id\":1000},{\"name\":\"dept_id\",\"transform\":\"identity\",\"source-id\":3,\"field-id\":1001}]}
format-version 2
iceberg.orc.files.only true
#### A masked pattern was here ####
numFiles 8
- numRows 18
+ numRows 10
parquet.compression zstd
#### A masked pattern was here ####
rawDataSize 0
serialization.format 1
- snapshot-count 16
+ snapshot-count 15
storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
table_type ICEBERG
totalSize #Masked#
@@ -405,8 +336,8 @@ PREHOOK: type: SHOW COMPACTIONS
POSTHOOK: query: show compactions order by 'partition'
POSTHOOK: type: SHOW COMPACTIONS
CompactionId Database Table Partition Type State Worker host Worker Enqueue Time Start Time Duration(ms) HadoopJobId Error message Initiator host Initiator Pool name TxnId Next TxnId Commit Time Highest WriteId
-#Masked# default ice_orc team_id=10 MAJOR refused #Masked# manual default 0 0 0 ---
-#Masked# default ice_orc team_id=11 MAJOR refused #Masked# manual default 0 0 0 ---
-#Masked# default ice_orc team_id=12 MAJOR refused #Masked# manual default 0 0 0 ---
-#Masked# default ice_orc team_id=13 MAJOR refused #Masked# manual default 0 0 0 ---
+#Masked# default ice_orc company_id=100/dept_id=1 MAJOR succeeded #Masked# manual default 0 0 0 ---
+#Masked# default ice_orc company_id=100/dept_id=2 MAJOR succeeded #Masked# manual default 0 0 0 ---
+#Masked# default ice_orc company_id=100/dept_id=3 MAJOR succeeded #Masked# manual default 0 0 0 ---
+#Masked# default ice_orc company_id=100/dept_id=4 MAJOR refused #Masked# manual default 0 0 0 ---
#Masked# default ice_orc --- MAJOR succeeded #Masked# manual default 0 0 0 ---
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 3dea702b7fcb..b56658019c7e 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
@@ -72,14 +72,14 @@ STAGE PLANS:
Tez
#### A masked pattern was here ####
Edges:
- Reducer 10 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE)
+ Reducer 10 <- Reducer 9 (SIMPLE_EDGE), Union 3 (CONTAINS)
Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE), Union 3 (CONTAINS)
Reducer 4 <- Union 3 (SIMPLE_EDGE)
- Reducer 5 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE)
- Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
- Reducer 7 <- Reducer 10 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE), Union 3 (CONTAINS)
- Reducer 8 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE)
- Reducer 9 <- Reducer 8 (SIMPLE_EDGE), Union 3 (CONTAINS)
+ Reducer 5 <- Map 1 (SIMPLE_EDGE), Map 12 (SIMPLE_EDGE)
+ Reducer 6 <- Reducer 5 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE), Union 3 (CONTAINS)
+ Reducer 7 <- Map 1 (SIMPLE_EDGE), Map 12 (SIMPLE_EDGE)
+ Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
+ Reducer 9 <- Map 1 (SIMPLE_EDGE), Map 12 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -128,6 +128,27 @@ STAGE PLANS:
Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized
Map 11
+ Map Operator Tree:
+ TableScan
+ alias: target_ice
+ filterExpr: (a <= 100) (type: boolean)
+ Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (a <= 100) (type: boolean)
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), a (type: 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
+ 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
+ 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
@@ -160,20 +181,6 @@ STAGE PLANS:
Map-reduce partition columns: _col4 (type: int)
Statistics: Num rows: 4 Data size: 1900 Basic stats: COMPLETE Column stats: COMPLETE
value expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: string), _col3 (type: string), _col5 (type: string), _col6 (type: int)
- Filter Operator
- predicate: (a <= 100) (type: boolean)
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), a (type: 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
- 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
- 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: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
@@ -190,29 +197,45 @@ STAGE PLANS:
value expressions: _col0 (type: string)
Execution mode: vectorized
Reducer 10
+ Execution mode: vectorized
Reduce Operator Tree:
- Merge Join Operator
- condition map:
- Left Outer Join 0 to 1
- keys:
- 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
- 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
- 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
- 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
- value expressions: _col0 (type: int), _col1 (type: bigint), _col3 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int)
+ 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
+ PTF Operator
+ Function definitions:
+ Input definition
+ input alias: ptf_0
+ type: WINDOWING
+ Windowing table definition
+ input alias: ptf_1
+ name: windowingtablefunction
+ order by: _col3 ASC NULLS FIRST
+ partition by: _col3
+ raw input shape:
+ window functions:
+ window function definition
+ alias: row_number_window_0
+ name: row_number
+ 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
+ Filter Operator
+ predicate: (row_number_window_0 = 1) (type: boolean)
+ Statistics: Num rows: 2 Data size: 950 Basic stats: COMPLETE Column stats: COMPLETE
+ 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
+ Reduce Output Operator
+ key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string)
+ null sort order: aa
+ 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
+ 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:
Merge Join Operator
@@ -250,6 +273,47 @@ STAGE PLANS:
serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
name: default.target_ice
Reducer 5
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 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
+ 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
+ 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
+ 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
+ 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:
+ Merge Join Operator
+ condition map:
+ Left Semi Join 0 to 1
+ keys:
+ 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
+ Reduce Output Operator
+ key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string)
+ null sort order: aa
+ 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
+ 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:
Merge Join Operator
condition map:
@@ -265,7 +329,7 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: _col1 (type: string)
Statistics: Num rows: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE
- Reducer 6
+ Reducer 8
Execution mode: vectorized
Reduce Operator Tree:
Select Operator
@@ -311,24 +375,7 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
- Reducer 7
- Reduce Operator Tree:
- Merge Join Operator
- condition map:
- Left Semi Join 0 to 1
- keys:
- 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
- Reduce Output Operator
- key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string)
- null sort order: aa
- 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
- 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
+ Reducer 9
Reduce Operator Tree:
Merge Join Operator
condition map:
@@ -345,46 +392,6 @@ STAGE PLANS:
Map-reduce partition columns: _col3 (type: string)
Statistics: Num rows: 4 Data size: 1900 Basic stats: COMPLETE Column stats: COMPLETE
value expressions: _col1 (type: int), _col2 (type: bigint), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int)
- Reducer 9
- Execution mode: vectorized
- Reduce Operator Tree:
- 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
- PTF Operator
- Function definitions:
- Input definition
- input alias: ptf_0
- type: WINDOWING
- Windowing table definition
- input alias: ptf_1
- name: windowingtablefunction
- order by: _col3 ASC NULLS FIRST
- partition by: _col3
- raw input shape:
- window functions:
- window function definition
- alias: row_number_window_0
- name: row_number
- 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
- Filter Operator
- predicate: (row_number_window_0 = 1) (type: boolean)
- Statistics: Num rows: 2 Data size: 950 Basic stats: COMPLETE Column stats: COMPLETE
- 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
- Reduce Output Operator
- key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string)
- null sort order: aa
- 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
- 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 3
Vertex: Union 3
diff --git a/iceberg/iceberg-handler/src/test/results/positive/query_iceberg_metadata_of_partitioned_table.q.out b/iceberg/iceberg-handler/src/test/results/positive/query_iceberg_metadata_of_partitioned_table.q.out
index 4b260fb55f82..74bcad25b34d 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/query_iceberg_metadata_of_partitioned_table.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/query_iceberg_metadata_of_partitioned_table.q.out
@@ -404,6 +404,10 @@ hdfs://### HDFS PATH ###
hdfs://### HDFS PATH ###
hdfs://### HDFS PATH ###
hdfs://### HDFS PATH ###
+hdfs://### HDFS PATH ###
+hdfs://### HDFS PATH ###
+hdfs://### HDFS PATH ###
+hdfs://### HDFS PATH ###
PREHOOK: query: select file from default.ice_meta_3.metadata_log_entries
PREHOOK: type: QUERY
PREHOOK: Input: default@ice_meta_3
@@ -704,6 +708,10 @@ hdfs://### HDFS PATH ###
hdfs://### HDFS PATH ###
hdfs://### HDFS PATH ###
hdfs://### HDFS PATH ###
+hdfs://### HDFS PATH ###
+hdfs://### HDFS PATH ###
+hdfs://### HDFS PATH ###
+hdfs://### HDFS PATH ###
PREHOOK: query: select file from default.ice_meta_3.metadata_log_entries
PREHOOK: type: QUERY
PREHOOK: Input: default@ice_meta_3
diff --git a/iceberg/iceberg-handler/src/test/results/positive/row_count.q.out b/iceberg/iceberg-handler/src/test/results/positive/row_count.q.out
index 5883a8e2f812..ffa5c73c3ea1 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/row_count.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/row_count.q.out
@@ -241,8 +241,8 @@ Stage-0
PARTITION_ONLY_SHUFFLE [RS_9]
Group By Operator [GBY_8] (rows=1 width=8)
Output:["_col0"],aggregations:["count()"]
- Select Operator [SEL_7] (rows=11 width=1####)
- TableScan [TS_0] (rows=11 width=1####)
+ Select Operator [SEL_7] (rows=14 width=5###)
+ TableScan [TS_0] (rows=14 width=5###)
default@llap_orders,llap_orders,Tbl:COMPLETE,Col:COMPLETE
PREHOOK: query: explain insert into llap_orders select * from llap_orders limit 100000
@@ -276,29 +276,29 @@ Stage-3
<-Reducer 2 [SIMPLE_EDGE] vectorized
PARTITION_ONLY_SHUFFLE [RS_21]
PartitionCols:_col4, _col5
- Select Operator [SEL_20] (rows=11 width=224)
+ Select Operator [SEL_20] (rows=14 width=224)
Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
- Limit [LIM_19] (rows=11 width=224)
+ Limit [LIM_19] (rows=14 width=224)
Number of rows:100000
<-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized
PARTITION_ONLY_SHUFFLE [RS_18]
- Select Operator [SEL_17] (rows=11 width=224)
+ Select Operator [SEL_17] (rows=14 width=224)
Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
- Limit [LIM_16] (rows=11 width=224)
+ Limit [LIM_16] (rows=14 width=224)
Number of rows:100000
- TableScan [TS_0] (rows=11 width=224)
+ TableScan [TS_0] (rows=14 width=224)
default@llap_orders,llap_orders,Tbl:COMPLETE,Col:COMPLETE,Output:["orderid","quantity","itemid","tradets","p1","p2"]
Reducer 4 vectorized
File Output Operator [FS_29]
- Select Operator [SEL_28] (rows=1 width=1####)
+ Select Operator [SEL_28] (rows=1 width=1###)
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"]
- Group By Operator [GBY_27] (rows=1 width=1####)
+ Group By Operator [GBY_27] (rows=1 width=1###)
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"],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)","min(VALUE._col9)","max(VALUE._col10)","count(VALUE._col11)","compute_bit_vector_hll(VALUE._col12)","min(VALUE._col13)","max(VALUE._col14)","count(VALUE._col15)","compute_bit_vector_hll(VALUE._col16)","max(VALUE._col17)","avg(VALUE._col18)","count(VALUE._col19)","compute_bit_vector_hll(VALUE._col20)","max(VALUE._col21)","avg(VALUE._col22)","count(VALUE._col23)","compute_bit_vector_hll(VALUE._col24)"]
<-Reducer 2 [CUSTOM_SIMPLE_EDGE] vectorized
PARTITION_ONLY_SHUFFLE [RS_24]
- Group By Operator [GBY_23] (rows=1 width=1####)
+ Group By Operator [GBY_23] (rows=1 width=1###)
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"],aggregations:["min(orderid)","max(orderid)","count(1)","count(orderid)","compute_bit_vector_hll(orderid)","min(quantity)","max(quantity)","count(quantity)","compute_bit_vector_hll(quantity)","min(itemid)","max(itemid)","count(itemid)","compute_bit_vector_hll(itemid)","min(tradets)","max(tradets)","count(tradets)","compute_bit_vector_hll(tradets)","max(length(p1))","avg(COALESCE(length(p1),0))","count(p1)","compute_bit_vector_hll(p1)","max(length(p2))","avg(COALESCE(length(p2),0))","count(p2)","compute_bit_vector_hll(p2)"]
- Select Operator [SEL_22] (rows=11 width=224)
+ Select Operator [SEL_22] (rows=14 width=224)
Output:["orderid","quantity","itemid","tradets","p1","p2"]
Please refer to the previous Select Operator [SEL_20]
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 e89292654475..9d48964b7b2b 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: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 3 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
predicate: ((a = 22) or (b) IN ('one', 'four')) (type: boolean)
- Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: PARTITION__SPEC__ID (type: int), PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: bigint), PARTITION__PROJECTION (type: string), a (type: int), 'Changed' (type: string), c (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
- Statistics: Num rows: 4 Data size: 1948 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 487 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string)
null sort order: aa
sort order: ++
Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string)
- Statistics: Num rows: 8 Data size: 3884 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 3 Data size: 1443 Basic stats: COMPLETE Column stats: COMPLETE
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: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
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: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
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: 7 Data size: 3388 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
key expressions: _col5 (type: string)
null sort order: z
sort order: +
Map-reduce partition columns: _col5 (type: string)
- Statistics: Num rows: 7 Data size: 3388 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE
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: 7 Data size: 672 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 3 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
predicate: ((a = 22) or (b) IN ('one', 'four')) (type: boolean)
- Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
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: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
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: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
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: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
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: 8 Data size: 3884 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 3 Data size: 1443 Basic stats: COMPLETE Column stats: COMPLETE
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: 2 Data size: 968 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE
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: 2 Data size: 968 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string)
null sort order: aa
sort order: ++
Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string)
- Statistics: Num rows: 8 Data size: 3884 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 3 Data size: 1443 Basic stats: COMPLETE Column stats: COMPLETE
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: 4 Data size: 1904 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 476 Basic stats: COMPLETE Column stats: COMPLETE
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: 4 Data size: 1904 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 476 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
predicate: (row_number_window_0 = 1) (type: boolean)
- Statistics: Num rows: 2 Data size: 952 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 476 Basic stats: COMPLETE Column stats: COMPLETE
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: 2 Data size: 968 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 484 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string)
null sort order: aa
sort order: ++
Map-reduce partition columns: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string)
- Statistics: Num rows: 8 Data size: 3884 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 3 Data size: 1443 Basic stats: COMPLETE Column stats: COMPLETE
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: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
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: 4 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
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: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: _col5 (type: string)
outputColumnNames: _col0
- Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
Group By Operator
keys: _col0 (type: string)
minReductionHashAggr: 0.4
mode: hash
outputColumnNames: _col0
- Statistics: Num rows: 2 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
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: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
Union 2
Vertex: Union 2
@@ -287,14 +287,14 @@ POSTHOOK: query: insert into tbl_ice values (444, 'hola', 800), (555, 'schola',
POSTHOOK: type: QUERY
POSTHOOK: Input: _dummy_database@_dummy_table
POSTHOOK: Output: default@tbl_ice
-Warning: Shuffle Join MERGEJOIN[282][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product
-Warning: Shuffle Join MERGEJOIN[295][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 4' is a cross product
-Warning: Shuffle Join MERGEJOIN[287][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 10' is a cross product
-Warning: Shuffle Join MERGEJOIN[289][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 12' is a cross product
-Warning: Shuffle Join MERGEJOIN[284][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 14' is a cross product
-Warning: Shuffle Join MERGEJOIN[293][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 16' is a cross product
-Warning: Shuffle Join MERGEJOIN[281][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 19' is a cross product
-Warning: Shuffle Join MERGEJOIN[291][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 21' is a cross product
+Warning: Shuffle Join MERGEJOIN[284][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 23' is a cross product
+Warning: Shuffle Join MERGEJOIN[293][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[287][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 14' is a cross product
+Warning: Shuffle Join MERGEJOIN[289][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 16' is a cross product
+Warning: Shuffle Join MERGEJOIN[281][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 25' is a cross product
+Warning: Shuffle Join MERGEJOIN[291][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 11' is a cross product
+Warning: Shuffle Join MERGEJOIN[282][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 24' is a cross product
+Warning: Shuffle Join MERGEJOIN[295][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 18' is a cross product
PREHOOK: query: explain update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800)
PREHOOK: type: QUERY
PREHOOK: Input: default@tbl_ice
@@ -314,37 +314,85 @@ STAGE PLANS:
Tez
#### A masked pattern was here ####
Edges:
- Reducer 10 <- Reducer 29 (XPROD_EDGE), Reducer 9 (XPROD_EDGE)
- Reducer 11 <- Reducer 10 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE)
- Reducer 12 <- Reducer 11 (XPROD_EDGE), Reducer 26 (XPROD_EDGE)
- Reducer 13 <- Reducer 12 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE), Union 7 (CONTAINS)
- Reducer 14 <- Map 1 (XPROD_EDGE), Reducer 28 (XPROD_EDGE)
- Reducer 15 <- Reducer 14 (SIMPLE_EDGE), Reducer 27 (SIMPLE_EDGE)
- Reducer 16 <- Reducer 15 (XPROD_EDGE), Reducer 23 (XPROD_EDGE)
- Reducer 17 <- Reducer 16 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE)
- Reducer 18 <- Reducer 17 (SIMPLE_EDGE)
- Reducer 19 <- Map 1 (XPROD_EDGE), Reducer 30 (XPROD_EDGE)
- Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 28 (XPROD_EDGE)
- Reducer 20 <- Reducer 19 (SIMPLE_EDGE), Reducer 27 (SIMPLE_EDGE)
- Reducer 21 <- Reducer 20 (XPROD_EDGE), Reducer 23 (XPROD_EDGE)
- Reducer 22 <- Reducer 21 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE), Union 7 (CONTAINS)
- Reducer 23 <- Map 1 (CUSTOM_SIMPLE_EDGE)
- Reducer 24 <- Map 1 (SIMPLE_EDGE)
- Reducer 25 <- Map 1 (CUSTOM_SIMPLE_EDGE)
- Reducer 26 <- Map 1 (CUSTOM_SIMPLE_EDGE)
- Reducer 27 <- Map 1 (SIMPLE_EDGE)
- Reducer 28 <- Map 1 (CUSTOM_SIMPLE_EDGE)
- Reducer 29 <- Map 1 (CUSTOM_SIMPLE_EDGE)
- Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 27 (SIMPLE_EDGE)
- Reducer 30 <- Map 1 (CUSTOM_SIMPLE_EDGE)
- Reducer 4 <- Reducer 25 (XPROD_EDGE), Reducer 3 (XPROD_EDGE)
- Reducer 5 <- Reducer 24 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
- Reducer 6 <- Reducer 5 (SIMPLE_EDGE), Union 7 (CONTAINS)
- Reducer 8 <- Union 7 (SIMPLE_EDGE)
- Reducer 9 <- Map 1 (SIMPLE_EDGE), Reducer 27 (SIMPLE_EDGE)
+ Reducer 10 <- Reducer 2 (SIMPLE_EDGE), Reducer 25 (SIMPLE_EDGE)
+ Reducer 11 <- Reducer 10 (XPROD_EDGE), Reducer 27 (XPROD_EDGE)
+ Reducer 12 <- Reducer 11 (SIMPLE_EDGE), Reducer 28 (SIMPLE_EDGE), Union 8 (CONTAINS)
+ Reducer 13 <- Map 26 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+ Reducer 14 <- Reducer 13 (XPROD_EDGE), Reducer 21 (XPROD_EDGE)
+ Reducer 15 <- Reducer 14 (SIMPLE_EDGE), Reducer 28 (SIMPLE_EDGE)
+ Reducer 16 <- Reducer 15 (XPROD_EDGE), Reducer 29 (XPROD_EDGE)
+ Reducer 17 <- Reducer 2 (SIMPLE_EDGE), Reducer 24 (SIMPLE_EDGE)
+ Reducer 18 <- Reducer 17 (XPROD_EDGE), Reducer 27 (XPROD_EDGE)
+ Reducer 19 <- Reducer 18 (SIMPLE_EDGE), Reducer 28 (SIMPLE_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 20 <- Reducer 19 (SIMPLE_EDGE), Union 8 (CONTAINS)
+ Reducer 21 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+ Reducer 22 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+ Reducer 23 <- Map 26 (XPROD_EDGE), Reducer 22 (XPROD_EDGE)
+ Reducer 24 <- Map 26 (XPROD_EDGE), Reducer 22 (XPROD_EDGE)
+ Reducer 25 <- Map 26 (XPROD_EDGE), Reducer 22 (XPROD_EDGE)
+ Reducer 27 <- Map 26 (CUSTOM_SIMPLE_EDGE)
+ Reducer 28 <- Map 26 (SIMPLE_EDGE)
+ Reducer 29 <- Map 26 (CUSTOM_SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE)
+ Reducer 4 <- Reducer 27 (XPROD_EDGE), Reducer 3 (XPROD_EDGE)
+ Reducer 5 <- Reducer 28 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+ Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+ Reducer 7 <- Reducer 16 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE), Union 8 (CONTAINS)
+ Reducer 9 <- Union 8 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
+ Map Operator Tree:
+ TableScan
+ alias: tbl_ice
+ filterExpr: (a <= 5) (type: boolean)
+ Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (a <= 5) (type: boolean)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: a (type: int)
+ outputColumnNames: a
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: a (type: int)
+ minReductionHashAggr: 0.4
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ 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
+ Group By Operator
+ aggregations: count(), count(a)
+ minReductionHashAggr: 0.4
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: bigint), _col1 (type: bigint)
+ Select Operator
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ aggregations: count()
+ minReductionHashAggr: 0.4
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: bigint)
+ Execution mode: vectorized
+ Map 26
Map Operator Tree:
TableScan
alias: tbl_ice
@@ -418,19 +466,6 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: _col0 (type: int)
Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- aggregations: count()
- minReductionHashAggr: 0.8888889
- mode: hash
- outputColumnNames: _col0
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint)
Group By Operator
aggregations: count(), count(c)
minReductionHashAggr: 0.8888889
@@ -442,60 +477,79 @@ STAGE PLANS:
sort order:
Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
value expressions: _col0 (type: bigint), _col1 (type: bigint)
- Filter Operator
- predicate: (a <= 5) (type: boolean)
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- keys: a (type: int)
- minReductionHashAggr: 0.4
- mode: hash
- outputColumnNames: _col0
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
- 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
- Select Operator
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- aggregations: count()
- minReductionHashAggr: 0.4
- mode: hash
- outputColumnNames: _col0
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint)
- Group By Operator
- aggregations: count(), count(a)
- minReductionHashAggr: 0.4
- mode: hash
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint), _col1 (type: bigint)
- Select Operator
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- aggregations: count()
- minReductionHashAggr: 0.4
- mode: hash
- outputColumnNames: _col0
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint)
Execution mode: vectorized
Reducer 10
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 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
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 9 Data size: 3660 Basic stats: COMPLETE Column stats: COMPLETE
+ 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:
+ Merge Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 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
+ 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
+ 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:
+ Merge Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 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
+ 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
+ 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
+ Reduce Output Operator
+ key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string)
+ null sort order: aa
+ 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
+ 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:
+ Merge Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 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
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 9 Data size: 4389 Basic stats: COMPLETE Column stats: COMPLETE
+ 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:
Merge Join Operator
condition map:
@@ -516,7 +570,7 @@ STAGE PLANS:
Map-reduce partition columns: _col2 (type: int)
Statistics: Num rows: 9 Data size: 4533 Basic stats: COMPLETE Column stats: COMPLETE
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 11
+ Reducer 15
Reduce Operator Tree:
Merge Join Operator
condition map:
@@ -531,7 +585,7 @@ STAGE PLANS:
sort order:
Statistics: Num rows: 9 Data size: 4569 Basic stats: COMPLETE Column stats: COMPLETE
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 12
+ Reducer 16
Reduce Operator Tree:
Merge Join Operator
condition map:
@@ -559,45 +613,7 @@ STAGE PLANS:
Map-reduce partition columns: _col5 (type: string)
Statistics: Num rows: 9 Data size: 4365 Basic stats: COMPLETE Column stats: COMPLETE
value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: bigint), _col7 (type: string)
- Reducer 13
- Reduce Operator Tree:
- Merge Join Operator
- condition map:
- Left Semi Join 0 to 1
- keys:
- 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
- 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
- Reduce Output Operator
- key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string)
- null sort order: aa
- 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
- 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 14
- Reduce Operator Tree:
- Merge Join Operator
- condition map:
- Inner Join 0 to 1
- keys:
- 0
- 1
- outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 9 Data size: 1800 Basic stats: COMPLETE Column stats: COMPLETE
- 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
- value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint)
- Reducer 15
+ Reducer 17
Reduce Operator Tree:
Merge Join Operator
condition map:
@@ -605,14 +621,14 @@ STAGE PLANS:
keys:
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
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9
+ Statistics: Num rows: 9 Data size: 4389 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
null sort order:
sort order:
- Statistics: Num rows: 13 Data size: 2568 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean)
- Reducer 16
+ Statistics: Num rows: 9 Data size: 4389 Basic stats: COMPLETE Column stats: COMPLETE
+ 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:
Merge Join Operator
condition map:
@@ -620,52 +636,92 @@ STAGE PLANS:
keys:
0
1
- outputColumnNames: _col1, _col2, _col3, _col5, _col6
- Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col9, _col10
+ Statistics: Num rows: 9 Data size: 4461 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
- key expressions: _col1 (type: int)
+ key expressions: _col2 (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
- value expressions: _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean), _col6 (type: bigint)
- Reducer 17
+ Map-reduce partition columns: _col2 (type: int)
+ Statistics: Num rows: 9 Data size: 4461 Basic stats: COMPLETE Column stats: COMPLETE
+ 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:
Merge Join Operator
condition map:
Left Outer Join 0 to 1
keys:
- 0 _col1 (type: int)
+ 0 _col2 (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
+ 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
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
+ 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
Reduce Output Operator
- key expressions: _col2 (type: string)
+ key expressions: _col5 (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
- Reducer 18
+ Map-reduce partition columns: _col5 (type: string)
+ Statistics: Num rows: 9 Data size: 4497 Basic stats: COMPLETE Column stats: COMPLETE
+ 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
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: int)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ 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
+ 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
+ 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
+ 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
+ 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
+ value expressions: _col1 (type: boolean)
+ Reducer 20
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
+ 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
PTF Operator
Function definitions:
Input definition
input alias: ptf_0
- output shape: _col2: string
type: WINDOWING
Windowing table definition
input alias: ptf_1
name: windowingtablefunction
- order by: _col2 ASC NULLS FIRST
- partition by: _col2
+ order by: _col5 ASC NULLS FIRST
+ partition by: _col5
raw input shape:
window functions:
window function definition
@@ -674,27 +730,58 @@ 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: 4293 Basic stats: COMPLETE Column stats: COMPLETE
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: 1908 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: _col2 (type: string)
- outputColumnNames: _col0
- Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- keys: _col0 (type: string)
- minReductionHashAggr: 0.4
- mode: hash
- outputColumnNames: _col0
- Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE
- 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
- Reducer 19
+ 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
+ Reduce Output Operator
+ key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string)
+ null sort order: aa
+ 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
+ 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
+ Reduce Operator Tree:
+ Group By Operator
+ 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
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: bigint), _col1 (type: bigint)
+ Reducer 22
+ Execution mode: vectorized
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ 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
+ 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
+ value expressions: _col0 (type: bigint)
+ Reducer 23
Reduce Operator Tree:
Merge Join Operator
condition map:
@@ -702,16 +789,16 @@ STAGE PLANS:
keys:
0
1
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
- Statistics: Num rows: 9 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 9 Data size: 1800 Basic stats: COMPLETE Column stats: COMPLETE
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
- value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: string), _col5 (type: bigint), _col6 (type: string), _col7 (type: bigint)
- Reducer 2
+ Statistics: Num rows: 9 Data size: 1800 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint)
+ Reducer 24
Reduce Operator Tree:
Merge Join Operator
condition map:
@@ -728,22 +815,7 @@ STAGE PLANS:
Map-reduce partition columns: _col0 (type: int)
Statistics: Num rows: 9 Data size: 4365 Basic stats: COMPLETE Column stats: COMPLETE
value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col5 (type: string), _col6 (type: string), _col7 (type: bigint)
- Reducer 20
- Reduce Operator Tree:
- Merge Join Operator
- condition map:
- Left Outer Join 0 to 1
- keys:
- 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
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 9 Data size: 3660 Basic stats: COMPLETE Column stats: COMPLETE
- 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 21
+ Reducer 25
Reduce Operator Tree:
Merge Join Operator
condition map:
@@ -751,40 +823,16 @@ STAGE PLANS:
keys:
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
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
+ Statistics: Num rows: 9 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
- key expressions: _col1 (type: int)
+ key expressions: _col0 (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
- 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 22
- Reduce Operator Tree:
- Merge Join Operator
- condition map:
- Left Outer Join 0 to 1
- keys:
- 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
- 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
- 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
- Reduce Output Operator
- key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string)
- null sort order: aa
- 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
- 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 23
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 9 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
+ 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
Reduce Operator Tree:
Group By Operator
@@ -802,7 +850,12 @@ STAGE PLANS:
sort order:
Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
value expressions: _col0 (type: bigint)
- Reducer 24
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: bigint)
+ Reducer 28
Execution mode: vectorized
Reduce Operator Tree:
Group By Operator
@@ -842,90 +895,6 @@ STAGE PLANS:
Map-reduce partition columns: _col0 (type: int)
Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
value expressions: _col1 (type: boolean)
- Reducer 25
- Execution mode: vectorized
- Reduce Operator Tree:
- Group By Operator
- aggregations: count(VALUE._col0)
- mode: mergepartial
- outputColumnNames: _col0
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint)
- Reducer 26
- Execution mode: vectorized
- Reduce Operator Tree:
- Group By Operator
- 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
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint), _col1 (type: bigint)
- Reducer 27
- Execution mode: vectorized
- Reduce Operator Tree:
- Group By Operator
- keys: KEY._col0 (type: int)
- mode: mergepartial
- outputColumnNames: _col0
- Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
- 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
- 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
- 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
- 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
- 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
- value expressions: _col1 (type: boolean)
- Reducer 28
- Execution mode: vectorized
- Reduce Operator Tree:
- Group By Operator
- aggregations: count(VALUE._col0)
- mode: mergepartial
- outputColumnNames: _col0
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- 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
- value expressions: _col0 (type: bigint)
Reducer 29
Execution mode: vectorized
Reduce Operator Tree:
@@ -947,26 +916,13 @@ STAGE PLANS:
keys:
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
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 9 Data size: 4389 Basic stats: COMPLETE Column stats: COMPLETE
- 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 30
- Execution mode: vectorized
- Reduce Operator Tree:
- Group By Operator
- aggregations: count(VALUE._col0)
- mode: mergepartial
- outputColumnNames: _col0
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ outputColumnNames: _col1, _col2, _col3, _col5
+ Statistics: Num rows: 13 Data size: 2568 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
null sort order:
sort order:
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: bigint)
+ Statistics: Num rows: 13 Data size: 2568 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean)
Reducer 4
Reduce Operator Tree:
Merge Join Operator
@@ -975,52 +931,52 @@ STAGE PLANS:
keys:
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
+ outputColumnNames: _col1, _col2, _col3, _col5, _col6
+ Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
- key expressions: _col2 (type: int)
+ key expressions: _col1 (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
- 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)
+ Map-reduce partition columns: _col1 (type: int)
+ Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col2 (type: string), _col3 (type: bigint), _col5 (type: boolean), _col6 (type: bigint)
Reducer 5
Reduce Operator Tree:
Merge Join Operator
condition map:
Left Outer Join 0 to 1
keys:
- 0 _col2 (type: int)
+ 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: 4497 Basic stats: COMPLETE Column stats: COMPLETE
+ outputColumnNames: _col2, _col3, _col5, _col6, _col8
+ Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE
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
+ 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
Reduce Output Operator
- key expressions: _col5 (type: string)
+ key expressions: _col2 (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
- value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: bigint), _col6 (type: string)
+ Map-reduce partition columns: _col2 (type: string)
+ Statistics: Num rows: 13 Data size: 2672 Basic stats: COMPLETE Column stats: COMPLETE
Reducer 6
Execution mode: vectorized
Reduce Operator Tree:
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
+ expressions: KEY.reducesinkkey0 (type: string)
+ outputColumnNames: _col2
+ Statistics: Num rows: 13 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE
PTF Operator
Function definitions:
Input definition
input alias: ptf_0
+ output shape: _col2: string
type: WINDOWING
Windowing table definition
input alias: ptf_1
name: windowingtablefunction
- order by: _col5 ASC NULLS FIRST
- partition by: _col5
+ order by: _col2 ASC NULLS FIRST
+ partition by: _col2
raw input shape:
window functions:
window function definition
@@ -1029,22 +985,48 @@ 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: 13 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE
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: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE
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
- Reduce Output Operator
- key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string)
- null sort order: aa
- 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
- 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
+ expressions: _col2 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: string)
+ minReductionHashAggr: 0.4
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: COMPLETE
+ 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
+ Reducer 7
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Semi Join 0 to 1
+ keys:
+ 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
+ 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
+ Reduce Output Operator
+ key expressions: iceberg_bucket(_col5, 16) (type: int), iceberg_truncate(_col6, 3) (type: string)
+ null sort order: aa
+ 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
+ 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
Reduce Operator Tree:
Select Operator
@@ -1059,23 +1041,8 @@ STAGE PLANS:
output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
name: default.tbl_ice
- Reducer 9
- Reduce Operator Tree:
- Merge Join Operator
- condition map:
- Left Outer Join 0 to 1
- keys:
- 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
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 9 Data size: 4389 Basic stats: COMPLETE Column stats: COMPLETE
- 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)
- Union 7
- Vertex: Union 7
+ Union 8
+ Vertex: Union 8
Stage: Stage-2
Dependency Collection
@@ -1094,14 +1061,14 @@ STAGE PLANS:
Stats Work
Basic Stats Work:
-Warning: Shuffle Join MERGEJOIN[282][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 2' is a cross product
-Warning: Shuffle Join MERGEJOIN[295][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 4' is a cross product
-Warning: Shuffle Join MERGEJOIN[287][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 10' is a cross product
-Warning: Shuffle Join MERGEJOIN[289][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 12' is a cross product
-Warning: Shuffle Join MERGEJOIN[284][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 14' is a cross product
-Warning: Shuffle Join MERGEJOIN[293][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 16' is a cross product
-Warning: Shuffle Join MERGEJOIN[281][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 19' is a cross product
-Warning: Shuffle Join MERGEJOIN[291][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 21' is a cross product
+Warning: Shuffle Join MERGEJOIN[284][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 23' is a cross product
+Warning: Shuffle Join MERGEJOIN[293][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[287][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 14' is a cross product
+Warning: Shuffle Join MERGEJOIN[289][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 16' is a cross product
+Warning: Shuffle Join MERGEJOIN[281][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 25' is a cross product
+Warning: Shuffle Join MERGEJOIN[291][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 11' is a cross product
+Warning: Shuffle Join MERGEJOIN[282][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 24' is a cross product
+Warning: Shuffle Join MERGEJOIN[295][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 18' is a cross product
PREHOOK: query: update tbl_ice set b='Changed again' where a in (select a from tbl_ice where a <= 5) or c in (select c from tbl_ice where c > 800)
PREHOOK: type: QUERY
PREHOOK: Input: default@tbl_ice
diff --git a/iceberg/patched-iceberg-core/pom.xml b/iceberg/patched-iceberg-core/pom.xml
index e870ddf42c03..c1745bb021b4 100644
--- a/iceberg/patched-iceberg-core/pom.xml
+++ b/iceberg/patched-iceberg-core/pom.xml
@@ -94,6 +94,11 @@
**/HadoopInputFile.class
**/HadoopTableOperations.class
+ org.apache.iceberg.avro.ValueReaders.class
+ org.apache.iceberg.avro.ValueWriters.class
+ org.apache.iceberg.BaseScan.class
+ org.apache.iceberg.PartitionsTable.class
+ org.apache.iceberg.Partitioning.class
diff --git a/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/BaseScan.java b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/BaseScan.java
new file mode 100644
index 000000000000..7496be6f7220
--- /dev/null
+++ b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/BaseScan.java
@@ -0,0 +1,312 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.stream.Collectors;
+import org.apache.iceberg.expressions.Binder;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.metrics.MetricsReporter;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+// TODO: remove class once upgraded to Iceberg v1.7.0
+
+abstract class BaseScan>
+ implements Scan {
+
+ protected static final List SCAN_COLUMNS =
+ ImmutableList.of(
+ "snapshot_id",
+ "file_path",
+ "file_ordinal",
+ "file_format",
+ "block_size_in_bytes",
+ "file_size_in_bytes",
+ "record_count",
+ "partition",
+ "key_metadata",
+ "split_offsets",
+ "sort_order_id");
+
+ private static final List STATS_COLUMNS =
+ ImmutableList.of(
+ "value_counts",
+ "null_value_counts",
+ "nan_value_counts",
+ "lower_bounds",
+ "upper_bounds",
+ "column_sizes");
+
+ protected static final List SCAN_WITH_STATS_COLUMNS =
+ ImmutableList.builder().addAll(SCAN_COLUMNS).addAll(STATS_COLUMNS).build();
+
+ protected static final List DELETE_SCAN_COLUMNS =
+ ImmutableList.of(
+ "snapshot_id",
+ "content",
+ "file_path",
+ "file_ordinal",
+ "file_format",
+ "block_size_in_bytes",
+ "file_size_in_bytes",
+ "record_count",
+ "partition",
+ "key_metadata",
+ "split_offsets",
+ "equality_ids");
+
+ protected static final List DELETE_SCAN_WITH_STATS_COLUMNS =
+ ImmutableList.builder().addAll(DELETE_SCAN_COLUMNS).addAll(STATS_COLUMNS).build();
+
+ protected static final boolean PLAN_SCANS_WITH_WORKER_POOL =
+ SystemConfigs.SCAN_THREAD_POOL_ENABLED.value();
+
+ private final Table table;
+ private final Schema schema;
+ private final TableScanContext context;
+
+ protected BaseScan(Table table, Schema schema, TableScanContext context) {
+ this.table = table;
+ this.schema = schema;
+ this.context = context;
+ }
+
+ public Table table() {
+ return table;
+ }
+
+ protected FileIO io() {
+ return table.io();
+ }
+
+ protected Schema tableSchema() {
+ return schema;
+ }
+
+ protected TableScanContext context() {
+ return context;
+ }
+
+ protected Map options() {
+ return context().options();
+ }
+
+ protected List scanColumns() {
+ return context.returnColumnStats() ? SCAN_WITH_STATS_COLUMNS : SCAN_COLUMNS;
+ }
+
+ protected boolean shouldReturnColumnStats() {
+ return context().returnColumnStats();
+ }
+
+ protected Set columnsToKeepStats() {
+ return context().columnsToKeepStats();
+ }
+
+ protected boolean shouldIgnoreResiduals() {
+ return context().ignoreResiduals();
+ }
+
+ protected Expression residualFilter() {
+ return shouldIgnoreResiduals() ? Expressions.alwaysTrue() : filter();
+ }
+
+ protected boolean shouldPlanWithExecutor() {
+ return PLAN_SCANS_WITH_WORKER_POOL || context().planWithCustomizedExecutor();
+ }
+
+ protected ExecutorService planExecutor() {
+ return context().planExecutor();
+ }
+
+ protected abstract ThisT newRefinedScan(
+ Table newTable, Schema newSchema, TableScanContext newContext);
+
+ @Override
+ public ThisT option(String property, String value) {
+ return newRefinedScan(table, schema, context.withOption(property, value));
+ }
+
+ @Override
+ public ThisT project(Schema projectedSchema) {
+ return newRefinedScan(table, schema, context.project(projectedSchema));
+ }
+
+ @Override
+ public ThisT caseSensitive(boolean caseSensitive) {
+ return newRefinedScan(table, schema, context.setCaseSensitive(caseSensitive));
+ }
+
+ @Override
+ public boolean isCaseSensitive() {
+ return context().caseSensitive();
+ }
+
+ @Override
+ public ThisT includeColumnStats() {
+ return newRefinedScan(table, schema, context.shouldReturnColumnStats(true));
+ }
+
+ @Override
+ public ThisT includeColumnStats(Collection requestedColumns) {
+ return newRefinedScan(
+ table,
+ schema,
+ context
+ .shouldReturnColumnStats(true)
+ .columnsToKeepStats(
+ requestedColumns.stream()
+ .map(c -> schema.findField(c).fieldId())
+ .collect(Collectors.toSet())));
+ }
+
+ @Override
+ public ThisT select(Collection columns) {
+ return newRefinedScan(table, schema, context.selectColumns(columns));
+ }
+
+ @Override
+ public ThisT filter(Expression expr) {
+ return newRefinedScan(
+ table, schema, context.filterRows(Expressions.and(context.rowFilter(), expr)));
+ }
+
+ @Override
+ public Expression filter() {
+ return context().rowFilter();
+ }
+
+ @Override
+ public ThisT ignoreResiduals() {
+ return newRefinedScan(table, schema, context.ignoreResiduals(true));
+ }
+
+ @Override
+ public ThisT planWith(ExecutorService executorService) {
+ return newRefinedScan(table, schema, context.planWith(executorService));
+ }
+
+ @Override
+ public Schema schema() {
+ return lazyColumnProjection(context, schema);
+ }
+
+ @Override
+ public long targetSplitSize() {
+ long tableValue =
+ PropertyUtil.propertyAsLong(
+ table().properties(), TableProperties.SPLIT_SIZE, TableProperties.SPLIT_SIZE_DEFAULT);
+ return PropertyUtil.propertyAsLong(context.options(), TableProperties.SPLIT_SIZE, tableValue);
+ }
+
+ @Override
+ public int splitLookback() {
+ int tableValue =
+ PropertyUtil.propertyAsInt(
+ table().properties(),
+ TableProperties.SPLIT_LOOKBACK,
+ TableProperties.SPLIT_LOOKBACK_DEFAULT);
+ return PropertyUtil.propertyAsInt(
+ context.options(), TableProperties.SPLIT_LOOKBACK, tableValue);
+ }
+
+ @Override
+ public long splitOpenFileCost() {
+ long tableValue =
+ PropertyUtil.propertyAsLong(
+ table().properties(),
+ TableProperties.SPLIT_OPEN_FILE_COST,
+ TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT);
+ return PropertyUtil.propertyAsLong(
+ context.options(), TableProperties.SPLIT_OPEN_FILE_COST, tableValue);
+ }
+
+ /**
+ * Resolve the schema to be projected lazily.
+ *
+ * If there are selected columns from scan context, selected columns are projected to the table
+ * schema. Otherwise, projected schema from scan context shall be returned.
+ *
+ * @param context scan context
+ * @param schema table schema
+ * @return the Schema to project
+ */
+ private static Schema lazyColumnProjection(TableScanContext context, Schema schema) {
+ Collection selectedColumns = context.selectedColumns();
+ if (selectedColumns != null) {
+ Set requiredFieldIds = Sets.newHashSet();
+
+ // all of the filter columns are required
+ requiredFieldIds.addAll(
+ Binder.boundReferences(
+ schema.asStruct(),
+ Collections.singletonList(context.rowFilter()),
+ context.caseSensitive()));
+
+ // all of the projection columns are required
+ Set selectedIds;
+ if (context.caseSensitive()) {
+ selectedIds = TypeUtil.getProjectedIds(schema.select(selectedColumns));
+ } else {
+ selectedIds = TypeUtil.getProjectedIds(schema.caseInsensitiveSelect(selectedColumns));
+ }
+ requiredFieldIds.addAll(selectedIds);
+
+ return TypeUtil.project(schema, requiredFieldIds);
+
+ } else if (context.projectedSchema() != null) {
+ return context.projectedSchema();
+ }
+
+ return schema;
+ }
+
+ @Override
+ public ThisT metricsReporter(MetricsReporter reporter) {
+ return newRefinedScan(table, schema, context.reportWith(reporter));
+ }
+
+ /**
+ * Retrieves a list of column names based on the type of manifest content provided.
+ *
+ * @param content the manifest content type to scan.
+ * @return a list of column names corresponding to the specified manifest content type.
+ */
+ static List scanColumns(ManifestContent content) {
+ switch (content) {
+ case DATA:
+ return BaseScan.SCAN_COLUMNS;
+ case DELETES:
+ return BaseScan.DELETE_SCAN_COLUMNS;
+ default:
+ throw new UnsupportedOperationException("Cannot read unknown manifest type: " + content);
+ }
+ }
+}
diff --git a/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/PartitionStats.java b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/PartitionStats.java
new file mode 100644
index 000000000000..35247b0451bf
--- /dev/null
+++ b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/PartitionStats.java
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+// TODO: remove class once upgraded to Iceberg v1.7.0
+
+public class PartitionStats implements StructLike {
+
+ private static final int STATS_COUNT = 12;
+
+ private StructLike partition;
+ private int specId;
+ private long dataRecordCount;
+ private int dataFileCount;
+ private long totalDataFileSizeInBytes;
+ private long positionDeleteRecordCount;
+ private int positionDeleteFileCount;
+ private long equalityDeleteRecordCount;
+ private int equalityDeleteFileCount;
+ private long totalRecordCount;
+ private Long lastUpdatedAt; // null by default
+ private Long lastUpdatedSnapshotId; // null by default
+
+ public PartitionStats(StructLike partition, int specId) {
+ this.partition = partition;
+ this.specId = specId;
+ }
+
+ public StructLike partition() {
+ return partition;
+ }
+
+ public int specId() {
+ return specId;
+ }
+
+ public long dataRecordCount() {
+ return dataRecordCount;
+ }
+
+ public int dataFileCount() {
+ return dataFileCount;
+ }
+
+ public long totalDataFileSizeInBytes() {
+ return totalDataFileSizeInBytes;
+ }
+
+ public long positionDeleteRecordCount() {
+ return positionDeleteRecordCount;
+ }
+
+ public int positionDeleteFileCount() {
+ return positionDeleteFileCount;
+ }
+
+ public long equalityDeleteRecordCount() {
+ return equalityDeleteRecordCount;
+ }
+
+ public int equalityDeleteFileCount() {
+ return equalityDeleteFileCount;
+ }
+
+ public long totalRecordCount() {
+ return totalRecordCount;
+ }
+
+ public Long lastUpdatedAt() {
+ return lastUpdatedAt;
+ }
+
+ public Long lastUpdatedSnapshotId() {
+ return lastUpdatedSnapshotId;
+ }
+
+ /**
+ * Updates the partition stats from the data/delete file.
+ *
+ * @param file the {@link ContentFile} from the manifest entry.
+ * @param snapshot the snapshot corresponding to the live entry.
+ */
+ public void liveEntry(ContentFile> file, Snapshot snapshot) {
+ Preconditions.checkArgument(specId == file.specId(), "Spec IDs must match");
+
+ switch (file.content()) {
+ case DATA:
+ this.dataRecordCount += file.recordCount();
+ this.dataFileCount += 1;
+ this.totalDataFileSizeInBytes += file.fileSizeInBytes();
+ break;
+ case POSITION_DELETES:
+ this.positionDeleteRecordCount += file.recordCount();
+ this.positionDeleteFileCount += 1;
+ break;
+ case EQUALITY_DELETES:
+ this.equalityDeleteRecordCount += file.recordCount();
+ this.equalityDeleteFileCount += 1;
+ break;
+ default:
+ throw new UnsupportedOperationException("Unsupported file content type: " + file.content());
+ }
+
+ if (snapshot != null) {
+ updateSnapshotInfo(snapshot.snapshotId(), snapshot.timestampMillis());
+ }
+
+ // Note: Not computing the `TOTAL_RECORD_COUNT` for now as it needs scanning the data.
+ }
+
+ /**
+ * Updates the modified time and snapshot ID for the deleted manifest entry.
+ *
+ * @param snapshot the snapshot corresponding to the deleted manifest entry.
+ */
+ public void deletedEntry(Snapshot snapshot) {
+ if (snapshot != null) {
+ updateSnapshotInfo(snapshot.snapshotId(), snapshot.timestampMillis());
+ }
+ }
+
+ /**
+ * Appends statistics from given entry to current entry.
+ *
+ * @param entry the entry from which statistics will be sourced.
+ */
+ public void appendStats(PartitionStats entry) {
+ Preconditions.checkArgument(specId == entry.specId(), "Spec IDs must match");
+
+ this.dataRecordCount += entry.dataRecordCount;
+ this.dataFileCount += entry.dataFileCount;
+ this.totalDataFileSizeInBytes += entry.totalDataFileSizeInBytes;
+ this.positionDeleteRecordCount += entry.positionDeleteRecordCount;
+ this.positionDeleteFileCount += entry.positionDeleteFileCount;
+ this.equalityDeleteRecordCount += entry.equalityDeleteRecordCount;
+ this.equalityDeleteFileCount += entry.equalityDeleteFileCount;
+ this.totalRecordCount += entry.totalRecordCount;
+
+ if (entry.lastUpdatedAt != null) {
+ updateSnapshotInfo(entry.lastUpdatedSnapshotId, entry.lastUpdatedAt);
+ }
+ }
+
+ private void updateSnapshotInfo(long snapshotId, long updatedAt) {
+ if (lastUpdatedAt == null || lastUpdatedAt < updatedAt) {
+ this.lastUpdatedAt = updatedAt;
+ this.lastUpdatedSnapshotId = snapshotId;
+ }
+ }
+
+ @Override
+ public int size() {
+ return STATS_COUNT;
+ }
+
+ @Override
+ public T get(int pos, Class javaClass) {
+ switch (pos) {
+ case 0:
+ return javaClass.cast(partition);
+ case 1:
+ return javaClass.cast(specId);
+ case 2:
+ return javaClass.cast(dataRecordCount);
+ case 3:
+ return javaClass.cast(dataFileCount);
+ case 4:
+ return javaClass.cast(totalDataFileSizeInBytes);
+ case 5:
+ return javaClass.cast(positionDeleteRecordCount);
+ case 6:
+ return javaClass.cast(positionDeleteFileCount);
+ case 7:
+ return javaClass.cast(equalityDeleteRecordCount);
+ case 8:
+ return javaClass.cast(equalityDeleteFileCount);
+ case 9:
+ return javaClass.cast(totalRecordCount);
+ case 10:
+ return javaClass.cast(lastUpdatedAt);
+ case 11:
+ return javaClass.cast(lastUpdatedSnapshotId);
+ default:
+ throw new UnsupportedOperationException("Unknown position: " + pos);
+ }
+ }
+
+ @Override
+ public void set(int pos, T value) {
+ switch (pos) {
+ case 0:
+ this.partition = (StructLike) value;
+ break;
+ case 1:
+ this.specId = (Integer) value;
+ break;
+ case 2:
+ this.dataRecordCount = (Long) value;
+ break;
+ case 3:
+ this.dataFileCount = (Integer) value;
+ break;
+ case 4:
+ this.totalDataFileSizeInBytes = (Long) value;
+ break;
+ case 5:
+ // optional field as per spec, implementation initialize to 0 for counters
+ this.positionDeleteRecordCount = value == null ? 0L : (Long) value;
+ break;
+ case 6:
+ // optional field as per spec, implementation initialize to 0 for counters
+ this.positionDeleteFileCount = value == null ? 0 : (Integer) value;
+ break;
+ case 7:
+ // optional field as per spec, implementation initialize to 0 for counters
+ this.equalityDeleteRecordCount = value == null ? 0L : (Long) value;
+ break;
+ case 8:
+ // optional field as per spec, implementation initialize to 0 for counters
+ this.equalityDeleteFileCount = value == null ? 0 : (Integer) value;
+ break;
+ case 9:
+ // optional field as per spec, implementation initialize to 0 for counters
+ this.totalRecordCount = value == null ? 0L : (Long) value;
+ break;
+ case 10:
+ this.lastUpdatedAt = (Long) value;
+ break;
+ case 11:
+ this.lastUpdatedSnapshotId = (Long) value;
+ break;
+ default:
+ throw new UnsupportedOperationException("Unknown position: " + pos);
+ }
+ }
+}
diff --git a/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/PartitionStatsUtil.java b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/PartitionStatsUtil.java
new file mode 100644
index 000000000000..1c70baba165d
--- /dev/null
+++ b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/PartitionStatsUtil.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Queue;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionMap;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+
+// TODO: remove class once upgraded to Iceberg v1.7.0
+
+public class PartitionStatsUtil {
+
+ private PartitionStatsUtil() {
+ }
+
+ /**
+ * Computes the partition stats for the given snapshot of the table.
+ *
+ * @param table the table for which partition stats to be computed.
+ * @param snapshot the snapshot for which partition stats is computed.
+ * @return the collection of {@link PartitionStats}
+ */
+ public static Collection computeStats(Table table, Snapshot snapshot) {
+ Preconditions.checkArgument(table != null, "table cannot be null");
+ Preconditions.checkArgument(Partitioning.isPartitioned(table), "table must be partitioned");
+ Preconditions.checkArgument(snapshot != null, "snapshot cannot be null");
+
+ StructType partitionType = Partitioning.partitionType(table);
+ List manifests = snapshot.allManifests(table.io());
+ Queue> statsByManifest = Queues.newConcurrentLinkedQueue();
+ Tasks.foreach(manifests)
+ .stopOnFailure()
+ .throwFailureWhenFinished()
+ .executeWith(ThreadPools.getWorkerPool())
+ .run(manifest -> statsByManifest.add(collectStats(table, manifest, partitionType)));
+
+ return mergeStats(statsByManifest, table.specs());
+ }
+
+ /**
+ * Sorts the {@link PartitionStats} based on the partition data.
+ *
+ * @param stats collection of {@link PartitionStats} which needs to be sorted.
+ * @param partitionType unified partition schema.
+ * @return the list of {@link PartitionStats}
+ */
+ public static List sortStats(
+ Collection stats, StructType partitionType) {
+ List entries = Lists.newArrayList(stats);
+ entries.sort(partitionStatsCmp(partitionType));
+ return entries;
+ }
+
+ private static Comparator partitionStatsCmp(StructType partitionType) {
+ return Comparator.comparing(PartitionStats::partition, Comparators.forType(partitionType));
+ }
+
+ private static PartitionMap collectStats(
+ Table table, ManifestFile manifest, StructType partitionType) {
+ try (ManifestReader> reader = openManifest(table, manifest)) {
+ PartitionMap statsMap = PartitionMap.create(table.specs());
+ int specId = manifest.partitionSpecId();
+ PartitionSpec spec = table.specs().get(specId);
+ PartitionData keyTemplate = new PartitionData(partitionType);
+
+ for (ManifestEntry> entry : reader.entries()) {
+ ContentFile> file = entry.file();
+ StructLike coercedPartition =
+ PartitionUtil.coercePartition(partitionType, spec, file.partition());
+ StructLike key = keyTemplate.copyFor(coercedPartition);
+ Snapshot snapshot = table.snapshot(entry.snapshotId());
+ PartitionStats stats =
+ statsMap.computeIfAbsent(specId, ((PartitionData) file.partition()).copy(),
+ () -> new PartitionStats(key, specId));
+ if (entry.isLive()) {
+ stats.liveEntry(file, snapshot);
+ } else {
+ stats.deletedEntry(snapshot);
+ }
+ }
+
+ return statsMap;
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+
+ private static ManifestReader> openManifest(Table table, ManifestFile manifest) {
+ List projection = BaseScan.scanColumns(manifest.content());
+ return ManifestFiles.open(manifest, table.io()).select(projection);
+ }
+
+ private static Collection mergeStats(
+ Queue> statsByManifest, Map specs) {
+ PartitionMap statsMap = PartitionMap.create(specs);
+
+ for (PartitionMap stats : statsByManifest) {
+ stats.forEach(
+ (key, value) ->
+ statsMap.merge(
+ key,
+ value,
+ (existingEntry, newEntry) -> {
+ existingEntry.appendStats(newEntry);
+ return existingEntry;
+ }));
+ }
+
+ return statsMap.values();
+ }
+
+ @SuppressWarnings("checkstyle:CyclomaticComplexity")
+ public static boolean isEqual(
+ Comparator partitionComparator, PartitionStats stats1, PartitionStats stats2) {
+ if (stats1 == stats2) {
+ return true;
+ } else if (stats1 == null || stats2 == null) {
+ return false;
+ }
+
+ return partitionComparator.compare(stats1.partition(), stats2.partition()) == 0 &&
+ stats1.specId() == stats2.specId() &&
+ stats1.dataRecordCount() == stats2.dataRecordCount() &&
+ stats1.dataFileCount() == stats2.dataFileCount() &&
+ stats1.totalDataFileSizeInBytes() == stats2.totalDataFileSizeInBytes() &&
+ stats1.positionDeleteRecordCount() == stats2.positionDeleteRecordCount() &&
+ stats1.positionDeleteFileCount() == stats2.positionDeleteFileCount() &&
+ stats1.equalityDeleteRecordCount() == stats2.equalityDeleteRecordCount() &&
+ stats1.equalityDeleteFileCount() == stats2.equalityDeleteFileCount() &&
+ stats1.totalRecordCount() == stats2.totalRecordCount() &&
+ Objects.equals(stats1.lastUpdatedAt(), stats2.lastUpdatedAt()) &&
+ Objects.equals(stats1.lastUpdatedSnapshotId(), stats2.lastUpdatedSnapshotId());
+ }
+}
diff --git a/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/Partitioning.java b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/Partitioning.java
new file mode 100644
index 000000000000..9cb395b81b22
--- /dev/null
+++ b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/Partitioning.java
@@ -0,0 +1,387 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.FluentIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.transforms.PartitionSpecVisitor;
+import org.apache.iceberg.transforms.Transform;
+import org.apache.iceberg.transforms.Transforms;
+import org.apache.iceberg.transforms.UnknownTransform;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+
+// TODO: remove class once upgraded to Iceberg v1.7.0
+
+public class Partitioning {
+
+ private Partitioning() {
+ }
+
+ /**
+ * Check whether the spec contains a bucketed partition field.
+ *
+ * @param spec a partition spec
+ * @return true if the spec has field with a bucket transform
+ */
+ public static boolean hasBucketField(PartitionSpec spec) {
+ List bucketList =
+ PartitionSpecVisitor.visit(
+ spec,
+ new PartitionSpecVisitor() {
+ @Override
+ public Boolean identity(int fieldId, String sourceName, int sourceId) {
+ return false;
+ }
+
+ @Override
+ public Boolean bucket(int fieldId, String sourceName, int sourceId, int width) {
+ return true;
+ }
+
+ @Override
+ public Boolean truncate(int fieldId, String sourceName, int sourceId, int width) {
+ return false;
+ }
+
+ @Override
+ public Boolean year(int fieldId, String sourceName, int sourceId) {
+ return false;
+ }
+
+ @Override
+ public Boolean month(int fieldId, String sourceName, int sourceId) {
+ return false;
+ }
+
+ @Override
+ public Boolean day(int fieldId, String sourceName, int sourceId) {
+ return false;
+ }
+
+ @Override
+ public Boolean hour(int fieldId, String sourceName, int sourceId) {
+ return false;
+ }
+
+ @Override
+ public Boolean alwaysNull(int fieldId, String sourceName, int sourceId) {
+ return false;
+ }
+
+ @Override
+ public Boolean unknown(
+ int fieldId, String sourceName, int sourceId, String transform) {
+ return false;
+ }
+ });
+
+ return bucketList.stream().anyMatch(Boolean::booleanValue);
+ }
+
+ /**
+ * Create a sort order that will group data for a partition spec.
+ *
+ * If the partition spec contains bucket columns, the sort order will also have a field to sort
+ * by a column that is bucketed in the spec. The column is selected by the highest number of
+ * buckets in the transform.
+ *
+ * @param spec a partition spec
+ * @return a sort order that will cluster data for the spec
+ */
+ public static SortOrder sortOrderFor(PartitionSpec spec) {
+ if (spec.isUnpartitioned()) {
+ return SortOrder.unsorted();
+ }
+
+ SortOrder.Builder builder = SortOrder.builderFor(spec.schema());
+ SpecToOrderVisitor converter = new SpecToOrderVisitor(builder);
+ PartitionSpecVisitor.visit(spec, converter);
+
+ // columns used for bucketing are high cardinality; add one to the sort at the end
+ String bucketColumn = converter.bucketColumn();
+ if (bucketColumn != null) {
+ builder.asc(bucketColumn);
+ }
+
+ return builder.build();
+ }
+
+ private static class SpecToOrderVisitor implements PartitionSpecVisitor {
+ private final SortOrder.Builder builder;
+ private String bucketColumn = null;
+ private int highestNumBuckets = 0;
+
+ private SpecToOrderVisitor(SortOrder.Builder builder) {
+ this.builder = builder;
+ }
+
+ String bucketColumn() {
+ return bucketColumn;
+ }
+
+ @Override
+ public Void identity(int fieldId, String sourceName, int sourceId) {
+ builder.asc(sourceName);
+ return null;
+ }
+
+ @Override
+ public Void bucket(int fieldId, String sourceName, int sourceId, int numBuckets) {
+ // the column with highest cardinality is usually the one with the highest number of buckets
+ if (numBuckets > highestNumBuckets) {
+ this.highestNumBuckets = numBuckets;
+ this.bucketColumn = sourceName;
+ }
+ builder.asc(Expressions.bucket(sourceName, numBuckets));
+ return null;
+ }
+
+ @Override
+ public Void truncate(int fieldId, String sourceName, int sourceId, int width) {
+ builder.asc(Expressions.truncate(sourceName, width));
+ return null;
+ }
+
+ @Override
+ public Void year(int fieldId, String sourceName, int sourceId) {
+ builder.asc(Expressions.year(sourceName));
+ return null;
+ }
+
+ @Override
+ public Void month(int fieldId, String sourceName, int sourceId) {
+ builder.asc(Expressions.month(sourceName));
+ return null;
+ }
+
+ @Override
+ public Void day(int fieldId, String sourceName, int sourceId) {
+ builder.asc(Expressions.day(sourceName));
+ return null;
+ }
+
+ @Override
+ public Void hour(int fieldId, String sourceName, int sourceId) {
+ builder.asc(Expressions.hour(sourceName));
+ return null;
+ }
+
+ @Override
+ public Void alwaysNull(int fieldId, String sourceName, int sourceId) {
+ // do nothing for alwaysNull, it doesn't need to be added to the sort
+ return null;
+ }
+ }
+
+ /**
+ * Builds a grouping key type considering the provided schema and specs.
+ *
+ * A grouping key defines how data is split between files and consists of partition fields with
+ * non-void transforms that are present in each provided spec. Iceberg guarantees that records
+ * with different values for the grouping key are disjoint and are stored in separate files.
+ *
+ *
If there is only one spec, the grouping key will include all partition fields with non-void
+ * transforms from that spec. Whenever there are multiple specs, the grouping key will represent
+ * an intersection of all partition fields with non-void transforms. If a partition field is
+ * present only in a subset of specs, Iceberg cannot guarantee data distribution on that field.
+ * That's why it will not be part of the grouping key. Unpartitioned tables or tables with
+ * non-overlapping specs have empty grouping keys.
+ *
+ *
When partition fields are dropped in v1 tables, they are replaced with new partition fields
+ * that have the same field ID but use a void transform under the hood. Such fields cannot be part
+ * of the grouping key as void transforms always return null.
+ *
+ *
If the provided schema is not null, this method will only take into account partition fields
+ * on top of columns present in the schema. Otherwise, all partition fields will be considered.
+ *
+ * @param schema a schema specifying a set of source columns to consider (null to consider all)
+ * @param specs one or many specs
+ * @return the constructed grouping key type
+ */
+ public static StructType groupingKeyType(Schema schema, Collection specs) {
+ return buildPartitionProjectionType("grouping key", specs, commonActiveFieldIds(schema, specs));
+ }
+
+ /**
+ * Builds a unified partition type considering all specs in a table.
+ *
+ * If there is only one spec, the partition type is that spec's partition type. Whenever there
+ * are multiple specs, the partition type is a struct containing all fields that have ever been a
+ * part of any spec in the table. In other words, the struct fields represent a union of all known
+ * partition fields.
+ *
+ * @param table a table with one or many specs
+ * @return the constructed unified partition type
+ */
+ public static StructType partitionType(Table table) {
+ Collection specs = table.specs().values();
+ return buildPartitionProjectionType("table partition", specs, allFieldIds(specs));
+ }
+
+ /**
+ * Checks if any of the specs in a table is partitioned.
+ *
+ * @param table the table to check.
+ * @return {@code true} if the table is partitioned, {@code false} otherwise.
+ */
+ public static boolean isPartitioned(Table table) {
+ return table.specs().values().stream().anyMatch(PartitionSpec::isPartitioned);
+ }
+
+ private static StructType buildPartitionProjectionType(
+ String typeName, Collection specs, Set projectedFieldIds) {
+
+ // we currently don't know the output type of unknown transforms
+ List> unknownTransforms = collectUnknownTransforms(specs);
+ ValidationException.check(
+ unknownTransforms.isEmpty(),
+ "Cannot build %s type, unknown transforms: %s",
+ typeName,
+ unknownTransforms);
+
+ Map fieldMap = Maps.newHashMap();
+ Map typeMap = Maps.newHashMap();
+ Map nameMap = Maps.newHashMap();
+
+ // sort specs by ID in descending order to pick up the most recent field names
+ List sortedSpecs =
+ specs.stream()
+ .sorted(Comparator.comparingLong(PartitionSpec::specId).reversed())
+ .collect(Collectors.toList());
+
+ for (PartitionSpec spec : sortedSpecs) {
+ for (PartitionField field : spec.fields()) {
+ int fieldId = field.fieldId();
+
+ if (!projectedFieldIds.contains(fieldId)) {
+ continue;
+ }
+
+ NestedField structField = spec.partitionType().field(fieldId);
+ PartitionField existingField = fieldMap.get(fieldId);
+
+ if (existingField == null) {
+ fieldMap.put(fieldId, field);
+ typeMap.put(fieldId, structField.type());
+ nameMap.put(fieldId, structField.name());
+
+ } else {
+ // verify the fields are compatible as they may conflict in v1 tables
+ ValidationException.check(
+ equivalentIgnoringNames(field, existingField),
+ "Conflicting partition fields: ['%s', '%s']",
+ field,
+ existingField);
+
+ // use the correct type for dropped partitions in v1 tables
+ if (isVoidTransform(existingField) && !isVoidTransform(field)) {
+ fieldMap.put(fieldId, field);
+ typeMap.put(fieldId, structField.type());
+ }
+ }
+ }
+ }
+
+ List sortedStructFields =
+ fieldMap.keySet().stream()
+ .sorted(Comparator.naturalOrder())
+ .map(
+ fieldId ->
+ NestedField.optional(fieldId, nameMap.get(fieldId), typeMap.get(fieldId)))
+ .collect(Collectors.toList());
+ return StructType.of(sortedStructFields);
+ }
+
+ private static boolean isVoidTransform(PartitionField field) {
+ return field.transform().equals(Transforms.alwaysNull());
+ }
+
+ private static List> collectUnknownTransforms(Collection specs) {
+ List> unknownTransforms = Lists.newArrayList();
+
+ for (PartitionSpec spec : specs) {
+ spec.fields().stream()
+ .map(PartitionField::transform)
+ .filter(transform -> transform instanceof UnknownTransform)
+ .forEach(unknownTransforms::add);
+ }
+
+ return unknownTransforms;
+ }
+
+ private static boolean equivalentIgnoringNames(
+ PartitionField field, PartitionField anotherField) {
+ return field.fieldId() == anotherField.fieldId() &&
+ field.sourceId() == anotherField.sourceId() &&
+ compatibleTransforms(field.transform(), anotherField.transform());
+ }
+
+ private static boolean compatibleTransforms(Transform, ?> t1, Transform, ?> t2) {
+ return t1.equals(t2) ||
+ t1.equals(Transforms.alwaysNull()) ||
+ t2.equals(Transforms.alwaysNull());
+ }
+
+ // collects IDs of all partition field used across specs
+ private static Set allFieldIds(Collection specs) {
+ return FluentIterable.from(specs)
+ .transformAndConcat(PartitionSpec::fields)
+ .transform(PartitionField::fieldId)
+ .toSet();
+ }
+
+ // collects IDs of partition fields with non-void transforms that are present in each spec
+ private static Set commonActiveFieldIds(Schema schema, Collection specs) {
+ Set commonActiveFieldIds = Sets.newHashSet();
+
+ int specIndex = 0;
+ for (PartitionSpec spec : specs) {
+ if (specIndex == 0) {
+ commonActiveFieldIds.addAll(activeFieldIds(schema, spec));
+ } else {
+ commonActiveFieldIds.retainAll(activeFieldIds(schema, spec));
+ }
+
+ specIndex++;
+ }
+
+ return commonActiveFieldIds;
+ }
+
+ private static List activeFieldIds(Schema schema, PartitionSpec spec) {
+ return spec.fields().stream()
+ .filter(field -> schema == null || schema.findField(field.sourceId()) != null)
+ .filter(field -> !isVoidTransform(field))
+ .map(PartitionField::fieldId)
+ .collect(Collectors.toList());
+ }
+}
diff --git a/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/PartitionsTable.java b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/PartitionsTable.java
new file mode 100644
index 000000000000..654b82faf095
--- /dev/null
+++ b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/PartitionsTable.java
@@ -0,0 +1,336 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg;
+
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import org.apache.iceberg.expressions.ManifestEvaluator;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ParallelIterable;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.StructLikeMap;
+
+// TODO: remove class once upgraded to Iceberg v1.7.0
+
+/** A {@link Table} implementation that exposes a table's partitions as rows. */
+public class PartitionsTable extends BaseMetadataTable {
+
+ private final Schema schema;
+
+ private final boolean unpartitionedTable;
+
+ PartitionsTable(Table table) {
+ this(table, table.name() + ".partitions");
+ }
+
+ PartitionsTable(Table table, String name) {
+ super(table, name);
+
+ this.schema =
+ new Schema(
+ Types.NestedField.required(1, "partition", Partitioning.partitionType(table)),
+ Types.NestedField.required(4, "spec_id", Types.IntegerType.get()),
+ Types.NestedField.required(
+ 2, "record_count", Types.LongType.get(), "Count of records in data files"),
+ Types.NestedField.required(
+ 3, "file_count", Types.IntegerType.get(), "Count of data files"),
+ Types.NestedField.required(
+ 11,
+ "total_data_file_size_in_bytes",
+ Types.LongType.get(),
+ "Total size in bytes of data files"),
+ Types.NestedField.required(
+ 5,
+ "position_delete_record_count",
+ Types.LongType.get(),
+ "Count of records in position delete files"),
+ Types.NestedField.required(
+ 6,
+ "position_delete_file_count",
+ Types.IntegerType.get(),
+ "Count of position delete files"),
+ Types.NestedField.required(
+ 7,
+ "equality_delete_record_count",
+ Types.LongType.get(),
+ "Count of records in equality delete files"),
+ Types.NestedField.required(
+ 8,
+ "equality_delete_file_count",
+ Types.IntegerType.get(),
+ "Count of equality delete files"),
+ Types.NestedField.optional(
+ 9,
+ "last_updated_at",
+ Types.TimestampType.withZone(),
+ "Commit time of snapshot that last updated this partition"),
+ Types.NestedField.optional(
+ 10,
+ "last_updated_snapshot_id",
+ Types.LongType.get(),
+ "Id of snapshot that last updated this partition"));
+ this.unpartitionedTable = Partitioning.partitionType(table).fields().isEmpty();
+ }
+
+ @Override
+ public TableScan newScan() {
+ return new PartitionsScan(table());
+ }
+
+ @Override
+ public Schema schema() {
+ if (unpartitionedTable) {
+ return schema.select(
+ "record_count",
+ "file_count",
+ "total_data_file_size_in_bytes",
+ "position_delete_record_count",
+ "position_delete_file_count",
+ "equality_delete_record_count",
+ "equality_delete_file_count",
+ "last_updated_at",
+ "last_updated_snapshot_id");
+ }
+ return schema;
+ }
+
+ @Override
+ MetadataTableType metadataTableType() {
+ return MetadataTableType.PARTITIONS;
+ }
+
+ private DataTask task(StaticTableScan scan) {
+ Iterable partitions = partitions(table(), scan);
+ if (unpartitionedTable) {
+ // the table is unpartitioned, partitions contains only the root partition
+ return StaticDataTask.of(
+ io().newInputFile(table().operations().current().metadataFileLocation()),
+ schema(),
+ scan.schema(),
+ partitions,
+ root ->
+ StaticDataTask.Row.of(
+ root.dataRecordCount,
+ root.dataFileCount,
+ root.dataFileSizeInBytes,
+ root.posDeleteRecordCount,
+ root.posDeleteFileCount,
+ root.eqDeleteRecordCount,
+ root.eqDeleteFileCount,
+ root.lastUpdatedAt,
+ root.lastUpdatedSnapshotId));
+ } else {
+ return StaticDataTask.of(
+ io().newInputFile(table().operations().current().metadataFileLocation()),
+ schema(),
+ scan.schema(),
+ partitions,
+ PartitionsTable::convertPartition);
+ }
+ }
+
+ private static StaticDataTask.Row convertPartition(Partition partition) {
+ return StaticDataTask.Row.of(
+ partition.partitionData,
+ partition.specId,
+ partition.dataRecordCount,
+ partition.dataFileCount,
+ partition.dataFileSizeInBytes,
+ partition.posDeleteRecordCount,
+ partition.posDeleteFileCount,
+ partition.eqDeleteRecordCount,
+ partition.eqDeleteFileCount,
+ partition.lastUpdatedAt,
+ partition.lastUpdatedSnapshotId);
+ }
+
+ private static Iterable partitions(Table table, StaticTableScan scan) {
+ Types.StructType partitionType = Partitioning.partitionType(table);
+ PartitionMap partitions = new PartitionMap(partitionType);
+ try (CloseableIterable>> entries = planEntries(scan)) {
+ for (ManifestEntry extends ContentFile>> entry : entries) {
+ Snapshot snapshot = table.snapshot(entry.snapshotId());
+ ContentFile> file = entry.file();
+ StructLike partition =
+ PartitionUtil.coercePartition(
+ partitionType, table.specs().get(file.specId()), file.partition());
+ partitions.get(partition).update(file, snapshot);
+ }
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+
+ return partitions.all();
+ }
+
+ @VisibleForTesting
+ static CloseableIterable> planEntries(StaticTableScan scan) {
+ Table table = scan.table();
+
+ CloseableIterable filteredManifests =
+ filteredManifests(scan, table, scan.snapshot().allManifests(table.io()));
+
+ Iterable>> tasks =
+ CloseableIterable.transform(filteredManifests, manifest -> readEntries(manifest, scan));
+
+ return new ParallelIterable<>(tasks, scan.planExecutor());
+ }
+
+ private static CloseableIterable> readEntries(
+ ManifestFile manifest, StaticTableScan scan) {
+ Table table = scan.table();
+ return CloseableIterable.transform(
+ ManifestFiles.open(manifest, table.io(), table.specs())
+ .caseSensitive(scan.isCaseSensitive())
+ .select(BaseScan.scanColumns(manifest.content())) // don't select stats columns
+ .liveEntries(),
+ t ->
+ (ManifestEntry extends ContentFile>>)
+ // defensive copy of manifest entry without stats columns
+ t.copyWithoutStats());
+ }
+
+ private static CloseableIterable filteredManifests(
+ StaticTableScan scan, Table table, List manifestFilesList) {
+ CloseableIterable manifestFiles =
+ CloseableIterable.withNoopClose(manifestFilesList);
+
+ LoadingCache evalCache =
+ Caffeine.newBuilder()
+ .build(
+ specId -> {
+ PartitionSpec spec = table.specs().get(specId);
+ PartitionSpec transformedSpec = transformSpec(scan.tableSchema(), spec);
+ return ManifestEvaluator.forRowFilter(
+ scan.filter(), transformedSpec, scan.isCaseSensitive());
+ });
+
+ return CloseableIterable.filter(
+ manifestFiles, manifest -> evalCache.get(manifest.partitionSpecId()).eval(manifest));
+ }
+
+ private class PartitionsScan extends StaticTableScan {
+ PartitionsScan(Table table) {
+ super(
+ table,
+ PartitionsTable.this.schema(),
+ MetadataTableType.PARTITIONS,
+ PartitionsTable.this::task);
+ }
+ }
+
+ static class PartitionMap {
+ private final StructLikeMap partitions;
+ private final Types.StructType keyType;
+
+ PartitionMap(Types.StructType type) {
+ this.partitions = StructLikeMap.create(type);
+ this.keyType = type;
+ }
+
+ Partition get(StructLike key) {
+ Partition partition = partitions.get(key);
+ if (partition == null) {
+ partition = new Partition(key, keyType);
+ partitions.put(key, partition);
+ }
+ return partition;
+ }
+
+ Iterable all() {
+ return partitions.values();
+ }
+ }
+
+ static class Partition {
+ private final PartitionData partitionData;
+ private int specId;
+ private long dataRecordCount;
+ private int dataFileCount;
+ private long dataFileSizeInBytes;
+ private long posDeleteRecordCount;
+ private int posDeleteFileCount;
+ private long eqDeleteRecordCount;
+ private int eqDeleteFileCount;
+ private Long lastUpdatedAt;
+ private Long lastUpdatedSnapshotId;
+
+ Partition(StructLike key, Types.StructType keyType) {
+ this.partitionData = toPartitionData(key, keyType);
+ this.specId = 0;
+ this.dataRecordCount = 0L;
+ this.dataFileCount = 0;
+ this.dataFileSizeInBytes = 0L;
+ this.posDeleteRecordCount = 0L;
+ this.posDeleteFileCount = 0;
+ this.eqDeleteRecordCount = 0L;
+ this.eqDeleteFileCount = 0;
+ }
+
+ void update(ContentFile> file, Snapshot snapshot) {
+ if (snapshot != null) {
+ long snapshotCommitTime = snapshot.timestampMillis() * 1000;
+ if (this.lastUpdatedAt == null || snapshotCommitTime > this.lastUpdatedAt) {
+ this.lastUpdatedAt = snapshotCommitTime;
+ this.lastUpdatedSnapshotId = snapshot.snapshotId();
+ }
+ }
+
+ switch (file.content()) {
+ case DATA:
+ this.dataRecordCount += file.recordCount();
+ this.dataFileCount += 1;
+ this.specId = file.specId();
+ this.dataFileSizeInBytes += file.fileSizeInBytes();
+ break;
+ case POSITION_DELETES:
+ this.posDeleteRecordCount += file.recordCount();
+ this.posDeleteFileCount += 1;
+ this.specId = file.specId();
+ break;
+ case EQUALITY_DELETES:
+ this.eqDeleteRecordCount += file.recordCount();
+ this.eqDeleteFileCount += 1;
+ this.specId = file.specId();
+ break;
+ default:
+ throw new UnsupportedOperationException(
+ "Unsupported file content type: " + file.content());
+ }
+ }
+
+ /** Needed because StructProjection is not serializable */
+ private PartitionData toPartitionData(StructLike key, Types.StructType keyType) {
+ PartitionData data = new PartitionData(keyType);
+ for (int i = 0; i < keyType.fields().size(); i++) {
+ Object val = key.get(i, keyType.fields().get(i).type().typeId().javaClass());
+ if (val != null) {
+ data.set(i, val);
+ }
+ }
+ return data;
+ }
+ }
+}
diff --git a/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/avro/BaseWriteBuilder.java b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/avro/BaseWriteBuilder.java
new file mode 100644
index 000000000000..34b9d9ce2980
--- /dev/null
+++ b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/avro/BaseWriteBuilder.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.avro;
+
+import java.util.List;
+import org.apache.avro.LogicalType;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+abstract class BaseWriteBuilder extends AvroSchemaVisitor> {
+
+ protected abstract ValueWriter> createRecordWriter(List> fields);
+
+ protected abstract ValueWriter> fixedWriter(int length);
+
+ @Override
+ public ValueWriter> record(Schema record, List names, List> fields) {
+ return createRecordWriter(fields);
+ }
+
+ @Override
+ public ValueWriter> union(Schema union, List> options) {
+ Preconditions.checkArgument(
+ options.size() == 2, "Cannot create writer for non-option union: %s", union);
+ if (union.getTypes().get(0).getType() == Schema.Type.NULL) {
+ return ValueWriters.option(0, options.get(1));
+ } else if (union.getTypes().get(1).getType() == Schema.Type.NULL) {
+ return ValueWriters.option(1, options.get(0));
+ } else {
+ throw new IllegalArgumentException(
+ String.format("Cannot create writer for non-option union: %s", union));
+ }
+ }
+
+ @Override
+ public ValueWriter> array(Schema array, ValueWriter> elementWriter) {
+ if (array.getLogicalType() instanceof LogicalMap) {
+ ValueWriters.StructWriter> keyValueWriter = (ValueWriters.StructWriter>) elementWriter;
+ return ValueWriters.arrayMap(keyValueWriter.writer(0), keyValueWriter.writer(1));
+ }
+
+ return ValueWriters.array(elementWriter);
+ }
+
+ @Override
+ public ValueWriter> map(Schema map, ValueWriter> valueWriter) {
+ return ValueWriters.map(ValueWriters.strings(), valueWriter);
+ }
+
+ @Override
+ public ValueWriter> primitive(Schema primitive) {
+ LogicalType logicalType = primitive.getLogicalType();
+ if (logicalType != null) {
+ switch (logicalType.getName()) {
+ case "date":
+ return ValueWriters.ints();
+
+ case "time-micros":
+ return ValueWriters.longs();
+
+ case "timestamp-micros":
+ return ValueWriters.longs();
+
+ case "decimal":
+ LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType;
+ return ValueWriters.decimal(decimal.getPrecision(), decimal.getScale());
+
+ case "uuid":
+ return ValueWriters.uuids();
+
+ default:
+ throw new IllegalArgumentException("Unsupported logical type: " + logicalType);
+ }
+ }
+
+ switch (primitive.getType()) {
+ case NULL:
+ return ValueWriters.nulls();
+ case BOOLEAN:
+ return ValueWriters.booleans();
+ case INT:
+ return ValueWriters.ints();
+ case LONG:
+ return ValueWriters.longs();
+ case FLOAT:
+ return ValueWriters.floats();
+ case DOUBLE:
+ return ValueWriters.doubles();
+ case STRING:
+ return ValueWriters.strings();
+ case FIXED:
+ return fixedWriter(primitive.getFixedSize());
+ case BYTES:
+ return ValueWriters.byteBuffers();
+ default:
+ throw new IllegalArgumentException("Unsupported type: " + primitive);
+ }
+ }
+}
diff --git a/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/avro/InternalReader.java b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/avro/InternalReader.java
new file mode 100644
index 000000000000..602b987d28bc
--- /dev/null
+++ b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/avro/InternalReader.java
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.avro;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.avro.LogicalType;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.Decoder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.Pair;
+
+/**
+ * A reader that produces Iceberg's internal in-memory object model.
+ *
+ * Iceberg's internal in-memory object model produces the types defined in {@link
+ * Type.TypeID#javaClass()}.
+ *
+ * @param Java type returned by the reader
+ */
+public class InternalReader implements DatumReader, SupportsRowPosition {
+ private static final int ROOT_ID = -1;
+
+ private final Types.StructType expectedType;
+ private final Map> typeMap = Maps.newHashMap();
+ private final Map idToConstant = ImmutableMap.of();
+ private Schema fileSchema = null;
+ private ValueReader reader = null;
+
+ public static InternalReader create(org.apache.iceberg.Schema schema) {
+ return new InternalReader<>(schema);
+ }
+
+ InternalReader(org.apache.iceberg.Schema readSchema) {
+ this.expectedType = readSchema.asStruct();
+ }
+
+ @SuppressWarnings("unchecked")
+ private void initReader() {
+ this.reader =
+ (ValueReader)
+ AvroWithPartnerVisitor.visit(
+ Pair.of(ROOT_ID, expectedType),
+ fileSchema,
+ new ResolvingReadBuilder(),
+ AccessByID.instance());
+ }
+
+ @Override
+ public void setSchema(Schema schema) {
+ this.fileSchema = schema;
+ initReader();
+ }
+
+ public InternalReader setRootType(Class extends StructLike> rootClass) {
+ typeMap.put(ROOT_ID, rootClass);
+ return this;
+ }
+
+ public InternalReader setCustomType(int fieldId, Class extends StructLike> structClass) {
+ typeMap.put(fieldId, structClass);
+ return this;
+ }
+
+ @Override
+ public void setRowPositionSupplier(Supplier posSupplier) {
+ if (reader instanceof SupportsRowPosition) {
+ ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier);
+ }
+ }
+
+ @Override
+ public T read(T reuse, Decoder decoder) throws IOException {
+ return reader.read(decoder, reuse);
+ }
+
+ private class ResolvingReadBuilder
+ extends AvroWithPartnerVisitor, ValueReader>> {
+ @Override
+ public ValueReader> record(
+ Pair partner, Schema record, List> fieldResults) {
+ if (partner == null) {
+ return ValueReaders.skipStruct(fieldResults);
+ }
+
+ Types.StructType expected = partner.second().asStructType();
+ List>> readPlan =
+ ValueReaders.buildReadPlan(expected, record, fieldResults, idToConstant);
+
+ return structReader(readPlan, partner.first(), expected);
+ }
+
+ private ValueReader> structReader(
+ List>> readPlan, int fieldId, Types.StructType struct) {
+
+ Class extends StructLike> structClass = typeMap.get(fieldId);
+ if (structClass != null) {
+ return InternalReaders.struct(struct, structClass, readPlan);
+ } else {
+ return InternalReaders.struct(struct, readPlan);
+ }
+ }
+
+ @Override
+ public ValueReader> union(
+ Pair partner, Schema union, List> options) {
+ return ValueReaders.union(options);
+ }
+
+ @Override
+ public ValueReader> arrayMap(
+ Pair partner,
+ Schema map,
+ ValueReader> keyReader,
+ ValueReader> valueReader) {
+ return ValueReaders.arrayMap(keyReader, valueReader);
+ }
+
+ @Override
+ public ValueReader> array(
+ Pair partner, Schema array, ValueReader> elementReader) {
+ return ValueReaders.array(elementReader);
+ }
+
+ @Override
+ public ValueReader> map(Pair partner, Schema map, ValueReader> valueReader) {
+ return ValueReaders.map(ValueReaders.strings(), valueReader);
+ }
+
+ @Override
+ public ValueReader> primitive(Pair partner, Schema primitive) {
+ LogicalType logicalType = primitive.getLogicalType();
+ if (logicalType != null) {
+ switch (logicalType.getName()) {
+ case "date":
+ return ValueReaders.ints();
+
+ case "time-micros":
+ return ValueReaders.longs();
+
+ case "timestamp-millis":
+ // adjust to microseconds
+ ValueReader longs = ValueReaders.longs();
+ return (ValueReader) (decoder, ignored) -> longs.read(decoder, null) * 1000L;
+
+ case "timestamp-micros":
+ return ValueReaders.longs();
+
+ case "decimal":
+ return ValueReaders.decimal(
+ ValueReaders.decimalBytesReader(primitive),
+ ((LogicalTypes.Decimal) logicalType).getScale());
+
+ case "uuid":
+ return ValueReaders.uuids();
+
+ default:
+ throw new IllegalArgumentException("Unknown logical type: " + logicalType);
+ }
+ }
+
+ switch (primitive.getType()) {
+ case NULL:
+ return ValueReaders.nulls();
+ case BOOLEAN:
+ return ValueReaders.booleans();
+ case INT:
+ if (partner != null && partner.second().typeId() == Type.TypeID.LONG) {
+ return ValueReaders.intsAsLongs();
+ }
+ return ValueReaders.ints();
+ case LONG:
+ return ValueReaders.longs();
+ case FLOAT:
+ if (partner != null && partner.second().typeId() == Type.TypeID.DOUBLE) {
+ return ValueReaders.floatsAsDoubles();
+ }
+ return ValueReaders.floats();
+ case DOUBLE:
+ return ValueReaders.doubles();
+ case STRING:
+ return ValueReaders.strings();
+ case FIXED:
+ case BYTES:
+ return ValueReaders.byteBuffers();
+ case ENUM:
+ return ValueReaders.enums(primitive.getEnumSymbols());
+ default:
+ throw new IllegalArgumentException("Unsupported type: " + primitive);
+ }
+ }
+ }
+
+ private static class AccessByID
+ implements AvroWithPartnerVisitor.PartnerAccessors> {
+ private static final AccessByID INSTANCE = new AccessByID();
+
+ public static AccessByID instance() {
+ return INSTANCE;
+ }
+
+ @Override
+ public Pair fieldPartner(
+ Pair partner, Integer fieldId, String name) {
+ Types.NestedField field = partner.second().asStructType().field(fieldId);
+ return field != null ? Pair.of(field.fieldId(), field.type()) : null;
+ }
+
+ @Override
+ public Pair mapKeyPartner(Pair partner) {
+ Types.MapType map = partner.second().asMapType();
+ return Pair.of(map.keyId(), map.keyType());
+ }
+
+ @Override
+ public Pair mapValuePartner(Pair partner) {
+ Types.MapType map = partner.second().asMapType();
+ return Pair.of(map.valueId(), map.valueType());
+ }
+
+ @Override
+ public Pair listElementPartner(Pair partner) {
+ Types.ListType list = partner.second().asListType();
+ return Pair.of(list.elementId(), list.elementType());
+ }
+ }
+}
diff --git a/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/avro/InternalReaders.java b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/avro/InternalReaders.java
new file mode 100644
index 000000000000..b426d9b53942
--- /dev/null
+++ b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/avro/InternalReaders.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.avro;
+
+import java.util.List;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.common.DynConstructors;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.Pair;
+
+class InternalReaders {
+ private InternalReaders() {
+ }
+
+ static ValueReader extends Record> struct(
+ Types.StructType struct, List>> readPlan) {
+ return new RecordReader(readPlan, struct);
+ }
+
+ static ValueReader struct(
+ Types.StructType struct, Class structClass, List>> readPlan) {
+ return new PlannedStructLikeReader<>(readPlan, struct, structClass);
+ }
+
+ private static class PlannedStructLikeReader
+ extends ValueReaders.PlannedStructReader {
+ private final Types.StructType structType;
+ private final Class structClass;
+ private final DynConstructors.Ctor ctor;
+
+ private PlannedStructLikeReader(
+ List>> readPlan,
+ Types.StructType structType,
+ Class structClass) {
+ super(readPlan);
+ this.structType = structType;
+ this.structClass = structClass;
+ this.ctor =
+ DynConstructors.builder(StructLike.class)
+ .hiddenImpl(structClass, Types.StructType.class)
+ .hiddenImpl(structClass)
+ .build();
+ }
+
+ @Override
+ protected S reuseOrCreate(Object reuse) {
+ if (structClass.isInstance(reuse)) {
+ return structClass.cast(reuse);
+ } else {
+ return ctor.newInstance(structType);
+ }
+ }
+
+ @Override
+ protected Object get(S struct, int pos) {
+ return struct.get(pos, Object.class);
+ }
+
+ @Override
+ protected void set(S struct, int pos, Object value) {
+ struct.set(pos, value);
+ }
+ }
+
+ private static class RecordReader extends ValueReaders.PlannedStructReader {
+ private final Types.StructType structType;
+
+ private RecordReader(
+ List>> readPlan, Types.StructType structType) {
+ super(readPlan);
+ this.structType = structType;
+ }
+
+ @Override
+ protected GenericRecord reuseOrCreate(Object reuse) {
+ if (reuse instanceof GenericRecord) {
+ return (GenericRecord) reuse;
+ } else {
+ return GenericRecord.create(structType);
+ }
+ }
+
+ @Override
+ protected Object get(GenericRecord struct, int pos) {
+ return struct.get(pos);
+ }
+
+ @Override
+ protected void set(GenericRecord struct, int pos, Object value) {
+ struct.set(pos, value);
+ }
+ }
+}
diff --git a/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/avro/InternalWriter.java b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/avro/InternalWriter.java
new file mode 100644
index 000000000000..0b6a406247ad
--- /dev/null
+++ b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/avro/InternalWriter.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.avro;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.stream.Stream;
+import org.apache.avro.Schema;
+import org.apache.avro.io.Encoder;
+import org.apache.iceberg.FieldMetrics;
+import org.apache.iceberg.types.Type;
+
+/**
+ * A Writer that consumes Iceberg's internal in-memory object model.
+ *
+ * Iceberg's internal in-memory object model produces the types defined in {@link
+ * Type.TypeID#javaClass()}.
+ */
+public class InternalWriter implements MetricsAwareDatumWriter {
+ private ValueWriter writer = null;
+
+ public static InternalWriter create(Schema schema) {
+ return new InternalWriter<>(schema);
+ }
+
+ InternalWriter(Schema schema) {
+ setSchema(schema);
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void setSchema(Schema schema) {
+ this.writer = (ValueWriter) AvroSchemaVisitor.visit(schema, new WriteBuilder());
+ }
+
+ @Override
+ public void write(T datum, Encoder out) throws IOException {
+ writer.write(datum, out);
+ }
+
+ @Override
+ public Stream metrics() {
+ return writer.metrics();
+ }
+
+ private static class WriteBuilder extends BaseWriteBuilder {
+
+ @Override
+ protected ValueWriter> createRecordWriter(List> fields) {
+ return ValueWriters.struct(fields);
+ }
+
+ @Override
+ protected ValueWriter> fixedWriter(int length) {
+ return ValueWriters.fixedBuffers(length);
+ }
+ }
+}
diff --git a/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/avro/ValueReaders.java b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/avro/ValueReaders.java
new file mode 100644
index 000000000000..080bc0b68d17
--- /dev/null
+++ b/iceberg/patched-iceberg-core/src/main/java/org/apache/iceberg/avro/ValueReaders.java
@@ -0,0 +1,1206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.avro;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.function.BiFunction;
+import java.util.function.Supplier;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.ResolvingDecoder;
+import org.apache.avro.util.Utf8;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.common.DynConstructors;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.UUIDUtil;
+
+import static java.util.Collections.emptyIterator;
+
+public class ValueReaders {
+ private ValueReaders() {
+ }
+
+ public static ValueReader nulls() {
+ return NullReader.INSTANCE;
+ }
+
+ public static ValueReader constant(T value) {
+ return new ConstantReader<>(value);
+ }
+
+ public static ValueReader replaceWithConstant(ValueReader> reader, T value) {
+ return new ReplaceWithConstantReader<>(reader, value);
+ }
+
+ public static ValueReader booleans() {
+ return BooleanReader.INSTANCE;
+ }
+
+ public static ValueReader ints() {
+ return IntegerReader.INSTANCE;
+ }
+
+ public static ValueReader intsAsLongs() {
+ return IntegerAsLongReader.INSTANCE;
+ }
+
+ public static ValueReader longs() {
+ return LongReader.INSTANCE;
+ }
+
+ public static ValueReader floats() {
+ return FloatReader.INSTANCE;
+ }
+
+ public static ValueReader floatsAsDoubles() {
+ return FloatAsDoubleReader.INSTANCE;
+ }
+
+ public static ValueReader doubles() {
+ return DoubleReader.INSTANCE;
+ }
+
+ public static ValueReader strings() {
+ return StringReader.INSTANCE;
+ }
+
+ public static ValueReader utf8s() {
+ return Utf8Reader.INSTANCE;
+ }
+
+ public static ValueReader enums(List