IMPALA-12598: Allow multiple equality field id lists for Iceberg tables

This patch adds support for reading Iceberg tables that have
different equality field ID lists associated to different equality
delete files. In practice this is a use case when one equality delete
file deletes by e.g. columnA and columnB while another one deletes by
columnB and columnC.

In order to achieve such functionality the plan tree creation needed
some adjustments so that it can create separate LEFT ANTI JOIN nodes
for the different equality field ID lists.

Testing:
  - Flink and NiFi was used for creating some test tables with the
    desired equality field IDs. Coverage on these tables are added to
    the test suite.

Change-Id: I3e52d7a5800bf1b479f0c234679be92442d09f79
Reviewed-on: http://gerrit.cloudera.org:8080/20951
Reviewed-by: Gabor Kaszab <gaborkaszab@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
This commit is contained in:
Gabor Kaszab
2024-01-23 11:14:18 +01:00
committed by Impala Public Jenkins
parent b81368a225
commit 65094a74f1
73 changed files with 1574 additions and 347 deletions

View File

@@ -50,6 +50,7 @@ table FbIcebergMetadata {
data_sequence_number : long; data_sequence_number : long;
spec_id : ushort; spec_id : ushort;
partition_keys : [FbIcebergPartitionTransformValue]; partition_keys : [FbIcebergPartitionTransformValue];
equality_field_ids : [int];
} }
table FbIcebergColumnStats { table FbIcebergColumnStats {

View File

@@ -630,10 +630,9 @@ struct TIcebergContentFileStore {
2: optional map<string, THdfsFileDesc> path_hash_to_data_file_with_deletes 2: optional map<string, THdfsFileDesc> path_hash_to_data_file_with_deletes
3: optional map<string, THdfsFileDesc> path_hash_to_position_delete_file 3: optional map<string, THdfsFileDesc> path_hash_to_position_delete_file
4: optional map<string, THdfsFileDesc> path_hash_to_equality_delete_file 4: optional map<string, THdfsFileDesc> path_hash_to_equality_delete_file
5: optional list<i32> equality_ids 5: optional bool has_avro
6: optional bool has_avro 6: optional bool has_orc
7: optional bool has_orc 7: optional bool has_parquet
8: optional bool has_parquet
} }
// Represents a drop partition request for Iceberg tables // Represents a drop partition request for Iceberg tables

View File

@@ -789,8 +789,7 @@ public interface FeIcebergTable extends FeFsTable {
} }
for (DeleteFile deleteFile : icebergFiles.equalityDeleteFiles) { for (DeleteFile deleteFile : icebergFiles.equalityDeleteFiles) {
pathHashAndFd = getPathHashAndFd(deleteFile, table, hdfsFileDescMap); pathHashAndFd = getPathHashAndFd(deleteFile, table, hdfsFileDescMap);
fileStore.addEqualityDeleteFile(pathHashAndFd.first, pathHashAndFd.second, fileStore.addEqualityDeleteFile(pathHashAndFd.first, pathHashAndFd.second);
deleteFile.equalityFieldIds());
} }
return fileStore; return fileStore;
} }

View File

@@ -21,17 +21,13 @@ import com.google.common.collect.Iterables;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
import hiveexec.com.google.common.collect.Lists;
import org.apache.curator.shaded.com.google.common.base.Preconditions; import org.apache.curator.shaded.com.google.common.base.Preconditions;
import org.apache.impala.catalog.HdfsPartition.FileDescriptor; import org.apache.impala.catalog.HdfsPartition.FileDescriptor;
import org.apache.impala.common.ImpalaRuntimeException;
import org.apache.impala.fb.FbIcebergDataFileFormat; import org.apache.impala.fb.FbIcebergDataFileFormat;
import org.apache.impala.thrift.THdfsFileDesc; import org.apache.impala.thrift.THdfsFileDesc;
import org.apache.impala.thrift.TIcebergContentFileStore; import org.apache.impala.thrift.TIcebergContentFileStore;
@@ -102,12 +98,6 @@ public class IcebergContentFileStore {
private MapListContainer positionDeleteFiles_ = new MapListContainer(); private MapListContainer positionDeleteFiles_ = new MapListContainer();
private MapListContainer equalityDeleteFiles_ = new MapListContainer(); private MapListContainer equalityDeleteFiles_ = new MapListContainer();
// The equality field IDs associated with the equality delete files.
// TODO IMPALA-12598: currently it's not supported to have equality delete files with
// different equality field ID lists, or in other words the equality delete files have
// to delete by the same columns.
private Set<Integer> equalityIds_ = new HashSet<>();
// Caches file descriptors loaded during time-travel queries. // Caches file descriptors loaded during time-travel queries.
private final ConcurrentMap<String, FileDescriptor> oldFileDescMap_ = private final ConcurrentMap<String, FileDescriptor> oldFileDescMap_ =
new ConcurrentHashMap<>(); new ConcurrentHashMap<>();
@@ -137,20 +127,9 @@ public class IcebergContentFileStore {
} }
} }
public void addEqualityDeleteFile(String pathHash, FileDescriptor desc, public void addEqualityDeleteFile(String pathHash, FileDescriptor desc) {
List<Integer> equalityIds) throws ImpalaRuntimeException { Preconditions.checkState(
if (equalityIds.isEmpty()) { desc.getFbFileMetadata().icebergMetadata().equalityFieldIdsLength() > 0);
throw new ImpalaRuntimeException("No equality IDs for equality delete file " +
desc.getPath());
}
if (equalityIds_.isEmpty()) {
equalityIds_.addAll(equalityIds);
} else if (equalityIds_.size() != equalityIds.size() ||
!equalityIds_.containsAll(equalityIds)) {
throw new ImpalaRuntimeException(String.format("Equality delete files with " +
"different equality field ID lists aren't supported. %s vs %s", equalityIds,
equalityIds_));
}
if (equalityDeleteFiles_.add(pathHash, desc)) updateFileFormats(desc); if (equalityDeleteFiles_.add(pathHash, desc)) updateFileFormats(desc);
} }
@@ -192,8 +171,6 @@ public class IcebergContentFileStore {
return equalityDeleteFiles_.getList(); return equalityDeleteFiles_.getList();
} }
public Set<Integer> getEqualityIds() { return equalityIds_; }
public long getNumFiles() { public long getNumFiles() {
return dataFilesWithoutDeletes_.getNumFiles() + return dataFilesWithoutDeletes_.getNumFiles() +
dataFilesWithDeletes_.getNumFiles() + dataFilesWithDeletes_.getNumFiles() +
@@ -236,7 +213,6 @@ public class IcebergContentFileStore {
ret.setPath_hash_to_data_file_with_deletes(dataFilesWithDeletes_.toThrift()); ret.setPath_hash_to_data_file_with_deletes(dataFilesWithDeletes_.toThrift());
ret.setPath_hash_to_position_delete_file(positionDeleteFiles_.toThrift()); ret.setPath_hash_to_position_delete_file(positionDeleteFiles_.toThrift());
ret.setPath_hash_to_equality_delete_file(equalityDeleteFiles_.toThrift()); ret.setPath_hash_to_equality_delete_file(equalityDeleteFiles_.toThrift());
ret.setEquality_ids(Lists.newArrayList(equalityIds_));
ret.setHas_avro(hasAvro_); ret.setHas_avro(hasAvro_);
ret.setHas_orc(hasOrc_); ret.setHas_orc(hasOrc_);
ret.setHas_parquet(hasParquet_); ret.setHas_parquet(hasParquet_);
@@ -267,7 +243,6 @@ public class IcebergContentFileStore {
tFileStore.getPath_hash_to_equality_delete_file(), tFileStore.getPath_hash_to_equality_delete_file(),
networkAddresses, hostIndex); networkAddresses, hostIndex);
} }
ret.equalityIds_.addAll(tFileStore.getEquality_ids());
ret.hasAvro_ = tFileStore.isSetHas_avro() ? tFileStore.isHas_avro() : false; ret.hasAvro_ = tFileStore.isSetHas_avro() ? tFileStore.isHas_avro() : false;
ret.hasOrc_ = tFileStore.isSetHas_orc() ? tFileStore.isHas_orc() : false; ret.hasOrc_ = tFileStore.isSetHas_orc() ? tFileStore.isHas_orc() : false;
ret.hasParquet_ = tFileStore.isSetHas_parquet() ? tFileStore.isHas_parquet() : false; ret.hasParquet_ = tFileStore.isSetHas_parquet() ? tFileStore.isHas_parquet() : false;

View File

@@ -36,7 +36,7 @@ import org.apache.iceberg.types.Types.NestedField;
public class IcebergEqualityDeleteTable extends IcebergDeleteTable { public class IcebergEqualityDeleteTable extends IcebergDeleteTable {
public IcebergEqualityDeleteTable(FeIcebergTable baseTable, String name, public IcebergEqualityDeleteTable(FeIcebergTable baseTable, String name,
Set<FileDescriptor> deleteFiles, Set<Integer> equalityIds, long deleteRecordsCount) Set<FileDescriptor> deleteFiles, List<Integer> equalityIds, long deleteRecordsCount)
throws ImpalaRuntimeException { throws ImpalaRuntimeException {
super(baseTable, name, deleteFiles, deleteRecordsCount); super(baseTable, name, deleteFiles, deleteRecordsCount);

View File

@@ -32,6 +32,8 @@ import java.util.Set;
import java.util.TreeSet; import java.util.TreeSet;
import java.util.function.Predicate; import java.util.function.Predicate;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.curator.shaded.com.google.common.collect.Lists; import org.apache.curator.shaded.com.google.common.collect.Lists;
import org.apache.iceberg.ContentFile; import org.apache.iceberg.ContentFile;
import org.apache.iceberg.DeleteFile; import org.apache.iceberg.DeleteFile;
@@ -76,6 +78,7 @@ import org.apache.impala.common.IcebergPredicateConverter;
import org.apache.impala.common.ImpalaException; import org.apache.impala.common.ImpalaException;
import org.apache.impala.common.ImpalaRuntimeException; import org.apache.impala.common.ImpalaRuntimeException;
import org.apache.impala.common.Pair; import org.apache.impala.common.Pair;
import org.apache.impala.fb.FbIcebergMetadata;
import org.apache.impala.planner.JoinNode.DistributionMode; import org.apache.impala.planner.JoinNode.DistributionMode;
import org.apache.impala.thrift.TColumnStats; import org.apache.impala.thrift.TColumnStats;
import org.apache.impala.thrift.TIcebergPartitionTransformType; import org.apache.impala.thrift.TIcebergPartitionTransformType;
@@ -117,16 +120,21 @@ public class IcebergScanPlanner {
private List<FileDescriptor> dataFilesWithoutDeletes_ = new ArrayList<>(); private List<FileDescriptor> dataFilesWithoutDeletes_ = new ArrayList<>();
private List<FileDescriptor> dataFilesWithDeletes_ = new ArrayList<>(); private List<FileDescriptor> dataFilesWithDeletes_ = new ArrayList<>();
private Set<FileDescriptor> positionDeleteFiles_ = new HashSet<>(); private Set<FileDescriptor> positionDeleteFiles_ = new HashSet<>();
private Set<FileDescriptor> equalityDeleteFiles_ = new HashSet<>();
// The equality field IDs to be used for the equality delete files. // Holds all the equalityFieldIds from the equality delete file descriptors involved in
private Set<Integer> equalityIds_ = new HashSet<>(); // this query.
private Set<Integer> allEqualityFieldIds_ = new HashSet<>();
private Map<List<Integer>, Set<FileDescriptor>> equalityIdsToDeleteFiles_ =
new HashMap<>();
// Statistics about the data and delete files. Useful for memory estimates of the // Statistics about the data and delete files. Useful for memory estimates of the
// ANTI JOIN // ANTI JOIN
private long positionDeletesRecordCount_ = 0; private long positionDeletesRecordCount_ = 0;
private long equalityDeletesRecordCount_ = 0;
private long dataFilesWithDeletesSumPaths_ = 0; private long dataFilesWithDeletesSumPaths_ = 0;
private long dataFilesWithDeletesMaxPath_ = 0; private long dataFilesWithDeletesMaxPath_ = 0;
// Stores how many delete records are involved broken down by equality field ID lists.
private Map<List<Integer>, Long> equalityDeletesRecordCount_ = new HashMap<>();
private Set<Long> equalityDeleteSequenceNumbers_ = new HashSet<>(); private Set<Long> equalityDeleteSequenceNumbers_ = new HashSet<>();
private final long snapshotId_; private final long snapshotId_;
@@ -174,14 +182,13 @@ public class IcebergScanPlanner {
dataFilesWithoutDeletes_ = fileStore.getDataFilesWithoutDeletes(); dataFilesWithoutDeletes_ = fileStore.getDataFilesWithoutDeletes();
dataFilesWithDeletes_ = fileStore.getDataFilesWithDeletes(); dataFilesWithDeletes_ = fileStore.getDataFilesWithDeletes();
positionDeleteFiles_ = new HashSet<>(fileStore.getPositionDeleteFiles()); positionDeleteFiles_ = new HashSet<>(fileStore.getPositionDeleteFiles());
equalityDeleteFiles_ = new HashSet<>(fileStore.getEqualityDeleteFiles()); initEqualityIds(fileStore.getEqualityDeleteFiles());
equalityIds_ = fileStore.getEqualityIds();
updateDeleteStatistics(); updateDeleteStatistics();
} }
private boolean noDeleteFiles() { private boolean noDeleteFiles() {
return positionDeleteFiles_.isEmpty() && equalityDeleteFiles_.isEmpty(); return positionDeleteFiles_.isEmpty() && equalityIdsToDeleteFiles_.isEmpty();
} }
private PlanNode createIcebergScanPlanImpl() throws ImpalaException { private PlanNode createIcebergScanPlanImpl() throws ImpalaException {
@@ -198,7 +205,7 @@ public class IcebergScanPlanner {
PlanNode joinNode = null; PlanNode joinNode = null;
if (!positionDeleteFiles_.isEmpty()) joinNode = createPositionJoinNode(); if (!positionDeleteFiles_.isEmpty()) joinNode = createPositionJoinNode();
if (!equalityDeleteFiles_.isEmpty()) joinNode = createEqualityJoinNode(joinNode); if (!equalityIdsToDeleteFiles_.isEmpty()) joinNode = createEqualityJoinNode(joinNode);
Preconditions.checkNotNull(joinNode); Preconditions.checkNotNull(joinNode);
// If the count star query can be optimized for Iceberg V2 table, the number of rows // If the count star query can be optimized for Iceberg V2 table, the number of rows
@@ -245,7 +252,7 @@ public class IcebergScanPlanner {
Arrays.asList(deleteTable.getDb().getName(), deleteTable.getName()), Arrays.asList(deleteTable.getDb().getName(), deleteTable.getName()),
tblRef_.getUniqueAlias() + "-position-delete"); tblRef_.getUniqueAlias() + "-position-delete");
addDataVirtualPositionSlots(tblRef_); addDataVirtualPositionSlots(tblRef_);
if (!equalityDeleteFiles_.isEmpty()) addSlotsForEqualityDelete(equalityIds_, tblRef_); if (!equalityIdsToDeleteFiles_.isEmpty()) addAllSlotsForEqualityDeletes(tblRef_);
addDeletePositionSlots(deleteDeltaRef); addDeletePositionSlots(deleteDeltaRef);
IcebergScanNode dataScanNode = new IcebergScanNode( IcebergScanNode dataScanNode = new IcebergScanNode(
dataScanNodeId, tblRef_, conjuncts_, aggInfo_, dataFilesWithDeletes_, dataScanNodeId, tblRef_, conjuncts_, aggInfo_, dataFilesWithDeletes_,
@@ -296,15 +303,20 @@ public class IcebergScanPlanner {
filePosSlotDesc.setStats(virtualFilePositionStats()); filePosSlotDesc.setStats(virtualFilePositionStats());
} }
private void addSlotsForEqualityDelete(Set<Integer> equalityIds, TableRef tblRef) private void addAllSlotsForEqualityDeletes(TableRef tblRef) throws AnalysisException {
addSlotsForEqualityDeletes(Lists.newArrayList(allEqualityFieldIds_), tblRef);
}
private void addSlotsForEqualityDeletes(List<Integer> equalityFieldIds, TableRef tblRef)
throws AnalysisException { throws AnalysisException {
Preconditions.checkState(!equalityFieldIds.isEmpty());
List<String> rawPath = Lists.newArrayList( List<String> rawPath = Lists.newArrayList(
tblRef.getUniqueAlias(), VirtualColumn.ICEBERG_DATA_SEQUENCE_NUMBER.getName()); tblRef.getUniqueAlias(), VirtualColumn.ICEBERG_DATA_SEQUENCE_NUMBER.getName());
SlotDescriptor slotDesc = SingleNodePlanner.addSlotRefToDesc(analyzer_, rawPath); SlotDescriptor slotDesc = SingleNodePlanner.addSlotRefToDesc(analyzer_, rawPath);
slotDesc.setStats(virtualDataSeqNumStats()); slotDesc.setStats(virtualDataSeqNumStats());
Preconditions.checkState(!equalityIds.isEmpty()); for (Integer eqId : equalityFieldIds) {
for (Integer eqId : equalityIds) {
String eqColName = getIceTable().getIcebergSchema().findColumnName(eqId); String eqColName = getIceTable().getIcebergSchema().findColumnName(eqId);
Preconditions.checkNotNull(eqColName); Preconditions.checkNotNull(eqColName);
rawPath = Lists.newArrayList(tblRef.getUniqueAlias(), eqColName); rawPath = Lists.newArrayList(tblRef.getUniqueAlias(), eqColName);
@@ -424,8 +436,7 @@ public class IcebergScanPlanner {
private PlanNode createEqualityJoinNode(PlanNode positionJoinNode) private PlanNode createEqualityJoinNode(PlanNode positionJoinNode)
throws ImpalaException { throws ImpalaException {
Preconditions.checkState(!equalityDeleteFiles_.isEmpty()); Preconditions.checkState(!equalityIdsToDeleteFiles_.isEmpty());
Preconditions.checkState(equalityDeletesRecordCount_ > 0);
if (getIceTable().getPartitionSpecs().size() > 1) { if (getIceTable().getPartitionSpecs().size() > 1) {
throw new ImpalaRuntimeException("Equality delete files are not supported for " + throw new ImpalaRuntimeException("Equality delete files are not supported for " +
@@ -440,36 +451,46 @@ public class IcebergScanPlanner {
IcebergScanNode dataScanNode = new IcebergScanNode( IcebergScanNode dataScanNode = new IcebergScanNode(
dataScanNodeId, tblRef_, conjuncts_, aggInfo_, dataFilesWithDeletes_, dataScanNodeId, tblRef_, conjuncts_, aggInfo_, dataFilesWithDeletes_,
nonIdentityConjuncts_, getSkippedConjuncts(), snapshotId_); nonIdentityConjuncts_, getSkippedConjuncts(), snapshotId_);
addSlotsForEqualityDelete(equalityIds_, tblRef_); addAllSlotsForEqualityDeletes(tblRef_);
dataScanNode.init(analyzer_); dataScanNode.init(analyzer_);
leftSideOfJoin = dataScanNode; leftSideOfJoin = dataScanNode;
} }
List<List<Integer>> orderedEqualityFieldIds =
getOrderedEqualityFieldIds(equalityDeletesRecordCount_);
JoinNode joinNode = null; JoinNode joinNode = null;
for (List<Integer> equalityIds : orderedEqualityFieldIds) {
Set<FileDescriptor> equalityDeleteFiles =
equalityIdsToDeleteFiles_.get(equalityIds);
Preconditions.checkState(equalityDeleteFiles != null &&
!equalityDeleteFiles.isEmpty());
Long numRecordsInDeletes = equalityDeletesRecordCount_.get(equalityIds);
Preconditions.checkState(numRecordsInDeletes != null && numRecordsInDeletes > 0);
PlanNodeId deleteScanNodeId = ctx_.getNextNodeId(); PlanNodeId deleteScanNodeId = ctx_.getNextNodeId();
IcebergEqualityDeleteTable deleteTable = IcebergEqualityDeleteTable deleteTable =
new IcebergEqualityDeleteTable(getIceTable(), new IcebergEqualityDeleteTable(getIceTable(),
getIceTable().getName() + "-EQUALITY-DELETE-" + deleteScanNodeId.toString(), getIceTable().getName() + "-EQUALITY-DELETE-" + deleteScanNodeId.toString(),
equalityDeleteFiles_, equalityIds_, equalityDeletesRecordCount_); equalityDeleteFiles, equalityIds, numRecordsInDeletes);
analyzer_.addVirtualTable(deleteTable); analyzer_.addVirtualTable(deleteTable);
TableRef deleteTblRef = TableRef.newTableRef(analyzer_, TableRef deleteTblRef = TableRef.newTableRef(analyzer_,
Arrays.asList(deleteTable.getDb().getName(), deleteTable.getName()), Arrays.asList(deleteTable.getDb().getName(), deleteTable.getName()),
tblRef_.getUniqueAlias() + "-equality-delete-" + deleteScanNodeId.toString()); tblRef_.getUniqueAlias() + "-equality-delete-" + deleteScanNodeId.toString());
addSlotsForEqualityDelete(equalityIds_, deleteTblRef); addSlotsForEqualityDeletes(equalityIds, deleteTblRef);
// TODO IMPALA-12608: As an optimization we can populate the conjuncts below that are // TODO IMPALA-12608: As an optimization we can populate the conjuncts below that
// relevant for the delete scan node. // are relevant for the delete scan node.
IcebergScanNode deleteScanNode = new IcebergScanNode( IcebergScanNode deleteScanNode = new IcebergScanNode(
deleteScanNodeId, deleteScanNodeId,
deleteTblRef, deleteTblRef,
Collections.emptyList(), /*conjuncts*/ Collections.emptyList(), /*conjuncts*/
aggInfo_, aggInfo_,
Lists.newArrayList(equalityDeleteFiles_), Lists.newArrayList(equalityDeleteFiles),
Collections.emptyList(), /*nonIdentityConjuncts*/ Collections.emptyList(), /*nonIdentityConjuncts*/
Collections.emptyList(), /*skippedConjuncts*/ Collections.emptyList(),
snapshotId_); snapshotId_); /*skippedConjuncts*/
deleteScanNode.init(analyzer_); deleteScanNode.init(analyzer_);
Pair<List<BinaryPredicate>, List<Expr>> equalityJoinConjuncts = Pair<List<BinaryPredicate>, List<Expr>> equalityJoinConjuncts =
@@ -481,10 +502,49 @@ public class IcebergScanPlanner {
equalityJoinConjuncts.first, equalityJoinConjuncts.second); equalityJoinConjuncts.first, equalityJoinConjuncts.second);
joinNode.setId(ctx_.getNextNodeId()); joinNode.setId(ctx_.getNextNodeId());
joinNode.init(analyzer_); joinNode.init(analyzer_);
leftSideOfJoin = joinNode;
}
return joinNode; return joinNode;
} }
/**
* Based on the equality delete fields ID lists and the number of delete rows associated
* to them this function gives back the equality field ID lists in an order we'd like to
* use them for join node creation.
* The more delete rows are involved the earlier the equality field ID list is placed in
* the ordering. If some equality field ID lists have the same amount of delete rows
* then the order between them is decided by preferring the longest lists first and in
* case of equal length the numerical order of the field IDs in the lists is used.
* E.g. If they have the same amount of delete rows then [1,2] comes before [1] because
* of the length and comes before [2,3] because of the numerical order.
*/
static List<List<Integer>> getOrderedEqualityFieldIds(
Map<List<Integer>, Long> equalityDeletesRecordCount) {
Preconditions.checkState(!equalityDeletesRecordCount.isEmpty());
return equalityDeletesRecordCount.entrySet().stream()
.sorted(Map.Entry.<List<Integer>, Long>comparingByValue().reversed()
.thenComparing((e1, e2) -> {
List<Integer> list1 = e1.getKey();
List<Integer> list2 = e2.getKey();
// Order the longest equality field ID lists first.
if (list1.size() < list2.size()) return 1;
if (list2.size() < list1.size()) return -1;
for (int i = 0; i < list1.size(); ++i) {
if (list1.get(i) < list2.get(i)) return -1;
if (list2.get(i) < list1.get(i)) return 1;
}
return 0;
}))
.map(e -> e.getKey())
.collect(Collectors.toList());
}
private void filterFileDescriptors() throws ImpalaException { private void filterFileDescriptors() throws ImpalaException {
Preconditions.checkState(allEqualityFieldIds_.isEmpty());
Preconditions.checkState(equalityIdsToDeleteFiles_.isEmpty());
TimeTravelSpec timeTravelSpec = tblRef_.getTimeTravelSpec(); TimeTravelSpec timeTravelSpec = tblRef_.getTimeTravelSpec();
try (CloseableIterable<FileScanTask> fileScanTasks = try (CloseableIterable<FileScanTask> fileScanTasks =
@@ -506,8 +566,7 @@ public class IcebergScanPlanner {
Pair<FileDescriptor, Boolean> delFileDesc = getFileDescriptor(delFile); Pair<FileDescriptor, Boolean> delFileDesc = getFileDescriptor(delFile);
if (!delFileDesc.second) ++dataFilesCacheMisses; if (!delFileDesc.second) ++dataFilesCacheMisses;
if (delFile.content() == FileContent.EQUALITY_DELETES) { if (delFile.content() == FileContent.EQUALITY_DELETES) {
equalityDeleteFiles_.add(delFileDesc.first); addEqualityDeletesAndIds(delFileDesc.first);
addEqualityIds(delFile.equalityFieldIds());
} else { } else {
Preconditions.checkState(delFile.content() == FileContent.POSITION_DELETES); Preconditions.checkState(delFile.content() == FileContent.POSITION_DELETES);
positionDeleteFiles_.add(delFileDesc.first); positionDeleteFiles_.add(delFileDesc.first);
@@ -528,17 +587,23 @@ public class IcebergScanPlanner {
updateDeleteStatistics(); updateDeleteStatistics();
} }
private void addEqualityDeletesAndIds(FileDescriptor fd) {
private void addEqualityIds(List<Integer> equalityFieldIds) FbIcebergMetadata fileMetadata = fd.getFbFileMetadata().icebergMetadata();
throws ImpalaRuntimeException { List<Integer> eqFieldIdList = new ArrayList<>();
if (equalityIds_.isEmpty()) { for (int i = 0; i < fileMetadata.equalityFieldIdsLength(); ++i) {
equalityIds_.addAll(equalityFieldIds); eqFieldIdList.add(fileMetadata.equalityFieldIds(i));
} else if (equalityIds_.size() != equalityFieldIds.size() || allEqualityFieldIds_.add(fileMetadata.equalityFieldIds(i));
!equalityIds_.containsAll(equalityFieldIds)) {
throw new ImpalaRuntimeException(String.format("Equality delete files with " +
"different equality field ID lists aren't supported. %s vs %s", equalityIds_,
equalityFieldIds));
} }
if (!equalityIdsToDeleteFiles_.containsKey(eqFieldIdList)) {
equalityIdsToDeleteFiles_.put(eqFieldIdList, new HashSet<>());
}
equalityIdsToDeleteFiles_.get(eqFieldIdList).add(fd);
}
private void initEqualityIds(List<FileDescriptor> equalityDeleteFiles) {
Preconditions.checkState(allEqualityFieldIds_.isEmpty());
Preconditions.checkState(equalityIdsToDeleteFiles_.isEmpty());
for (FileDescriptor fd : equalityDeleteFiles) addEqualityDeletesAndIds(fd);
} }
private void filterConjuncts() { private void filterConjuncts() {
@@ -585,8 +650,9 @@ public class IcebergScanPlanner {
for (FileDescriptor fd : positionDeleteFiles_) { for (FileDescriptor fd : positionDeleteFiles_) {
updatePositionDeleteFilesStatistics(fd); updatePositionDeleteFilesStatistics(fd);
} }
for (FileDescriptor fd : equalityDeleteFiles_) { for (Map.Entry<List<Integer>, Set<FileDescriptor>> entry :
updateEqualityDeleteFilesStatistics(fd); equalityIdsToDeleteFiles_.entrySet()) {
updateEqualityDeleteFilesStatistics(entry.getKey(), entry.getValue());
} }
} }
@@ -603,11 +669,16 @@ public class IcebergScanPlanner {
positionDeletesRecordCount_ += getRecordCount(fd); positionDeletesRecordCount_ += getRecordCount(fd);
} }
private void updateEqualityDeleteFilesStatistics(FileDescriptor fd) { private void updateEqualityDeleteFilesStatistics(List<Integer> equalityIds,
equalityDeletesRecordCount_ += getRecordCount(fd); Set<FileDescriptor> fileDescriptors) {
long numRecords = 0;
for (FileDescriptor fd : fileDescriptors) {
numRecords += getRecordCount(fd);
equalityDeleteSequenceNumbers_.add( equalityDeleteSequenceNumbers_.add(
fd.getFbFileMetadata().icebergMetadata().dataSequenceNumber()); fd.getFbFileMetadata().icebergMetadata().dataSequenceNumber());
} }
equalityDeletesRecordCount_.put(equalityIds, numRecords);
}
private long getRecordCount(FileDescriptor fd) { private long getRecordCount(FileDescriptor fd) {
long recordCount = fd.getFbFileMetadata().icebergMetadata().recordCount(); long recordCount = fd.getFbFileMetadata().icebergMetadata().recordCount();

View File

@@ -1057,6 +1057,12 @@ public class IcebergUtil {
if (spec != null && !spec.fields().isEmpty()) { if (spec != null && !spec.fields().isEmpty()) {
partKeysOffset = createPartitionKeys(feTbl, fbb, spec, cf); partKeysOffset = createPartitionKeys(feTbl, fbb, spec, cf);
} }
int eqFieldIdsOffset = -1;
List<Integer> eqFieldIds = cf.equalityFieldIds();
if (eqFieldIds != null && !eqFieldIds.isEmpty()) {
eqFieldIdsOffset = FbIcebergMetadata.createEqualityFieldIdsVector(fbb,
eqFieldIds.stream().mapToInt(i -> i).sorted().toArray());
}
FbIcebergMetadata.startFbIcebergMetadata(fbb); FbIcebergMetadata.startFbIcebergMetadata(fbb);
byte fileFormat = -1; byte fileFormat = -1;
if (cf.format() == FileFormat.PARQUET) fileFormat = FbIcebergDataFileFormat.PARQUET; if (cf.format() == FileFormat.PARQUET) fileFormat = FbIcebergDataFileFormat.PARQUET;
@@ -1081,6 +1087,11 @@ public class IcebergUtil {
// for manifest entries with status DELETED (older Iceberg versions)." // for manifest entries with status DELETED (older Iceberg versions)."
FbIcebergMetadata.addDataSequenceNumber(fbb, -1l); FbIcebergMetadata.addDataSequenceNumber(fbb, -1l);
} }
if (eqFieldIdsOffset != -1) {
FbIcebergMetadata.addEqualityFieldIds(fbb, eqFieldIdsOffset);
}
return FbIcebergMetadata.endFbIcebergMetadata(fbb); return FbIcebergMetadata.endFbIcebergMetadata(fbb);
} }

View File

@@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Set; import java.util.Set;
import org.apache.impala.catalog.Catalog; import org.apache.impala.catalog.Catalog;
@@ -32,6 +33,7 @@ import org.apache.impala.catalog.HBaseColumn;
import org.apache.impala.catalog.Type; import org.apache.impala.catalog.Type;
import org.apache.impala.common.ImpalaException; import org.apache.impala.common.ImpalaException;
import org.apache.impala.datagenerator.HBaseTestDataRegionAssignment; import org.apache.impala.datagenerator.HBaseTestDataRegionAssignment;
import org.apache.impala.planner.IcebergScanPlanner;
import org.apache.impala.service.Frontend.PlanCtx; import org.apache.impala.service.Frontend.PlanCtx;
import org.apache.impala.testutil.TestUtils; import org.apache.impala.testutil.TestUtils;
import org.apache.impala.testutil.TestUtils.IgnoreValueFilter; import org.apache.impala.testutil.TestUtils.IgnoreValueFilter;
@@ -51,6 +53,7 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.Collections2; import com.google.common.collect.Collections2;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
// All planner tests, except for S3 specific tests should go here. // All planner tests, except for S3 specific tests should go here.
public class PlannerTest extends PlannerTestBase { public class PlannerTest extends PlannerTestBase {
@@ -1511,4 +1514,25 @@ public class PlannerTest extends PlannerTestBase {
runPlannerTestFile("runtime-filter-cardinality-reduction-on-kudu", "tpch_kudu", runPlannerTestFile("runtime-filter-cardinality-reduction-on-kudu", "tpch_kudu",
ImmutableSet.of(PlannerTestOption.VALIDATE_CARDINALITY)); ImmutableSet.of(PlannerTestOption.VALIDATE_CARDINALITY));
} }
/**
* Test that how the different equality delete field ID lists are ordered for reading.
*/
@Test
public void testEqualityDeleteFieldIdOrdering() {
Map<List<Integer>, Long> inp = Maps.newHashMap();
inp.put(Lists.newArrayList(2, 3), 2L);
inp.put(Lists.newArrayList(1, 2, 3), 2L);
inp.put(Lists.newArrayList(1, 2), 2L);
inp.put(Lists.newArrayList(4), 3L);
inp.put(Lists.newArrayList(1), 3L);
Assert.assertEquals(
Lists.newArrayList(
Lists.newArrayList(1),
Lists.newArrayList(4),
Lists.newArrayList(1, 2, 3),
Lists.newArrayList(1, 2),
Lists.newArrayList(2, 3)),
IcebergScanPlanner.getOrderedEqualityFieldIds(inp));
}
} }

54
testdata/data/README vendored
View File

@@ -803,12 +803,6 @@ Steps:
(2, 'str2_updated', to_date('2023-12-13')); (2, 'str2_updated', to_date('2023-12-13'));
4-Impala: delete from functional_parquet.iceberg_v2_delete_both_eq_and_pos where i = 1; 4-Impala: delete from functional_parquet.iceberg_v2_delete_both_eq_and_pos where i = 1;
iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_different_equality_ids:
Used a hacked Impala to write Iceberg metadata where one equality delete file deletes by
field ID [1] while another deletes by field IDs [1,2]. Note, the underlying delete files
are position delete files written by Impala. It's not possible to add this table to the
data load because it'd give a TableLoadingException when doing a refresh on the table.
iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned: iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned:
Flink is used for creating this test table. The statements executed are the follow: Flink is used for creating this test table. The statements executed are the follow:
1: Create the table with the partition column part of the primary key. This is enforced 1: Create the table with the partition column part of the primary key. This is enforced
@@ -848,6 +842,54 @@ Impala is used for doing some partition evolution on this table.
alter table functional_parquet.iceberg_v2_delete_equality_partition_evolution alter table functional_parquet.iceberg_v2_delete_equality_partition_evolution
set partition spec (d, i); set partition spec (d, i);
iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids:
Used Impala and Nifi to create a table that has equality delete files with different
equality field ID lists. Steps:
1-Impala:
create table functional_parquet.iceberg_v2_delete_equality_multi_eq_ids
(i int not null, s string not null, primary key(i))
STORED AS ICEBERG
TBLPROPERTIES('iceberg.catalog'='hadoop.catalog',
'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog',
'iceberg.table_identifier'='ice.iceberg_v2_delete_equality_multi_eq_ids',
'format-version'='2');
2-Nifi: Insert 3 rows in one data file:
(1, "str1"), (2, "str2"), (3, "str3")
3-Nifi: Update a row using column 'i' as PK:
(3, "str3_updated")
4: Manually edited 'identifier-field-ids' from [1] to [2]
5-Nifi: In One step insert new rows and update existing ones using column 's' as PK:
Insert (4, "str4"), (5, "str5")
Update (2222, "str2"), (3333, "str3_updated")
6: Manually edited 'identifier-field-ids' from [2] to [1,2]
7: Update rows using columns [i,s] as PK:
(4, "str4") -> (4, "str4_updated"), (3333, "str3_updated") -> (33, "str3_updated_twice")
iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids:
Used Flink and Impala to create a table that has both positional and equality delete
files where some of the equality deletes have different equality field IDs.
1-Flink:
create table hadoop_catalog.ice.iceberg_v2_delete_pos_and_multi_eq_ids
(i int not null, s string not null, d date not null, primary key(i,s) not enforced)
with ('format-version'='2', 'write.upsert.enabled'='true');
2-Flink:
insert into hadoop_catalog.ice.iceberg_v2_delete_pos_and_multi_eq_ids values
(1, 'str1', to_date('2024-01-23')),
(2, 'str2', to_date('2024-01-24')),
(3, 'str3', to_date('2024-01-25'));
3-Flink:
insert into hadoop_catalog.ice.iceberg_v2_delete_pos_and_multi_eq_ids values
(1, 'str1', to_date('2020-12-01')),
(4, 'str4', to_date('2024-01-26'));
4-Impala:
delete from functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids where s = 'str2';
5: Manually edited 'identifier-field-ids' from [1,2] to [3,2].
6: Restarted Flink to forget the table metadata.
7-Flink:
insert into hadoop_catalog.ice.iceberg_v2_delete_pos_and_multi_eq_ids values
(333333, 'str3', to_date('2024-01-25')),
(5, 'str5', to_date('2024-01-27'));
iceberg_test/iceberg_migrated_alter_test iceberg_test/iceberg_migrated_alter_test
Generated and migrated by Hive Generated and migrated by Hive
CREATE TABLE iceberg_migrated_alter_test (int_col int, string_col string, double_col double) stored as parquet; CREATE TABLE iceberg_migrated_alter_test (int_col int, string_col string, double_col double) stored as parquet;

View File

@@ -1,137 +0,0 @@
{
"format-version" : 2,
"table-uuid" : "3d258856-43d4-4715-9ed9-7824750c4652",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_different_equality_ids",
"last-sequence-number" : 3,
"last-updated-ms" : 1702391589205,
"last-column-id" : 2,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : false,
"type" : "int"
}, {
"id" : 2,
"name" : "s",
"required" : false,
"type" : "string"
} ]
} ],
"default-spec-id" : 0,
"partition-specs" : [ {
"spec-id" : 0,
"fields" : [ ]
} ],
"last-partition-id" : 999,
"default-sort-order-id" : 0,
"sort-orders" : [ {
"order-id" : 0,
"fields" : [ ]
} ],
"properties" : {
"engine.hive.enabled" : "true",
"write.merge.mode" : "merge-on-read",
"write.format.default" : "parquet",
"write.delete.mode" : "merge-on-read",
"iceberg.catalog_location" : "/test-warehouse/iceberg_test/hadoop_catalog",
"OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
"write.update.mode" : "merge-on-read",
"storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler",
"iceberg.catalog" : "hadoop.catalog",
"iceberg.table_identifier" : "ice.iceberg_v2_delete_different_equality_ids"
},
"current-snapshot-id" : 6117850509763739078,
"refs" : {
"main" : {
"snapshot-id" : 6117850509763739078,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 397031335297740726,
"timestamp-ms" : 1702391239615,
"summary" : {
"operation" : "append",
"added-data-files" : "1",
"added-records" : "3",
"added-files-size" : "616",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "616",
"total-data-files" : "1",
"total-delete-files" : "0",
"total-position-deletes" : "0",
"total-equality-deletes" : "0"
},
"manifest-list" : "hdfs://localhost:20500/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_different_equality_ids/metadata/snap-397031335297740726-1-2d3fafd7-bce6-483f-be82-e0ccce9203fc.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
"snapshot-id" : 8494861454990126958,
"parent-snapshot-id" : 397031335297740726,
"timestamp-ms" : 1702391268756,
"summary" : {
"operation" : "overwrite",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-files-size" : "1630",
"added-equality-deletes" : "1",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "2246",
"total-data-files" : "1",
"total-delete-files" : "1",
"total-position-deletes" : "0",
"total-equality-deletes" : "1"
},
"manifest-list" : "hdfs://localhost:20500/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_different_equality_ids/metadata/snap-8494861454990126958-1-8bd425d8-25fb-4603-8cc7-aeb5ad2a3917.avro",
"schema-id" : 0
}, {
"sequence-number" : 3,
"snapshot-id" : 6117850509763739078,
"parent-snapshot-id" : 8494861454990126958,
"timestamp-ms" : 1702391589205,
"summary" : {
"operation" : "overwrite",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-files-size" : "1630",
"added-equality-deletes" : "1",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "3876",
"total-data-files" : "1",
"total-delete-files" : "2",
"total-position-deletes" : "0",
"total-equality-deletes" : "2"
},
"manifest-list" : "hdfs://localhost:20500/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_different_equality_ids/metadata/snap-6117850509763739078-1-57a963d3-0e4e-4540-8080-a57afd51ba99.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1702391239615,
"snapshot-id" : 397031335297740726
}, {
"timestamp-ms" : 1702391268756,
"snapshot-id" : 8494861454990126958
}, {
"timestamp-ms" : 1702391589205,
"snapshot-id" : 6117850509763739078
} ],
"metadata-log" : [ {
"timestamp-ms" : 1702391192239,
"metadata-file" : "hdfs://localhost:20500/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_different_equality_ids/metadata/v1.metadata.json"
}, {
"timestamp-ms" : 1702391239615,
"metadata-file" : "hdfs://localhost:20500/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_different_equality_ids/metadata/v2.metadata.json"
}, {
"timestamp-ms" : 1702391268756,
"metadata-file" : "hdfs://localhost:20500/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_different_equality_ids/metadata/v3.metadata.json"
} ]
}

View File

@@ -1,23 +1,24 @@
{ {
"format-version" : 2, "format-version" : 2,
"table-uuid" : "3d258856-43d4-4715-9ed9-7824750c4652", "table-uuid" : "0f020fe9-a36d-4e02-9963-a55946c487e2",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_different_equality_ids", "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids",
"last-sequence-number" : 0, "last-sequence-number" : 0,
"last-updated-ms" : 1702391192239, "last-updated-ms" : 1707392530371,
"last-column-id" : 2, "last-column-id" : 2,
"current-schema-id" : 0, "current-schema-id" : 0,
"schemas" : [ { "schemas" : [ {
"type" : "struct", "type" : "struct",
"schema-id" : 0, "schema-id" : 0,
"identifier-field-ids" : [ 1 ],
"fields" : [ { "fields" : [ {
"id" : 1, "id" : 1,
"name" : "i", "name" : "i",
"required" : false, "required" : true,
"type" : "int" "type" : "int"
}, { }, {
"id" : 2, "id" : 2,
"name" : "s", "name" : "s",
"required" : false, "required" : true,
"type" : "string" "type" : "string"
} ] } ]
} ], } ],
@@ -42,7 +43,7 @@
"write.update.mode" : "merge-on-read", "write.update.mode" : "merge-on-read",
"storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler", "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler",
"iceberg.catalog" : "hadoop.catalog", "iceberg.catalog" : "hadoop.catalog",
"iceberg.table_identifier" : "ice.iceberg_v2_delete_different_equality_ids" "iceberg.table_identifier" : "ice.iceberg_v2_delete_equality_multi_eq_ids"
}, },
"current-snapshot-id" : -1, "current-snapshot-id" : -1,
"refs" : { }, "refs" : { },

View File

@@ -1,23 +1,24 @@
{ {
"format-version" : 2, "format-version" : 2,
"table-uuid" : "3d258856-43d4-4715-9ed9-7824750c4652", "table-uuid" : "0f020fe9-a36d-4e02-9963-a55946c487e2",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_different_equality_ids", "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids",
"last-sequence-number" : 1, "last-sequence-number" : 1,
"last-updated-ms" : 1702391239615, "last-updated-ms" : 1707392713570,
"last-column-id" : 2, "last-column-id" : 2,
"current-schema-id" : 0, "current-schema-id" : 0,
"schemas" : [ { "schemas" : [ {
"type" : "struct", "type" : "struct",
"schema-id" : 0, "schema-id" : 0,
"identifier-field-ids" : [ 1 ],
"fields" : [ { "fields" : [ {
"id" : 1, "id" : 1,
"name" : "i", "name" : "i",
"required" : false, "required" : true,
"type" : "int" "type" : "int"
}, { }, {
"id" : 2, "id" : 2,
"name" : "s", "name" : "s",
"required" : false, "required" : true,
"type" : "string" "type" : "string"
} ] } ]
} ], } ],
@@ -42,42 +43,42 @@
"write.update.mode" : "merge-on-read", "write.update.mode" : "merge-on-read",
"storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler", "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler",
"iceberg.catalog" : "hadoop.catalog", "iceberg.catalog" : "hadoop.catalog",
"iceberg.table_identifier" : "ice.iceberg_v2_delete_different_equality_ids" "iceberg.table_identifier" : "ice.iceberg_v2_delete_equality_multi_eq_ids"
}, },
"current-snapshot-id" : 397031335297740726, "current-snapshot-id" : 2374780975027972430,
"refs" : { "refs" : {
"main" : { "main" : {
"snapshot-id" : 397031335297740726, "snapshot-id" : 2374780975027972430,
"type" : "branch" "type" : "branch"
} }
}, },
"snapshots" : [ { "snapshots" : [ {
"sequence-number" : 1, "sequence-number" : 1,
"snapshot-id" : 397031335297740726, "snapshot-id" : 2374780975027972430,
"timestamp-ms" : 1702391239615, "timestamp-ms" : 1707392713570,
"summary" : { "summary" : {
"operation" : "append", "operation" : "overwrite",
"added-data-files" : "1", "added-data-files" : "1",
"added-records" : "3", "added-records" : "3",
"added-files-size" : "616", "added-files-size" : "655",
"changed-partition-count" : "1", "changed-partition-count" : "1",
"total-records" : "3", "total-records" : "3",
"total-files-size" : "616", "total-files-size" : "655",
"total-data-files" : "1", "total-data-files" : "1",
"total-delete-files" : "0", "total-delete-files" : "0",
"total-position-deletes" : "0", "total-position-deletes" : "0",
"total-equality-deletes" : "0" "total-equality-deletes" : "0"
}, },
"manifest-list" : "hdfs://localhost:20500/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_different_equality_ids/metadata/snap-397031335297740726-1-2d3fafd7-bce6-483f-be82-e0ccce9203fc.avro", "manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/snap-2374780975027972430-1-ea25da34-c91b-4a0f-a003-3958e87caffd.avro",
"schema-id" : 0 "schema-id" : 0
} ], } ],
"statistics" : [ ], "statistics" : [ ],
"snapshot-log" : [ { "snapshot-log" : [ {
"timestamp-ms" : 1702391239615, "timestamp-ms" : 1707392713570,
"snapshot-id" : 397031335297740726 "snapshot-id" : 2374780975027972430
} ], } ],
"metadata-log" : [ { "metadata-log" : [ {
"timestamp-ms" : 1702391192239, "timestamp-ms" : 1707392530371,
"metadata-file" : "hdfs://localhost:20500/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_different_equality_ids/metadata/v1.metadata.json" "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/v1.metadata.json"
} ] } ]
} }

View File

@@ -1,23 +1,24 @@
{ {
"format-version" : 2, "format-version" : 2,
"table-uuid" : "3d258856-43d4-4715-9ed9-7824750c4652", "table-uuid" : "0f020fe9-a36d-4e02-9963-a55946c487e2",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_different_equality_ids", "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids",
"last-sequence-number" : 2, "last-sequence-number" : 2,
"last-updated-ms" : 1702391268756, "last-updated-ms" : 1707392778729,
"last-column-id" : 2, "last-column-id" : 2,
"current-schema-id" : 0, "current-schema-id" : 0,
"schemas" : [ { "schemas" : [ {
"type" : "struct", "type" : "struct",
"schema-id" : 0, "schema-id" : 0,
"identifier-field-ids" : [ 2 ],
"fields" : [ { "fields" : [ {
"id" : 1, "id" : 1,
"name" : "i", "name" : "i",
"required" : false, "required" : true,
"type" : "int" "type" : "int"
}, { }, {
"id" : 2, "id" : 2,
"name" : "s", "name" : "s",
"required" : false, "required" : true,
"type" : "string" "type" : "string"
} ] } ]
} ], } ],
@@ -42,69 +43,71 @@
"write.update.mode" : "merge-on-read", "write.update.mode" : "merge-on-read",
"storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler", "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler",
"iceberg.catalog" : "hadoop.catalog", "iceberg.catalog" : "hadoop.catalog",
"iceberg.table_identifier" : "ice.iceberg_v2_delete_different_equality_ids" "iceberg.table_identifier" : "ice.iceberg_v2_delete_equality_multi_eq_ids"
}, },
"current-snapshot-id" : 8494861454990126958, "current-snapshot-id" : 8127619959873391049,
"refs" : { "refs" : {
"main" : { "main" : {
"snapshot-id" : 8494861454990126958, "snapshot-id" : 8127619959873391049,
"type" : "branch" "type" : "branch"
} }
}, },
"snapshots" : [ { "snapshots" : [ {
"sequence-number" : 1, "sequence-number" : 1,
"snapshot-id" : 397031335297740726, "snapshot-id" : 2374780975027972430,
"timestamp-ms" : 1702391239615, "timestamp-ms" : 1707392713570,
"summary" : { "summary" : {
"operation" : "append", "operation" : "overwrite",
"added-data-files" : "1", "added-data-files" : "1",
"added-records" : "3", "added-records" : "3",
"added-files-size" : "616", "added-files-size" : "655",
"changed-partition-count" : "1", "changed-partition-count" : "1",
"total-records" : "3", "total-records" : "3",
"total-files-size" : "616", "total-files-size" : "655",
"total-data-files" : "1", "total-data-files" : "1",
"total-delete-files" : "0", "total-delete-files" : "0",
"total-position-deletes" : "0", "total-position-deletes" : "0",
"total-equality-deletes" : "0" "total-equality-deletes" : "0"
}, },
"manifest-list" : "hdfs://localhost:20500/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_different_equality_ids/metadata/snap-397031335297740726-1-2d3fafd7-bce6-483f-be82-e0ccce9203fc.avro", "manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/snap-2374780975027972430-1-ea25da34-c91b-4a0f-a003-3958e87caffd.avro",
"schema-id" : 0 "schema-id" : 0
}, { }, {
"sequence-number" : 2, "sequence-number" : 2,
"snapshot-id" : 8494861454990126958, "snapshot-id" : 8127619959873391049,
"parent-snapshot-id" : 397031335297740726, "parent-snapshot-id" : 2374780975027972430,
"timestamp-ms" : 1702391268756, "timestamp-ms" : 1707392778729,
"summary" : { "summary" : {
"operation" : "overwrite", "operation" : "overwrite",
"added-data-files" : "1",
"added-equality-delete-files" : "1", "added-equality-delete-files" : "1",
"added-delete-files" : "1", "added-delete-files" : "1",
"added-files-size" : "1630", "added-records" : "1",
"added-files-size" : "1156",
"added-equality-deletes" : "1", "added-equality-deletes" : "1",
"changed-partition-count" : "1", "changed-partition-count" : "1",
"total-records" : "3", "total-records" : "4",
"total-files-size" : "2246", "total-files-size" : "1811",
"total-data-files" : "1", "total-data-files" : "2",
"total-delete-files" : "1", "total-delete-files" : "1",
"total-position-deletes" : "0", "total-position-deletes" : "0",
"total-equality-deletes" : "1" "total-equality-deletes" : "1"
}, },
"manifest-list" : "hdfs://localhost:20500/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_different_equality_ids/metadata/snap-8494861454990126958-1-8bd425d8-25fb-4603-8cc7-aeb5ad2a3917.avro", "manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/snap-8127619959873391049-1-b8fe0a34-e755-4ba0-92f1-c72bef85a82a.avro",
"schema-id" : 0 "schema-id" : 0
} ], } ],
"statistics" : [ ], "statistics" : [ ],
"snapshot-log" : [ { "snapshot-log" : [ {
"timestamp-ms" : 1702391239615, "timestamp-ms" : 1707392713570,
"snapshot-id" : 397031335297740726 "snapshot-id" : 2374780975027972430
}, { }, {
"timestamp-ms" : 1702391268756, "timestamp-ms" : 1707392778729,
"snapshot-id" : 8494861454990126958 "snapshot-id" : 8127619959873391049
} ], } ],
"metadata-log" : [ { "metadata-log" : [ {
"timestamp-ms" : 1702391192239, "timestamp-ms" : 1707392530371,
"metadata-file" : "hdfs://localhost:20500/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_different_equality_ids/metadata/v1.metadata.json" "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/v1.metadata.json"
}, { }, {
"timestamp-ms" : 1702391239615, "timestamp-ms" : 1707392713570,
"metadata-file" : "hdfs://localhost:20500/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_different_equality_ids/metadata/v2.metadata.json" "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/v2.metadata.json"
} ] } ]
} }

View File

@@ -0,0 +1,142 @@
{
"format-version" : 2,
"table-uuid" : "0f020fe9-a36d-4e02-9963-a55946c487e2",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids",
"last-sequence-number" : 3,
"last-updated-ms" : 1707392867744,
"last-column-id" : 2,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 1, 2 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : true,
"type" : "int"
}, {
"id" : 2,
"name" : "s",
"required" : true,
"type" : "string"
} ]
} ],
"default-spec-id" : 0,
"partition-specs" : [ {
"spec-id" : 0,
"fields" : [ ]
} ],
"last-partition-id" : 999,
"default-sort-order-id" : 0,
"sort-orders" : [ {
"order-id" : 0,
"fields" : [ ]
} ],
"properties" : {
"engine.hive.enabled" : "true",
"write.merge.mode" : "merge-on-read",
"write.format.default" : "parquet",
"write.delete.mode" : "merge-on-read",
"iceberg.catalog_location" : "/test-warehouse/iceberg_test/hadoop_catalog",
"OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
"write.update.mode" : "merge-on-read",
"storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler",
"iceberg.catalog" : "hadoop.catalog",
"iceberg.table_identifier" : "ice.iceberg_v2_delete_equality_multi_eq_ids"
},
"current-snapshot-id" : 5777805847908928861,
"refs" : {
"main" : {
"snapshot-id" : 5777805847908928861,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 2374780975027972430,
"timestamp-ms" : 1707392713570,
"summary" : {
"operation" : "overwrite",
"added-data-files" : "1",
"added-records" : "3",
"added-files-size" : "655",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "655",
"total-data-files" : "1",
"total-delete-files" : "0",
"total-position-deletes" : "0",
"total-equality-deletes" : "0"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/snap-2374780975027972430-1-ea25da34-c91b-4a0f-a003-3958e87caffd.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
"snapshot-id" : 8127619959873391049,
"parent-snapshot-id" : 2374780975027972430,
"timestamp-ms" : 1707392778729,
"summary" : {
"operation" : "overwrite",
"added-data-files" : "1",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-records" : "1",
"added-files-size" : "1156",
"added-equality-deletes" : "1",
"changed-partition-count" : "1",
"total-records" : "4",
"total-files-size" : "1811",
"total-data-files" : "2",
"total-delete-files" : "1",
"total-position-deletes" : "0",
"total-equality-deletes" : "1"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/snap-8127619959873391049-1-b8fe0a34-e755-4ba0-92f1-c72bef85a82a.avro",
"schema-id" : 0
}, {
"sequence-number" : 3,
"snapshot-id" : 5777805847908928861,
"parent-snapshot-id" : 8127619959873391049,
"timestamp-ms" : 1707392867744,
"summary" : {
"operation" : "overwrite",
"added-data-files" : "1",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-records" : "4",
"added-files-size" : "1146",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "8",
"total-files-size" : "2957",
"total-data-files" : "3",
"total-delete-files" : "2",
"total-position-deletes" : "0",
"total-equality-deletes" : "3"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/snap-5777805847908928861-1-2eb78a39-190a-4eab-b24a-df07f32f2cc0.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1707392713570,
"snapshot-id" : 2374780975027972430
}, {
"timestamp-ms" : 1707392778729,
"snapshot-id" : 8127619959873391049
}, {
"timestamp-ms" : 1707392867744,
"snapshot-id" : 5777805847908928861
} ],
"metadata-log" : [ {
"timestamp-ms" : 1707392530371,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/v1.metadata.json"
}, {
"timestamp-ms" : 1707392713570,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/v2.metadata.json"
}, {
"timestamp-ms" : 1707392778729,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/v3.metadata.json"
} ]
}

View File

@@ -0,0 +1,171 @@
{
"format-version" : 2,
"table-uuid" : "0f020fe9-a36d-4e02-9963-a55946c487e2",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids",
"last-sequence-number" : 4,
"last-updated-ms" : 1707393145897,
"last-column-id" : 2,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 1, 2 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : true,
"type" : "int"
}, {
"id" : 2,
"name" : "s",
"required" : true,
"type" : "string"
} ]
} ],
"default-spec-id" : 0,
"partition-specs" : [ {
"spec-id" : 0,
"fields" : [ ]
} ],
"last-partition-id" : 999,
"default-sort-order-id" : 0,
"sort-orders" : [ {
"order-id" : 0,
"fields" : [ ]
} ],
"properties" : {
"engine.hive.enabled" : "true",
"write.merge.mode" : "merge-on-read",
"write.format.default" : "parquet",
"write.delete.mode" : "merge-on-read",
"iceberg.catalog_location" : "/test-warehouse/iceberg_test/hadoop_catalog",
"OBJCAPABILITIES" : "EXTREAD,EXTWRITE",
"write.update.mode" : "merge-on-read",
"storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler",
"iceberg.catalog" : "hadoop.catalog",
"iceberg.table_identifier" : "ice.iceberg_v2_delete_equality_multi_eq_ids"
},
"current-snapshot-id" : 4077234998626563290,
"refs" : {
"main" : {
"snapshot-id" : 4077234998626563290,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 2374780975027972430,
"timestamp-ms" : 1707392713570,
"summary" : {
"operation" : "overwrite",
"added-data-files" : "1",
"added-records" : "3",
"added-files-size" : "655",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "655",
"total-data-files" : "1",
"total-delete-files" : "0",
"total-position-deletes" : "0",
"total-equality-deletes" : "0"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/snap-2374780975027972430-1-ea25da34-c91b-4a0f-a003-3958e87caffd.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
"snapshot-id" : 8127619959873391049,
"parent-snapshot-id" : 2374780975027972430,
"timestamp-ms" : 1707392778729,
"summary" : {
"operation" : "overwrite",
"added-data-files" : "1",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-records" : "1",
"added-files-size" : "1156",
"added-equality-deletes" : "1",
"changed-partition-count" : "1",
"total-records" : "4",
"total-files-size" : "1811",
"total-data-files" : "2",
"total-delete-files" : "1",
"total-position-deletes" : "0",
"total-equality-deletes" : "1"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/snap-8127619959873391049-1-b8fe0a34-e755-4ba0-92f1-c72bef85a82a.avro",
"schema-id" : 0
}, {
"sequence-number" : 3,
"snapshot-id" : 5777805847908928861,
"parent-snapshot-id" : 8127619959873391049,
"timestamp-ms" : 1707392867744,
"summary" : {
"operation" : "overwrite",
"added-data-files" : "1",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-records" : "4",
"added-files-size" : "1146",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "8",
"total-files-size" : "2957",
"total-data-files" : "3",
"total-delete-files" : "2",
"total-position-deletes" : "0",
"total-equality-deletes" : "3"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/snap-5777805847908928861-1-2eb78a39-190a-4eab-b24a-df07f32f2cc0.avro",
"schema-id" : 0
}, {
"sequence-number" : 4,
"snapshot-id" : 4077234998626563290,
"parent-snapshot-id" : 5777805847908928861,
"timestamp-ms" : 1707393145897,
"summary" : {
"operation" : "overwrite",
"added-data-files" : "1",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-records" : "2",
"added-files-size" : "1439",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "10",
"total-files-size" : "4396",
"total-data-files" : "4",
"total-delete-files" : "3",
"total-position-deletes" : "0",
"total-equality-deletes" : "5"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/snap-4077234998626563290-1-58a6295d-0076-4e3e-ac77-84ce48c406cf.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1707392713570,
"snapshot-id" : 2374780975027972430
}, {
"timestamp-ms" : 1707392778729,
"snapshot-id" : 8127619959873391049
}, {
"timestamp-ms" : 1707392867744,
"snapshot-id" : 5777805847908928861
}, {
"timestamp-ms" : 1707393145897,
"snapshot-id" : 4077234998626563290
} ],
"metadata-log" : [ {
"timestamp-ms" : 1707392530371,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/v1.metadata.json"
}, {
"timestamp-ms" : 1707392713570,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/v2.metadata.json"
}, {
"timestamp-ms" : 1707392778729,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/v3.metadata.json"
}, {
"timestamp-ms" : 1707392867744,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids/metadata/v4.metadata.json"
} ]
}

View File

@@ -0,0 +1,51 @@
{
"format-version" : 2,
"table-uuid" : "ec1c2bce-1cc7-4449-be37-5fdc16d1233a",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids",
"last-sequence-number" : 0,
"last-updated-ms" : 1706016347276,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 1, 2 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : true,
"type" : "int"
}, {
"id" : 2,
"name" : "s",
"required" : true,
"type" : "string"
}, {
"id" : 3,
"name" : "d",
"required" : true,
"type" : "date"
} ]
} ],
"default-spec-id" : 0,
"partition-specs" : [ {
"spec-id" : 0,
"fields" : [ ]
} ],
"last-partition-id" : 999,
"default-sort-order-id" : 0,
"sort-orders" : [ {
"order-id" : 0,
"fields" : [ ]
} ],
"properties" : {
"write.parquet.compression-codec" : "zstd",
"write.upsert.enabled" : "true"
},
"current-snapshot-id" : -1,
"refs" : { },
"snapshots" : [ ],
"statistics" : [ ],
"snapshot-log" : [ ],
"metadata-log" : [ ]
}

View File

@@ -0,0 +1,87 @@
{
"format-version" : 2,
"table-uuid" : "ec1c2bce-1cc7-4449-be37-5fdc16d1233a",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids",
"last-sequence-number" : 1,
"last-updated-ms" : 1706016363578,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 1, 2 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : true,
"type" : "int"
}, {
"id" : 2,
"name" : "s",
"required" : true,
"type" : "string"
}, {
"id" : 3,
"name" : "d",
"required" : true,
"type" : "date"
} ]
} ],
"default-spec-id" : 0,
"partition-specs" : [ {
"spec-id" : 0,
"fields" : [ ]
} ],
"last-partition-id" : 999,
"default-sort-order-id" : 0,
"sort-orders" : [ {
"order-id" : 0,
"fields" : [ ]
} ],
"properties" : {
"write.parquet.compression-codec" : "zstd",
"write.upsert.enabled" : "true"
},
"current-snapshot-id" : 2066775081852432762,
"refs" : {
"main" : {
"snapshot-id" : 2066775081852432762,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 2066775081852432762,
"timestamp-ms" : 1706016363578,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "751aac623aff5b6900c68eac5be5ffe6",
"flink.job-id" : "89aa9b1a4c993a9fc3f9b84a3aa87c75",
"flink.max-committed-checkpoint-id" : "9223372036854775807",
"added-data-files" : "1",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-records" : "3",
"added-files-size" : "1525",
"added-equality-deletes" : "3",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "1525",
"total-data-files" : "1",
"total-delete-files" : "1",
"total-position-deletes" : "0",
"total-equality-deletes" : "3"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/snap-2066775081852432762-1-f9fa006c-0078-4caf-8eaf-f9d499fc6939.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1706016363578,
"snapshot-id" : 2066775081852432762
} ],
"metadata-log" : [ {
"timestamp-ms" : 1706016347276,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/v1.metadata.json"
} ]
}

View File

@@ -0,0 +1,119 @@
{
"format-version" : 2,
"table-uuid" : "ec1c2bce-1cc7-4449-be37-5fdc16d1233a",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids",
"last-sequence-number" : 2,
"last-updated-ms" : 1706016371153,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 1, 2 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : true,
"type" : "int"
}, {
"id" : 2,
"name" : "s",
"required" : true,
"type" : "string"
}, {
"id" : 3,
"name" : "d",
"required" : true,
"type" : "date"
} ]
} ],
"default-spec-id" : 0,
"partition-specs" : [ {
"spec-id" : 0,
"fields" : [ ]
} ],
"last-partition-id" : 999,
"default-sort-order-id" : 0,
"sort-orders" : [ {
"order-id" : 0,
"fields" : [ ]
} ],
"properties" : {
"write.parquet.compression-codec" : "zstd",
"write.upsert.enabled" : "true"
},
"current-snapshot-id" : 7591397613223797435,
"refs" : {
"main" : {
"snapshot-id" : 7591397613223797435,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 2066775081852432762,
"timestamp-ms" : 1706016363578,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "751aac623aff5b6900c68eac5be5ffe6",
"flink.job-id" : "89aa9b1a4c993a9fc3f9b84a3aa87c75",
"flink.max-committed-checkpoint-id" : "9223372036854775807",
"added-data-files" : "1",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-records" : "3",
"added-files-size" : "1525",
"added-equality-deletes" : "3",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "1525",
"total-data-files" : "1",
"total-delete-files" : "1",
"total-position-deletes" : "0",
"total-equality-deletes" : "3"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/snap-2066775081852432762-1-f9fa006c-0078-4caf-8eaf-f9d499fc6939.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
"snapshot-id" : 7591397613223797435,
"parent-snapshot-id" : 2066775081852432762,
"timestamp-ms" : 1706016371153,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "a5df36bf3be49d13acf7c9a290f491bf",
"flink.job-id" : "f7d34181dd8012779d0736df2c3a160c",
"flink.max-committed-checkpoint-id" : "9223372036854775807",
"added-data-files" : "1",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-records" : "2",
"added-files-size" : "1509",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "5",
"total-files-size" : "3034",
"total-data-files" : "2",
"total-delete-files" : "2",
"total-position-deletes" : "0",
"total-equality-deletes" : "5"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/snap-7591397613223797435-1-d7fa3972-f84c-4b35-aa37-2079458ccea8.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1706016363578,
"snapshot-id" : 2066775081852432762
}, {
"timestamp-ms" : 1706016371153,
"snapshot-id" : 7591397613223797435
} ],
"metadata-log" : [ {
"timestamp-ms" : 1706016347276,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/v1.metadata.json"
}, {
"timestamp-ms" : 1706016363578,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/v2.metadata.json"
} ]
}

View File

@@ -0,0 +1,146 @@
{
"format-version" : 2,
"table-uuid" : "ec1c2bce-1cc7-4449-be37-5fdc16d1233a",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids",
"last-sequence-number" : 3,
"last-updated-ms" : 1706016398098,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 3, 2 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : true,
"type" : "int"
}, {
"id" : 2,
"name" : "s",
"required" : true,
"type" : "string"
}, {
"id" : 3,
"name" : "d",
"required" : true,
"type" : "date"
} ]
} ],
"default-spec-id" : 0,
"partition-specs" : [ {
"spec-id" : 0,
"fields" : [ ]
} ],
"last-partition-id" : 999,
"default-sort-order-id" : 0,
"sort-orders" : [ {
"order-id" : 0,
"fields" : [ ]
} ],
"properties" : {
"write.parquet.compression-codec" : "zstd",
"write.upsert.enabled" : "true"
},
"current-snapshot-id" : 6283211732171745116,
"refs" : {
"main" : {
"snapshot-id" : 6283211732171745116,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 2066775081852432762,
"timestamp-ms" : 1706016363578,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "751aac623aff5b6900c68eac5be5ffe6",
"flink.job-id" : "89aa9b1a4c993a9fc3f9b84a3aa87c75",
"flink.max-committed-checkpoint-id" : "9223372036854775807",
"added-data-files" : "1",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-records" : "3",
"added-files-size" : "1525",
"added-equality-deletes" : "3",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "1525",
"total-data-files" : "1",
"total-delete-files" : "1",
"total-position-deletes" : "0",
"total-equality-deletes" : "3"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/snap-2066775081852432762-1-f9fa006c-0078-4caf-8eaf-f9d499fc6939.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
"snapshot-id" : 7591397613223797435,
"parent-snapshot-id" : 2066775081852432762,
"timestamp-ms" : 1706016371153,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "a5df36bf3be49d13acf7c9a290f491bf",
"flink.job-id" : "f7d34181dd8012779d0736df2c3a160c",
"flink.max-committed-checkpoint-id" : "9223372036854775807",
"added-data-files" : "1",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-records" : "2",
"added-files-size" : "1509",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "5",
"total-files-size" : "3034",
"total-data-files" : "2",
"total-delete-files" : "2",
"total-position-deletes" : "0",
"total-equality-deletes" : "5"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/snap-7591397613223797435-1-d7fa3972-f84c-4b35-aa37-2079458ccea8.avro",
"schema-id" : 0
}, {
"sequence-number" : 3,
"snapshot-id" : 6283211732171745116,
"parent-snapshot-id" : 7591397613223797435,
"timestamp-ms" : 1706016398098,
"summary" : {
"operation" : "overwrite",
"added-position-delete-files" : "1",
"added-delete-files" : "1",
"added-files-size" : "1633",
"added-position-deletes" : "1",
"changed-partition-count" : "1",
"total-records" : "5",
"total-files-size" : "4667",
"total-data-files" : "2",
"total-delete-files" : "3",
"total-position-deletes" : "1",
"total-equality-deletes" : "5"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/snap-6283211732171745116-1-103b5b20-fb15-41bb-a97d-1e2ddc147650.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1706016363578,
"snapshot-id" : 2066775081852432762
}, {
"timestamp-ms" : 1706016371153,
"snapshot-id" : 7591397613223797435
}, {
"timestamp-ms" : 1706016398098,
"snapshot-id" : 6283211732171745116
} ],
"metadata-log" : [ {
"timestamp-ms" : 1706016347276,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/v1.metadata.json"
}, {
"timestamp-ms" : 1706016363578,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/v2.metadata.json"
}, {
"timestamp-ms" : 1706016371153,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/v3.metadata.json"
} ]
}

View File

@@ -0,0 +1,178 @@
{
"format-version" : 2,
"table-uuid" : "ec1c2bce-1cc7-4449-be37-5fdc16d1233a",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids",
"last-sequence-number" : 4,
"last-updated-ms" : 1706016531605,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 3, 2 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : true,
"type" : "int"
}, {
"id" : 2,
"name" : "s",
"required" : true,
"type" : "string"
}, {
"id" : 3,
"name" : "d",
"required" : true,
"type" : "date"
} ]
} ],
"default-spec-id" : 0,
"partition-specs" : [ {
"spec-id" : 0,
"fields" : [ ]
} ],
"last-partition-id" : 999,
"default-sort-order-id" : 0,
"sort-orders" : [ {
"order-id" : 0,
"fields" : [ ]
} ],
"properties" : {
"write.parquet.compression-codec" : "zstd",
"write.upsert.enabled" : "true"
},
"current-snapshot-id" : 152862018760071153,
"refs" : {
"main" : {
"snapshot-id" : 152862018760071153,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 2066775081852432762,
"timestamp-ms" : 1706016363578,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "751aac623aff5b6900c68eac5be5ffe6",
"flink.job-id" : "89aa9b1a4c993a9fc3f9b84a3aa87c75",
"flink.max-committed-checkpoint-id" : "9223372036854775807",
"added-data-files" : "1",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-records" : "3",
"added-files-size" : "1525",
"added-equality-deletes" : "3",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "1525",
"total-data-files" : "1",
"total-delete-files" : "1",
"total-position-deletes" : "0",
"total-equality-deletes" : "3"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/snap-2066775081852432762-1-f9fa006c-0078-4caf-8eaf-f9d499fc6939.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
"snapshot-id" : 7591397613223797435,
"parent-snapshot-id" : 2066775081852432762,
"timestamp-ms" : 1706016371153,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "a5df36bf3be49d13acf7c9a290f491bf",
"flink.job-id" : "f7d34181dd8012779d0736df2c3a160c",
"flink.max-committed-checkpoint-id" : "9223372036854775807",
"added-data-files" : "1",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-records" : "2",
"added-files-size" : "1509",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "5",
"total-files-size" : "3034",
"total-data-files" : "2",
"total-delete-files" : "2",
"total-position-deletes" : "0",
"total-equality-deletes" : "5"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/snap-7591397613223797435-1-d7fa3972-f84c-4b35-aa37-2079458ccea8.avro",
"schema-id" : 0
}, {
"sequence-number" : 3,
"snapshot-id" : 6283211732171745116,
"parent-snapshot-id" : 7591397613223797435,
"timestamp-ms" : 1706016398098,
"summary" : {
"operation" : "overwrite",
"added-position-delete-files" : "1",
"added-delete-files" : "1",
"added-files-size" : "1633",
"added-position-deletes" : "1",
"changed-partition-count" : "1",
"total-records" : "5",
"total-files-size" : "4667",
"total-data-files" : "2",
"total-delete-files" : "3",
"total-position-deletes" : "1",
"total-equality-deletes" : "5"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/snap-6283211732171745116-1-103b5b20-fb15-41bb-a97d-1e2ddc147650.avro",
"schema-id" : 0
}, {
"sequence-number" : 4,
"snapshot-id" : 152862018760071153,
"parent-snapshot-id" : 6283211732171745116,
"timestamp-ms" : 1706016531605,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "a5df36bf3be49d13acf7c9a290f491bf",
"flink.job-id" : "02c6784e5fd3db022944c7d57ee8aaa8",
"flink.max-committed-checkpoint-id" : "9223372036854775807",
"added-data-files" : "1",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-records" : "2",
"added-files-size" : "1516",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "7",
"total-files-size" : "6183",
"total-data-files" : "3",
"total-delete-files" : "4",
"total-position-deletes" : "1",
"total-equality-deletes" : "7"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/snap-152862018760071153-1-c0500e2e-00c0-48fb-9c29-31bbafc91d57.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1706016363578,
"snapshot-id" : 2066775081852432762
}, {
"timestamp-ms" : 1706016371153,
"snapshot-id" : 7591397613223797435
}, {
"timestamp-ms" : 1706016398098,
"snapshot-id" : 6283211732171745116
}, {
"timestamp-ms" : 1706016531605,
"snapshot-id" : 152862018760071153
} ],
"metadata-log" : [ {
"timestamp-ms" : 1706016347276,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/v1.metadata.json"
}, {
"timestamp-ms" : 1706016363578,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/v2.metadata.json"
}, {
"timestamp-ms" : 1706016371153,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/v3.metadata.json"
}, {
"timestamp-ms" : 1706016398098,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids/metadata/v4.metadata.json"
} ]
}

View File

@@ -3550,7 +3550,7 @@ STORED AS ICEBERG
TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.catalog', TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.catalog',
'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog', 'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog',
'iceberg.table_identifier'='ice.iceberg_v2_delete_positional', 'iceberg.table_identifier'='ice.iceberg_v2_delete_positional',
'format-version'='2', 'write.update.mode'='merge-on-read'); 'format-version'='2');
---- DEPENDENT_LOAD ---- DEPENDENT_LOAD
`hadoop fs -mkdir -p /test-warehouse/iceberg_test/hadoop_catalog/ice && \ `hadoop fs -mkdir -p /test-warehouse/iceberg_test/hadoop_catalog/ice && \
hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_positional /test-warehouse/iceberg_test/hadoop_catalog/ice hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_positional /test-warehouse/iceberg_test/hadoop_catalog/ice
@@ -3565,7 +3565,7 @@ STORED AS ICEBERG
TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.catalog', TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.catalog',
'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog', 'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog',
'iceberg.table_identifier'='ice.iceberg_v2_delete_equality', 'iceberg.table_identifier'='ice.iceberg_v2_delete_equality',
'format-version'='2', 'write.update.mode'='merge-on-read'); 'format-version'='2');
---- DEPENDENT_LOAD ---- DEPENDENT_LOAD
`hadoop fs -mkdir -p /test-warehouse/iceberg_test/hadoop_catalog/ice && \ `hadoop fs -mkdir -p /test-warehouse/iceberg_test/hadoop_catalog/ice && \
hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality /test-warehouse/iceberg_test/hadoop_catalog/ice hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality /test-warehouse/iceberg_test/hadoop_catalog/ice
@@ -3580,7 +3580,7 @@ STORED AS ICEBERG
TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.catalog', TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.catalog',
'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog', 'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog',
'iceberg.table_identifier'='ice.iceberg_v2_delete_equality_nulls', 'iceberg.table_identifier'='ice.iceberg_v2_delete_equality_nulls',
'format-version'='2', 'write.update.mode'='merge-on-read'); 'format-version'='2');
---- DEPENDENT_LOAD ---- DEPENDENT_LOAD
`hadoop fs -mkdir -p /test-warehouse/iceberg_test/hadoop_catalog/ice && \ `hadoop fs -mkdir -p /test-warehouse/iceberg_test/hadoop_catalog/ice && \
hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_nulls /test-warehouse/iceberg_test/hadoop_catalog/ice hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_nulls /test-warehouse/iceberg_test/hadoop_catalog/ice
@@ -3595,7 +3595,7 @@ STORED AS ICEBERG
TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.catalog', TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.catalog',
'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog', 'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog',
'iceberg.table_identifier'='ice.iceberg_v2_delete_both_eq_and_pos', 'iceberg.table_identifier'='ice.iceberg_v2_delete_both_eq_and_pos',
'format-version'='2', 'write.update.mode'='merge-on-read'); 'format-version'='2');
---- DEPENDENT_LOAD ---- DEPENDENT_LOAD
`hadoop fs -mkdir -p /test-warehouse/iceberg_test/hadoop_catalog/ice && \ `hadoop fs -mkdir -p /test-warehouse/iceberg_test/hadoop_catalog/ice && \
hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos /test-warehouse/iceberg_test/hadoop_catalog/ice hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos /test-warehouse/iceberg_test/hadoop_catalog/ice
@@ -3610,7 +3610,7 @@ STORED AS ICEBERG
TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.catalog', TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.catalog',
'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog', 'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog',
'iceberg.table_identifier'='ice.iceberg_v2_delete_equality_partitioned', 'iceberg.table_identifier'='ice.iceberg_v2_delete_equality_partitioned',
'format-version'='2', 'write.update.mode'='merge-on-read'); 'format-version'='2');
---- DEPENDENT_LOAD ---- DEPENDENT_LOAD
`hadoop fs -mkdir -p /test-warehouse/iceberg_test/hadoop_catalog/ice && \ `hadoop fs -mkdir -p /test-warehouse/iceberg_test/hadoop_catalog/ice && \
hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned /test-warehouse/iceberg_test/hadoop_catalog/ice hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned /test-warehouse/iceberg_test/hadoop_catalog/ice
@@ -3625,7 +3625,7 @@ STORED AS ICEBERG
TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.catalog', TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.catalog',
'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog', 'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog',
'iceberg.table_identifier'='ice.iceberg_v2_delete_equality_partition_evolution', 'iceberg.table_identifier'='ice.iceberg_v2_delete_equality_partition_evolution',
'format-version'='2', 'write.update.mode'='merge-on-read'); 'format-version'='2');
---- DEPENDENT_LOAD ---- DEPENDENT_LOAD
`hadoop fs -mkdir -p /test-warehouse/iceberg_test/hadoop_catalog/ice && \ `hadoop fs -mkdir -p /test-warehouse/iceberg_test/hadoop_catalog/ice && \
hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partition_evolution /test-warehouse/iceberg_test/hadoop_catalog/ice hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partition_evolution /test-warehouse/iceberg_test/hadoop_catalog/ice
@@ -3633,6 +3633,36 @@ hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/i
---- DATASET ---- DATASET
functional functional
---- BASE_TABLE_NAME ---- BASE_TABLE_NAME
iceberg_v2_delete_equality_multi_eq_ids
---- CREATE
CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name}
STORED AS ICEBERG
TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.catalog',
'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog',
'iceberg.table_identifier'='ice.iceberg_v2_delete_equality_multi_eq_ids',
'format-version'='2');
---- DEPENDENT_LOAD
`hadoop fs -mkdir -p /test-warehouse/iceberg_test/hadoop_catalog/ice && \
hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_multi_eq_ids /test-warehouse/iceberg_test/hadoop_catalog/ice
====
---- DATASET
functional
---- BASE_TABLE_NAME
iceberg_v2_delete_pos_and_multi_eq_ids
---- CREATE
CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name}
STORED AS ICEBERG
TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.catalog',
'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog',
'iceberg.table_identifier'='ice.iceberg_v2_delete_pos_and_multi_eq_ids',
'format-version'='2');
---- DEPENDENT_LOAD
`hadoop fs -mkdir -p /test-warehouse/iceberg_test/hadoop_catalog/ice && \
hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_pos_and_multi_eq_ids /test-warehouse/iceberg_test/hadoop_catalog/ice
====
---- DATASET
functional
---- BASE_TABLE_NAME
iceberg_multiple_storage_locations iceberg_multiple_storage_locations
---- CREATE ---- CREATE
CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name} CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name}

View File

@@ -88,6 +88,8 @@ table_name:iceberg_v2_delete_equality_nulls, constraint:restrict_to, table_forma
table_name:iceberg_v2_delete_both_eq_and_pos, constraint:restrict_to, table_format:parquet/none/none table_name:iceberg_v2_delete_both_eq_and_pos, constraint:restrict_to, table_format:parquet/none/none
table_name:iceberg_v2_delete_equality_partitioned, constraint:restrict_to, table_format:parquet/none/none table_name:iceberg_v2_delete_equality_partitioned, constraint:restrict_to, table_format:parquet/none/none
table_name:iceberg_v2_delete_equality_partition_evolution, constraint:restrict_to, table_format:parquet/none/none table_name:iceberg_v2_delete_equality_partition_evolution, constraint:restrict_to, table_format:parquet/none/none
table_name:iceberg_v2_delete_equality_multi_eq_ids, constraint:restrict_to, table_format:parquet/none/none
table_name:iceberg_v2_delete_pos_and_multi_eq_ids, constraint:restrict_to, table_format:parquet/none/none
table_name:iceberg_v2_no_deletes, constraint:restrict_to, table_format:parquet/none/none table_name:iceberg_v2_no_deletes, constraint:restrict_to, table_format:parquet/none/none
table_name:iceberg_v2_no_deletes_orc, constraint:restrict_to, table_format:parquet/none/none table_name:iceberg_v2_no_deletes_orc, constraint:restrict_to, table_format:parquet/none/none
table_name:iceberg_v2_positional_update_all_rows, constraint:restrict_to, table_format:parquet/none/none table_name:iceberg_v2_positional_update_all_rows, constraint:restrict_to, table_format:parquet/none/none
1 # Table level constraints:
88 table_name:iceberg_v2_positional_update_all_rows, constraint:restrict_to, table_format:parquet/none/none table_name:iceberg_v2_no_deletes, constraint:restrict_to, table_format:parquet/none/none
89 table_name:iceberg_v2_positional_delete_all_rows, constraint:restrict_to, table_format:parquet/none/none table_name:iceberg_v2_no_deletes_orc, constraint:restrict_to, table_format:parquet/none/none
90 table_name:iceberg_v2_positional_delete_all_rows_orc, constraint:restrict_to, table_format:parquet/none/none table_name:iceberg_v2_positional_update_all_rows, constraint:restrict_to, table_format:parquet/none/none
91 table_name:iceberg_v2_positional_delete_all_rows, constraint:restrict_to, table_format:parquet/none/none
92 table_name:iceberg_v2_positional_delete_all_rows_orc, constraint:restrict_to, table_format:parquet/none/none
93 table_name:iceberg_v2_positional_not_all_data_files_have_delete_files, constraint:restrict_to, table_format:parquet/none/none
94 table_name:iceberg_v2_positional_not_all_data_files_have_delete_files_orc, constraint:restrict_to, table_format:parquet/none/none
95 table_name:iceberg_v2_partitioned_position_deletes, constraint:restrict_to, table_format:parquet/none/none

View File

@@ -1566,3 +1566,291 @@ PLAN-ROOT SINK
Iceberg snapshot id: 4821964189199835313 Iceberg snapshot id: 4821964189199835313
row-size=28B cardinality=3 row-size=28B cardinality=3
==== ====
# Table has some equality delete files with different equality field ID lists. These have
# separate HASH JOINs. The equality delete file SCANs with higher cardinality happen
# earlier.
select * from functional_parquet.iceberg_v2_delete_equality_multi_eq_ids;
---- PLAN
PLAN-ROOT SINK
|
08:UNION
| pass-through-operands: all
| row-size=24B cardinality=10
|
|--06:HASH JOIN [LEFT ANTI JOIN]
| | hash predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.i IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-05.i
| | other join predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.iceberg__data__sequence__number < functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-05.iceberg__data__sequence__number
| | row-size=24B cardinality=8
| |
| |--05:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-05 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-05]
| | HDFS partitions=1/1 files=1 size=444B
| | Iceberg snapshot id: 4077234998626563290
| | row-size=12B cardinality=1
| |
| 04:HASH JOIN [LEFT ANTI JOIN]
| | hash predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.s IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03.s
| | other join predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.iceberg__data__sequence__number < functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03.iceberg__data__sequence__number
| | row-size=24B cardinality=8
| |
| |--03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-03 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03]
| | HDFS partitions=1/1 files=1 size=473B
| | Iceberg snapshot id: 4077234998626563290
| | row-size=20B cardinality=2
| |
| 02:HASH JOIN [LEFT ANTI JOIN]
| | hash predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.i IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01.i, functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.s IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01.s
| | other join predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.iceberg__data__sequence__number < functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01.iceberg__data__sequence__number
| | row-size=24B cardinality=8
| |
| |--01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01]
| | HDFS partitions=1/1 files=1 size=726B
| | Iceberg snapshot id: 4077234998626563290
| | row-size=24B cardinality=2
| |
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
| HDFS partitions=1/1 files=3 size=1.99KB
| Iceberg snapshot id: 4077234998626563290
| row-size=24B cardinality=8
|
07:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
HDFS partitions=1/1 files=1 size=713B
Iceberg snapshot id: 4077234998626563290
row-size=24B cardinality=2
---- DISTRIBUTEDPLAN
PLAN-ROOT SINK
|
12:EXCHANGE [UNPARTITIONED]
|
08:UNION
| pass-through-operands: all
| row-size=24B cardinality=10
|
|--06:HASH JOIN [LEFT ANTI JOIN, BROADCAST]
| | hash predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.i IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-05.i
| | other join predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.iceberg__data__sequence__number < functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-05.iceberg__data__sequence__number
| | row-size=24B cardinality=8
| |
| |--11:EXCHANGE [BROADCAST]
| | |
| | 05:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-05 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-05]
| | HDFS partitions=1/1 files=1 size=444B
| | Iceberg snapshot id: 4077234998626563290
| | row-size=12B cardinality=1
| |
| 04:HASH JOIN [LEFT ANTI JOIN, BROADCAST]
| | hash predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.s IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03.s
| | other join predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.iceberg__data__sequence__number < functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03.iceberg__data__sequence__number
| | row-size=24B cardinality=8
| |
| |--10:EXCHANGE [BROADCAST]
| | |
| | 03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-03 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03]
| | HDFS partitions=1/1 files=1 size=473B
| | Iceberg snapshot id: 4077234998626563290
| | row-size=20B cardinality=2
| |
| 02:HASH JOIN [LEFT ANTI JOIN, BROADCAST]
| | hash predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.i IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01.i, functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.s IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01.s
| | other join predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.iceberg__data__sequence__number < functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01.iceberg__data__sequence__number
| | row-size=24B cardinality=8
| |
| |--09:EXCHANGE [BROADCAST]
| | |
| | 01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01]
| | HDFS partitions=1/1 files=1 size=726B
| | Iceberg snapshot id: 4077234998626563290
| | row-size=24B cardinality=2
| |
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
| HDFS partitions=1/1 files=3 size=1.99KB
| Iceberg snapshot id: 4077234998626563290
| row-size=24B cardinality=8
|
07:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
HDFS partitions=1/1 files=1 size=713B
Iceberg snapshot id: 4077234998626563290
row-size=24B cardinality=2
====
# Similar as above but doing time travel to a snapshot where we don't have delete files
# for [i,s].
select * from functional_parquet.iceberg_v2_delete_equality_multi_eq_ids for system_version as of 5777805847908928861;
---- PLAN
PLAN-ROOT SINK
|
06:UNION
| pass-through-operands: all
| row-size=24B cardinality=8
|
|--04:HASH JOIN [LEFT ANTI JOIN]
| | hash predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.i IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03.i
| | other join predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.iceberg__data__sequence__number < functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03.iceberg__data__sequence__number
| | row-size=24B cardinality=4
| |
| |--03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-03 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03]
| | HDFS partitions=1/1 files=1 size=444B
| | Iceberg snapshot id: 5777805847908928861
| | row-size=12B cardinality=1
| |
| 02:HASH JOIN [LEFT ANTI JOIN]
| | hash predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.s IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01.s
| | other join predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.iceberg__data__sequence__number < functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01.iceberg__data__sequence__number
| | row-size=24B cardinality=4
| |
| |--01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01]
| | HDFS partitions=1/1 files=1 size=473B
| | Iceberg snapshot id: 5777805847908928861
| | row-size=20B cardinality=2
| |
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
| HDFS partitions=1/1 files=2 size=1.33KB
| Iceberg snapshot id: 5777805847908928861
| row-size=24B cardinality=4
|
05:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
HDFS partitions=1/1 files=1 size=673B
Iceberg snapshot id: 5777805847908928861
row-size=24B cardinality=4
---- DISTRIBUTEDPLAN
PLAN-ROOT SINK
|
09:EXCHANGE [UNPARTITIONED]
|
06:UNION
| pass-through-operands: all
| row-size=24B cardinality=8
|
|--04:HASH JOIN [LEFT ANTI JOIN, BROADCAST]
| | hash predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.i IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03.i
| | other join predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.iceberg__data__sequence__number < functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03.iceberg__data__sequence__number
| | row-size=24B cardinality=4
| |
| |--08:EXCHANGE [BROADCAST]
| | |
| | 03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-03 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03]
| | HDFS partitions=1/1 files=1 size=444B
| | Iceberg snapshot id: 5777805847908928861
| | row-size=12B cardinality=1
| |
| 02:HASH JOIN [LEFT ANTI JOIN, BROADCAST]
| | hash predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.s IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01.s
| | other join predicates: functional_parquet.iceberg_v2_delete_equality_multi_eq_ids.iceberg__data__sequence__number < functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01.iceberg__data__sequence__number
| | row-size=24B cardinality=4
| |
| |--07:EXCHANGE [BROADCAST]
| | |
| | 01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01]
| | HDFS partitions=1/1 files=1 size=473B
| | Iceberg snapshot id: 5777805847908928861
| | row-size=20B cardinality=2
| |
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
| HDFS partitions=1/1 files=2 size=1.33KB
| Iceberg snapshot id: 5777805847908928861
| row-size=24B cardinality=4
|
05:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
HDFS partitions=1/1 files=1 size=673B
Iceberg snapshot id: 5777805847908928861
row-size=24B cardinality=4
====
# Table has both positional and equality delete files where some equality delete files
# have different equality field ID lists. First the join for the positional deletes is
# executed and then the joins for the different equality field id lists.
select * from functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids;
---- PLAN
PLAN-ROOT SINK
|
08:UNION
| pass-through-operands: all
| row-size=48B cardinality=6
|
|--06:HASH JOIN [LEFT ANTI JOIN]
| | hash predicates: functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids.d IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-equality-delete-05.d, functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids.s IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-equality-delete-05.s
| | other join predicates: functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids.iceberg__data__sequence__number < functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-equality-delete-05.iceberg__data__sequence__number
| | row-size=48B cardinality=4
| |
| |--05:SCAN HDFS [functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-EQUALITY-DELETE-05 functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-equality-delete-05]
| | HDFS partitions=1/1 files=1 size=656B
| | Iceberg snapshot id: 152862018760071153
| | row-size=24B cardinality=2
| |
| 04:HASH JOIN [LEFT ANTI JOIN]
| | hash predicates: functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids.i IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-equality-delete-03.i, functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids.s IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-equality-delete-03.s
| | other join predicates: functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids.iceberg__data__sequence__number < functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-equality-delete-03.iceberg__data__sequence__number
| | row-size=48B cardinality=4
| |
| |--03:SCAN HDFS [functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-EQUALITY-DELETE-03 functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-equality-delete-03]
| | HDFS partitions=1/1 files=1 size=663B
| | Iceberg snapshot id: 152862018760071153
| | row-size=24B cardinality=2
| |
| 02:DELETE EVENTS ICEBERG DELETE [ICEBERG DELETE JOIN]
| | row-size=48B cardinality=4
| |
| |--01:SCAN HDFS [functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-POSITION-DELETE-01 functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-position-delete]
| | HDFS partitions=1/1 files=1 size=1.59KB
| | Iceberg snapshot id: 152862018760071153
| | row-size=192B cardinality=1
| |
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids]
| HDFS partitions=1/1 files=2 size=1.68KB
| Iceberg snapshot id: 152862018760071153
| row-size=48B cardinality=5
|
07:SCAN HDFS [functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids]
HDFS partitions=1/1 files=1 size=853B
Iceberg snapshot id: 152862018760071153
row-size=48B cardinality=2
---- DISTRIBUTEDPLAN
PLAN-ROOT SINK
|
12:EXCHANGE [UNPARTITIONED]
|
08:UNION
| pass-through-operands: all
| row-size=48B cardinality=6
|
|--06:HASH JOIN [LEFT ANTI JOIN, BROADCAST]
| | hash predicates: functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids.d IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-equality-delete-05.d, functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids.s IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-equality-delete-05.s
| | other join predicates: functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids.iceberg__data__sequence__number < functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-equality-delete-05.iceberg__data__sequence__number
| | row-size=48B cardinality=4
| |
| |--11:EXCHANGE [BROADCAST]
| | |
| | 05:SCAN HDFS [functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-EQUALITY-DELETE-05 functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-equality-delete-05]
| | HDFS partitions=1/1 files=1 size=656B
| | Iceberg snapshot id: 152862018760071153
| | row-size=24B cardinality=2
| |
| 04:HASH JOIN [LEFT ANTI JOIN, BROADCAST]
| | hash predicates: functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids.i IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-equality-delete-03.i, functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids.s IS NOT DISTINCT FROM functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-equality-delete-03.s
| | other join predicates: functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids.iceberg__data__sequence__number < functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-equality-delete-03.iceberg__data__sequence__number
| | row-size=48B cardinality=4
| |
| |--10:EXCHANGE [BROADCAST]
| | |
| | 03:SCAN HDFS [functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-EQUALITY-DELETE-03 functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-equality-delete-03]
| | HDFS partitions=1/1 files=1 size=663B
| | Iceberg snapshot id: 152862018760071153
| | row-size=24B cardinality=2
| |
| 02:DELETE EVENTS ICEBERG DELETE [ICEBERG DELETE JOIN, DIRECTED]
| | row-size=48B cardinality=4
| |
| |--09:EXCHANGE [DIRECTED]
| | |
| | 01:SCAN HDFS [functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-POSITION-DELETE-01 functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids-position-delete]
| | HDFS partitions=1/1 files=1 size=1.59KB
| | Iceberg snapshot id: 152862018760071153
| | row-size=192B cardinality=1
| |
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids]
| HDFS partitions=1/1 files=2 size=1.68KB
| Iceberg snapshot id: 152862018760071153
| row-size=48B cardinality=5
|
07:SCAN HDFS [functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids]
HDFS partitions=1/1 files=1 size=853B
Iceberg snapshot id: 152862018760071153
row-size=48B cardinality=2
====

View File

@@ -71,3 +71,65 @@ select * from functional_parquet.iceberg_v2_delete_equality_partitioned order by
---- TYPES ---- TYPES
INT,STRING,DATE INT,STRING,DATE
==== ====
---- QUERY
# There are equality deletes with different equality field ids.
select * from functional_parquet.iceberg_v2_delete_equality_multi_eq_ids;
---- RESULTS
1,'str1'
2222,'str2'
33,'str3_updated_twice'
4,'str4_updated'
5,'str5'
---- TYPES
INT,STRING
====
---- QUERY
# Similar as above, but do a time travel to the latest snapshot so that we can exercise a
# different code path.
select * from functional_parquet.iceberg_v2_delete_equality_multi_eq_ids
for system_version as of 4077234998626563290;
---- RESULTS
1,'str1'
2222,'str2'
33,'str3_updated_twice'
4,'str4_updated'
5,'str5'
---- TYPES
INT,STRING
====
---- QUERY
# Do a time travel to have a single equality field id list.
select * from functional_parquet.iceberg_v2_delete_equality_multi_eq_ids
for system_version as of 8127619959873391049;
---- RESULTS
1,'str1'
2,'str2'
3,'str3_updated'
---- TYPES
INT,STRING
====
---- QUERY
# The table has both equality and positional delete files where some of the equality
# delete files have different equality field ids.
select * from functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids;
---- RESULTS
1,'str1',2020-12-01
333333,'str3',2024-01-25
4,'str4',2024-01-26
5,'str5',2024-01-27
---- TYPES
INT,STRING,DATE
====
---- QUERY
# Similar as above, but do a time travel to the latest snapshot so that we can exercise a
# different code path.
select * from functional_parquet.iceberg_v2_delete_pos_and_multi_eq_ids
for system_version as of 152862018760071153;
---- RESULTS
1,'str1',2020-12-01
333333,'str3',2024-01-25
4,'str4',2024-01-26
5,'str5',2024-01-27
---- TYPES
INT,STRING,DATE
====

View File

@@ -1435,46 +1435,6 @@ class TestIcebergV2Table(IcebergTestSuite):
def test_read_equality_deletes(self, vector): def test_read_equality_deletes(self, vector):
self.run_test_case('QueryTest/iceberg-v2-read-equality-deletes', vector) self.run_test_case('QueryTest/iceberg-v2-read-equality-deletes', vector)
@SkipIfDockerizedCluster.internal_hostname
@SkipIf.hardcoded_uris
def test_multiple_equality_ids(self, unique_database):
"""This test loads an Iceberg table that has 2 equality delete files with different
equality ID lists. A query on such a table fails due to lack of support."""
SRC_DIR = os.path.join(os.environ['IMPALA_HOME'],
"testdata/data/iceberg_test/hadoop_catalog/ice/"
"iceberg_v2_delete_different_equality_ids")
DST_DIR = "/test-warehouse/iceberg_test/hadoop_catalog/ice/" \
"iceberg_v2_delete_different_equality_ids"
TBL_NAME = "iceberg_v2_delete_different_equality_ids"
FULL_TBL_NAME = unique_database + "." + TBL_NAME
try:
self.filesystem_client.make_dir(DST_DIR, permission=777)
self.filesystem_client.copy_from_local(os.path.join(SRC_DIR, "data"), DST_DIR)
self.filesystem_client.copy_from_local(os.path.join(SRC_DIR, "metadata"), DST_DIR)
self.execute_query_expect_success(self.client,
"""create external table {0} stored as iceberg
tblproperties('format-version'='2', 'iceberg.catalog'='hadoop.catalog',
'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog',
'iceberg.table_identifier'=
'ice.iceberg_v2_delete_different_equality_ids')""".format(FULL_TBL_NAME))
err = self.execute_query_expect_failure(self.client,
"select * from " + FULL_TBL_NAME)
assert "Equality delete files with different equality field ID lists aren't " \
"supported." in str(err)
# The error message also contains the mismatching equality ID lists but the order
# of the lists can change so we assert them separately. The end of the original
# message looks like: "[1] vs [1, 2]" or "[1, 2] vs [1]"
assert "[1, 2]" in str(err)
assert "[1]" in str(err)
finally:
# Clean up the test directory
self.filesystem_client.delete_file_dir(DST_DIR, True)
@SkipIfDockerizedCluster.internal_hostname @SkipIfDockerizedCluster.internal_hostname
@SkipIf.hardcoded_uris @SkipIf.hardcoded_uris
def test_read_position_deletes_orc(self, vector): def test_read_position_deletes_orc(self, vector):