IMPALA-12597: Basic Equality delete read support for Iceberg tables

In general, applying equality deletes is similar to how position
deletes are applied to data files: using a LEFT ANTI JOIN where the
SCAN for the data rows is on the left side while the SCAN for the
delete rows is on the right side of the JOIN. The difference is the
virtual columns and the conjuncts being used.
For equality deletes the data sequence number of a delete file has to
be greater than the data sequence number of the data file being
investigated. This information is added as a virtual column to the
scans and a conjunct is created in the JOIN node to check the relation.
The equality delete fields from the delete files are checked agains the
respective columns of the data SCANS.

This patch makes it possible for Impala to read Iceberg tables with
basic equality delete files. The Iceberg spec gives great flexibility
for engines for writing equality deletes, however in practice Flink,
one of the engines that write EQ-deletes supports only a subset of the
use cases. This patch focuses on reading the EQ-deletes written by
Flink.

The restrictions are the following:
- All equality delete files in a table should have the same equality
  field ID list.
- For partitioned Iceberg tables it is expected that the partition
  values are also written into the equality delete files.
- Tables with equality deletes shouldn't have partition or schema
  evolution.
- Floating point equality columns aren't supported.
- If a malformed equality delete file doesn't have some of the equality
  field IDs then Parquet reader will fill those missing fields with
  NULLs. As a side effect this will drop the rows from the result where
  the corresponding data columns have a null value.
See IMPALA-11388 epic Jira for more details.

Testing:
- Checked if the existing functional_parquet.iceberg_v2_delete_equality
  table can be read successfully.
- Added new test tables so that E2E tests can validate correctness.

Change-Id: I2053e6f321c69f1c82059a84a5d99aeaa9814cad
Reviewed-on: http://gerrit.cloudera.org:8080/20753
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
This commit is contained in:
Gabor Kaszab
2023-10-30 14:44:12 +01:00
committed by Impala Public Jenkins
parent 8c3cb56f64
commit 012996a06b
107 changed files with 3229 additions and 251 deletions

View File

@@ -82,7 +82,9 @@ class PhjBuilderConfig : public JoinBuilderConfig {
std::vector<ScalarExpr*> build_exprs_;
/// is_not_distinct_from_[i] is true if and only if the ith equi-join predicate is IS
/// NOT DISTINCT FROM, rather than equality.
/// NOT DISTINCT FROM, rather than equality. This is the case when IS NOT DISTINCT FROM
/// is explicitly used as a join predicate or when joining Iceberg equality delete
/// files to data files.
/// Set in InitExprsAndFilters() and constant thereafter.
std::vector<bool> is_not_distinct_from_;

View File

@@ -53,7 +53,9 @@ class PartitionedHashJoinPlanNode : public BlockingJoinPlanNode {
std::vector<ScalarExpr*> probe_exprs_;
/// is_not_distinct_from_[i] is true if and only if the ith equi-join predicate is IS
/// NOT DISTINCT FROM, rather than equality.
/// NOT DISTINCT FROM, rather than equality. This is the case when IS NOT DISTINCT FROM
/// is explicitly used as a join predicate or when joining Iceberg equality delete
/// files to data files.
std::vector<bool> is_not_distinct_from_;
/// Non-equi-join conjuncts from the ON clause.

View File

@@ -627,11 +627,13 @@ struct TIcebergContentFileStore {
1: optional map<string, THdfsFileDesc> path_hash_to_data_file_without_deletes
2: optional map<string, THdfsFileDesc> path_hash_to_data_file_with_deletes
3: optional map<string, THdfsFileDesc> path_hash_to_position_delete_file
7: optional map<string, THdfsFileDesc> path_hash_to_equality_delete_file
4: optional bool has_avro
5: optional bool has_orc
6: optional bool has_parquet
4: optional map<string, THdfsFileDesc> path_hash_to_equality_delete_file
5: optional list<i32> equality_ids
6: optional bool has_avro
7: optional bool has_orc
8: optional bool has_parquet
}
// Represents a drop partition request for Iceberg tables
struct TIcebergDropPartitionRequest {
// List of affected file paths (could be empty if the drop partition

View File

@@ -398,7 +398,13 @@ struct TEqJoinCondition {
1: required Exprs.TExpr left;
// right-hand side of "<a> = <b>"
2: required Exprs.TExpr right;
// true if and only if operator is "<=>", also known as "IS NOT DISTINCT FROM"
// In SQL NULL values aren't equal to each other, in other words NULL == NULL is false.
// However, there are some cases when joining tables where we'd like to have the
// NULL == NULL comparison to return true. This flag is true in this case.
// One example is when we join Iceberg equality delete files to the data files where we
// want the NULLs in the delete files to match with the NULLs in the data files.
// Another example is when this operator is a "<=>", also known as "IS NOT DISTINCT
// FROM".
3: required bool is_not_distinct_from;
}

View File

@@ -55,9 +55,8 @@ public class BinaryPredicate extends Predicate {
GT(">", "gt", TComparisonOp.GT),
DISTINCT_FROM("IS DISTINCT FROM", "distinctfrom", TComparisonOp.DISTINCT_FROM),
NOT_DISTINCT("IS NOT DISTINCT FROM", "notdistinct", TComparisonOp.NOT_DISTINCT),
// Same as EQ, except it returns True if the rhs is NULL. There is no backend
// function for this. The functionality is embedded in the hash-join
// implementation.
// Same as EQ, except it returns True if both sides are NULL. There is no backend
// function for this. The functionality is embedded in the hash-join implementation.
NULL_MATCHING_EQ("=", "null_matching_eq", TComparisonOp.EQ);
private final String description_;

View File

@@ -761,7 +761,7 @@ public interface FeIcebergTable extends FeFsTable {
*/
public static IcebergContentFileStore loadAllPartition(
IcebergTable table, GroupedContentFiles icebergFiles)
throws IOException {
throws IOException, ImpalaRuntimeException {
Map<String, HdfsPartition.FileDescriptor> hdfsFileDescMap = new HashMap<>();
Collection<HdfsPartition> partitions =
((HdfsTable)table.getFeFsTable()).partitionMap_.values();
@@ -781,16 +781,14 @@ public interface FeIcebergTable extends FeFsTable {
pathHashAndFd = getPathHashAndFd(dataFile, table, hdfsFileDescMap);
fileStore.addDataFileWithDeletes(pathHashAndFd.first, pathHashAndFd.second);
}
for (DeleteFile deleteFile : icebergFiles.deleteFiles) {
for (DeleteFile deleteFile : icebergFiles.positionDeleteFiles) {
pathHashAndFd = getPathHashAndFd(deleteFile, table, hdfsFileDescMap);
if (deleteFile.content().equals(FileContent.POSITION_DELETES)) {
fileStore.addPositionDeleteFile(pathHashAndFd.first, pathHashAndFd.second);
} else if (deleteFile.content().equals(FileContent.EQUALITY_DELETES)) {
fileStore.addEqualityDeleteFile(pathHashAndFd.first, pathHashAndFd.second);
} else {
Preconditions.checkState(false,
"Delete file with unknown kind: " + deleteFile.path().toString());
}
for (DeleteFile deleteFile : icebergFiles.equalityDeleteFiles) {
pathHashAndFd = getPathHashAndFd(deleteFile, table, hdfsFileDescMap);
fileStore.addEqualityDeleteFile(pathHashAndFd.first, pathHashAndFd.second,
deleteFile.equalityFieldIds());
}
return fileStore;
}

View File

@@ -21,13 +21,17 @@ import com.google.common.collect.Iterables;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
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.impala.catalog.HdfsPartition.FileDescriptor;
import org.apache.impala.common.ImpalaRuntimeException;
import org.apache.impala.fb.FbIcebergDataFileFormat;
import org.apache.impala.thrift.THdfsFileDesc;
import org.apache.impala.thrift.TIcebergContentFileStore;
@@ -98,6 +102,12 @@ public class IcebergContentFileStore {
private MapListContainer positionDeleteFiles_ = 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.
private final ConcurrentMap<String, FileDescriptor> oldFileDescMap_ =
new ConcurrentHashMap<>();
@@ -127,10 +137,21 @@ public class IcebergContentFileStore {
}
}
public void addEqualityDeleteFile(String pathHash, FileDescriptor desc) {
if (equalityDeleteFiles_.add(pathHash, desc)) {
updateFileFormats(desc);
public void addEqualityDeleteFile(String pathHash, FileDescriptor desc,
List<Integer> equalityIds) throws ImpalaRuntimeException {
if (equalityIds.isEmpty()) {
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);
}
// This is only invoked during time travel, when we are querying a snapshot that has
@@ -171,6 +192,8 @@ public class IcebergContentFileStore {
return equalityDeleteFiles_.getList();
}
public Set<Integer> getEqualityIds() { return equalityIds_; }
public long getNumFiles() {
return dataFilesWithoutDeletes_.getNumFiles() +
dataFilesWithDeletes_.getNumFiles() +
@@ -213,6 +236,7 @@ public class IcebergContentFileStore {
ret.setPath_hash_to_data_file_with_deletes(dataFilesWithDeletes_.toThrift());
ret.setPath_hash_to_position_delete_file(positionDeleteFiles_.toThrift());
ret.setPath_hash_to_equality_delete_file(equalityDeleteFiles_.toThrift());
ret.setEquality_ids(Lists.newArrayList(equalityIds_));
ret.setHas_avro(hasAvro_);
ret.setHas_orc(hasOrc_);
ret.setHas_parquet(hasParquet_);
@@ -243,6 +267,7 @@ public class IcebergContentFileStore {
tFileStore.getPath_hash_to_equality_delete_file(),
networkAddresses, hostIndex);
}
ret.equalityIds_.addAll(tFileStore.getEquality_ids());
ret.hasAvro_ = tFileStore.isSetHas_avro() ? tFileStore.isHas_avro() : false;
ret.hasOrc_ = tFileStore.isSetHas_orc() ? tFileStore.isHas_orc() : false;
ret.hasParquet_ = tFileStore.isSetHas_parquet() ? tFileStore.isHas_parquet() : false;

View File

@@ -0,0 +1,169 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.impala.catalog;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.commons.lang.NotImplementedException;
import org.apache.iceberg.Table;
import org.apache.impala.analysis.IcebergPartitionSpec;
import org.apache.impala.catalog.CatalogObject.ThriftObjectType;
import org.apache.impala.catalog.HdfsPartition.FileDescriptor;
import org.apache.impala.thrift.TCompressionCodec;
import org.apache.impala.thrift.THdfsTable;
import org.apache.impala.thrift.TIcebergCatalog;
import org.apache.impala.thrift.TIcebergFileFormat;
import org.apache.impala.thrift.TIcebergPartitionStats;
import org.apache.impala.thrift.TTableDescriptor;
import org.apache.impala.thrift.TTableStats;
/**
* Base class for the virtual table implementations for Iceberg deletes, like position or
* equality deletes.
*/
public abstract class IcebergDeleteTable extends VirtualTable implements FeIcebergTable {
protected final static int INVALID_MAP_KEY_ID = -1;
protected final static int INVALID_MAP_VALUE_ID = -1;
protected FeIcebergTable baseTable_;
protected Set<FileDescriptor> deleteFiles_;
protected long deleteRecordsCount_;
public IcebergDeleteTable(FeIcebergTable baseTable, String name,
Set<FileDescriptor> deleteFiles, long deleteRecordsCount) {
super(baseTable.getMetaStoreTable(), baseTable.getDb(), name,
baseTable.getOwnerUser());
baseTable_ = baseTable;
deleteFiles_ = deleteFiles;
deleteRecordsCount_ = deleteRecordsCount;
}
public FeIcebergTable getBaseTable() { return baseTable_; }
@Override
public long getNumRows() {
return deleteRecordsCount_;
}
@Override
public TTableStats getTTableStats() {
long totalBytes = 0;
for (FileDescriptor df : deleteFiles_) {
totalBytes += df.getFileLength();
}
TTableStats ret = new TTableStats(getNumRows());
ret.setTotal_file_bytes(totalBytes);
return ret;
}
/**
* Return same descriptor as the base table, but with a schema that corresponds to
* the schema of this delete table (including virtual columns).
*/
@Override
public TTableDescriptor toThriftDescriptor(int tableId,
Set<Long> referencedPartitions) {
TTableDescriptor desc =
baseTable_.toThriftDescriptor(tableId, referencedPartitions);
desc.setColumnDescriptors(FeCatalogUtils.getTColumnDescriptors(this));
return desc;
}
@Override
public IcebergContentFileStore getContentFileStore() {
throw new NotImplementedException("This should never be called.");
}
@Override
public Map<String, TIcebergPartitionStats> getIcebergPartitionStats(){
return null;
}
@Override
public FeFsTable getFeFsTable() {
return baseTable_.getFeFsTable();
}
@Override
public TIcebergCatalog getIcebergCatalog() {
return null;
}
@Override
public Table getIcebergApiTable() {
return baseTable_.getIcebergApiTable();
}
@Override
public String getIcebergCatalogLocation() {
return null;
}
@Override
public TIcebergFileFormat getIcebergFileFormat() {
return baseTable_.getIcebergFileFormat();
}
@Override
public TCompressionCodec getIcebergParquetCompressionCodec() {
return null;
}
@Override
public long getIcebergParquetRowGroupSize() {
return baseTable_.getIcebergParquetRowGroupSize();
}
@Override
public long getIcebergParquetPlainPageSize() {
return baseTable_.getIcebergParquetPlainPageSize();
}
@Override
public long getIcebergParquetDictPageSize() {
return baseTable_.getIcebergParquetDictPageSize();
}
@Override
public String getIcebergTableLocation() {
return null;
}
@Override
public List<IcebergPartitionSpec> getPartitionSpecs() {
return baseTable_.getPartitionSpecs();
}
@Override
public IcebergPartitionSpec getDefaultPartitionSpec() {
return null;
}
@Override
public int getDefaultPartitionSpecId() {
return -1;
}
@Override
public THdfsTable transformToTHdfsTable(boolean updatePartitionFlag,
ThriftObjectType type) {
throw new IllegalStateException("not implemented here");
}
}

View File

@@ -0,0 +1,68 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.impala.catalog;
import java.util.Arrays;
import java.util.List;
import java.util.Set;
import org.apache.impala.catalog.HdfsPartition.FileDescriptor;
import org.apache.impala.common.ImpalaRuntimeException;
import org.apache.impala.util.IcebergSchemaConverter;
import org.apache.iceberg.types.Types.NestedField;
/**
* Iceberg equality delete table is created on the fly during planning. It belongs to an
* actual Iceberg table (referred to as 'baseTable_'), but has a schema that corresponds
* to the file schema of equality delete files. Therefore with the help of it we can
* do an ANTI JOIN between data files and equality delete files.
*/
public class IcebergEqualityDeleteTable extends IcebergDeleteTable {
public IcebergEqualityDeleteTable(FeIcebergTable baseTable, String name,
Set<FileDescriptor> deleteFiles, Set<Integer> equalityIds, long deleteRecordsCount)
throws ImpalaRuntimeException {
super(baseTable, name, deleteFiles, deleteRecordsCount);
int columnPos = 0;
for (Integer eqId : equalityIds) {
++columnPos;
NestedField field = baseTable.getIcebergSchema().findField(eqId);
Type colType = IcebergSchemaConverter.toImpalaType(field.type());
if (colType.isComplexType()) {
throw new ImpalaRuntimeException(
"Equality ID for nested types isn't supported: '" + field.name() + "'");
} else if (colType.isFloatingPointType()) {
throw new ImpalaRuntimeException(
"Equality ID for floating point types isn't supported: '" +
field.name() + "'");
}
Column equalityCol = new IcebergColumn(field.name(), colType, field.doc(),
columnPos, field.fieldId(), INVALID_MAP_KEY_ID, INVALID_MAP_VALUE_ID,
field.isOptional());
addColumn(equalityCol);
}
}
@Override
public List<VirtualColumn> getVirtualColumns() {
return Arrays.asList(VirtualColumn.ICEBERG_DATA_SEQUENCE_NUMBER);
}
}

View File

@@ -18,35 +18,18 @@
package org.apache.impala.catalog;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.commons.lang.NotImplementedException;
import org.apache.iceberg.Table;
import org.apache.impala.catalog.CatalogObject.ThriftObjectType;
import org.apache.impala.catalog.HdfsPartition.FileDescriptor;
import org.apache.impala.analysis.IcebergPartitionSpec;
import org.apache.impala.thrift.TColumnStats;
import org.apache.impala.thrift.TCompressionCodec;
import org.apache.impala.thrift.THdfsTable;
import org.apache.impala.thrift.TIcebergCatalog;
import org.apache.impala.thrift.TIcebergFileFormat;
import org.apache.impala.thrift.TIcebergPartitionStats;
import org.apache.impala.thrift.TTableDescriptor;
import org.apache.impala.thrift.TTableStats;
/**
* Iceberg position delete table is created on the fly during planning. It belongs to an
* actual Iceberg table (referred to as 'baseTable_'), but has a schema that corresponds
* to the file schema of position delete files. Therefore with the help of it we can
* do an ANTI JOIN between data files and delete files.
* do an ANTI JOIN between data files and position delete files.
*/
public class IcebergPositionDeleteTable extends VirtualTable implements FeIcebergTable {
private FeIcebergTable baseTable_;
private Set<FileDescriptor> deleteFiles_;
private long deleteRecordsCount_;
public class IcebergPositionDeleteTable extends IcebergDeleteTable {
public static String FILE_PATH_COLUMN = "file_path";
public static String POS_COLUMN = "pos";
@@ -58,24 +41,19 @@ public class IcebergPositionDeleteTable extends VirtualTable implements FeIceber
public IcebergPositionDeleteTable(FeIcebergTable baseTable, String name,
Set<FileDescriptor> deleteFiles,
long deleteRecordsCount, TColumnStats filePathsStats) {
super(baseTable.getMetaStoreTable(), baseTable.getDb(), name,
baseTable.getOwnerUser());
baseTable_ = baseTable;
deleteFiles_ = deleteFiles;
deleteRecordsCount_ = deleteRecordsCount;
super(baseTable, name, deleteFiles, deleteRecordsCount);
Column filePath = new IcebergColumn(FILE_PATH_COLUMN, Type.STRING, /*comment=*/"",
colsByPos_.size(), IcebergTable.V2_FILE_PATH_FIELD_ID, -1, -1,
/*nullable=*/false);
colsByPos_.size(), IcebergTable.V2_FILE_PATH_FIELD_ID, INVALID_MAP_KEY_ID,
INVALID_MAP_VALUE_ID, /*nullable=*/false);
Column pos = new IcebergColumn(POS_COLUMN, Type.BIGINT, /*comment=*/"",
colsByPos_.size(), IcebergTable.V2_POS_FIELD_ID, -1, -1, /*nullable=*/false);
colsByPos_.size(), IcebergTable.V2_POS_FIELD_ID, INVALID_MAP_KEY_ID,
INVALID_MAP_VALUE_ID, /*nullable=*/false);
filePath.updateStats(filePathsStats);
pos.updateStats(getPosStats(pos));
addColumn(filePath);
addColumn(pos);
}
public FeIcebergTable getBaseTable() { return baseTable_; }
private TColumnStats getPosStats(Column pos) {
TColumnStats colStats = new TColumnStats();
colStats.num_distinct_values = deleteRecordsCount_;
@@ -83,114 +61,4 @@ public class IcebergPositionDeleteTable extends VirtualTable implements FeIceber
colStats.max_size = pos.getType().getSlotSize();
return colStats;
}
@Override
public long getNumRows() {
return deleteRecordsCount_;
}
@Override
public TTableStats getTTableStats() {
long totalBytes = 0;
for (FileDescriptor df : deleteFiles_) {
totalBytes += df.getFileLength();
}
TTableStats ret = new TTableStats(getNumRows());
ret.setTotal_file_bytes(totalBytes);
return ret;
}
/**
* Return same descriptor as the base table, but with a schema that corresponds to
* the position delete file schema ('file_path', 'pos').
*/
@Override
public TTableDescriptor toThriftDescriptor(int tableId,
Set<Long> referencedPartitions) {
TTableDescriptor desc = baseTable_.toThriftDescriptor(tableId, referencedPartitions);
desc.setColumnDescriptors(FeCatalogUtils.getTColumnDescriptors(this));
return desc;
}
@Override
public IcebergContentFileStore getContentFileStore() {
throw new NotImplementedException("This should never be called.");
}
@Override
public Map<String, TIcebergPartitionStats> getIcebergPartitionStats() {
return null;
}
@Override
public FeFsTable getFeFsTable() {
return baseTable_.getFeFsTable();
}
@Override
public TIcebergCatalog getIcebergCatalog() {
return null;
}
@Override
public Table getIcebergApiTable() {
return baseTable_.getIcebergApiTable();
}
@Override
public String getIcebergCatalogLocation() {
return null;
}
@Override
public TIcebergFileFormat getIcebergFileFormat() {
return baseTable_.getIcebergFileFormat();
}
@Override
public TCompressionCodec getIcebergParquetCompressionCodec() {
return null;
}
@Override
public long getIcebergParquetRowGroupSize() {
return baseTable_.getIcebergParquetRowGroupSize();
}
@Override
public long getIcebergParquetPlainPageSize() {
return baseTable_.getIcebergParquetPlainPageSize();
}
@Override
public long getIcebergParquetDictPageSize() {
return baseTable_.getIcebergParquetDictPageSize();
}
@Override
public String getIcebergTableLocation() {
return null;
}
@Override
public List<IcebergPartitionSpec> getPartitionSpecs() {
return baseTable_.getPartitionSpecs();
}
@Override
public IcebergPartitionSpec getDefaultPartitionSpec() {
return null;
}
@Override
public int getDefaultPartitionSpecId() {
return -1;
}
@Override
public THdfsTable transformToTHdfsTable(boolean updatePartitionFlag,
ThriftObjectType type) {
throw new IllegalStateException("not implemented here");
}
}

View File

@@ -22,23 +22,27 @@ import java.util.HashSet;
import java.util.List;
import java.util.Set;
import com.google.common.collect.Iterables;
import org.apache.iceberg.ContentFile;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DeleteFile;
import org.apache.iceberg.FileContent;
import org.apache.iceberg.FileScanTask;
import org.apache.iceberg.io.CloseableIterable;
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables;
/**
* Struct-like object to group different Iceberg content files:
* - data files without deleted rows
* - data files with deleted rows
* - delete files
* - delete files (position and equality)
*/
public class GroupedContentFiles {
public List<DataFile> dataFilesWithoutDeletes = new ArrayList<>();
public List<DataFile> dataFilesWithDeletes = new ArrayList<>();
public Set<DeleteFile> deleteFiles = new HashSet<>();
public Set<DeleteFile> positionDeleteFiles = new HashSet<>();
public Set<DeleteFile> equalityDeleteFiles = new HashSet<>();
public GroupedContentFiles() { }
@@ -48,18 +52,26 @@ public class GroupedContentFiles {
dataFilesWithoutDeletes.add(scanTask.file());
} else {
dataFilesWithDeletes.add(scanTask.file());
deleteFiles.addAll(scanTask.deletes());
for (DeleteFile delFile : scanTask.deletes()) {
if (delFile.content() == FileContent.POSITION_DELETES) {
positionDeleteFiles.add(delFile);
} else {
Preconditions.checkState(delFile.content() == FileContent.EQUALITY_DELETES);
equalityDeleteFiles.add(delFile);
}
}
}
}
}
public Iterable<ContentFile<?>> getAllContentFiles() {
return Iterables.concat(dataFilesWithoutDeletes, dataFilesWithDeletes, deleteFiles);
return Iterables.concat(dataFilesWithoutDeletes, dataFilesWithDeletes,
positionDeleteFiles, equalityDeleteFiles);
}
public int size() {
return dataFilesWithDeletes.size() + dataFilesWithoutDeletes.size() +
deleteFiles.size();
positionDeleteFiles.size() + equalityDeleteFiles.size();
}
public boolean isEmpty() {

View File

@@ -49,8 +49,6 @@ import org.apache.impala.analysis.BinaryPredicate;
import org.apache.impala.analysis.BinaryPredicate.Operator;
import org.apache.impala.analysis.Expr;
import org.apache.impala.analysis.IcebergExpressionCollector;
import org.apache.impala.analysis.InPredicate;
import org.apache.impala.analysis.IsNullPredicate;
import org.apache.impala.analysis.JoinOperator;
import org.apache.impala.analysis.MultiAggregateInfo;
import org.apache.impala.analysis.SlotDescriptor;
@@ -66,6 +64,7 @@ import org.apache.impala.catalog.FeIcebergTable;
import org.apache.impala.catalog.HdfsPartition.FileDescriptor;
import org.apache.impala.catalog.IcebergColumn;
import org.apache.impala.catalog.IcebergContentFileStore;
import org.apache.impala.catalog.IcebergEqualityDeleteTable;
import org.apache.impala.catalog.IcebergPositionDeleteTable;
import org.apache.impala.catalog.IcebergTable;
import org.apache.impala.catalog.TableLoadingException;
@@ -113,16 +112,22 @@ public class IcebergScanPlanner {
private final List<Expr> untranslatedExpressions_ = new ArrayList<>();
// Conjuncts on columns not involved in IDENTITY-partitioning.
private List<Expr> nonIdentityConjuncts_ = new ArrayList<>();
// Containers for different groupings of file descriptors.
private List<FileDescriptor> dataFilesWithoutDeletes_ = new ArrayList<>();
private List<FileDescriptor> dataFilesWithDeletes_ = new ArrayList<>();
private Set<FileDescriptor> deleteFiles_ = 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.
private Set<Integer> equalityIds_ = new HashSet<>();
// Statistics about the data and delete files. Useful for memory estimates of the
// ANTI JOIN
private long deletesRecordCount_ = 0;
private long positionDeletesRecordCount_ = 0;
private long equalityDeletesRecordCount_ = 0;
private long dataFilesWithDeletesSumPaths_ = 0;
private long dataFilesWithDeletesMaxPath_ = 0;
private Set<Long> equalityDeleteSequenceNumbers_ = new HashSet<>();
public IcebergScanPlanner(Analyzer analyzer, PlannerContext ctx,
TableRef iceTblRef, List<Expr> conjuncts, MultiAggregateInfo aggInfo)
@@ -163,23 +168,21 @@ public class IcebergScanPlanner {
private void setFileDescriptorsBasedOnFileStore() throws ImpalaException {
IcebergContentFileStore fileStore = getIceTable().getContentFileStore();
if (!fileStore.getEqualityDeleteFiles().isEmpty()) {
// TODO(IMPALA-11388): Add support for equality deletes.
FileDescriptor firstEqualityDeleteFile = fileStore.getEqualityDeleteFiles().get(0);
throw new ImpalaRuntimeException(String.format(
"Iceberg table %s has EQUALITY delete file which is currently " +
"not supported by Impala, for example: %s",
getIceTable().getFullName(),
firstEqualityDeleteFile.getAbsolutePath(getIceTable().getLocation())));
}
dataFilesWithoutDeletes_ = fileStore.getDataFilesWithoutDeletes();
dataFilesWithDeletes_ = fileStore.getDataFilesWithDeletes();
deleteFiles_ = new HashSet<>(fileStore.getPositionDeleteFiles());
positionDeleteFiles_ = new HashSet<>(fileStore.getPositionDeleteFiles());
equalityDeleteFiles_ = new HashSet<>(fileStore.getEqualityDeleteFiles());
equalityIds_ = fileStore.getEqualityIds();
updateDeleteStatistics();
}
private boolean noDeleteFiles() {
return positionDeleteFiles_.isEmpty() && equalityDeleteFiles_.isEmpty();
}
private PlanNode createIcebergScanPlanImpl() throws ImpalaException {
if (deleteFiles_.isEmpty()) {
if (noDeleteFiles()) {
// If there are no delete files we can just create a single SCAN node.
Preconditions.checkState(dataFilesWithDeletes_.isEmpty());
PlanNode ret = new IcebergScanNode(ctx_.getNextNodeId(), tblRef_, conjuncts_,
@@ -188,7 +191,12 @@ public class IcebergScanPlanner {
ret.init(analyzer_);
return ret;
}
PlanNode joinNode = createPositionJoinNode();
PlanNode joinNode = null;
if (!positionDeleteFiles_.isEmpty()) joinNode = createPositionJoinNode();
if (!equalityDeleteFiles_.isEmpty()) joinNode = createEqualityJoinNode(joinNode);
Preconditions.checkNotNull(joinNode);
// If the count star query can be optimized for Iceberg V2 table, the number of rows
// of all DataFiles without corresponding DeleteFiles can be calculated by Iceberg
@@ -219,7 +227,7 @@ public class IcebergScanPlanner {
}
private PlanNode createPositionJoinNode() throws ImpalaException {
Preconditions.checkState(deletesRecordCount_ != 0);
Preconditions.checkState(positionDeletesRecordCount_ != 0);
Preconditions.checkState(dataFilesWithDeletesSumPaths_ != 0);
Preconditions.checkState(dataFilesWithDeletesMaxPath_ != 0);
// The followings just create separate scan nodes for data files and position delete
@@ -228,12 +236,13 @@ public class IcebergScanPlanner {
PlanNodeId deleteScanNodeId = ctx_.getNextNodeId();
IcebergPositionDeleteTable deleteTable = new IcebergPositionDeleteTable(getIceTable(),
getIceTable().getName() + "-POSITION-DELETE-" + deleteScanNodeId.toString(),
deleteFiles_, deletesRecordCount_, getFilePathStats());
positionDeleteFiles_, positionDeletesRecordCount_, getFilePathStats());
analyzer_.addVirtualTable(deleteTable);
TableRef deleteDeltaRef = TableRef.newTableRef(analyzer_,
Arrays.asList(deleteTable.getDb().getName(), deleteTable.getName()),
tblRef_.getUniqueAlias() + "-position-delete");
addDataVirtualPositionSlots(tblRef_);
if (!equalityDeleteFiles_.isEmpty()) addSlotsForEqualityDelete(equalityIds_, tblRef_);
addDeletePositionSlots(deleteDeltaRef);
IcebergScanNode dataScanNode = new IcebergScanNode(
dataScanNodeId, tblRef_, conjuncts_, aggInfo_, dataFilesWithDeletes_,
@@ -244,7 +253,7 @@ public class IcebergScanPlanner {
deleteDeltaRef,
Collections.emptyList(), /*conjuncts*/
aggInfo_,
Lists.newArrayList(deleteFiles_),
Lists.newArrayList(positionDeleteFiles_),
Collections.emptyList(), /*nonIdentityConjuncts*/
Collections.emptyList()); /*skippedConjuncts*/
deleteScanNode.init(analyzer_);
@@ -270,38 +279,43 @@ public class IcebergScanPlanner {
}
private void addDataVirtualPositionSlots(TableRef tblRef) throws AnalysisException {
List<String> rawPath = new ArrayList<>();
rawPath.add(tblRef.getUniqueAlias());
// Add slot refs for position delete fields;
String[] posFields = {VirtualColumn.INPUT_FILE_NAME.getName(),
VirtualColumn.FILE_POSITION.getName()};
for (String posField : posFields) {
rawPath.add(posField);
List<String> rawPath = Lists.newArrayList(
tblRef.getUniqueAlias(), VirtualColumn.INPUT_FILE_NAME.getName());
SlotDescriptor fileNameSlotDesc =
SingleNodePlanner.addSlotRefToDesc(analyzer_, rawPath);
rawPath.remove(rawPath.size() - 1);
}
for (SlotDescriptor insertSlotDesc : tblRef.getDesc().getSlots()) {
TVirtualColumnType virtColType = insertSlotDesc.getVirtualColumnType();
if (virtColType == TVirtualColumnType.INPUT_FILE_NAME) {
insertSlotDesc.setStats(virtualInputFileNameStats());
} else if (virtColType == TVirtualColumnType.FILE_POSITION) {
insertSlotDesc.setStats(virtualFilePositionStats());
fileNameSlotDesc.setStats(virtualInputFileNameStats());
rawPath = Lists.newArrayList(
tblRef.getUniqueAlias(), VirtualColumn.FILE_POSITION.getName());
SlotDescriptor filePosSlotDesc =
SingleNodePlanner.addSlotRefToDesc(analyzer_, rawPath);
filePosSlotDesc.setStats(virtualFilePositionStats());
}
private void addSlotsForEqualityDelete(Set<Integer> equalityIds, TableRef tblRef)
throws AnalysisException {
List<String> rawPath = Lists.newArrayList(
tblRef.getUniqueAlias(), VirtualColumn.ICEBERG_DATA_SEQUENCE_NUMBER.getName());
SlotDescriptor slotDesc = SingleNodePlanner.addSlotRefToDesc(analyzer_, rawPath);
slotDesc.setStats(virtualDataSeqNumStats());
Preconditions.checkState(!equalityIds.isEmpty());
for (Integer eqId : equalityIds) {
String eqColName = getIceTable().getIcebergSchema().findColumnName(eqId);
Preconditions.checkNotNull(eqColName);
rawPath = Lists.newArrayList(tblRef.getUniqueAlias(), eqColName);
SingleNodePlanner.addSlotRefToDesc(analyzer_, rawPath);
}
}
private void addDeletePositionSlots(TableRef tblRef)
throws AnalysisException {
List<String> rawPath = new ArrayList<>();
rawPath.add(tblRef.getUniqueAlias());
// Add slot refs for position delete fields;
String[] posFields = {IcebergPositionDeleteTable.FILE_PATH_COLUMN,
IcebergPositionDeleteTable.POS_COLUMN};
for (String posField : posFields) {
rawPath.add(posField);
SingleNodePlanner.addSlotRefToDesc(analyzer_, rawPath);
rawPath.remove(rawPath.size() - 1);
}
SingleNodePlanner.addSlotRefToDesc(analyzer_,
Lists.newArrayList(
tblRef.getUniqueAlias(), IcebergPositionDeleteTable.FILE_PATH_COLUMN));
SingleNodePlanner.addSlotRefToDesc(analyzer_,
Lists.newArrayList(
tblRef.getUniqueAlias(), IcebergPositionDeleteTable.POS_COLUMN));
}
private List<BinaryPredicate> createPositionJoinConjuncts(Analyzer analyzer,
@@ -344,6 +358,48 @@ public class IcebergScanPlanner {
return ret;
}
private Pair<List<BinaryPredicate>, List<Expr>> createEqualityJoinConjuncts(
Analyzer analyzer, TupleDescriptor dataTupleDesc, TupleDescriptor deleteTupleDesc)
throws AnalysisException, ImpalaRuntimeException {
// Pre-process the slots for faster lookup by field ID.
Map<Integer, SlotDescriptor> fieldIdToIcebergColumn = new HashMap<>();
SlotDescriptor dataSeqNumSlot = null;
for (SlotDescriptor dataSlotDesc : dataTupleDesc.getSlots()) {
if (dataSlotDesc.getVirtualColumnType() ==
TVirtualColumnType.ICEBERG_DATA_SEQUENCE_NUMBER) {
dataSeqNumSlot = dataSlotDesc;
} else if (dataSlotDesc.getColumn() instanceof IcebergColumn) {
IcebergColumn icebergCol = (IcebergColumn)dataSlotDesc.getColumn();
fieldIdToIcebergColumn.put(icebergCol.getFieldId(), dataSlotDesc);
}
}
List<BinaryPredicate> eqPredicates = new ArrayList<>();
List<Expr> seqNumPredicate = new ArrayList<>();
for (SlotDescriptor deleteSlotDesc : deleteTupleDesc.getSlots()) {
if (deleteSlotDesc.getVirtualColumnType() ==
TVirtualColumnType.ICEBERG_DATA_SEQUENCE_NUMBER) {
BinaryPredicate pred = new BinaryPredicate(Operator.LT,
new SlotRef(dataSeqNumSlot), new SlotRef(deleteSlotDesc));
pred.analyze(analyzer);
seqNumPredicate.add(pred);
} else {
Preconditions.checkState(deleteSlotDesc.getColumn() instanceof IcebergColumn);
int fieldId = ((IcebergColumn)deleteSlotDesc.getColumn()).getFieldId();
if (!fieldIdToIcebergColumn.containsKey(fieldId)) {
throw new ImpalaRuntimeException("Field ID not found in table: " + fieldId);
}
SlotRef dataSlotRef = new SlotRef(fieldIdToIcebergColumn.get(fieldId));
SlotRef deleteSlotRef = new SlotRef(deleteSlotDesc);
BinaryPredicate eqColPred = new BinaryPredicate(
Operator.NOT_DISTINCT, dataSlotRef, deleteSlotRef);
eqColPred.analyze(analyzer);
eqPredicates.add(eqColPred);
}
}
return new Pair<>(eqPredicates, seqNumPredicate);
}
private ColumnStats virtualInputFileNameStats() {
ColumnStats ret = new ColumnStats(Type.STRING);
ret.setNumDistinctValues(dataFilesWithDeletes_.size());
@@ -352,10 +408,77 @@ public class IcebergScanPlanner {
private ColumnStats virtualFilePositionStats() {
ColumnStats ret = new ColumnStats(Type.BIGINT);
ret.setNumDistinctValues(deletesRecordCount_ / dataFilesWithDeletes_.size());
ret.setNumDistinctValues(positionDeletesRecordCount_ / dataFilesWithDeletes_.size());
return ret;
}
private ColumnStats virtualDataSeqNumStats() {
ColumnStats ret = new ColumnStats(Type.BIGINT);
ret.setNumDistinctValues(equalityDeleteSequenceNumbers_.size());
return ret;
}
private PlanNode createEqualityJoinNode(PlanNode positionJoinNode)
throws ImpalaException {
Preconditions.checkState(!equalityDeleteFiles_.isEmpty());
Preconditions.checkState(equalityDeletesRecordCount_ > 0);
if (getIceTable().getPartitionSpecs().size() > 1) {
throw new ImpalaRuntimeException("Equality delete files are not supported for " +
"tables with partition evolution");
}
PlanNode leftSideOfJoin = null;
if (positionJoinNode != null) {
leftSideOfJoin = positionJoinNode;
} else {
PlanNodeId dataScanNodeId = ctx_.getNextNodeId();
IcebergScanNode dataScanNode = new IcebergScanNode(
dataScanNodeId, tblRef_, conjuncts_, aggInfo_, dataFilesWithDeletes_,
nonIdentityConjuncts_, getSkippedConjuncts());
addSlotsForEqualityDelete(equalityIds_, tblRef_);
dataScanNode.init(analyzer_);
leftSideOfJoin = dataScanNode;
}
JoinNode joinNode = null;
PlanNodeId deleteScanNodeId = ctx_.getNextNodeId();
IcebergEqualityDeleteTable deleteTable =
new IcebergEqualityDeleteTable(getIceTable(),
getIceTable().getName() + "-EQUALITY-DELETE-" + deleteScanNodeId.toString(),
equalityDeleteFiles_, equalityIds_, equalityDeletesRecordCount_);
analyzer_.addVirtualTable(deleteTable);
TableRef deleteTblRef = TableRef.newTableRef(analyzer_,
Arrays.asList(deleteTable.getDb().getName(), deleteTable.getName()),
tblRef_.getUniqueAlias() + "-equality-delete-" + deleteScanNodeId.toString());
addSlotsForEqualityDelete(equalityIds_, deleteTblRef);
// TODO IMPALA-12608: As an optimization we can populate the conjuncts below that are
// relevant for the delete scan node.
IcebergScanNode deleteScanNode = new IcebergScanNode(
deleteScanNodeId,
deleteTblRef,
Collections.emptyList(), /*conjuncts*/
aggInfo_,
Lists.newArrayList(equalityDeleteFiles_),
Collections.emptyList(), /*nonIdentityConjuncts*/
Collections.emptyList()); /*skippedConjuncts*/
deleteScanNode.init(analyzer_);
Pair<List<BinaryPredicate>, List<Expr>> equalityJoinConjuncts =
createEqualityJoinConjuncts(
analyzer_, tblRef_.getDesc(), deleteTblRef.getDesc());
joinNode = new HashJoinNode(leftSideOfJoin, deleteScanNode,
/*straight_join=*/true, DistributionMode.NONE, JoinOperator.LEFT_ANTI_JOIN,
equalityJoinConjuncts.first, equalityJoinConjuncts.second);
joinNode.setId(ctx_.getNextNodeId());
joinNode.init(analyzer_);
return joinNode;
}
private void filterFileDescriptors() throws ImpalaException {
TimeTravelSpec timeTravelSpec = tblRef_.getTimeTravelSpec();
@@ -375,16 +498,15 @@ public class IcebergScanPlanner {
} else {
dataFilesWithDeletes_.add(fileDesc.first);
for (DeleteFile delFile : fileScanTask.deletes()) {
// TODO(IMPALA-11388): Add support for equality deletes.
if (delFile.content() == FileContent.EQUALITY_DELETES) {
throw new ImpalaRuntimeException(String.format(
"Iceberg table %s has EQUALITY delete file which is currently " +
"not supported by Impala, for example: %s", getIceTable().getFullName(),
delFile.path()));
}
Pair<FileDescriptor, Boolean> delFileDesc = getFileDescriptor(delFile);
if (!delFileDesc.second) ++dataFilesCacheMisses;
deleteFiles_.add(delFileDesc.first);
if (delFile.content() == FileContent.EQUALITY_DELETES) {
equalityDeleteFiles_.add(delFileDesc.first);
addEqualityIds(delFile.equalityFieldIds());
} else {
Preconditions.checkState(delFile.content() == FileContent.POSITION_DELETES);
positionDeleteFiles_.add(delFileDesc.first);
}
}
}
}
@@ -401,6 +523,19 @@ public class IcebergScanPlanner {
updateDeleteStatistics();
}
private void addEqualityIds(List<Integer> equalityFieldIds)
throws ImpalaRuntimeException {
if (equalityIds_.isEmpty()) {
equalityIds_.addAll(equalityFieldIds);
} else if (equalityIds_.size() != equalityFieldIds.size() ||
!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));
}
}
private void filterConjuncts() {
if (residualExpressions_.isEmpty()) {
conjuncts_.removeAll(impalaIcebergPredicateMapping_.values());
@@ -442,8 +577,11 @@ public class IcebergScanPlanner {
for (FileDescriptor fd : dataFilesWithDeletes_) {
updateDataFilesWithDeletesStatistics(fd);
}
for (FileDescriptor fd : deleteFiles_) {
updateDeleteFilesStatistics(fd);
for (FileDescriptor fd : positionDeleteFiles_) {
updatePositionDeleteFilesStatistics(fd);
}
for (FileDescriptor fd : equalityDeleteFiles_) {
updateEqualityDeleteFilesStatistics(fd);
}
}
@@ -456,8 +594,14 @@ public class IcebergScanPlanner {
}
}
private void updateDeleteFilesStatistics(FileDescriptor fd) {
deletesRecordCount_ += getRecordCount(fd);
private void updatePositionDeleteFilesStatistics(FileDescriptor fd) {
positionDeletesRecordCount_ += getRecordCount(fd);
}
private void updateEqualityDeleteFilesStatistics(FileDescriptor fd) {
equalityDeletesRecordCount_ += getRecordCount(fd);
equalityDeleteSequenceNumbers_.add(
fd.getFbFileMetadata().icebergMetadata().dataSequenceNumber());
}
private long getRecordCount(FileDescriptor fd) {

View File

@@ -1716,9 +1716,10 @@ public class SingleNodePlanner {
/**
* Adds a new slot ref with path 'rawPath' to its tuple descriptor. This is a no-op if
* the tuple descriptor already has a slot ref with the given raw path.
* the tuple descriptor already has a slot ref with the given raw path. Returns the slot
* descriptor (new or already existing) for 'rawPath'.
*/
public static void addSlotRefToDesc(Analyzer analyzer, List<String> rawPath)
public static SlotDescriptor addSlotRefToDesc(Analyzer analyzer, List<String> rawPath)
throws AnalysisException {
Path resolvedPath = null;
try {
@@ -1730,6 +1731,7 @@ public class SingleNodePlanner {
Preconditions.checkNotNull(resolvedPath);
SlotDescriptor desc = analyzer.registerSlotRef(resolvedPath);
desc.setIsMaterialized(true);
return desc;
}
/**

78
testdata/data/README vendored
View File

@@ -770,6 +770,84 @@ statements:
This table was created with HDFS absolute paths, which were replaced with the script
specified in `iceberg_test/hadoop_catalog/ice/iceberg_v2_no_deletes`.
iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_nulls:
This table has an equality delete file that contains NULL value. Created by a hacked
Impala where IcebergCatalogOpExecutor is changed to write equality delete metadata
instead of position delete metadata when running a DELETE FROM statement. In a second
step the underlying delete file was replaced by another parquet file with the desired
content.
The content:
1: insert into functional_parquet.iceberg_v2_delete_equality_nulls values (1, "str1"), (null, "str2"), (3, "str3");
2: EQ-delete file for the first column with values: (null), (3)
3: insert into functional_parquet.iceberg_v2_delete_equality_nulls values (4, "str4"), (null, "str5");
As a result 2 values (including the row with null) will be dropped from the first data
file, while there is going to be another data file containing a null value that has
greater data sequence number than the delete file.
iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos:
This table is created by Flink with 2 columns as primary key. Some data and equality
delete files were added by Flink, and then Impala was used for dropping a row by writing
a positional delete file.
Steps:
1-Flink:
create table ice.iceberg_v2_delete_both_eq_and_pos
(i int, s string, d date, primary key (i, d) not enforced)
with ('format-version'='2', 'write.upsert.enabled'='true');
2: Flink:
insert into ice.iceberg_v2_delete_both_eq_and_pos values
(1, 'str1', to_date('2023-12-13')),
(2, 'str2', to_date('2023-12-13'));
3-Flink:
insert into ice.iceberg_v2_delete_both_eq_and_pos values
(3, 'str3', to_date('2023-12-23')),
(2, 'str2_updated', to_date('2023-12-13'));
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:
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
by Flink:
create table ice.iceberg_v2_delete_equality_partitioned
(i int, s string, d date, primary key (d, s) not enforced)
partitioned by (d)
w ith ('format-version'='2', 'write.upsert.enabled'='true');
2: Populate one partition.
insert into ice.iceberg_v2_delete_equality_partitioned partition (d='2023-12-24') values
(1, 'str1'), (2, 'str2'), (3, 'str3');
3: Populate another partition
insert into ice.iceberg_v2_delete_equality_partitioned partition (d='2023-12-25') values
(1, 'str1'), (2, 'str2');
4: Update one row in the first partiton and add a new row.
insert into ice.iceberg_v2_delete_equality_partitioned partition (d='2023-12-24') values
(333333, 'str3'), (4, 'str4');
5: Update one row in the second partition.
insert into ice.iceberg_v2_delete_equality_partitioned partition (d='2023-12-25') values
(222, 'str2');
iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partition_evolution:
Flink is used to create and populate this simple table to have an equlity delete file.
Impala is used for doing some partition evolution on this table.
1-Flink:
create table ice.iceberg_v2_delete_equality_partition_evolution
(i int, s string, d date, primary key (d, s) not enforced)
partitioned by (d)
with ('format-version'='2', 'write.upsert.enabled'='true');
2-Flink:
insert into ice.iceberg_v2_delete_equality_partition_evolution
partition (d='2023-12-24') values (1, 'str1'), (2, 'str2');
3-Flink:
insert into ice.iceberg_v2_delete_equality_partition_evolution
partition (d='2023-12-24') values (111, 'str1');
4-Impala:
alter table functional_parquet.iceberg_v2_delete_equality_partition_evolution
set partition spec (d, i);
iceberg_test/iceberg_migrated_alter_test
Generated and migrated by Hive
CREATE TABLE iceberg_migrated_alter_test (int_col int, string_col string, double_col double) stored as parquet;

View File

@@ -0,0 +1,51 @@
{
"format-version" : 2,
"table-uuid" : "352654fb-efac-4192-9944-a3beae0068cf",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos",
"last-sequence-number" : 0,
"last-updated-ms" : 1702459661521,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 1, 3 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : true,
"type" : "int"
}, {
"id" : 2,
"name" : "s",
"required" : false,
"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" : "352654fb-efac-4192-9944-a3beae0068cf",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos",
"last-sequence-number" : 1,
"last-updated-ms" : 1702459703855,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 1, 3 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : true,
"type" : "int"
}, {
"id" : 2,
"name" : "s",
"required" : false,
"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" : 3802179086205335895,
"refs" : {
"main" : {
"snapshot-id" : 3802179086205335895,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 3802179086205335895,
"timestamp-ms" : 1702459703855,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "a5df36bf3be49d13acf7c9a290f491bf",
"flink.job-id" : "94ebe46e1c7ca3559289f8bff73d9a4e",
"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" : "1595",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "2",
"total-files-size" : "1595",
"total-data-files" : "1",
"total-delete-files" : "1",
"total-position-deletes" : "0",
"total-equality-deletes" : "2"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos/metadata/snap-3802179086205335895-1-3d36bf90-2625-4625-b09b-d4359b979df9.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1702459703855,
"snapshot-id" : 3802179086205335895
} ],
"metadata-log" : [ {
"timestamp-ms" : 1702459661521,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos/metadata/v1.metadata.json"
} ]
}

View File

@@ -0,0 +1,119 @@
{
"format-version" : 2,
"table-uuid" : "352654fb-efac-4192-9944-a3beae0068cf",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos",
"last-sequence-number" : 2,
"last-updated-ms" : 1702459737285,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 1, 3 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : true,
"type" : "int"
}, {
"id" : 2,
"name" : "s",
"required" : false,
"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" : 8985205515767142888,
"refs" : {
"main" : {
"snapshot-id" : 8985205515767142888,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 3802179086205335895,
"timestamp-ms" : 1702459703855,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "a5df36bf3be49d13acf7c9a290f491bf",
"flink.job-id" : "94ebe46e1c7ca3559289f8bff73d9a4e",
"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" : "1595",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "2",
"total-files-size" : "1595",
"total-data-files" : "1",
"total-delete-files" : "1",
"total-position-deletes" : "0",
"total-equality-deletes" : "2"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos/metadata/snap-3802179086205335895-1-3d36bf90-2625-4625-b09b-d4359b979df9.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
"snapshot-id" : 8985205515767142888,
"parent-snapshot-id" : 3802179086205335895,
"timestamp-ms" : 1702459737285,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "a5df36bf3be49d13acf7c9a290f491bf",
"flink.job-id" : "7ab03b339acf084740dccfbcf5f26440",
"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" : "1542",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "4",
"total-files-size" : "3137",
"total-data-files" : "2",
"total-delete-files" : "2",
"total-position-deletes" : "0",
"total-equality-deletes" : "4"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos/metadata/snap-8985205515767142888-1-0cf1a310-d39c-4c6a-bfef-c3fe33cd0c25.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1702459703855,
"snapshot-id" : 3802179086205335895
}, {
"timestamp-ms" : 1702459737285,
"snapshot-id" : 8985205515767142888
} ],
"metadata-log" : [ {
"timestamp-ms" : 1702459661521,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos/metadata/v1.metadata.json"
}, {
"timestamp-ms" : 1702459703855,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos/metadata/v2.metadata.json"
} ]
}

View File

@@ -0,0 +1,146 @@
{
"format-version" : 2,
"table-uuid" : "352654fb-efac-4192-9944-a3beae0068cf",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos",
"last-sequence-number" : 3,
"last-updated-ms" : 1702460042344,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 1, 3 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : true,
"type" : "int"
}, {
"id" : 2,
"name" : "s",
"required" : false,
"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" : 911559291487642581,
"refs" : {
"main" : {
"snapshot-id" : 911559291487642581,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 3802179086205335895,
"timestamp-ms" : 1702459703855,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "a5df36bf3be49d13acf7c9a290f491bf",
"flink.job-id" : "94ebe46e1c7ca3559289f8bff73d9a4e",
"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" : "1595",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "2",
"total-files-size" : "1595",
"total-data-files" : "1",
"total-delete-files" : "1",
"total-position-deletes" : "0",
"total-equality-deletes" : "2"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos/metadata/snap-3802179086205335895-1-3d36bf90-2625-4625-b09b-d4359b979df9.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
"snapshot-id" : 8985205515767142888,
"parent-snapshot-id" : 3802179086205335895,
"timestamp-ms" : 1702459737285,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "a5df36bf3be49d13acf7c9a290f491bf",
"flink.job-id" : "7ab03b339acf084740dccfbcf5f26440",
"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" : "1542",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "4",
"total-files-size" : "3137",
"total-data-files" : "2",
"total-delete-files" : "2",
"total-position-deletes" : "0",
"total-equality-deletes" : "4"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos/metadata/snap-8985205515767142888-1-0cf1a310-d39c-4c6a-bfef-c3fe33cd0c25.avro",
"schema-id" : 0
}, {
"sequence-number" : 3,
"snapshot-id" : 911559291487642581,
"parent-snapshot-id" : 8985205515767142888,
"timestamp-ms" : 1702460042344,
"summary" : {
"operation" : "overwrite",
"added-position-delete-files" : "1",
"added-delete-files" : "1",
"added-files-size" : "1606",
"added-position-deletes" : "1",
"changed-partition-count" : "1",
"total-records" : "4",
"total-files-size" : "4743",
"total-data-files" : "2",
"total-delete-files" : "3",
"total-position-deletes" : "1",
"total-equality-deletes" : "4"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos/metadata/snap-911559291487642581-1-bb4b8c07-84e1-421a-bb6c-594f297d118e.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1702459703855,
"snapshot-id" : 3802179086205335895
}, {
"timestamp-ms" : 1702459737285,
"snapshot-id" : 8985205515767142888
}, {
"timestamp-ms" : 1702460042344,
"snapshot-id" : 911559291487642581
} ],
"metadata-log" : [ {
"timestamp-ms" : 1702459661521,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos/metadata/v1.metadata.json"
}, {
"timestamp-ms" : 1702459703855,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos/metadata/v2.metadata.json"
}, {
"timestamp-ms" : 1702459737285,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_both_eq_and_pos/metadata/v3.metadata.json"
} ]
}

View File

@@ -0,0 +1,53 @@
{
"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" : 0,
"last-updated-ms" : 1702391192239,
"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" : -1,
"refs" : { },
"snapshots" : [ ],
"statistics" : [ ],
"snapshot-log" : [ ],
"metadata-log" : [ ]
}

View File

@@ -0,0 +1,83 @@
{
"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" : 1,
"last-updated-ms" : 1702391239615,
"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" : 397031335297740726,
"refs" : {
"main" : {
"snapshot-id" : 397031335297740726,
"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
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1702391239615,
"snapshot-id" : 397031335297740726
} ],
"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"
} ]
}

View File

@@ -0,0 +1,110 @@
{
"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" : 2,
"last-updated-ms" : 1702391268756,
"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" : 8494861454990126958,
"refs" : {
"main" : {
"snapshot-id" : 8494861454990126958,
"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
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1702391239615,
"snapshot-id" : 397031335297740726
}, {
"timestamp-ms" : 1702391268756,
"snapshot-id" : 8494861454990126958
} ],
"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"
} ]
}

View File

@@ -0,0 +1,137 @@
{
"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

@@ -0,0 +1,53 @@
{
"format-version" : 2,
"table-uuid" : "56e56cde-1aac-49d0-810a-15d2998778b2",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_nulls",
"last-sequence-number" : 0,
"last-updated-ms" : 1702383834781,
"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_equality_nulls"
},
"current-snapshot-id" : -1,
"refs" : { },
"snapshots" : [ ],
"statistics" : [ ],
"snapshot-log" : [ ],
"metadata-log" : [ ]
}

View File

@@ -0,0 +1,83 @@
{
"format-version" : 2,
"table-uuid" : "56e56cde-1aac-49d0-810a-15d2998778b2",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_nulls",
"last-sequence-number" : 1,
"last-updated-ms" : 1702383846889,
"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_equality_nulls"
},
"current-snapshot-id" : 4346796256488077976,
"refs" : {
"main" : {
"snapshot-id" : 4346796256488077976,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 4346796256488077976,
"timestamp-ms" : 1702383846889,
"summary" : {
"operation" : "append",
"added-data-files" : "1",
"added-records" : "3",
"added-files-size" : "611",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "611",
"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_nulls/metadata/snap-4346796256488077976-1-87d3b6df-f00d-40a4-aafa-5d7f20e3299b.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1702383846889,
"snapshot-id" : 4346796256488077976
} ],
"metadata-log" : [ {
"timestamp-ms" : 1702383834781,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_nulls/metadata/v1.metadata.json"
} ]
}

View File

@@ -0,0 +1,110 @@
{
"format-version" : 2,
"table-uuid" : "56e56cde-1aac-49d0-810a-15d2998778b2",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_nulls",
"last-sequence-number" : 2,
"last-updated-ms" : 1702383865598,
"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_equality_nulls"
},
"current-snapshot-id" : 9091814429631192676,
"refs" : {
"main" : {
"snapshot-id" : 9091814429631192676,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 4346796256488077976,
"timestamp-ms" : 1702383846889,
"summary" : {
"operation" : "append",
"added-data-files" : "1",
"added-records" : "3",
"added-files-size" : "611",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "611",
"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_nulls/metadata/snap-4346796256488077976-1-87d3b6df-f00d-40a4-aafa-5d7f20e3299b.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
"snapshot-id" : 9091814429631192676,
"parent-snapshot-id" : 4346796256488077976,
"timestamp-ms" : 1702383865598,
"summary" : {
"operation" : "overwrite",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-files-size" : "1589",
"added-equality-deletes" : "1",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "2200",
"total-data-files" : "1",
"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_nulls/metadata/snap-9091814429631192676-1-c8b17188-94bf-4496-9069-3eda900cd71d.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1702383846889,
"snapshot-id" : 4346796256488077976
}, {
"timestamp-ms" : 1702383865598,
"snapshot-id" : 9091814429631192676
} ],
"metadata-log" : [ {
"timestamp-ms" : 1702383834781,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_nulls/metadata/v1.metadata.json"
}, {
"timestamp-ms" : 1702383846889,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_nulls/metadata/v2.metadata.json"
} ]
}

View File

@@ -0,0 +1,136 @@
{
"format-version" : 2,
"table-uuid" : "56e56cde-1aac-49d0-810a-15d2998778b2",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_nulls",
"last-sequence-number" : 3,
"last-updated-ms" : 1702384074863,
"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_equality_nulls"
},
"current-snapshot-id" : 4301391241829251636,
"refs" : {
"main" : {
"snapshot-id" : 4301391241829251636,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 4346796256488077976,
"timestamp-ms" : 1702383846889,
"summary" : {
"operation" : "append",
"added-data-files" : "1",
"added-records" : "3",
"added-files-size" : "611",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "611",
"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_nulls/metadata/snap-4346796256488077976-1-87d3b6df-f00d-40a4-aafa-5d7f20e3299b.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
"snapshot-id" : 9091814429631192676,
"parent-snapshot-id" : 4346796256488077976,
"timestamp-ms" : 1702383865598,
"summary" : {
"operation" : "overwrite",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-files-size" : "1589",
"added-equality-deletes" : "1",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "2200",
"total-data-files" : "1",
"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_nulls/metadata/snap-9091814429631192676-1-c8b17188-94bf-4496-9069-3eda900cd71d.avro",
"schema-id" : 0
}, {
"sequence-number" : 3,
"snapshot-id" : 4301391241829251636,
"parent-snapshot-id" : 9091814429631192676,
"timestamp-ms" : 1702384074863,
"summary" : {
"operation" : "append",
"added-data-files" : "1",
"added-records" : "2",
"added-files-size" : "598",
"changed-partition-count" : "1",
"total-records" : "5",
"total-files-size" : "2798",
"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_nulls/metadata/snap-4301391241829251636-1-25ce5480-23b6-4c70-a724-63931f8d84c6.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1702383846889,
"snapshot-id" : 4346796256488077976
}, {
"timestamp-ms" : 1702383865598,
"snapshot-id" : 9091814429631192676
}, {
"timestamp-ms" : 1702384074863,
"snapshot-id" : 4301391241829251636
} ],
"metadata-log" : [ {
"timestamp-ms" : 1702383834781,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_nulls/metadata/v1.metadata.json"
}, {
"timestamp-ms" : 1702383846889,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_nulls/metadata/v2.metadata.json"
}, {
"timestamp-ms" : 1702383865598,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_nulls/metadata/v3.metadata.json"
} ]
}

View File

@@ -0,0 +1,56 @@
{
"format-version" : 2,
"table-uuid" : "9258fcb2-d821-426d-8a88-304ff9e7364c",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partition_evolution",
"last-sequence-number" : 0,
"last-updated-ms" : 1702498563674,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 2, 3 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : false,
"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" : [ {
"name" : "d",
"transform" : "identity",
"source-id" : 3,
"field-id" : 1000
} ]
} ],
"last-partition-id" : 1000,
"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,92 @@
{
"format-version" : 2,
"table-uuid" : "9258fcb2-d821-426d-8a88-304ff9e7364c",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partition_evolution",
"last-sequence-number" : 1,
"last-updated-ms" : 1702498592419,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 2, 3 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : false,
"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" : [ {
"name" : "d",
"transform" : "identity",
"source-id" : 3,
"field-id" : 1000
} ]
} ],
"last-partition-id" : 1000,
"default-sort-order-id" : 0,
"sort-orders" : [ {
"order-id" : 0,
"fields" : [ ]
} ],
"properties" : {
"write.parquet.compression-codec" : "zstd",
"write.upsert.enabled" : "true"
},
"current-snapshot-id" : 3409033829588781878,
"refs" : {
"main" : {
"snapshot-id" : 3409033829588781878,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 3409033829588781878,
"timestamp-ms" : 1702498592419,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "a5df36bf3be49d13acf7c9a290f491bf",
"flink.job-id" : "1bf562ae989b81282dd5d3a188bf905c",
"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" : "1600",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "2",
"total-files-size" : "1600",
"total-data-files" : "1",
"total-delete-files" : "1",
"total-position-deletes" : "0",
"total-equality-deletes" : "2"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partition_evolution/metadata/snap-3409033829588781878-1-c9c459ff-9747-4dab-b65f-cace0f31e669.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1702498592419,
"snapshot-id" : 3409033829588781878
} ],
"metadata-log" : [ {
"timestamp-ms" : 1702498563674,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partition_evolution/metadata/v1.metadata.json"
} ]
}

View File

@@ -0,0 +1,124 @@
{
"format-version" : 2,
"table-uuid" : "9258fcb2-d821-426d-8a88-304ff9e7364c",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partition_evolution",
"last-sequence-number" : 2,
"last-updated-ms" : 1702498647202,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 2, 3 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : false,
"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" : [ {
"name" : "d",
"transform" : "identity",
"source-id" : 3,
"field-id" : 1000
} ]
} ],
"last-partition-id" : 1000,
"default-sort-order-id" : 0,
"sort-orders" : [ {
"order-id" : 0,
"fields" : [ ]
} ],
"properties" : {
"write.parquet.compression-codec" : "zstd",
"write.upsert.enabled" : "true"
},
"current-snapshot-id" : 586812101365618837,
"refs" : {
"main" : {
"snapshot-id" : 586812101365618837,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 3409033829588781878,
"timestamp-ms" : 1702498592419,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "a5df36bf3be49d13acf7c9a290f491bf",
"flink.job-id" : "1bf562ae989b81282dd5d3a188bf905c",
"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" : "1600",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "2",
"total-files-size" : "1600",
"total-data-files" : "1",
"total-delete-files" : "1",
"total-position-deletes" : "0",
"total-equality-deletes" : "2"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partition_evolution/metadata/snap-3409033829588781878-1-c9c459ff-9747-4dab-b65f-cace0f31e669.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
"snapshot-id" : 586812101365618837,
"parent-snapshot-id" : 3409033829588781878,
"timestamp-ms" : 1702498647202,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "fbb4ef531e002f8fb3a2052db255adf5",
"flink.job-id" : "dd939cbb702c58c53221a09712f171c3",
"flink.max-committed-checkpoint-id" : "9223372036854775807",
"added-data-files" : "1",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-records" : "1",
"added-files-size" : "1519",
"added-equality-deletes" : "1",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "3119",
"total-data-files" : "2",
"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_partition_evolution/metadata/snap-586812101365618837-1-807d7d2a-0557-4bbe-9f07-9467de72598a.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1702498592419,
"snapshot-id" : 3409033829588781878
}, {
"timestamp-ms" : 1702498647202,
"snapshot-id" : 586812101365618837
} ],
"metadata-log" : [ {
"timestamp-ms" : 1702498563674,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partition_evolution/metadata/v1.metadata.json"
}, {
"timestamp-ms" : 1702498592419,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partition_evolution/metadata/v2.metadata.json"
} ]
}

View File

@@ -0,0 +1,142 @@
{
"format-version" : 2,
"table-uuid" : "9258fcb2-d821-426d-8a88-304ff9e7364c",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partition_evolution",
"last-sequence-number" : 2,
"last-updated-ms" : 1702498757874,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 2, 3 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : false,
"type" : "int"
}, {
"id" : 2,
"name" : "s",
"required" : true,
"type" : "string"
}, {
"id" : 3,
"name" : "d",
"required" : true,
"type" : "date"
} ]
} ],
"default-spec-id" : 1,
"partition-specs" : [ {
"spec-id" : 0,
"fields" : [ {
"name" : "d",
"transform" : "identity",
"source-id" : 3,
"field-id" : 1000
} ]
}, {
"spec-id" : 1,
"fields" : [ {
"name" : "d",
"transform" : "identity",
"source-id" : 3,
"field-id" : 1000
}, {
"name" : "i",
"transform" : "identity",
"source-id" : 1,
"field-id" : 1001
} ]
} ],
"last-partition-id" : 1001,
"default-sort-order-id" : 0,
"sort-orders" : [ {
"order-id" : 0,
"fields" : [ ]
} ],
"properties" : {
"impala.events.catalogServiceId" : "82734f9389d94c17:adba9a6a60c75a10",
"impala.events.catalogVersion" : "1777",
"write.parquet.compression-codec" : "zstd",
"write.upsert.enabled" : "true"
},
"current-snapshot-id" : 586812101365618837,
"refs" : {
"main" : {
"snapshot-id" : 586812101365618837,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 3409033829588781878,
"timestamp-ms" : 1702498592419,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "a5df36bf3be49d13acf7c9a290f491bf",
"flink.job-id" : "1bf562ae989b81282dd5d3a188bf905c",
"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" : "1600",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "2",
"total-files-size" : "1600",
"total-data-files" : "1",
"total-delete-files" : "1",
"total-position-deletes" : "0",
"total-equality-deletes" : "2"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partition_evolution/metadata/snap-3409033829588781878-1-c9c459ff-9747-4dab-b65f-cace0f31e669.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
"snapshot-id" : 586812101365618837,
"parent-snapshot-id" : 3409033829588781878,
"timestamp-ms" : 1702498647202,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "fbb4ef531e002f8fb3a2052db255adf5",
"flink.job-id" : "dd939cbb702c58c53221a09712f171c3",
"flink.max-committed-checkpoint-id" : "9223372036854775807",
"added-data-files" : "1",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-records" : "1",
"added-files-size" : "1519",
"added-equality-deletes" : "1",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "3119",
"total-data-files" : "2",
"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_partition_evolution/metadata/snap-586812101365618837-1-807d7d2a-0557-4bbe-9f07-9467de72598a.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1702498592419,
"snapshot-id" : 3409033829588781878
}, {
"timestamp-ms" : 1702498647202,
"snapshot-id" : 586812101365618837
} ],
"metadata-log" : [ {
"timestamp-ms" : 1702498563674,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partition_evolution/metadata/v1.metadata.json"
}, {
"timestamp-ms" : 1702498592419,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partition_evolution/metadata/v2.metadata.json"
}, {
"timestamp-ms" : 1702498647202,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partition_evolution/metadata/v3.metadata.json"
} ]
}

View File

@@ -0,0 +1,56 @@
{
"format-version" : 2,
"table-uuid" : "72376038-8325-453e-a757-20e8f8feed5e",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned",
"last-sequence-number" : 0,
"last-updated-ms" : 1702476288199,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 2, 3 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : false,
"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" : [ {
"name" : "d",
"transform" : "identity",
"source-id" : 3,
"field-id" : 1000
} ]
} ],
"last-partition-id" : 1000,
"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,92 @@
{
"format-version" : 2,
"table-uuid" : "72376038-8325-453e-a757-20e8f8feed5e",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned",
"last-sequence-number" : 1,
"last-updated-ms" : 1702476300728,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 2, 3 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : false,
"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" : [ {
"name" : "d",
"transform" : "identity",
"source-id" : 3,
"field-id" : 1000
} ]
} ],
"last-partition-id" : 1000,
"default-sort-order-id" : 0,
"sort-orders" : [ {
"order-id" : 0,
"fields" : [ ]
} ],
"properties" : {
"write.parquet.compression-codec" : "zstd",
"write.upsert.enabled" : "true"
},
"current-snapshot-id" : 3217166167862484560,
"refs" : {
"main" : {
"snapshot-id" : 3217166167862484560,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 3217166167862484560,
"timestamp-ms" : 1702476300728,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "751aac623aff5b6900c68eac5be5ffe6",
"flink.job-id" : "d790a22a176d9ddd8657533e41628505",
"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" : "1608",
"added-equality-deletes" : "3",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "1608",
"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_equality_partitioned/metadata/snap-3217166167862484560-1-db5041df-259d-48b9-ade1-1bf382a93d5a.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1702476300728,
"snapshot-id" : 3217166167862484560
} ],
"metadata-log" : [ {
"timestamp-ms" : 1702476288199,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned/metadata/v1.metadata.json"
} ]
}

View File

@@ -0,0 +1,124 @@
{
"format-version" : 2,
"table-uuid" : "72376038-8325-453e-a757-20e8f8feed5e",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned",
"last-sequence-number" : 2,
"last-updated-ms" : 1702476310943,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 2, 3 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : false,
"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" : [ {
"name" : "d",
"transform" : "identity",
"source-id" : 3,
"field-id" : 1000
} ]
} ],
"last-partition-id" : 1000,
"default-sort-order-id" : 0,
"sort-orders" : [ {
"order-id" : 0,
"fields" : [ ]
} ],
"properties" : {
"write.parquet.compression-codec" : "zstd",
"write.upsert.enabled" : "true"
},
"current-snapshot-id" : 7564375228633944060,
"refs" : {
"main" : {
"snapshot-id" : 7564375228633944060,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 3217166167862484560,
"timestamp-ms" : 1702476300728,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "751aac623aff5b6900c68eac5be5ffe6",
"flink.job-id" : "d790a22a176d9ddd8657533e41628505",
"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" : "1608",
"added-equality-deletes" : "3",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "1608",
"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_equality_partitioned/metadata/snap-3217166167862484560-1-db5041df-259d-48b9-ade1-1bf382a93d5a.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
"snapshot-id" : 7564375228633944060,
"parent-snapshot-id" : 3217166167862484560,
"timestamp-ms" : 1702476310943,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "a5df36bf3be49d13acf7c9a290f491bf",
"flink.job-id" : "2df40cec755b051393a30a9ff5247e4d",
"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" : "1600",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "5",
"total-files-size" : "3208",
"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_equality_partitioned/metadata/snap-7564375228633944060-1-57f8cd32-619c-46fc-a683-1dee7473c990.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1702476300728,
"snapshot-id" : 3217166167862484560
}, {
"timestamp-ms" : 1702476310943,
"snapshot-id" : 7564375228633944060
} ],
"metadata-log" : [ {
"timestamp-ms" : 1702476288199,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned/metadata/v1.metadata.json"
}, {
"timestamp-ms" : 1702476300728,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned/metadata/v2.metadata.json"
} ]
}

View File

@@ -0,0 +1,156 @@
{
"format-version" : 2,
"table-uuid" : "72376038-8325-453e-a757-20e8f8feed5e",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned",
"last-sequence-number" : 3,
"last-updated-ms" : 1702476332813,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 2, 3 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : false,
"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" : [ {
"name" : "d",
"transform" : "identity",
"source-id" : 3,
"field-id" : 1000
} ]
} ],
"last-partition-id" : 1000,
"default-sort-order-id" : 0,
"sort-orders" : [ {
"order-id" : 0,
"fields" : [ ]
} ],
"properties" : {
"write.parquet.compression-codec" : "zstd",
"write.upsert.enabled" : "true"
},
"current-snapshot-id" : 3979814664665937114,
"refs" : {
"main" : {
"snapshot-id" : 3979814664665937114,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 3217166167862484560,
"timestamp-ms" : 1702476300728,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "751aac623aff5b6900c68eac5be5ffe6",
"flink.job-id" : "d790a22a176d9ddd8657533e41628505",
"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" : "1608",
"added-equality-deletes" : "3",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "1608",
"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_equality_partitioned/metadata/snap-3217166167862484560-1-db5041df-259d-48b9-ade1-1bf382a93d5a.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
"snapshot-id" : 7564375228633944060,
"parent-snapshot-id" : 3217166167862484560,
"timestamp-ms" : 1702476310943,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "a5df36bf3be49d13acf7c9a290f491bf",
"flink.job-id" : "2df40cec755b051393a30a9ff5247e4d",
"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" : "1600",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "5",
"total-files-size" : "3208",
"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_equality_partitioned/metadata/snap-7564375228633944060-1-57f8cd32-619c-46fc-a683-1dee7473c990.avro",
"schema-id" : 0
}, {
"sequence-number" : 3,
"snapshot-id" : 3979814664665937114,
"parent-snapshot-id" : 7564375228633944060,
"timestamp-ms" : 1702476332813,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "a5df36bf3be49d13acf7c9a290f491bf",
"flink.job-id" : "0cbe9ac985f1c730a7953b16e5cec31f",
"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" : "1600",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "7",
"total-files-size" : "4808",
"total-data-files" : "3",
"total-delete-files" : "3",
"total-position-deletes" : "0",
"total-equality-deletes" : "7"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned/metadata/snap-3979814664665937114-1-09ca0acc-c19e-4073-80f7-b476a6e568c7.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1702476300728,
"snapshot-id" : 3217166167862484560
}, {
"timestamp-ms" : 1702476310943,
"snapshot-id" : 7564375228633944060
}, {
"timestamp-ms" : 1702476332813,
"snapshot-id" : 3979814664665937114
} ],
"metadata-log" : [ {
"timestamp-ms" : 1702476288199,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned/metadata/v1.metadata.json"
}, {
"timestamp-ms" : 1702476300728,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned/metadata/v2.metadata.json"
}, {
"timestamp-ms" : 1702476310943,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned/metadata/v3.metadata.json"
} ]
}

View File

@@ -0,0 +1,188 @@
{
"format-version" : 2,
"table-uuid" : "72376038-8325-453e-a757-20e8f8feed5e",
"location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned",
"last-sequence-number" : 4,
"last-updated-ms" : 1702476380992,
"last-column-id" : 3,
"current-schema-id" : 0,
"schemas" : [ {
"type" : "struct",
"schema-id" : 0,
"identifier-field-ids" : [ 2, 3 ],
"fields" : [ {
"id" : 1,
"name" : "i",
"required" : false,
"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" : [ {
"name" : "d",
"transform" : "identity",
"source-id" : 3,
"field-id" : 1000
} ]
} ],
"last-partition-id" : 1000,
"default-sort-order-id" : 0,
"sort-orders" : [ {
"order-id" : 0,
"fields" : [ ]
} ],
"properties" : {
"write.parquet.compression-codec" : "zstd",
"write.upsert.enabled" : "true"
},
"current-snapshot-id" : 4821964189199835313,
"refs" : {
"main" : {
"snapshot-id" : 4821964189199835313,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
"snapshot-id" : 3217166167862484560,
"timestamp-ms" : 1702476300728,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "751aac623aff5b6900c68eac5be5ffe6",
"flink.job-id" : "d790a22a176d9ddd8657533e41628505",
"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" : "1608",
"added-equality-deletes" : "3",
"changed-partition-count" : "1",
"total-records" : "3",
"total-files-size" : "1608",
"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_equality_partitioned/metadata/snap-3217166167862484560-1-db5041df-259d-48b9-ade1-1bf382a93d5a.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
"snapshot-id" : 7564375228633944060,
"parent-snapshot-id" : 3217166167862484560,
"timestamp-ms" : 1702476310943,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "a5df36bf3be49d13acf7c9a290f491bf",
"flink.job-id" : "2df40cec755b051393a30a9ff5247e4d",
"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" : "1600",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "5",
"total-files-size" : "3208",
"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_equality_partitioned/metadata/snap-7564375228633944060-1-57f8cd32-619c-46fc-a683-1dee7473c990.avro",
"schema-id" : 0
}, {
"sequence-number" : 3,
"snapshot-id" : 3979814664665937114,
"parent-snapshot-id" : 7564375228633944060,
"timestamp-ms" : 1702476332813,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "a5df36bf3be49d13acf7c9a290f491bf",
"flink.job-id" : "0cbe9ac985f1c730a7953b16e5cec31f",
"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" : "1600",
"added-equality-deletes" : "2",
"changed-partition-count" : "1",
"total-records" : "7",
"total-files-size" : "4808",
"total-data-files" : "3",
"total-delete-files" : "3",
"total-position-deletes" : "0",
"total-equality-deletes" : "7"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned/metadata/snap-3979814664665937114-1-09ca0acc-c19e-4073-80f7-b476a6e568c7.avro",
"schema-id" : 0
}, {
"sequence-number" : 4,
"snapshot-id" : 4821964189199835313,
"parent-snapshot-id" : 3979814664665937114,
"timestamp-ms" : 1702476380992,
"summary" : {
"operation" : "overwrite",
"flink.operator-id" : "fbb4ef531e002f8fb3a2052db255adf5",
"flink.job-id" : "0909766ab9d7637470b746e6f14cbde6",
"flink.max-committed-checkpoint-id" : "9223372036854775807",
"added-data-files" : "1",
"added-equality-delete-files" : "1",
"added-delete-files" : "1",
"added-records" : "1",
"added-files-size" : "1519",
"added-equality-deletes" : "1",
"changed-partition-count" : "1",
"total-records" : "8",
"total-files-size" : "6327",
"total-data-files" : "4",
"total-delete-files" : "4",
"total-position-deletes" : "0",
"total-equality-deletes" : "8"
},
"manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned/metadata/snap-4821964189199835313-1-e3dac70e-a8aa-4d15-9d35-20c4f25f36d5.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"snapshot-log" : [ {
"timestamp-ms" : 1702476300728,
"snapshot-id" : 3217166167862484560
}, {
"timestamp-ms" : 1702476310943,
"snapshot-id" : 7564375228633944060
}, {
"timestamp-ms" : 1702476332813,
"snapshot-id" : 3979814664665937114
}, {
"timestamp-ms" : 1702476380992,
"snapshot-id" : 4821964189199835313
} ],
"metadata-log" : [ {
"timestamp-ms" : 1702476288199,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned/metadata/v1.metadata.json"
}, {
"timestamp-ms" : 1702476300728,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned/metadata/v2.metadata.json"
}, {
"timestamp-ms" : 1702476310943,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned/metadata/v3.metadata.json"
}, {
"timestamp-ms" : 1702476332813,
"metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_v2_delete_equality_partitioned/metadata/v4.metadata.json"
} ]
}

Some files were not shown because too many files have changed in this diff Show More