mirror of
https://github.com/apache/impala.git
synced 2025-12-19 18:12:08 -05:00
IMPALA-13267: Display number of partitions for Iceberg tables
Before this change, query plans and profile reported only a single partition even for partitioned Iceberg tables, which was misleading for users. Now we can display the number of scanned partitions correctly for both partitioned and unpartitioned Iceberg tables. This is achieved by extracting the partition values from the file descriptors and storing them in the IcebergContentFileStore. Instead of storing this information redundantly in all file descriptors, we store them in one place and reference the partition metadata in the FDs with an id. This also gives the opportunity to optimize memory consumption in the Catalog and Coordinator as well as reduce network traffic between them in the future. Time travel is handled similarly to oldFileDescMap. In that case we don't know the total number of partitions in the old snapshot, so the output is [Num scanned partitions]/unknown. Testing: - Planner tests - E2E tests - partition transforms - partition evolution - DROP PARTITION - time travel Change-Id: Ifb2f654bc6c9bdf9cfafc27b38b5ca2f7b6b4872 Reviewed-on: http://gerrit.cloudera.org:8080/23113 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:
committed by
Impala Public Jenkins
parent
968559da26
commit
821c7347d1
@@ -48,6 +48,7 @@ table FbIcebergMetadata {
|
|||||||
spec_id : ushort;
|
spec_id : ushort;
|
||||||
partition_keys : [FbIcebergPartitionTransformValue];
|
partition_keys : [FbIcebergPartitionTransformValue];
|
||||||
equality_field_ids : [int];
|
equality_field_ids : [int];
|
||||||
|
part_id : int = -1;
|
||||||
}
|
}
|
||||||
|
|
||||||
table FbIcebergColumnStats {
|
table FbIcebergColumnStats {
|
||||||
|
|||||||
@@ -641,6 +641,11 @@ struct TIcebergPartitionSpec {
|
|||||||
2: optional list<TIcebergPartitionField> partition_fields
|
2: optional list<TIcebergPartitionField> partition_fields
|
||||||
}
|
}
|
||||||
|
|
||||||
|
struct TIcebergPartition {
|
||||||
|
1: required i32 spec_id
|
||||||
|
2: required list<string> partition_values
|
||||||
|
}
|
||||||
|
|
||||||
struct TIcebergPartitionStats {
|
struct TIcebergPartitionStats {
|
||||||
1: required i64 num_files;
|
1: required i64 num_files;
|
||||||
2: required i64 num_rows;
|
2: required i64 num_rows;
|
||||||
@@ -657,6 +662,7 @@ struct TIcebergContentFileStore {
|
|||||||
6: optional bool has_orc
|
6: optional bool has_orc
|
||||||
7: optional bool has_parquet
|
7: optional bool has_parquet
|
||||||
8: optional list<string> missing_files
|
8: optional list<string> missing_files
|
||||||
|
9: optional list<TIcebergPartition> partitions
|
||||||
}
|
}
|
||||||
|
|
||||||
// Represents a drop partition request for Iceberg tables
|
// Represents a drop partition request for Iceberg tables
|
||||||
|
|||||||
@@ -42,6 +42,7 @@ import org.apache.impala.util.IcebergUtil;
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
@@ -239,7 +240,7 @@ public class OptimizeStmt extends DmlStatementBase {
|
|||||||
IcebergContentFileStore selectedFiles =
|
IcebergContentFileStore selectedFiles =
|
||||||
new IcebergContentFileStore(iceTable.getIcebergApiTable(),
|
new IcebergContentFileStore(iceTable.getIcebergApiTable(),
|
||||||
iceTable.getContentFileStore().getDataFilesWithoutDeletes(),
|
iceTable.getContentFileStore().getDataFilesWithoutDeletes(),
|
||||||
selectedContentFiles);
|
selectedContentFiles, new HashMap<>());
|
||||||
return selectedFiles.getDataFilesWithoutDeletes();
|
return selectedFiles.getDataFilesWithoutDeletes();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -18,6 +18,7 @@
|
|||||||
package org.apache.impala.catalog;
|
package org.apache.impala.catalog;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
|
||||||
@@ -45,19 +46,17 @@ import org.apache.impala.fb.FbIcebergDataFileFormat;
|
|||||||
import org.apache.impala.fb.FbIcebergMetadata;
|
import org.apache.impala.fb.FbIcebergMetadata;
|
||||||
import org.apache.impala.thrift.THdfsFileDesc;
|
import org.apache.impala.thrift.THdfsFileDesc;
|
||||||
import org.apache.impala.thrift.TIcebergContentFileStore;
|
import org.apache.impala.thrift.TIcebergContentFileStore;
|
||||||
|
import org.apache.impala.thrift.TIcebergPartition;
|
||||||
import org.apache.impala.thrift.TNetworkAddress;
|
import org.apache.impala.thrift.TNetworkAddress;
|
||||||
import org.apache.impala.util.IcebergUtil;
|
import org.apache.impala.util.IcebergUtil;
|
||||||
import org.apache.impala.util.ListMap;
|
import org.apache.impala.util.ListMap;
|
||||||
|
|
||||||
import org.slf4j.Logger;
|
|
||||||
import org.slf4j.LoggerFactory;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Helper class for storing Iceberg file descriptors. It stores data and delete files
|
* Helper class for storing Iceberg file descriptors. It stores data and delete files
|
||||||
* separately, while also storing file descriptors belonging to earlier snapshots.
|
* separately, while also storing file descriptors belonging to earlier snapshots.
|
||||||
|
* Shared between queries on the Coordinator side.
|
||||||
*/
|
*/
|
||||||
public class IcebergContentFileStore {
|
public class IcebergContentFileStore {
|
||||||
final static Logger LOG = LoggerFactory.getLogger(IcebergContentFileStore.class);
|
|
||||||
|
|
||||||
private static class EncodedFileDescriptor {
|
private static class EncodedFileDescriptor {
|
||||||
public final byte[] fileDesc_;
|
public final byte[] fileDesc_;
|
||||||
@@ -157,10 +156,16 @@ public class IcebergContentFileStore {
|
|||||||
private MapListContainer positionDeleteFiles_ = new MapListContainer();
|
private MapListContainer positionDeleteFiles_ = new MapListContainer();
|
||||||
private MapListContainer equalityDeleteFiles_ = new MapListContainer();
|
private MapListContainer equalityDeleteFiles_ = new MapListContainer();
|
||||||
private Set<String> missingFiles_ = new HashSet<>();
|
private Set<String> missingFiles_ = new HashSet<>();
|
||||||
|
// Partitions with their corresponding ids that are used to refer to the partition info
|
||||||
|
// from the IcebergFileDescriptors.
|
||||||
|
private Map<TIcebergPartition, Integer> partitions_;
|
||||||
|
|
||||||
// Caches file descriptors loaded during time-travel queries.
|
// Caches file descriptors loaded during time-travel queries.
|
||||||
private final ConcurrentMap<String, EncodedFileDescriptor> oldFileDescMap_ =
|
private final ConcurrentMap<String, EncodedFileDescriptor> oldFileDescMap_ =
|
||||||
new ConcurrentHashMap<>();
|
new ConcurrentHashMap<>();
|
||||||
|
// Caches the partitions of file descriptors loaded during time-travel queries.
|
||||||
|
private final ConcurrentMap<TIcebergPartition, Integer> oldPartitionMap_ =
|
||||||
|
new ConcurrentHashMap<>();
|
||||||
|
|
||||||
// Flags to indicate file formats used in the table.
|
// Flags to indicate file formats used in the table.
|
||||||
private boolean hasAvro_ = false;
|
private boolean hasAvro_ = false;
|
||||||
@@ -171,10 +176,13 @@ public class IcebergContentFileStore {
|
|||||||
|
|
||||||
public IcebergContentFileStore(
|
public IcebergContentFileStore(
|
||||||
Table iceApiTable, List<IcebergFileDescriptor> fileDescriptors,
|
Table iceApiTable, List<IcebergFileDescriptor> fileDescriptors,
|
||||||
GroupedContentFiles icebergFiles) {
|
GroupedContentFiles icebergFiles, Map<TIcebergPartition, Integer> partitions) {
|
||||||
Preconditions.checkNotNull(iceApiTable);
|
Preconditions.checkNotNull(iceApiTable);
|
||||||
Preconditions.checkNotNull(fileDescriptors);
|
Preconditions.checkNotNull(fileDescriptors);
|
||||||
Preconditions.checkNotNull(icebergFiles);
|
Preconditions.checkNotNull(icebergFiles);
|
||||||
|
Preconditions.checkNotNull(partitions);
|
||||||
|
|
||||||
|
partitions_ = partitions;
|
||||||
|
|
||||||
Map<String, IcebergFileDescriptor> fileDescMap = new HashMap<>();
|
Map<String, IcebergFileDescriptor> fileDescMap = new HashMap<>();
|
||||||
for (IcebergFileDescriptor fileDesc : fileDescriptors) {
|
for (IcebergFileDescriptor fileDesc : fileDescriptors) {
|
||||||
@@ -213,6 +221,12 @@ public class IcebergContentFileStore {
|
|||||||
oldFileDescMap_.put(pathHash, encode(desc));
|
oldFileDescMap_.put(pathHash, encode(desc));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// This is only invoked during time travel, when we are querying a snapshot that has
|
||||||
|
// partitions which have been removed since.
|
||||||
|
public void addOldPartition(TIcebergPartition partition, Integer id) {
|
||||||
|
oldPartitionMap_.put(partition, id);
|
||||||
|
}
|
||||||
|
|
||||||
public IcebergFileDescriptor getDataFileDescriptor(String pathHash) {
|
public IcebergFileDescriptor getDataFileDescriptor(String pathHash) {
|
||||||
IcebergFileDescriptor desc = dataFilesWithoutDeletes_.get(pathHash);
|
IcebergFileDescriptor desc = dataFilesWithoutDeletes_.get(pathHash);
|
||||||
if (desc != null) return desc;
|
if (desc != null) return desc;
|
||||||
@@ -230,6 +244,14 @@ public class IcebergContentFileStore {
|
|||||||
return decode(oldFileDescMap_.get(pathHash));
|
return decode(oldFileDescMap_.get(pathHash));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Integer getOldPartition(TIcebergPartition partition) {
|
||||||
|
return oldPartitionMap_.get(partition);
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getOldPartitionsSize() {
|
||||||
|
return oldPartitionMap_.size();
|
||||||
|
}
|
||||||
|
|
||||||
public List<IcebergFileDescriptor> getDataFilesWithoutDeletes() {
|
public List<IcebergFileDescriptor> getDataFilesWithoutDeletes() {
|
||||||
return dataFilesWithoutDeletes_.getList();
|
return dataFilesWithoutDeletes_.getList();
|
||||||
}
|
}
|
||||||
@@ -281,6 +303,18 @@ public class IcebergContentFileStore {
|
|||||||
equalityDeleteFiles_.getList());
|
equalityDeleteFiles_.getList());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Map<TIcebergPartition, Integer> getPartitionMap() {
|
||||||
|
return partitions_;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<TIcebergPartition> getPartitionList() {
|
||||||
|
return convertPartitionMapToList(partitions_);
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getNumPartitions() {
|
||||||
|
return partitions_.size();
|
||||||
|
}
|
||||||
|
|
||||||
public boolean hasAvro() { return hasAvro_; }
|
public boolean hasAvro() { return hasAvro_; }
|
||||||
public boolean hasOrc() { return hasOrc_; }
|
public boolean hasOrc() { return hasOrc_; }
|
||||||
public boolean hasParquet() { return hasParquet_; }
|
public boolean hasParquet() { return hasParquet_; }
|
||||||
@@ -335,6 +369,7 @@ public class IcebergContentFileStore {
|
|||||||
ret.setHas_orc(hasOrc_);
|
ret.setHas_orc(hasOrc_);
|
||||||
ret.setHas_parquet(hasParquet_);
|
ret.setHas_parquet(hasParquet_);
|
||||||
ret.setMissing_files(new ArrayList<>(missingFiles_));
|
ret.setMissing_files(new ArrayList<>(missingFiles_));
|
||||||
|
ret.setPartitions(convertPartitionMapToList(partitions_));
|
||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -367,6 +402,30 @@ public class IcebergContentFileStore {
|
|||||||
ret.hasParquet_ = tFileStore.isSetHas_parquet() ? tFileStore.isHas_parquet() : false;
|
ret.hasParquet_ = tFileStore.isSetHas_parquet() ? tFileStore.isHas_parquet() : false;
|
||||||
ret.missingFiles_ = tFileStore.isSetMissing_files() ?
|
ret.missingFiles_ = tFileStore.isSetMissing_files() ?
|
||||||
new HashSet<>(tFileStore.getMissing_files()) : Collections.emptySet();
|
new HashSet<>(tFileStore.getMissing_files()) : Collections.emptySet();
|
||||||
|
ret.partitions_ = tFileStore.isSetPartitions() ?
|
||||||
|
convertPartitionListToMap(tFileStore.getPartitions()) : new HashMap<>();
|
||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static List<TIcebergPartition> convertPartitionMapToList(
|
||||||
|
Map<TIcebergPartition, Integer> partitionMap) {
|
||||||
|
List<TIcebergPartition> partitionList = new ArrayList<>(partitionMap.size());
|
||||||
|
for (int i = 0; i < partitionMap.size(); i++) {
|
||||||
|
partitionList.add(null);
|
||||||
|
}
|
||||||
|
for (Map.Entry<TIcebergPartition, Integer> partition : partitionMap.entrySet()) {
|
||||||
|
partitionList.set(partition.getValue(), partition.getKey());
|
||||||
|
}
|
||||||
|
return partitionList;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static ImmutableMap<TIcebergPartition, Integer> convertPartitionListToMap(
|
||||||
|
List<TIcebergPartition> partitionList) {
|
||||||
|
Preconditions.checkState(partitionList != null);
|
||||||
|
ImmutableMap.Builder<TIcebergPartition, Integer> builder = ImmutableMap.builder();
|
||||||
|
for (int i = 0; i < partitionList.size(); ++i) {
|
||||||
|
builder.put(partitionList.get(i), i);
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -32,9 +32,11 @@ import java.util.Collections;
|
|||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
@@ -51,6 +53,7 @@ import org.apache.impala.common.FileSystemUtil;
|
|||||||
import org.apache.impala.common.PrintUtils;
|
import org.apache.impala.common.PrintUtils;
|
||||||
import org.apache.impala.common.Reference;
|
import org.apache.impala.common.Reference;
|
||||||
import org.apache.impala.common.Pair;
|
import org.apache.impala.common.Pair;
|
||||||
|
import org.apache.impala.thrift.TIcebergPartition;
|
||||||
import org.apache.impala.thrift.TNetworkAddress;
|
import org.apache.impala.thrift.TNetworkAddress;
|
||||||
import org.apache.impala.util.IcebergUtil;
|
import org.apache.impala.util.IcebergUtil;
|
||||||
import org.apache.impala.util.ListMap;
|
import org.apache.impala.util.ListMap;
|
||||||
@@ -69,16 +72,22 @@ public class IcebergFileMetadataLoader extends FileMetadataLoader {
|
|||||||
private final org.apache.iceberg.Table iceTbl_;
|
private final org.apache.iceberg.Table iceTbl_;
|
||||||
private final Path tablePath_;
|
private final Path tablePath_;
|
||||||
private final GroupedContentFiles icebergFiles_;
|
private final GroupedContentFiles icebergFiles_;
|
||||||
|
private final List<TIcebergPartition> oldIcebergPartitions_;
|
||||||
|
private AtomicInteger nextPartitionId_ = new AtomicInteger(0);
|
||||||
|
// Map of the freshly loaded Iceberg partitions and their corresponding ids.
|
||||||
|
private ConcurrentHashMap<TIcebergPartition, Integer> loadedIcebergPartitions_;
|
||||||
private final boolean requiresDataFilesInTableLocation_;
|
private final boolean requiresDataFilesInTableLocation_;
|
||||||
|
|
||||||
public IcebergFileMetadataLoader(org.apache.iceberg.Table iceTbl,
|
public IcebergFileMetadataLoader(org.apache.iceberg.Table iceTbl,
|
||||||
Iterable<IcebergFileDescriptor> oldFds, ListMap<TNetworkAddress> hostIndex,
|
Iterable<IcebergFileDescriptor> oldFds, ListMap<TNetworkAddress> hostIndex,
|
||||||
GroupedContentFiles icebergFiles, boolean requiresDataFilesInTableLocation) {
|
GroupedContentFiles icebergFiles, List<TIcebergPartition> partitions,
|
||||||
|
boolean requiresDataFilesInTableLocation) {
|
||||||
super(iceTbl.location(), true, oldFds, hostIndex, null, null,
|
super(iceTbl.location(), true, oldFds, hostIndex, null, null,
|
||||||
HdfsFileFormat.ICEBERG);
|
HdfsFileFormat.ICEBERG);
|
||||||
iceTbl_ = iceTbl;
|
iceTbl_ = iceTbl;
|
||||||
tablePath_ = FileSystemUtil.createFullyQualifiedPath(new Path(iceTbl.location()));
|
tablePath_ = FileSystemUtil.createFullyQualifiedPath(new Path(iceTbl.location()));
|
||||||
icebergFiles_ = icebergFiles;
|
icebergFiles_ = icebergFiles;
|
||||||
|
oldIcebergPartitions_ = partitions;
|
||||||
requiresDataFilesInTableLocation_ = requiresDataFilesInTableLocation;
|
requiresDataFilesInTableLocation_ = requiresDataFilesInTableLocation;
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -101,8 +110,18 @@ public class IcebergFileMetadataLoader extends FileMetadataLoader {
|
|||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Map<TIcebergPartition, Integer> getIcebergPartitions() {
|
||||||
|
Preconditions.checkNotNull(loadedIcebergPartitions_);
|
||||||
|
return Collections.unmodifiableMap(loadedIcebergPartitions_);
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<TIcebergPartition> getIcebergPartitionList() {
|
||||||
|
return IcebergContentFileStore.convertPartitionMapToList(getIcebergPartitions());
|
||||||
|
}
|
||||||
|
|
||||||
private void loadInternal() throws CatalogException, IOException {
|
private void loadInternal() throws CatalogException, IOException {
|
||||||
loadedFds_ = new ArrayList<>();
|
loadedFds_ = new ArrayList<>();
|
||||||
|
loadedIcebergPartitions_ = new ConcurrentHashMap<>();
|
||||||
loadStats_ = new LoadStats(partDir_);
|
loadStats_ = new LoadStats(partDir_);
|
||||||
fileMetadataStats_ = new FileMetadataStats();
|
fileMetadataStats_ = new FileMetadataStats();
|
||||||
|
|
||||||
@@ -161,10 +180,20 @@ public class IcebergFileMetadataLoader extends FileMetadataLoader {
|
|||||||
}
|
}
|
||||||
List<ContentFile<?>> newContentFiles = Lists.newArrayList();
|
List<ContentFile<?>> newContentFiles = Lists.newArrayList();
|
||||||
for (ContentFile<?> contentFile : icebergFiles_.getAllContentFiles()) {
|
for (ContentFile<?> contentFile : icebergFiles_.getAllContentFiles()) {
|
||||||
FileDescriptor fd = getOldFd(contentFile, partPath);
|
IcebergFileDescriptor fd = getOldFd(contentFile, partPath);
|
||||||
if (fd == null) {
|
if (fd == null) {
|
||||||
newContentFiles.add(contentFile);
|
newContentFiles.add(contentFile);
|
||||||
} else {
|
} else {
|
||||||
|
int oldPartId = fd.getFbFileMetadata().icebergMetadata().partId();
|
||||||
|
TIcebergPartition partition = oldIcebergPartitions_.get(oldPartId);
|
||||||
|
Integer newPartId = loadedIcebergPartitions_.computeIfAbsent(
|
||||||
|
partition, k -> nextPartitionId_.getAndIncrement());
|
||||||
|
// Look up the partition info in this old file descriptor from the partition list.
|
||||||
|
// Put the partition info in the new partitions map and write the new partition id
|
||||||
|
// to the file metadata of the fd.
|
||||||
|
if (!fd.getFbFileMetadata().icebergMetadata().mutatePartId(newPartId)) {
|
||||||
|
throw new TableLoadingException("Error modifying the Iceberg file descriptor.");
|
||||||
|
}
|
||||||
++loadStats_.skippedFiles;
|
++loadStats_.skippedFiles;
|
||||||
loadedFds_.add(fd);
|
loadedFds_.add(fd);
|
||||||
fileMetadataStats_.accumulate(fd);
|
fileMetadataStats_.accumulate(fd);
|
||||||
@@ -183,10 +212,11 @@ public class IcebergFileMetadataLoader extends FileMetadataLoader {
|
|||||||
Pair<String, String> absPathRelPath = getAbsPathRelPath(partPath, stat);
|
Pair<String, String> absPathRelPath = getAbsPathRelPath(partPath, stat);
|
||||||
String absPath = absPathRelPath.first;
|
String absPath = absPathRelPath.first;
|
||||||
String relPath = absPathRelPath.second;
|
String relPath = absPathRelPath.second;
|
||||||
|
int partitionId = addPartitionInfo(contentFile);
|
||||||
|
|
||||||
return IcebergFileDescriptor.cloneWithFileMetadata(
|
return IcebergFileDescriptor.cloneWithFileMetadata(
|
||||||
createFd(fs, stat, relPath, null, absPath),
|
createFd(fs, stat, relPath, null, absPath),
|
||||||
IcebergUtil.createIcebergMetadata(iceTbl_, contentFile));
|
IcebergUtil.createIcebergMetadata(iceTbl_, contentFile, partitionId));
|
||||||
}
|
}
|
||||||
|
|
||||||
private IcebergFileDescriptor createLocatedFd(ContentFile<?> contentFile,
|
private IcebergFileDescriptor createLocatedFd(ContentFile<?> contentFile,
|
||||||
@@ -197,10 +227,18 @@ public class IcebergFileMetadataLoader extends FileMetadataLoader {
|
|||||||
Pair<String, String> absPathRelPath = getAbsPathRelPath(partPath, stat);
|
Pair<String, String> absPathRelPath = getAbsPathRelPath(partPath, stat);
|
||||||
String absPath = absPathRelPath.first;
|
String absPath = absPathRelPath.first;
|
||||||
String relPath = absPathRelPath.second;
|
String relPath = absPathRelPath.second;
|
||||||
|
int partitionId = addPartitionInfo(contentFile);
|
||||||
|
|
||||||
return IcebergFileDescriptor.cloneWithFileMetadata(
|
return IcebergFileDescriptor.cloneWithFileMetadata(
|
||||||
createFd(null, stat, relPath, numUnknownDiskIds, absPath),
|
createFd(null, stat, relPath, numUnknownDiskIds, absPath),
|
||||||
IcebergUtil.createIcebergMetadata(iceTbl_, contentFile));
|
IcebergUtil.createIcebergMetadata(iceTbl_, contentFile, partitionId));
|
||||||
|
}
|
||||||
|
|
||||||
|
private int addPartitionInfo(ContentFile<?> contentFile) {
|
||||||
|
TIcebergPartition partition =
|
||||||
|
IcebergUtil.createIcebergPartitionInfo(iceTbl_, contentFile);
|
||||||
|
return loadedIcebergPartitions_.computeIfAbsent(
|
||||||
|
partition, k -> nextPartitionId_.getAndIncrement());
|
||||||
}
|
}
|
||||||
|
|
||||||
Pair<String, String> getAbsPathRelPath(Path partPath, FileStatus stat)
|
Pair<String, String> getAbsPathRelPath(Path partPath, FileStatus stat)
|
||||||
|
|||||||
@@ -526,11 +526,12 @@ public class IcebergTable extends Table implements FeIcebergTable {
|
|||||||
icebergApiTable_,
|
icebergApiTable_,
|
||||||
fileStore_ == null ? Collections.emptyList() : fileStore_.getAllFiles(),
|
fileStore_ == null ? Collections.emptyList() : fileStore_.getAllFiles(),
|
||||||
getHostIndex(), Preconditions.checkNotNull(icebergFiles),
|
getHostIndex(), Preconditions.checkNotNull(icebergFiles),
|
||||||
|
fileStore_ == null ? Collections.emptyList() : fileStore_.getPartitionList(),
|
||||||
Utils.requiresDataFilesInTableLocation(this));
|
Utils.requiresDataFilesInTableLocation(this));
|
||||||
loader.load();
|
loader.load();
|
||||||
catalogTimeline.markEvent("Loaded Iceberg file descriptors");
|
catalogTimeline.markEvent("Loaded Iceberg file descriptors");
|
||||||
fileStore_ = new IcebergContentFileStore(
|
fileStore_ = new IcebergContentFileStore(icebergApiTable_,
|
||||||
icebergApiTable_, loader.getLoadedIcebergFds(), icebergFiles);
|
loader.getLoadedIcebergFds(), icebergFiles, loader.getIcebergPartitions());
|
||||||
partitionStats_ = Utils.loadPartitionStats(this, icebergFiles);
|
partitionStats_ = Utils.loadPartitionStats(this, icebergFiles);
|
||||||
|
|
||||||
setAvroSchema(msClient, msTable_, fileStore_, catalogTimeline);
|
setAvroSchema(msClient, msTable_, fileStore_, catalogTimeline);
|
||||||
|
|||||||
@@ -581,10 +581,10 @@ public class IcebergMetaProvider implements MetaProvider {
|
|||||||
GroupedContentFiles groupedFiles = new GroupedContentFiles(scan.planFiles());
|
GroupedContentFiles groupedFiles = new GroupedContentFiles(scan.planFiles());
|
||||||
IcebergFileMetadataLoader iceFml = new IcebergFileMetadataLoader(
|
IcebergFileMetadataLoader iceFml = new IcebergFileMetadataLoader(
|
||||||
apiTable, Collections.emptyList(), hostIndex, groupedFiles,
|
apiTable, Collections.emptyList(), hostIndex, groupedFiles,
|
||||||
false);
|
new ArrayList<>(), false);
|
||||||
iceFml.load();
|
iceFml.load();
|
||||||
IcebergContentFileStore contentFileStore = new IcebergContentFileStore(
|
IcebergContentFileStore contentFileStore = new IcebergContentFileStore(apiTable,
|
||||||
apiTable, iceFml.getLoadedIcebergFds(), groupedFiles);
|
iceFml.getLoadedIcebergFds(), groupedFiles, iceFml.getIcebergPartitions());
|
||||||
return contentFileStore.toThrift();
|
return contentFileStore.toThrift();
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
throw new IllegalStateException(
|
throw new IllegalStateException(
|
||||||
|
|||||||
@@ -1945,48 +1945,7 @@ public class HdfsScanNode extends ScanNode {
|
|||||||
// This is in testing mode.
|
// This is in testing mode.
|
||||||
testTableSize = desc_.getTable().getTTableStats().total_file_bytes;
|
testTableSize = desc_.getTable().getTTableStats().total_file_bytes;
|
||||||
}
|
}
|
||||||
String partMetaTemplate = "partitions=%d/%d files=%d size=%s\n";
|
getPartitionExplainString(output, detailPrefix, table, testTableSize);
|
||||||
String erasureCodeTemplate = "erasure coded: files=%d size=%s\n";
|
|
||||||
if (!numPartitionsPerFs_.isEmpty()) {
|
|
||||||
// The table is partitioned; print a line for each filesystem we are reading
|
|
||||||
// partitions from
|
|
||||||
for (Map.Entry<FileSystemUtil.FsType, Long> partsPerFs :
|
|
||||||
numPartitionsPerFs_.entrySet()) {
|
|
||||||
FileSystemUtil.FsType fsType = partsPerFs.getKey();
|
|
||||||
output.append(detailPrefix);
|
|
||||||
output.append(fsType).append(" ");
|
|
||||||
long bytesToDisplay = totalBytesPerFs_.get(fsType);
|
|
||||||
if (testTableSize > -1) {
|
|
||||||
bytesToDisplay = (long) ((double) partsPerFs.getValue()
|
|
||||||
/ table.getPartitions().size() * testTableSize);
|
|
||||||
}
|
|
||||||
output.append(String.format(partMetaTemplate, partsPerFs.getValue(),
|
|
||||||
table.getPartitions().size(), totalFilesPerFs_.get(fsType),
|
|
||||||
PrintUtils.printBytes(bytesToDisplay)));
|
|
||||||
|
|
||||||
// Report the total number of erasure coded files and total bytes, if any.
|
|
||||||
if (totalFilesPerFsEC_.containsKey(fsType)) {
|
|
||||||
long totalNumECFiles = totalFilesPerFsEC_.get(fsType);
|
|
||||||
long totalECSize = totalBytesPerFsEC_.get(fsType);
|
|
||||||
output.append(String.format("%s", detailPrefix))
|
|
||||||
.append(String.format(erasureCodeTemplate, totalNumECFiles,
|
|
||||||
PrintUtils.printBytes(totalECSize)));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} else if (tbl_.getNumClusteringCols() == 0) {
|
|
||||||
// There are no partitions so we use the FsType of the base table. No report
|
|
||||||
// on EC related info.
|
|
||||||
output.append(detailPrefix);
|
|
||||||
output.append(table.getFsType()).append(" ");
|
|
||||||
output.append(String.format(partMetaTemplate, 1, table.getPartitions().size(),
|
|
||||||
0, PrintUtils.printBytes(0)));
|
|
||||||
} else {
|
|
||||||
// The table is partitioned, but no partitions are selected; in this case we
|
|
||||||
// exclude the FsType completely. No report on EC related info.
|
|
||||||
output.append(detailPrefix);
|
|
||||||
output.append(String.format(partMetaTemplate, 0, table.getPartitions().size(),
|
|
||||||
0, PrintUtils.printBytes(0)));
|
|
||||||
}
|
|
||||||
|
|
||||||
// Add information about whether this uses deterministic scan range scheduling
|
// Add information about whether this uses deterministic scan range scheduling
|
||||||
// To avoid polluting the explain output, only add this if mt_dop>0 and
|
// To avoid polluting the explain output, only add this if mt_dop>0 and
|
||||||
@@ -2068,6 +2027,65 @@ public class HdfsScanNode extends ScanNode {
|
|||||||
return output.toString();
|
return output.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected void getPartitionExplainString(StringBuilder output,
|
||||||
|
String detailPrefix, FeFsTable table, long testTableSize) {
|
||||||
|
String partMetaTemplate = "partitions=%d/%s files=%d size=%s\n";
|
||||||
|
String erasureCodeTemplate = "erasure coded: files=%d size=%s\n";
|
||||||
|
if (!numPartitionsPerFs_.isEmpty()) {
|
||||||
|
// The table is partitioned; print a line for each filesystem we are reading
|
||||||
|
// partitions from
|
||||||
|
for (Map.Entry<FileSystemUtil.FsType, Long> partsPerFs :
|
||||||
|
numPartitionsPerFs_.entrySet()) {
|
||||||
|
FileSystemUtil.FsType fsType = partsPerFs.getKey();
|
||||||
|
output.append(detailPrefix);
|
||||||
|
output.append(fsType).append(" ");
|
||||||
|
long bytesToDisplay = totalBytesPerFs_.get(fsType);
|
||||||
|
if (testTableSize > -1) {
|
||||||
|
bytesToDisplay = (long) ((double) partsPerFs.getValue()
|
||||||
|
/ getNumPartitions(table) * testTableSize);
|
||||||
|
}
|
||||||
|
output.append(String.format(
|
||||||
|
partMetaTemplate, getNumSelectedPartitions(partsPerFs.getValue()),
|
||||||
|
getNumPartitionString(table), totalFilesPerFs_.get(fsType),
|
||||||
|
PrintUtils.printBytes(bytesToDisplay)));
|
||||||
|
|
||||||
|
// Report the total number of erasure coded files and total bytes, if any.
|
||||||
|
if (totalFilesPerFsEC_.containsKey(fsType)) {
|
||||||
|
long totalNumECFiles = totalFilesPerFsEC_.get(fsType);
|
||||||
|
long totalECSize = totalBytesPerFsEC_.get(fsType);
|
||||||
|
output.append(String.format("%s", detailPrefix))
|
||||||
|
.append(String.format(erasureCodeTemplate, totalNumECFiles,
|
||||||
|
PrintUtils.printBytes(totalECSize)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} else if (tbl_.getNumClusteringCols() == 0) {
|
||||||
|
// There are no partitions so we use the FsType of the base table. No report
|
||||||
|
// on EC related info.
|
||||||
|
output.append(detailPrefix);
|
||||||
|
output.append(table.getFsType()).append(" ");
|
||||||
|
output.append(String.format(partMetaTemplate, 1, getNumPartitionString(table),
|
||||||
|
0, PrintUtils.printBytes(0)));
|
||||||
|
} else {
|
||||||
|
// The table is partitioned, but no partitions are selected; in this case we
|
||||||
|
// exclude the FsType completely. No report on EC related info.
|
||||||
|
output.append(detailPrefix);
|
||||||
|
output.append(String.format(partMetaTemplate, 0, getNumPartitionString(table),
|
||||||
|
0, PrintUtils.printBytes(0)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected long getNumSelectedPartitions(long partsPerFs) {
|
||||||
|
return partsPerFs;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int getNumPartitions(FeFsTable table) {
|
||||||
|
return table.getPartitions().size();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected String getNumPartitionString(FeFsTable table) {
|
||||||
|
return Integer.toString(getNumPartitions(table));
|
||||||
|
}
|
||||||
|
|
||||||
// Overriding this function can be used to add extra information to the explain string
|
// Overriding this function can be used to add extra information to the explain string
|
||||||
// of the HDFS Scan node from the derived classes (e.g. IcebergScanNode). Each new line
|
// of the HDFS Scan node from the derived classes (e.g. IcebergScanNode). Each new line
|
||||||
// in the output should be appended to 'explainLevel' to have the correct indentation.
|
// in the output should be appended to 'explainLevel' to have the correct indentation.
|
||||||
|
|||||||
@@ -20,11 +20,11 @@ package org.apache.impala.planner;
|
|||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.BitSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Random;
|
|
||||||
|
|
||||||
|
import org.apache.iceberg.Snapshot;
|
||||||
import org.apache.impala.analysis.Analyzer;
|
import org.apache.impala.analysis.Analyzer;
|
||||||
import org.apache.impala.analysis.Expr;
|
import org.apache.impala.analysis.Expr;
|
||||||
import org.apache.impala.analysis.MultiAggregateInfo;
|
import org.apache.impala.analysis.MultiAggregateInfo;
|
||||||
@@ -46,7 +46,6 @@ import org.slf4j.Logger;
|
|||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.Lists;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Scan of a single iceberg table.
|
* Scan of a single iceberg table.
|
||||||
@@ -59,6 +58,12 @@ public class IcebergScanNode extends HdfsScanNode {
|
|||||||
// See IMPALA-12765 for details.
|
// See IMPALA-12765 for details.
|
||||||
private List<IcebergFileDescriptor> fileDescs_;
|
private List<IcebergFileDescriptor> fileDescs_;
|
||||||
|
|
||||||
|
// Number of actual partitions in the table, inferred from file metadata.
|
||||||
|
// We treat Iceberg tables as unpartitioned HDFS tables, so instead of storing the file
|
||||||
|
// descriptors per partition in the partition map in FeFsTable, we keep track of
|
||||||
|
// the partition values and file descriptors separately in IcebergContentFileStore.
|
||||||
|
private int numIcebergPartitions_;
|
||||||
|
|
||||||
// Indicates that the files in 'fileDescs_' are sorted.
|
// Indicates that the files in 'fileDescs_' are sorted.
|
||||||
private boolean filesAreSorted_ = false;
|
private boolean filesAreSorted_ = false;
|
||||||
|
|
||||||
@@ -83,13 +88,15 @@ public class IcebergScanNode extends HdfsScanNode {
|
|||||||
|
|
||||||
public IcebergScanNode(PlanNodeId id, TableRef tblRef, List<Expr> conjuncts,
|
public IcebergScanNode(PlanNodeId id, TableRef tblRef, List<Expr> conjuncts,
|
||||||
MultiAggregateInfo aggInfo, List<IcebergFileDescriptor> fileDescs,
|
MultiAggregateInfo aggInfo, List<IcebergFileDescriptor> fileDescs,
|
||||||
|
int numPartitions,
|
||||||
List<Expr> nonIdentityConjuncts, List<Expr> skippedConjuncts, long snapshotId) {
|
List<Expr> nonIdentityConjuncts, List<Expr> skippedConjuncts, long snapshotId) {
|
||||||
this(id, tblRef, conjuncts, aggInfo, fileDescs, nonIdentityConjuncts,
|
this(id, tblRef, conjuncts, aggInfo, fileDescs, numPartitions, nonIdentityConjuncts,
|
||||||
skippedConjuncts, null, snapshotId);
|
skippedConjuncts, null, snapshotId);
|
||||||
}
|
}
|
||||||
|
|
||||||
public IcebergScanNode(PlanNodeId id, TableRef tblRef, List<Expr> conjuncts,
|
public IcebergScanNode(PlanNodeId id, TableRef tblRef, List<Expr> conjuncts,
|
||||||
MultiAggregateInfo aggInfo, List<IcebergFileDescriptor> fileDescs,
|
MultiAggregateInfo aggInfo, List<IcebergFileDescriptor> fileDescs,
|
||||||
|
int numPartitions,
|
||||||
List<Expr> nonIdentityConjuncts, List<Expr> skippedConjuncts, PlanNodeId deleteId,
|
List<Expr> nonIdentityConjuncts, List<Expr> skippedConjuncts, PlanNodeId deleteId,
|
||||||
long snapshotId) {
|
long snapshotId) {
|
||||||
super(id, tblRef.getDesc(), conjuncts,
|
super(id, tblRef.getDesc(), conjuncts,
|
||||||
@@ -99,6 +106,7 @@ public class IcebergScanNode extends HdfsScanNode {
|
|||||||
Preconditions.checkState(partitions_.size() == 1);
|
Preconditions.checkState(partitions_.size() == 1);
|
||||||
|
|
||||||
fileDescs_ = fileDescs;
|
fileDescs_ = fileDescs;
|
||||||
|
numIcebergPartitions_ = numPartitions;
|
||||||
if (((FeIcebergTable)tblRef.getTable()).isPartitioned()) {
|
if (((FeIcebergTable)tblRef.getTable()).isPartitioned()) {
|
||||||
// Let's order the file descriptors for better scheduling.
|
// Let's order the file descriptors for better scheduling.
|
||||||
// See IMPALA-12765 for details.
|
// See IMPALA-12765 for details.
|
||||||
@@ -221,6 +229,32 @@ public class IcebergScanNode extends HdfsScanNode {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected long getNumSelectedPartitions(long partsPerFs) {
|
||||||
|
BitSet selectedPartitions = new BitSet(numIcebergPartitions_);
|
||||||
|
for (FileDescriptor fd : fileDescs_) {
|
||||||
|
selectedPartitions.set(
|
||||||
|
((IcebergFileDescriptor)fd).getFbFileMetadata().icebergMetadata().partId());
|
||||||
|
}
|
||||||
|
return selectedPartitions.cardinality();
|
||||||
|
}
|
||||||
|
|
||||||
|
// Returns the number of partitions in the current snapshot, cached by catalog after
|
||||||
|
// loading. In case of time travel, the number of all partitions is not available
|
||||||
|
// in the old snapshot, therefore this info is omitted from the explain string.
|
||||||
|
@Override
|
||||||
|
protected String getNumPartitionString(FeFsTable table) {
|
||||||
|
Preconditions.checkState(table instanceof FeIcebergTable);
|
||||||
|
Snapshot currentSnapshot =
|
||||||
|
((FeIcebergTable) table).getIcebergApiTable().currentSnapshot();
|
||||||
|
if (currentSnapshot != null) {
|
||||||
|
if (snapshotId_ != currentSnapshot.snapshotId()) {
|
||||||
|
return "unknown";
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return Integer.toString(numIcebergPartitions_);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected String getDerivedExplainString(
|
protected String getDerivedExplainString(
|
||||||
String indentPrefix, TExplainLevel detailLevel) {
|
String indentPrefix, TExplainLevel detailLevel) {
|
||||||
|
|||||||
@@ -84,6 +84,7 @@ import org.apache.impala.fb.FbIcebergMetadata;
|
|||||||
import org.apache.impala.planner.JoinNode.DistributionMode;
|
import org.apache.impala.planner.JoinNode.DistributionMode;
|
||||||
import org.apache.impala.service.Frontend;
|
import org.apache.impala.service.Frontend;
|
||||||
import org.apache.impala.thrift.TColumnStats;
|
import org.apache.impala.thrift.TColumnStats;
|
||||||
|
import org.apache.impala.thrift.TIcebergPartition;
|
||||||
import org.apache.impala.thrift.TIcebergPartitionTransformType;
|
import org.apache.impala.thrift.TIcebergPartitionTransformType;
|
||||||
import org.apache.impala.thrift.TQueryOptions;
|
import org.apache.impala.thrift.TQueryOptions;
|
||||||
import org.apache.impala.thrift.TVirtualColumnType;
|
import org.apache.impala.thrift.TVirtualColumnType;
|
||||||
@@ -251,7 +252,9 @@ public class IcebergScanPlanner {
|
|||||||
// If there are no delete files we can just create a single SCAN node.
|
// If there are no delete files we can just create a single SCAN node.
|
||||||
Preconditions.checkState(dataFilesWithDeletes_.isEmpty());
|
Preconditions.checkState(dataFilesWithDeletes_.isEmpty());
|
||||||
PlanNode ret = new IcebergScanNode(ctx_.getNextNodeId(), tblRef_, conjuncts_,
|
PlanNode ret = new IcebergScanNode(ctx_.getNextNodeId(), tblRef_, conjuncts_,
|
||||||
aggInfo_, dataFilesWithoutDeletes_, nonIdentityConjuncts_,
|
aggInfo_, dataFilesWithoutDeletes_,
|
||||||
|
getIceTable().getContentFileStore().getNumPartitions(),
|
||||||
|
nonIdentityConjuncts_,
|
||||||
getSkippedConjuncts(), snapshotId_);
|
getSkippedConjuncts(), snapshotId_);
|
||||||
ret.init(analyzer_);
|
ret.init(analyzer_);
|
||||||
return ret;
|
return ret;
|
||||||
@@ -276,6 +279,7 @@ public class IcebergScanPlanner {
|
|||||||
// can just create a SCAN node for these and do a UNION ALL with the ANTI JOIN.
|
// can just create a SCAN node for these and do a UNION ALL with the ANTI JOIN.
|
||||||
IcebergScanNode dataScanNode = new IcebergScanNode(
|
IcebergScanNode dataScanNode = new IcebergScanNode(
|
||||||
ctx_.getNextNodeId(), tblRef_, conjuncts_, aggInfo_, dataFilesWithoutDeletes_,
|
ctx_.getNextNodeId(), tblRef_, conjuncts_, aggInfo_, dataFilesWithoutDeletes_,
|
||||||
|
getIceTable().getContentFileStore().getNumPartitions(),
|
||||||
nonIdentityConjuncts_, getSkippedConjuncts(), snapshotId_);
|
nonIdentityConjuncts_, getSkippedConjuncts(), snapshotId_);
|
||||||
dataScanNode.init(analyzer_);
|
dataScanNode.init(analyzer_);
|
||||||
List<Expr> outputExprs = tblRef_.getDesc().getSlots().stream().map(
|
List<Expr> outputExprs = tblRef_.getDesc().getSlots().stream().map(
|
||||||
@@ -315,6 +319,7 @@ public class IcebergScanPlanner {
|
|||||||
addDeletePositionSlots(deleteDeltaRef);
|
addDeletePositionSlots(deleteDeltaRef);
|
||||||
IcebergScanNode dataScanNode = new IcebergScanNode(
|
IcebergScanNode dataScanNode = new IcebergScanNode(
|
||||||
dataScanNodeId, tblRef_, conjuncts_, aggInfo_, dataFilesWithDeletes_,
|
dataScanNodeId, tblRef_, conjuncts_, aggInfo_, dataFilesWithDeletes_,
|
||||||
|
getIceTable().getContentFileStore().getNumPartitions(),
|
||||||
nonIdentityConjuncts_, getSkippedConjuncts(), deleteScanNodeId, snapshotId_);
|
nonIdentityConjuncts_, getSkippedConjuncts(), deleteScanNodeId, snapshotId_);
|
||||||
dataScanNode.init(analyzer_);
|
dataScanNode.init(analyzer_);
|
||||||
IcebergScanNode deleteScanNode = new IcebergScanNode(
|
IcebergScanNode deleteScanNode = new IcebergScanNode(
|
||||||
@@ -323,6 +328,7 @@ public class IcebergScanPlanner {
|
|||||||
Collections.emptyList(), /*conjuncts*/
|
Collections.emptyList(), /*conjuncts*/
|
||||||
aggInfo_,
|
aggInfo_,
|
||||||
Lists.newArrayList(positionDeleteFiles_),
|
Lists.newArrayList(positionDeleteFiles_),
|
||||||
|
getIceTable().getContentFileStore().getNumPartitions(),
|
||||||
Collections.emptyList(), /*nonIdentityConjuncts*/
|
Collections.emptyList(), /*nonIdentityConjuncts*/
|
||||||
Collections.emptyList(), /*skippedConjuncts*/
|
Collections.emptyList(), /*skippedConjuncts*/
|
||||||
snapshotId_);
|
snapshotId_);
|
||||||
@@ -526,6 +532,7 @@ public class IcebergScanPlanner {
|
|||||||
PlanNodeId dataScanNodeId = ctx_.getNextNodeId();
|
PlanNodeId dataScanNodeId = ctx_.getNextNodeId();
|
||||||
IcebergScanNode dataScanNode = new IcebergScanNode(
|
IcebergScanNode dataScanNode = new IcebergScanNode(
|
||||||
dataScanNodeId, tblRef_, conjuncts_, aggInfo_, dataFilesWithDeletes_,
|
dataScanNodeId, tblRef_, conjuncts_, aggInfo_, dataFilesWithDeletes_,
|
||||||
|
getIceTable().getContentFileStore().getNumPartitions(),
|
||||||
nonIdentityConjuncts_, getSkippedConjuncts(), snapshotId_);
|
nonIdentityConjuncts_, getSkippedConjuncts(), snapshotId_);
|
||||||
addAllSlotsForEqualityDeletes(tblRef_);
|
addAllSlotsForEqualityDeletes(tblRef_);
|
||||||
dataScanNode.init(analyzer_);
|
dataScanNode.init(analyzer_);
|
||||||
@@ -564,6 +571,7 @@ public class IcebergScanPlanner {
|
|||||||
Collections.emptyList(), /*conjuncts*/
|
Collections.emptyList(), /*conjuncts*/
|
||||||
aggInfo_,
|
aggInfo_,
|
||||||
Lists.newArrayList(equalityDeleteFiles),
|
Lists.newArrayList(equalityDeleteFiles),
|
||||||
|
getIceTable().getContentFileStore().getNumPartitions(),
|
||||||
Collections.emptyList(), /*nonIdentityConjuncts*/
|
Collections.emptyList(), /*nonIdentityConjuncts*/
|
||||||
Collections.emptyList(),
|
Collections.emptyList(),
|
||||||
snapshotId_); /*skippedConjuncts*/
|
snapshotId_); /*skippedConjuncts*/
|
||||||
@@ -622,6 +630,7 @@ public class IcebergScanPlanner {
|
|||||||
Preconditions.checkState(equalityIdsToDeleteFiles_.isEmpty());
|
Preconditions.checkState(equalityIdsToDeleteFiles_.isEmpty());
|
||||||
|
|
||||||
TimeTravelSpec timeTravelSpec = tblRef_.getTimeTravelSpec();
|
TimeTravelSpec timeTravelSpec = tblRef_.getTimeTravelSpec();
|
||||||
|
IcebergContentFileStore fileStore = getIceTable().getContentFileStore();
|
||||||
|
|
||||||
// 'metricsReporter_' is filled when the try-with-resources releases the FileScanTask
|
// 'metricsReporter_' is filled when the try-with-resources releases the FileScanTask
|
||||||
// iterable, i.e. not when the call to IcebergUtil.planFiles() returns.
|
// iterable, i.e. not when the call to IcebergUtil.planFiles() returns.
|
||||||
@@ -638,14 +647,15 @@ public class IcebergScanPlanner {
|
|||||||
residualExpressions_.add(residualExpr);
|
residualExpressions_.add(residualExpr);
|
||||||
}
|
}
|
||||||
Pair<IcebergFileDescriptor, Boolean> fileDesc =
|
Pair<IcebergFileDescriptor, Boolean> fileDesc =
|
||||||
getFileDescriptor(fileScanTask.file());
|
getFileDescriptor(fileScanTask.file(), fileStore);
|
||||||
if (!fileDesc.second) ++dataFilesCacheMisses;
|
if (!fileDesc.second) ++dataFilesCacheMisses;
|
||||||
if (fileScanTask.deletes().isEmpty()) {
|
if (fileScanTask.deletes().isEmpty()) {
|
||||||
dataFilesWithoutDeletes_.add(fileDesc.first);
|
dataFilesWithoutDeletes_.add(fileDesc.first);
|
||||||
} else {
|
} else {
|
||||||
dataFilesWithDeletes_.add(fileDesc.first);
|
dataFilesWithDeletes_.add(fileDesc.first);
|
||||||
for (DeleteFile delFile : fileScanTask.deletes()) {
|
for (DeleteFile delFile : fileScanTask.deletes()) {
|
||||||
Pair<IcebergFileDescriptor, Boolean> delFileDesc = getFileDescriptor(delFile);
|
Pair<IcebergFileDescriptor, Boolean> delFileDesc =
|
||||||
|
getFileDescriptor(delFile, fileStore);
|
||||||
if (!delFileDesc.second) ++dataFilesCacheMisses;
|
if (!delFileDesc.second) ++dataFilesCacheMisses;
|
||||||
if (delFile.content() == FileContent.EQUALITY_DELETES) {
|
if (delFile.content() == FileContent.EQUALITY_DELETES) {
|
||||||
addEqualityDeletesAndIds(delFileDesc.first);
|
addEqualityDeletesAndIds(delFileDesc.first);
|
||||||
@@ -789,10 +799,10 @@ public class IcebergScanPlanner {
|
|||||||
* Returns an IcebergFileDescriptor and an indicator whether it was found in the cache.
|
* Returns an IcebergFileDescriptor and an indicator whether it was found in the cache.
|
||||||
* True for cache hit and false for cache miss.
|
* True for cache hit and false for cache miss.
|
||||||
*/
|
*/
|
||||||
private Pair<IcebergFileDescriptor, Boolean> getFileDescriptor(ContentFile<?> cf)
|
private Pair<IcebergFileDescriptor, Boolean> getFileDescriptor(ContentFile<?> cf,
|
||||||
|
IcebergContentFileStore fileStore)
|
||||||
throws ImpalaRuntimeException {
|
throws ImpalaRuntimeException {
|
||||||
String pathHash = IcebergUtil.getFilePathHash(cf);
|
String pathHash = IcebergUtil.getFilePathHash(cf);
|
||||||
IcebergContentFileStore fileStore = getIceTable().getContentFileStore();
|
|
||||||
|
|
||||||
IcebergFileDescriptor iceFileDesc = cf.content() == FileContent.DATA ?
|
IcebergFileDescriptor iceFileDesc = cf.content() == FileContent.DATA ?
|
||||||
fileStore.getDataFileDescriptor(pathHash) :
|
fileStore.getDataFileDescriptor(pathHash) :
|
||||||
@@ -832,10 +842,28 @@ public class IcebergScanPlanner {
|
|||||||
|
|
||||||
Preconditions.checkNotNull(hdfsFileDesc);
|
Preconditions.checkNotNull(hdfsFileDesc);
|
||||||
|
|
||||||
|
Integer partitionId = 0;
|
||||||
|
|
||||||
|
TIcebergPartition partition =
|
||||||
|
IcebergUtil.createIcebergPartitionInfo(getIceTable().getIcebergApiTable(), cf);
|
||||||
|
Map<TIcebergPartition, Integer> partitions = fileStore.getPartitionMap();
|
||||||
|
partitionId = partitions.get(partition);
|
||||||
|
// If we do not find the partition among the current partitions cached by catalog, try
|
||||||
|
// looking it up among the old descriptors' partitions.
|
||||||
|
if (partitionId == null) {
|
||||||
|
synchronized (fileStore) {
|
||||||
|
partitionId = fileStore.getOldPartition(partition);
|
||||||
|
if (partitionId == null) {
|
||||||
|
partitionId = partitions.size() + fileStore.getOldPartitionsSize();
|
||||||
|
fileStore.addOldPartition(partition, partitionId);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// Add file descriptor to the cache.
|
// Add file descriptor to the cache.
|
||||||
iceFileDesc = IcebergFileDescriptor.cloneWithFileMetadata(
|
iceFileDesc = IcebergFileDescriptor.cloneWithFileMetadata(
|
||||||
hdfsFileDesc,
|
hdfsFileDesc, IcebergUtil.createIcebergMetadata(
|
||||||
IcebergUtil.createIcebergMetadata(getIceTable().getIcebergApiTable(), cf));
|
getIceTable().getIcebergApiTable(), cf, partitionId));
|
||||||
fileStore.addOldFileDescriptor(pathHash, iceFileDesc);
|
fileStore.addOldFileDescriptor(pathHash, iceFileDesc);
|
||||||
|
|
||||||
return new Pair<>(iceFileDesc, false);
|
return new Pair<>(iceFileDesc, false);
|
||||||
|
|||||||
@@ -116,6 +116,7 @@ import org.apache.impala.thrift.THdfsCompression;
|
|||||||
import org.apache.impala.thrift.THdfsFileFormat;
|
import org.apache.impala.thrift.THdfsFileFormat;
|
||||||
import org.apache.impala.thrift.TIcebergCatalog;
|
import org.apache.impala.thrift.TIcebergCatalog;
|
||||||
import org.apache.impala.thrift.TIcebergFileFormat;
|
import org.apache.impala.thrift.TIcebergFileFormat;
|
||||||
|
import org.apache.impala.thrift.TIcebergPartition;
|
||||||
import org.apache.impala.thrift.TIcebergPartitionField;
|
import org.apache.impala.thrift.TIcebergPartitionField;
|
||||||
import org.apache.impala.thrift.TIcebergPartitionSpec;
|
import org.apache.impala.thrift.TIcebergPartitionSpec;
|
||||||
import org.apache.impala.thrift.TIcebergPartitionTransformType;
|
import org.apache.impala.thrift.TIcebergPartitionTransformType;
|
||||||
@@ -1060,15 +1061,34 @@ public class IcebergUtil {
|
|||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static TIcebergPartition createIcebergPartitionInfo(
|
||||||
|
Table iceApiTbl, ContentFile cf) {
|
||||||
|
TIcebergPartition partInfo = new TIcebergPartition(cf.specId(), new ArrayList<>());
|
||||||
|
PartitionSpec spec = iceApiTbl.specs().get(cf.specId());
|
||||||
|
Preconditions.checkState(spec.fields().size() == cf.partition().size());
|
||||||
|
List<String> partitionKeys = new ArrayList<>();
|
||||||
|
for (int i = 0; i < spec.fields().size(); ++i) {
|
||||||
|
Object partValue = cf.partition().get(i, Object.class);
|
||||||
|
if (partValue != null) {
|
||||||
|
partitionKeys.add(partValue.toString());
|
||||||
|
} else {
|
||||||
|
partitionKeys.add("NULL");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
partInfo.setPartition_values(partitionKeys);
|
||||||
|
return partInfo;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Extracts metadata from Iceberg data file object 'cf'. Such metadata is the file
|
* Extracts metadata from Iceberg data file object 'cf'. Such metadata is the file
|
||||||
* format of the data file, also the partition information the data file belongs.
|
* format of the data file, also the partition information the data file belongs.
|
||||||
* It creates a flatbuffer so it can be passed between machines and processes without
|
* It creates a flatbuffer so it can be passed between machines and processes without
|
||||||
* further de/serialization.
|
* further de/serialization.
|
||||||
*/
|
*/
|
||||||
public static FbFileMetadata createIcebergMetadata(Table iceApiTbl, ContentFile cf) {
|
public static FbFileMetadata createIcebergMetadata(
|
||||||
|
Table iceApiTbl, ContentFile cf, int partId) {
|
||||||
FlatBufferBuilder fbb = new FlatBufferBuilder(1);
|
FlatBufferBuilder fbb = new FlatBufferBuilder(1);
|
||||||
int iceOffset = createIcebergMetadata(iceApiTbl, fbb, cf);
|
int iceOffset = createIcebergMetadata(iceApiTbl, fbb, cf, partId);
|
||||||
fbb.finish(FbFileMetadata.createFbFileMetadata(fbb, iceOffset));
|
fbb.finish(FbFileMetadata.createFbFileMetadata(fbb, iceOffset));
|
||||||
ByteBuffer bb = fbb.dataBuffer().slice();
|
ByteBuffer bb = fbb.dataBuffer().slice();
|
||||||
ByteBuffer compressedBb = ByteBuffer.allocate(bb.capacity());
|
ByteBuffer compressedBb = ByteBuffer.allocate(bb.capacity());
|
||||||
@@ -1077,7 +1097,7 @@ public class IcebergUtil {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private static int createIcebergMetadata(Table iceApiTbl, FlatBufferBuilder fbb,
|
private static int createIcebergMetadata(Table iceApiTbl, FlatBufferBuilder fbb,
|
||||||
ContentFile cf) {
|
ContentFile cf, int partId) {
|
||||||
int partKeysOffset = -1;
|
int partKeysOffset = -1;
|
||||||
PartitionSpec spec = iceApiTbl.specs().get(cf.specId());
|
PartitionSpec spec = iceApiTbl.specs().get(cf.specId());
|
||||||
if (spec != null && !spec.fields().isEmpty()) {
|
if (spec != null && !spec.fields().isEmpty()) {
|
||||||
@@ -1117,7 +1137,7 @@ public class IcebergUtil {
|
|||||||
if (eqFieldIdsOffset != -1) {
|
if (eqFieldIdsOffset != -1) {
|
||||||
FbIcebergMetadata.addEqualityFieldIds(fbb, eqFieldIdsOffset);
|
FbIcebergMetadata.addEqualityFieldIds(fbb, eqFieldIdsOffset);
|
||||||
}
|
}
|
||||||
|
FbIcebergMetadata.addPartId(fbb, partId);
|
||||||
return FbIcebergMetadata.endFbIcebergMetadata(fbb);
|
return FbIcebergMetadata.endFbIcebergMetadata(fbb);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -35,6 +35,7 @@ import org.apache.impala.common.FileSystemUtil;
|
|||||||
import org.apache.impala.compat.MetastoreShim;
|
import org.apache.impala.compat.MetastoreShim;
|
||||||
import org.apache.impala.service.BackendConfig;
|
import org.apache.impala.service.BackendConfig;
|
||||||
import org.apache.impala.testutil.CatalogServiceTestCatalog;
|
import org.apache.impala.testutil.CatalogServiceTestCatalog;
|
||||||
|
import org.apache.impala.thrift.TIcebergPartition;
|
||||||
import org.apache.impala.thrift.TNetworkAddress;
|
import org.apache.impala.thrift.TNetworkAddress;
|
||||||
import org.apache.impala.util.IcebergUtil;
|
import org.apache.impala.util.IcebergUtil;
|
||||||
import org.apache.impala.util.ListMap;
|
import org.apache.impala.util.ListMap;
|
||||||
@@ -145,7 +146,7 @@ public class FileMetadataLoaderTest {
|
|||||||
CatalogServiceCatalog catalog = CatalogServiceTestCatalog.create();
|
CatalogServiceCatalog catalog = CatalogServiceTestCatalog.create();
|
||||||
IcebergFileMetadataLoader fml1 = getLoaderForIcebergTable(catalog,
|
IcebergFileMetadataLoader fml1 = getLoaderForIcebergTable(catalog,
|
||||||
"functional_parquet", "iceberg_partitioned",
|
"functional_parquet", "iceberg_partitioned",
|
||||||
/* oldFds = */ Collections.emptyList(),
|
/* oldFds = */ Collections.emptyList(), Collections.emptyList(),
|
||||||
/* requiresDataFilesInTableLocation = */ true);
|
/* requiresDataFilesInTableLocation = */ true);
|
||||||
fml1.load();
|
fml1.load();
|
||||||
assertEquals(20, fml1.getStats().loadedFiles);
|
assertEquals(20, fml1.getStats().loadedFiles);
|
||||||
@@ -160,7 +161,7 @@ public class FileMetadataLoaderTest {
|
|||||||
|
|
||||||
IcebergFileMetadataLoader fml2 = getLoaderForIcebergTable(catalog,
|
IcebergFileMetadataLoader fml2 = getLoaderForIcebergTable(catalog,
|
||||||
"functional_parquet", "iceberg_non_partitioned",
|
"functional_parquet", "iceberg_non_partitioned",
|
||||||
/* oldFds = */ Collections.emptyList(),
|
/* oldFds = */ Collections.emptyList(), Collections.emptyList(),
|
||||||
/* requiresDataFilesInTableLocation = */ true);
|
/* requiresDataFilesInTableLocation = */ true);
|
||||||
fml2.load();
|
fml2.load();
|
||||||
assertEquals(20, fml2.getStats().loadedFiles);
|
assertEquals(20, fml2.getStats().loadedFiles);
|
||||||
@@ -185,7 +186,7 @@ public class FileMetadataLoaderTest {
|
|||||||
CatalogServiceCatalog catalog = CatalogServiceTestCatalog.create();
|
CatalogServiceCatalog catalog = CatalogServiceTestCatalog.create();
|
||||||
IcebergFileMetadataLoader fml = getLoaderForIcebergTable(catalog,
|
IcebergFileMetadataLoader fml = getLoaderForIcebergTable(catalog,
|
||||||
"functional_parquet", "iceberg_partitioned",
|
"functional_parquet", "iceberg_partitioned",
|
||||||
/* oldFds = */ Collections.emptyList(),
|
/* oldFds = */ Collections.emptyList(), Collections.emptyList(),
|
||||||
/* requiresDataFilesInTableLocation = */ true);
|
/* requiresDataFilesInTableLocation = */ true);
|
||||||
fml.load();
|
fml.load();
|
||||||
List<IcebergFileDescriptor> fileDescs = fml.getLoadedIcebergFds();
|
List<IcebergFileDescriptor> fileDescs = fml.getLoadedIcebergFds();
|
||||||
@@ -203,13 +204,13 @@ public class FileMetadataLoaderTest {
|
|||||||
CatalogServiceCatalog catalog = CatalogServiceTestCatalog.create();
|
CatalogServiceCatalog catalog = CatalogServiceTestCatalog.create();
|
||||||
IcebergFileMetadataLoader fml1 = getLoaderForIcebergTable(catalog,
|
IcebergFileMetadataLoader fml1 = getLoaderForIcebergTable(catalog,
|
||||||
"functional_parquet", "iceberg_partitioned",
|
"functional_parquet", "iceberg_partitioned",
|
||||||
/* oldFds = */ Collections.emptyList(),
|
/* oldFds = */ Collections.emptyList(), Collections.emptyList(),
|
||||||
/* requiresDataFilesInTableLocation = */ true);
|
/* requiresDataFilesInTableLocation = */ true);
|
||||||
fml1.load();
|
fml1.load();
|
||||||
|
|
||||||
IcebergFileMetadataLoader fml1Refresh = getLoaderForIcebergTable(catalog,
|
IcebergFileMetadataLoader fml1Refresh = getLoaderForIcebergTable(catalog,
|
||||||
"functional_parquet", "iceberg_partitioned",
|
"functional_parquet", "iceberg_partitioned",
|
||||||
/* oldFds = */ fml1.getLoadedIcebergFds(),
|
/* oldFds = */ fml1.getLoadedIcebergFds(), fml1.getIcebergPartitionList(),
|
||||||
/* requiresDataFilesInTableLocation = */ true);
|
/* requiresDataFilesInTableLocation = */ true);
|
||||||
fml1Refresh.load();
|
fml1Refresh.load();
|
||||||
assertEquals(0, fml1Refresh.getStats().loadedFiles);
|
assertEquals(0, fml1Refresh.getStats().loadedFiles);
|
||||||
@@ -225,13 +226,13 @@ public class FileMetadataLoaderTest {
|
|||||||
|
|
||||||
IcebergFileMetadataLoader fml2 = getLoaderForIcebergTable(catalog,
|
IcebergFileMetadataLoader fml2 = getLoaderForIcebergTable(catalog,
|
||||||
"functional_parquet", "iceberg_non_partitioned",
|
"functional_parquet", "iceberg_non_partitioned",
|
||||||
/* oldFds = */ Collections.emptyList(),
|
/* oldFds = */ Collections.emptyList(), Collections.emptyList(),
|
||||||
/* requiresDataFilesInTableLocation = */ true);
|
/* requiresDataFilesInTableLocation = */ true);
|
||||||
fml2.load();
|
fml2.load();
|
||||||
|
|
||||||
IcebergFileMetadataLoader fml2Refresh = getLoaderForIcebergTable(catalog,
|
IcebergFileMetadataLoader fml2Refresh = getLoaderForIcebergTable(catalog,
|
||||||
"functional_parquet", "iceberg_non_partitioned",
|
"functional_parquet", "iceberg_non_partitioned",
|
||||||
/* oldFds = */ fml2.getLoadedIcebergFds(),
|
/* oldFds = */ fml2.getLoadedIcebergFds(), fml1.getIcebergPartitionList(),
|
||||||
/* requiresDataFilesInTableLocation = */ true);
|
/* requiresDataFilesInTableLocation = */ true);
|
||||||
fml2Refresh.load();
|
fml2Refresh.load();
|
||||||
assertEquals(0, fml2Refresh.getStats().loadedFiles);
|
assertEquals(0, fml2Refresh.getStats().loadedFiles);
|
||||||
@@ -251,13 +252,14 @@ public class FileMetadataLoaderTest {
|
|||||||
CatalogServiceCatalog catalog = CatalogServiceTestCatalog.create();
|
CatalogServiceCatalog catalog = CatalogServiceTestCatalog.create();
|
||||||
IcebergFileMetadataLoader fml1 = getLoaderForIcebergTable(catalog,
|
IcebergFileMetadataLoader fml1 = getLoaderForIcebergTable(catalog,
|
||||||
"functional_parquet", "iceberg_partitioned",
|
"functional_parquet", "iceberg_partitioned",
|
||||||
/* oldFds = */ Collections.emptyList(),
|
/* oldFds = */ Collections.emptyList(), Collections.emptyList(),
|
||||||
/* requiresDataFilesInTableLocation = */ true);
|
/* requiresDataFilesInTableLocation = */ true);
|
||||||
fml1.load();
|
fml1.load();
|
||||||
|
|
||||||
IcebergFileMetadataLoader fml1Refresh = getLoaderForIcebergTable(catalog,
|
IcebergFileMetadataLoader fml1Refresh = getLoaderForIcebergTable(catalog,
|
||||||
"functional_parquet", "iceberg_partitioned",
|
"functional_parquet", "iceberg_partitioned",
|
||||||
/* oldFds = */ fml1.getLoadedIcebergFds().subList(0, 10),
|
/* oldFds = */ fml1.getLoadedIcebergFds().subList(0, 10),
|
||||||
|
fml1.getIcebergPartitionList(),
|
||||||
/* requiresDataFilesInTableLocation = */ true);
|
/* requiresDataFilesInTableLocation = */ true);
|
||||||
fml1Refresh.load();
|
fml1Refresh.load();
|
||||||
assertEquals(10, fml1Refresh.getStats().loadedFiles);
|
assertEquals(10, fml1Refresh.getStats().loadedFiles);
|
||||||
@@ -266,13 +268,14 @@ public class FileMetadataLoaderTest {
|
|||||||
|
|
||||||
IcebergFileMetadataLoader fml2 = getLoaderForIcebergTable(catalog,
|
IcebergFileMetadataLoader fml2 = getLoaderForIcebergTable(catalog,
|
||||||
"functional_parquet", "iceberg_non_partitioned",
|
"functional_parquet", "iceberg_non_partitioned",
|
||||||
/* oldFds = */ Collections.emptyList(),
|
/* oldFds = */ Collections.emptyList(), Collections.emptyList(),
|
||||||
/* requiresDataFilesInTableLocation = */ true);
|
/* requiresDataFilesInTableLocation = */ true);
|
||||||
fml2.load();
|
fml2.load();
|
||||||
|
|
||||||
IcebergFileMetadataLoader fml2Refresh = getLoaderForIcebergTable(catalog,
|
IcebergFileMetadataLoader fml2Refresh = getLoaderForIcebergTable(catalog,
|
||||||
"functional_parquet", "iceberg_non_partitioned",
|
"functional_parquet", "iceberg_non_partitioned",
|
||||||
/* oldFds = */ fml2.getLoadedIcebergFds().subList(0, 10),
|
/* oldFds = */ fml2.getLoadedIcebergFds().subList(0, 10),
|
||||||
|
fml1.getIcebergPartitionList(),
|
||||||
/* requiresDataFilesInTableLocation = */ true);
|
/* requiresDataFilesInTableLocation = */ true);
|
||||||
fml2Refresh.load();
|
fml2Refresh.load();
|
||||||
assertEquals(10, fml2Refresh.getStats().loadedFiles);
|
assertEquals(10, fml2Refresh.getStats().loadedFiles);
|
||||||
@@ -286,13 +289,14 @@ public class FileMetadataLoaderTest {
|
|||||||
BackendConfig.INSTANCE.setIcebergAllowDatafileInTableLocationOnly(false);
|
BackendConfig.INSTANCE.setIcebergAllowDatafileInTableLocationOnly(false);
|
||||||
IcebergFileMetadataLoader fml1 = getLoaderForIcebergTable(catalog,
|
IcebergFileMetadataLoader fml1 = getLoaderForIcebergTable(catalog,
|
||||||
"functional_parquet", "iceberg_multiple_storage_locations",
|
"functional_parquet", "iceberg_multiple_storage_locations",
|
||||||
/* oldFds = */ Collections.emptyList(),
|
/* oldFds = */ Collections.emptyList(), Collections.emptyList(),
|
||||||
/* requiresDataFilesInTableLocation = */ false);
|
/* requiresDataFilesInTableLocation = */ false);
|
||||||
fml1.load();
|
fml1.load();
|
||||||
|
|
||||||
IcebergFileMetadataLoader fml1Refresh1 = getLoaderForIcebergTable(catalog,
|
IcebergFileMetadataLoader fml1Refresh1 = getLoaderForIcebergTable(catalog,
|
||||||
"functional_parquet", "iceberg_multiple_storage_locations",
|
"functional_parquet", "iceberg_multiple_storage_locations",
|
||||||
/* oldFds = */ fml1.getLoadedIcebergFds().subList(0, 1),
|
/* oldFds = */ fml1.getLoadedIcebergFds().subList(0, 1),
|
||||||
|
fml1.getIcebergPartitionList(),
|
||||||
/* requiresDataFilesInTableLocation = */ false);
|
/* requiresDataFilesInTableLocation = */ false);
|
||||||
fml1Refresh1.load();
|
fml1Refresh1.load();
|
||||||
assertEquals(5, fml1Refresh1.getStats().loadedFiles);
|
assertEquals(5, fml1Refresh1.getStats().loadedFiles);
|
||||||
@@ -302,6 +306,7 @@ public class FileMetadataLoaderTest {
|
|||||||
IcebergFileMetadataLoader fml1Refresh5 = getLoaderForIcebergTable(catalog,
|
IcebergFileMetadataLoader fml1Refresh5 = getLoaderForIcebergTable(catalog,
|
||||||
"functional_parquet", "iceberg_multiple_storage_locations",
|
"functional_parquet", "iceberg_multiple_storage_locations",
|
||||||
/* oldFds = */ fml1.getLoadedIcebergFds().subList(0, 5),
|
/* oldFds = */ fml1.getLoadedIcebergFds().subList(0, 5),
|
||||||
|
fml1.getIcebergPartitionList(),
|
||||||
/* requiresDataFilesInTableLocation = */ false);
|
/* requiresDataFilesInTableLocation = */ false);
|
||||||
fml1Refresh5.load();
|
fml1Refresh5.load();
|
||||||
assertEquals(1, fml1Refresh5.getStats().loadedFiles);
|
assertEquals(1, fml1Refresh5.getStats().loadedFiles);
|
||||||
@@ -311,7 +316,8 @@ public class FileMetadataLoaderTest {
|
|||||||
|
|
||||||
private IcebergFileMetadataLoader getLoaderForIcebergTable(
|
private IcebergFileMetadataLoader getLoaderForIcebergTable(
|
||||||
CatalogServiceCatalog catalog, String dbName, String tblName,
|
CatalogServiceCatalog catalog, String dbName, String tblName,
|
||||||
List<IcebergFileDescriptor> oldFds, boolean requiresDataFilesInTableLocation)
|
List<IcebergFileDescriptor> oldFds, List<TIcebergPartition> oldPartitions,
|
||||||
|
boolean requiresDataFilesInTableLocation)
|
||||||
throws CatalogException {
|
throws CatalogException {
|
||||||
ListMap<TNetworkAddress> hostIndex = new ListMap<>();
|
ListMap<TNetworkAddress> hostIndex = new ListMap<>();
|
||||||
FeIcebergTable iceT = (FeIcebergTable)catalog.getOrLoadTable(
|
FeIcebergTable iceT = (FeIcebergTable)catalog.getOrLoadTable(
|
||||||
@@ -320,7 +326,7 @@ public class FileMetadataLoaderTest {
|
|||||||
GroupedContentFiles iceFiles = IcebergUtil.getIcebergFiles(iceT,
|
GroupedContentFiles iceFiles = IcebergUtil.getIcebergFiles(iceT,
|
||||||
/*predicates=*/Collections.emptyList(), /*timeTravelSpec=*/null);
|
/*predicates=*/Collections.emptyList(), /*timeTravelSpec=*/null);
|
||||||
return new IcebergFileMetadataLoader(iceT.getIcebergApiTable(),
|
return new IcebergFileMetadataLoader(iceT.getIcebergApiTable(),
|
||||||
oldFds, hostIndex, iceFiles, requiresDataFilesInTableLocation);
|
oldFds, hostIndex, iceFiles, oldPartitions, requiresDataFilesInTableLocation);
|
||||||
}
|
}
|
||||||
|
|
||||||
private FileMetadataLoader getLoaderForAcidTable(
|
private FileMetadataLoader getLoaderForAcidTable(
|
||||||
|
|||||||
@@ -4,8 +4,8 @@ using functional_parquet.iceberg_partitioned source
|
|||||||
on target.i = source.id
|
on target.i = source.id
|
||||||
when not matched then insert values (source.id, source.user)
|
when not matched then insert values (source.id, source.user)
|
||||||
---- PLAN
|
---- PLAN
|
||||||
Max Per-Host Resource Reservation: Memory=1.98MB Threads=3
|
Max Per-Host Resource Reservation: Memory=2.00MB Threads=3
|
||||||
Per-Host Resource Estimates: Memory=98MB
|
Per-Host Resource Estimates: Memory=130MB
|
||||||
WARNING: The following tables are missing relevant table and/or column statistics.
|
WARNING: The following tables are missing relevant table and/or column statistics.
|
||||||
functional_parquet.iceberg_partitioned, functional_parquet.iceberg_v2_no_deletes
|
functional_parquet.iceberg_partitioned, functional_parquet.iceberg_v2_no_deletes
|
||||||
Analyzed query: SELECT /* +straight_join */ CAST(CAST(CAST(TupleIsNull(0) AS
|
Analyzed query: SELECT /* +straight_join */ CAST(CAST(CAST(TupleIsNull(0) AS
|
||||||
@@ -26,7 +26,7 @@ WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
|||||||
| | result expressions: source.id, source.`user`
|
| | result expressions: source.id, source.`user`
|
||||||
| | type: INSERT
|
| | type: INSERT
|
||||||
| mem-estimate=0B mem-reservation=0B thread-reservation=0
|
| mem-estimate=0B mem-reservation=0B thread-reservation=0
|
||||||
| tuple-ids=0N,1N,2 row-size=60B cardinality=23
|
| tuple-ids=0N,1N,2 row-size=80B cardinality=23
|
||||||
| in pipelines: 00(GETNEXT)
|
| in pipelines: 00(GETNEXT)
|
||||||
|
|
|
|
||||||
02:HASH JOIN [FULL OUTER JOIN]
|
02:HASH JOIN [FULL OUTER JOIN]
|
||||||
@@ -37,7 +37,7 @@ WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
|||||||
| in pipelines: 00(GETNEXT), 01(OPEN)
|
| in pipelines: 00(GETNEXT), 01(OPEN)
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
|--01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
||||||
| HDFS partitions=1/1 files=20 size=22.90KB
|
| HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
| Iceberg snapshot id: 8270633197658268308
|
| Iceberg snapshot id: 8270633197658268308
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=20 size=22.90KB
|
| table: rows=20 size=22.90KB
|
||||||
@@ -54,12 +54,12 @@ WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
|||||||
table: rows=3 size=625B
|
table: rows=3 size=625B
|
||||||
columns missing stats: i, s
|
columns missing stats: i, s
|
||||||
extrapolated-rows=disabled max-scan-range-rows=3
|
extrapolated-rows=disabled max-scan-range-rows=3
|
||||||
mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=1
|
mem-estimate=64.00MB mem-reservation=32.00KB thread-reservation=1
|
||||||
tuple-ids=0 row-size=16B cardinality=3
|
tuple-ids=0 row-size=36B cardinality=3
|
||||||
in pipelines: 00(GETNEXT)
|
in pipelines: 00(GETNEXT)
|
||||||
---- DISTRIBUTEDPLAN
|
---- DISTRIBUTEDPLAN
|
||||||
Max Per-Host Resource Reservation: Memory=1.98MB Threads=5
|
Max Per-Host Resource Reservation: Memory=2.00MB Threads=5
|
||||||
Per-Host Resource Estimates: Memory=98MB
|
Per-Host Resource Estimates: Memory=130MB
|
||||||
WARNING: The following tables are missing relevant table and/or column statistics.
|
WARNING: The following tables are missing relevant table and/or column statistics.
|
||||||
functional_parquet.iceberg_partitioned, functional_parquet.iceberg_v2_no_deletes
|
functional_parquet.iceberg_partitioned, functional_parquet.iceberg_v2_no_deletes
|
||||||
Analyzed query: SELECT /* +straight_join */ CAST(CAST(CAST(TupleIsNull(0) AS
|
Analyzed query: SELECT /* +straight_join */ CAST(CAST(CAST(TupleIsNull(0) AS
|
||||||
@@ -73,21 +73,21 @@ F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
|||||||
| Per-Host Resources: mem-estimate=1.97MB mem-reservation=1.94MB thread-reservation=1
|
| Per-Host Resources: mem-estimate=1.97MB mem-reservation=1.94MB thread-reservation=1
|
||||||
WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
||||||
| output exprs: target.i, target.s
|
| output exprs: target.i, target.s
|
||||||
| mem-estimate=1.35KB mem-reservation=0B thread-reservation=0
|
| mem-estimate=1.80KB mem-reservation=0B thread-reservation=0
|
||||||
|
|
|
|
||||||
03:MERGE
|
03:MERGE
|
||||||
| CASE 0: NOT MATCHED BY TARGET
|
| CASE 0: NOT MATCHED BY TARGET
|
||||||
| | result expressions: source.id, source.`user`
|
| | result expressions: source.id, source.`user`
|
||||||
| | type: INSERT
|
| | type: INSERT
|
||||||
| mem-estimate=0B mem-reservation=0B thread-reservation=0
|
| mem-estimate=0B mem-reservation=0B thread-reservation=0
|
||||||
| tuple-ids=0N,1N,2 row-size=60B cardinality=23
|
| tuple-ids=0N,1N,2 row-size=80B cardinality=23
|
||||||
| in pipelines: 00(GETNEXT)
|
| in pipelines: 00(GETNEXT)
|
||||||
|
|
|
|
||||||
02:HASH JOIN [FULL OUTER JOIN, PARTITIONED]
|
02:HASH JOIN [FULL OUTER JOIN, PARTITIONED]
|
||||||
| hash predicates: target.i = source.id
|
| hash predicates: target.i = source.id
|
||||||
| fk/pk conjuncts: assumed fk/pk
|
| fk/pk conjuncts: assumed fk/pk
|
||||||
| mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
|
| mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
|
||||||
| tuple-ids=0N,1N row-size=60B cardinality=23
|
| tuple-ids=0N,1N row-size=80B cardinality=23
|
||||||
| in pipelines: 00(GETNEXT), 01(OPEN)
|
| in pipelines: 00(GETNEXT), 01(OPEN)
|
||||||
|
|
|
|
||||||
|--05:EXCHANGE [HASH(source.id)]
|
|--05:EXCHANGE [HASH(source.id)]
|
||||||
@@ -98,7 +98,7 @@ WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
|||||||
| F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
|
| F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
|
||||||
| Per-Host Resources: mem-estimate=64.19MB mem-reservation=32.00KB thread-reservation=2
|
| Per-Host Resources: mem-estimate=64.19MB mem-reservation=32.00KB thread-reservation=2
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_partitioned source, RANDOM]
|
| 01:SCAN HDFS [functional_parquet.iceberg_partitioned source, RANDOM]
|
||||||
| HDFS partitions=1/1 files=20 size=22.90KB
|
| HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
| Iceberg snapshot id: 8270633197658268308
|
| Iceberg snapshot id: 8270633197658268308
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=20 size=22.90KB
|
| table: rows=20 size=22.90KB
|
||||||
@@ -110,11 +110,11 @@ WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
|||||||
|
|
|
|
||||||
04:EXCHANGE [HASH(target.i)]
|
04:EXCHANGE [HASH(target.i)]
|
||||||
| mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
|
| mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
|
||||||
| tuple-ids=0 row-size=16B cardinality=3
|
| tuple-ids=0 row-size=36B cardinality=3
|
||||||
| in pipelines: 00(GETNEXT)
|
| in pipelines: 00(GETNEXT)
|
||||||
|
|
|
|
||||||
F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
||||||
Per-Host Resources: mem-estimate=32.08MB mem-reservation=16.00KB thread-reservation=2
|
Per-Host Resources: mem-estimate=64.16MB mem-reservation=32.00KB thread-reservation=2
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_no_deletes target, RANDOM]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_no_deletes target, RANDOM]
|
||||||
HDFS partitions=1/1 files=1 size=625B
|
HDFS partitions=1/1 files=1 size=625B
|
||||||
Iceberg snapshot id: 728158873687794725
|
Iceberg snapshot id: 728158873687794725
|
||||||
@@ -122,8 +122,8 @@ Per-Host Resources: mem-estimate=32.08MB mem-reservation=16.00KB thread-reservat
|
|||||||
table: rows=3 size=625B
|
table: rows=3 size=625B
|
||||||
columns missing stats: i, s
|
columns missing stats: i, s
|
||||||
extrapolated-rows=disabled max-scan-range-rows=3
|
extrapolated-rows=disabled max-scan-range-rows=3
|
||||||
mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=1
|
mem-estimate=64.00MB mem-reservation=32.00KB thread-reservation=1
|
||||||
tuple-ids=0 row-size=16B cardinality=3
|
tuple-ids=0 row-size=36B cardinality=3
|
||||||
in pipelines: 00(GETNEXT)
|
in pipelines: 00(GETNEXT)
|
||||||
====
|
====
|
||||||
# Merge into an unpartitioned Iceberg table with multiple insert cases
|
# Merge into an unpartitioned Iceberg table with multiple insert cases
|
||||||
@@ -134,8 +134,8 @@ when not matched and id < 10 then insert values (cast(source.id + 10 as int) , s
|
|||||||
when not matched and id < 20 then insert values (cast(source.id + 20 as int), source.user)
|
when not matched and id < 20 then insert values (cast(source.id + 20 as int), source.user)
|
||||||
when not matched then insert values (source.id, source.user)
|
when not matched then insert values (source.id, source.user)
|
||||||
---- PLAN
|
---- PLAN
|
||||||
Max Per-Host Resource Reservation: Memory=1.98MB Threads=3
|
Max Per-Host Resource Reservation: Memory=2.00MB Threads=3
|
||||||
Per-Host Resource Estimates: Memory=98MB
|
Per-Host Resource Estimates: Memory=130MB
|
||||||
WARNING: The following tables are missing relevant table and/or column statistics.
|
WARNING: The following tables are missing relevant table and/or column statistics.
|
||||||
functional_parquet.iceberg_partitioned, functional_parquet.iceberg_v2_no_deletes
|
functional_parquet.iceberg_partitioned, functional_parquet.iceberg_v2_no_deletes
|
||||||
Analyzed query: SELECT /* +straight_join */ CAST(CAST(CAST(TupleIsNull(0) AS
|
Analyzed query: SELECT /* +straight_join */ CAST(CAST(CAST(TupleIsNull(0) AS
|
||||||
@@ -146,10 +146,10 @@ functional_parquet.iceberg_v2_no_deletes target FULL OUTER JOIN
|
|||||||
functional_parquet.iceberg_partitioned source ON target.i = source.id
|
functional_parquet.iceberg_partitioned source ON target.i = source.id
|
||||||
|
|
||||||
F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
|
F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
|
||||||
| Per-Host Resources: mem-estimate=97.94MB mem-reservation=1.98MB thread-reservation=3
|
| Per-Host Resources: mem-estimate=129.94MB mem-reservation=2.00MB thread-reservation=3
|
||||||
WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
||||||
| output exprs: target.i, target.s
|
| output exprs: target.i, target.s
|
||||||
| mem-estimate=1.35KB mem-reservation=0B thread-reservation=0
|
| mem-estimate=1.80KB mem-reservation=0B thread-reservation=0
|
||||||
|
|
|
|
||||||
03:MERGE
|
03:MERGE
|
||||||
| CASE 0: NOT MATCHED BY TARGET
|
| CASE 0: NOT MATCHED BY TARGET
|
||||||
@@ -164,18 +164,18 @@ WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
|||||||
| | result expressions: source.id, source.`user`
|
| | result expressions: source.id, source.`user`
|
||||||
| | type: INSERT
|
| | type: INSERT
|
||||||
| mem-estimate=0B mem-reservation=0B thread-reservation=0
|
| mem-estimate=0B mem-reservation=0B thread-reservation=0
|
||||||
| tuple-ids=0N,1N,2 row-size=60B cardinality=23
|
| tuple-ids=0N,1N,2 row-size=80B cardinality=23
|
||||||
| in pipelines: 00(GETNEXT)
|
| in pipelines: 00(GETNEXT)
|
||||||
|
|
|
|
||||||
02:HASH JOIN [FULL OUTER JOIN]
|
02:HASH JOIN [FULL OUTER JOIN]
|
||||||
| hash predicates: target.i = source.id
|
| hash predicates: target.i = source.id
|
||||||
| fk/pk conjuncts: assumed fk/pk
|
| fk/pk conjuncts: assumed fk/pk
|
||||||
| mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
|
| mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
|
||||||
| tuple-ids=0N,1N row-size=60B cardinality=23
|
| tuple-ids=0N,1N row-size=80B cardinality=23
|
||||||
| in pipelines: 00(GETNEXT), 01(OPEN)
|
| in pipelines: 00(GETNEXT), 01(OPEN)
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
|--01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
||||||
| HDFS partitions=1/1 files=20 size=22.90KB
|
| HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
| Iceberg snapshot id: 8270633197658268308
|
| Iceberg snapshot id: 8270633197658268308
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=20 size=22.90KB
|
| table: rows=20 size=22.90KB
|
||||||
@@ -192,12 +192,12 @@ WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
|||||||
table: rows=3 size=625B
|
table: rows=3 size=625B
|
||||||
columns missing stats: i, s
|
columns missing stats: i, s
|
||||||
extrapolated-rows=disabled max-scan-range-rows=3
|
extrapolated-rows=disabled max-scan-range-rows=3
|
||||||
mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=1
|
mem-estimate=64.00MB mem-reservation=32.00KB thread-reservation=1
|
||||||
tuple-ids=0 row-size=16B cardinality=3
|
tuple-ids=0 row-size=36B cardinality=3
|
||||||
in pipelines: 00(GETNEXT)
|
in pipelines: 00(GETNEXT)
|
||||||
---- DISTRIBUTEDPLAN
|
---- DISTRIBUTEDPLAN
|
||||||
Max Per-Host Resource Reservation: Memory=1.98MB Threads=5
|
Max Per-Host Resource Reservation: Memory=2.00MB Threads=5
|
||||||
Per-Host Resource Estimates: Memory=98MB
|
Per-Host Resource Estimates: Memory=130MB
|
||||||
WARNING: The following tables are missing relevant table and/or column statistics.
|
WARNING: The following tables are missing relevant table and/or column statistics.
|
||||||
functional_parquet.iceberg_partitioned, functional_parquet.iceberg_v2_no_deletes
|
functional_parquet.iceberg_partitioned, functional_parquet.iceberg_v2_no_deletes
|
||||||
Analyzed query: SELECT /* +straight_join */ CAST(CAST(CAST(TupleIsNull(0) AS
|
Analyzed query: SELECT /* +straight_join */ CAST(CAST(CAST(TupleIsNull(0) AS
|
||||||
@@ -211,7 +211,7 @@ F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
|||||||
| Per-Host Resources: mem-estimate=1.97MB mem-reservation=1.94MB thread-reservation=1
|
| Per-Host Resources: mem-estimate=1.97MB mem-reservation=1.94MB thread-reservation=1
|
||||||
WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
||||||
| output exprs: target.i, target.s
|
| output exprs: target.i, target.s
|
||||||
| mem-estimate=1.35KB mem-reservation=0B thread-reservation=0
|
| mem-estimate=1.80KB mem-reservation=0B thread-reservation=0
|
||||||
|
|
|
|
||||||
03:MERGE
|
03:MERGE
|
||||||
| CASE 0: NOT MATCHED BY TARGET
|
| CASE 0: NOT MATCHED BY TARGET
|
||||||
@@ -226,14 +226,14 @@ WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
|||||||
| | result expressions: source.id, source.`user`
|
| | result expressions: source.id, source.`user`
|
||||||
| | type: INSERT
|
| | type: INSERT
|
||||||
| mem-estimate=0B mem-reservation=0B thread-reservation=0
|
| mem-estimate=0B mem-reservation=0B thread-reservation=0
|
||||||
| tuple-ids=0N,1N,2 row-size=60B cardinality=23
|
| tuple-ids=0N,1N,2 row-size=80B cardinality=23
|
||||||
| in pipelines: 00(GETNEXT)
|
| in pipelines: 00(GETNEXT)
|
||||||
|
|
|
|
||||||
02:HASH JOIN [FULL OUTER JOIN, PARTITIONED]
|
02:HASH JOIN [FULL OUTER JOIN, PARTITIONED]
|
||||||
| hash predicates: target.i = source.id
|
| hash predicates: target.i = source.id
|
||||||
| fk/pk conjuncts: assumed fk/pk
|
| fk/pk conjuncts: assumed fk/pk
|
||||||
| mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
|
| mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
|
||||||
| tuple-ids=0N,1N row-size=60B cardinality=23
|
| tuple-ids=0N,1N row-size=80B cardinality=23
|
||||||
| in pipelines: 00(GETNEXT), 01(OPEN)
|
| in pipelines: 00(GETNEXT), 01(OPEN)
|
||||||
|
|
|
|
||||||
|--05:EXCHANGE [HASH(source.id)]
|
|--05:EXCHANGE [HASH(source.id)]
|
||||||
@@ -244,7 +244,7 @@ WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
|||||||
| F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
|
| F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
|
||||||
| Per-Host Resources: mem-estimate=64.19MB mem-reservation=32.00KB thread-reservation=2
|
| Per-Host Resources: mem-estimate=64.19MB mem-reservation=32.00KB thread-reservation=2
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_partitioned source, RANDOM]
|
| 01:SCAN HDFS [functional_parquet.iceberg_partitioned source, RANDOM]
|
||||||
| HDFS partitions=1/1 files=20 size=22.90KB
|
| HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
| Iceberg snapshot id: 8270633197658268308
|
| Iceberg snapshot id: 8270633197658268308
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=20 size=22.90KB
|
| table: rows=20 size=22.90KB
|
||||||
@@ -256,11 +256,11 @@ WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
|||||||
|
|
|
|
||||||
04:EXCHANGE [HASH(target.i)]
|
04:EXCHANGE [HASH(target.i)]
|
||||||
| mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
|
| mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
|
||||||
| tuple-ids=0 row-size=16B cardinality=3
|
| tuple-ids=0 row-size=36B cardinality=3
|
||||||
| in pipelines: 00(GETNEXT)
|
| in pipelines: 00(GETNEXT)
|
||||||
|
|
|
|
||||||
F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
||||||
Per-Host Resources: mem-estimate=32.08MB mem-reservation=16.00KB thread-reservation=2
|
Per-Host Resources: mem-estimate=64.16MB mem-reservation=32.00KB thread-reservation=2
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_no_deletes target, RANDOM]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_no_deletes target, RANDOM]
|
||||||
HDFS partitions=1/1 files=1 size=625B
|
HDFS partitions=1/1 files=1 size=625B
|
||||||
Iceberg snapshot id: 728158873687794725
|
Iceberg snapshot id: 728158873687794725
|
||||||
@@ -268,8 +268,8 @@ Per-Host Resources: mem-estimate=32.08MB mem-reservation=16.00KB thread-reservat
|
|||||||
table: rows=3 size=625B
|
table: rows=3 size=625B
|
||||||
columns missing stats: i, s
|
columns missing stats: i, s
|
||||||
extrapolated-rows=disabled max-scan-range-rows=3
|
extrapolated-rows=disabled max-scan-range-rows=3
|
||||||
mem-estimate=32.00MB mem-reservation=16.00KB thread-reservation=1
|
mem-estimate=64.00MB mem-reservation=32.00KB thread-reservation=1
|
||||||
tuple-ids=0 row-size=16B cardinality=3
|
tuple-ids=0 row-size=36B cardinality=3
|
||||||
in pipelines: 00(GETNEXT)
|
in pipelines: 00(GETNEXT)
|
||||||
====
|
====
|
||||||
# Merge into a partitioned Iceberg table with multiple insert cases
|
# Merge into a partitioned Iceberg table with multiple insert cases
|
||||||
@@ -279,8 +279,8 @@ on target.i = source.i
|
|||||||
when not matched and source.i < 20 then insert values (source.ts, source.s, source.i, source.j)
|
when not matched and source.i < 20 then insert values (source.ts, source.s, source.i, source.j)
|
||||||
when not matched then insert *
|
when not matched then insert *
|
||||||
---- PLAN
|
---- PLAN
|
||||||
Max Per-Host Resource Reservation: Memory=13.98MB Threads=3
|
Max Per-Host Resource Reservation: Memory=14.00MB Threads=3
|
||||||
Per-Host Resource Estimates: Memory=162MB
|
Per-Host Resource Estimates: Memory=194MB
|
||||||
WARNING: The following tables are missing relevant table and/or column statistics.
|
WARNING: The following tables are missing relevant table and/or column statistics.
|
||||||
functional_parquet.iceberg_partition_transforms_zorder
|
functional_parquet.iceberg_partition_transforms_zorder
|
||||||
Analyzed query: SELECT /* +straight_join */ CAST(CAST(CAST(TupleIsNull(0) AS
|
Analyzed query: SELECT /* +straight_join */ CAST(CAST(CAST(TupleIsNull(0) AS
|
||||||
@@ -294,16 +294,16 @@ functional_parquet.iceberg_partition_transforms_zorder) source ON target.i =
|
|||||||
source.i
|
source.i
|
||||||
|
|
||||||
F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
|
F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
|
||||||
| Per-Host Resources: mem-estimate=161.94MB mem-reservation=13.98MB thread-reservation=3
|
| Per-Host Resources: mem-estimate=193.94MB mem-reservation=14.00MB thread-reservation=3
|
||||||
WRITE TO HDFS [functional_parquet.iceberg_partition_transforms_zorder, OVERWRITE=false, PARTITION-KEYS=(year(target.ts),iceberg_bucket_transform(target.s, 5))]
|
WRITE TO HDFS [functional_parquet.iceberg_partition_transforms_zorder, OVERWRITE=false, PARTITION-KEYS=(year(target.ts),iceberg_bucket_transform(target.s, 5))]
|
||||||
| output exprs: ts, s, i, j
|
| output exprs: ts, s, i, j
|
||||||
| mem-estimate=122B mem-reservation=0B thread-reservation=0
|
| mem-estimate=162B mem-reservation=0B thread-reservation=0
|
||||||
|
|
|
|
||||||
04:SORT
|
04:SORT
|
||||||
| order by: LEXICAL: year(target.ts) ASC NULLS LAST, iceberg_bucket_transform(target.s, 5) ASC NULLS LAST, ZORDER: i, j
|
| order by: LEXICAL: year(target.ts) ASC NULLS LAST, iceberg_bucket_transform(target.s, 5) ASC NULLS LAST, ZORDER: i, j
|
||||||
| materialized: year(target.ts), iceberg_bucket_transform(target.s, 5)
|
| materialized: year(target.ts), iceberg_bucket_transform(target.s, 5)
|
||||||
| mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
|
| mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
|
||||||
| tuple-ids=6 row-size=61B cardinality=2
|
| tuple-ids=6 row-size=81B cardinality=2
|
||||||
| in pipelines: 04(GETNEXT), 00(OPEN)
|
| in pipelines: 04(GETNEXT), 00(OPEN)
|
||||||
|
|
|
|
||||||
03:MERGE
|
03:MERGE
|
||||||
@@ -315,19 +315,19 @@ WRITE TO HDFS [functional_parquet.iceberg_partition_transforms_zorder, OVERWRITE
|
|||||||
| | result expressions: ts, s, CAST(CAST(i AS BIGINT) + CAST(10 AS BIGINT) AS INT), j
|
| | result expressions: ts, s, CAST(CAST(i AS BIGINT) + CAST(10 AS BIGINT) AS INT), j
|
||||||
| | type: INSERT
|
| | type: INSERT
|
||||||
| mem-estimate=0B mem-reservation=0B thread-reservation=0
|
| mem-estimate=0B mem-reservation=0B thread-reservation=0
|
||||||
| tuple-ids=0N,1N,3 row-size=88B cardinality=2
|
| tuple-ids=0N,1N,3 row-size=108B cardinality=2
|
||||||
| in pipelines: 00(GETNEXT)
|
| in pipelines: 00(GETNEXT)
|
||||||
|
|
|
|
||||||
02:HASH JOIN [FULL OUTER JOIN]
|
02:HASH JOIN [FULL OUTER JOIN]
|
||||||
| hash predicates: target.i = CAST(i + 10 AS INT)
|
| hash predicates: target.i = CAST(i + 10 AS INT)
|
||||||
| fk/pk conjuncts: assumed fk/pk
|
| fk/pk conjuncts: assumed fk/pk
|
||||||
| mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
|
| mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
|
||||||
| tuple-ids=0N,1N row-size=88B cardinality=2
|
| tuple-ids=0N,1N row-size=108B cardinality=2
|
||||||
| in pipelines: 00(GETNEXT), 01(OPEN)
|
| in pipelines: 00(GETNEXT), 01(OPEN)
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_partition_transforms_zorder]
|
|--01:SCAN HDFS [functional_parquet.iceberg_partition_transforms_zorder]
|
||||||
| HDFS partitions=1/1 files=1 size=1.08KB
|
| HDFS partitions=1/1 files=1 size=1.08KB
|
||||||
| Iceberg snapshot id: 7739113163707801067
|
| Iceberg snapshot id: 2888221093496088725
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=1 size=1.08KB
|
| table: rows=1 size=1.08KB
|
||||||
| columns: unavailable
|
| columns: unavailable
|
||||||
@@ -338,12 +338,12 @@ WRITE TO HDFS [functional_parquet.iceberg_partition_transforms_zorder, OVERWRITE
|
|||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partition_transforms_zorder target]
|
00:SCAN HDFS [functional_parquet.iceberg_partition_transforms_zorder target]
|
||||||
HDFS partitions=1/1 files=1 size=1.08KB
|
HDFS partitions=1/1 files=1 size=1.08KB
|
||||||
Iceberg snapshot id: 7739113163707801067
|
Iceberg snapshot id: 2888221093496088725
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=1 size=1.08KB
|
table: rows=1 size=1.08KB
|
||||||
columns missing stats: ts, s, i, j
|
columns missing stats: ts, s, i, j
|
||||||
extrapolated-rows=disabled max-scan-range-rows=1
|
extrapolated-rows=disabled max-scan-range-rows=1
|
||||||
mem-estimate=96.00MB mem-reservation=48.00KB thread-reservation=1
|
mem-estimate=128.00MB mem-reservation=64.00KB thread-reservation=1
|
||||||
tuple-ids=0 row-size=52B cardinality=1
|
tuple-ids=0 row-size=72B cardinality=1
|
||||||
in pipelines: 00(GETNEXT)
|
in pipelines: 00(GETNEXT)
|
||||||
====
|
====
|
||||||
|
|||||||
@@ -18,7 +18,7 @@ BUFFERED DELETE FROM ICEBERG [functional_parquet.iceberg_v2_no_deletes-POSITION-
|
|||||||
| row-size=80B cardinality=3
|
| row-size=80B cardinality=3
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
|--01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
||||||
| HDFS partitions=1/1 files=20 size=22.90KB
|
| HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
| Iceberg snapshot id: 8270633197658268308
|
| Iceberg snapshot id: 8270633197658268308
|
||||||
| row-size=44B cardinality=20
|
| row-size=44B cardinality=20
|
||||||
|
|
|
|
||||||
@@ -44,7 +44,7 @@ BUFFERED DELETE FROM ICEBERG [functional_parquet.iceberg_v2_no_deletes-POSITION-
|
|||||||
|--04:EXCHANGE [BROADCAST]
|
|--04:EXCHANGE [BROADCAST]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
| 01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
||||||
| HDFS partitions=1/1 files=20 size=22.90KB
|
| HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
| Iceberg snapshot id: 8270633197658268308
|
| Iceberg snapshot id: 8270633197658268308
|
||||||
| row-size=44B cardinality=20
|
| row-size=44B cardinality=20
|
||||||
|
|
|
|
||||||
@@ -73,7 +73,7 @@ WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
|||||||
| row-size=80B cardinality=23
|
| row-size=80B cardinality=23
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
|--01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
||||||
| HDFS partitions=1/1 files=20 size=22.90KB
|
| HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
| Iceberg snapshot id: 8270633197658268308
|
| Iceberg snapshot id: 8270633197658268308
|
||||||
| row-size=44B cardinality=20
|
| row-size=44B cardinality=20
|
||||||
|
|
|
|
||||||
@@ -97,7 +97,7 @@ WRITE TO HDFS [functional_parquet.iceberg_v2_no_deletes, OVERWRITE=false]
|
|||||||
|--05:EXCHANGE [HASH(source.id)]
|
|--05:EXCHANGE [HASH(source.id)]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
| 01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
||||||
| HDFS partitions=1/1 files=20 size=22.90KB
|
| HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
| Iceberg snapshot id: 8270633197658268308
|
| Iceberg snapshot id: 8270633197658268308
|
||||||
| row-size=44B cardinality=20
|
| row-size=44B cardinality=20
|
||||||
|
|
|
|
||||||
@@ -130,7 +130,7 @@ MERGE SINK
|
|||||||
| row-size=80B cardinality=3
|
| row-size=80B cardinality=3
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
|--01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
||||||
| HDFS partitions=1/1 files=20 size=22.90KB
|
| HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
| Iceberg snapshot id: 8270633197658268308
|
| Iceberg snapshot id: 8270633197658268308
|
||||||
| row-size=44B cardinality=20
|
| row-size=44B cardinality=20
|
||||||
|
|
|
|
||||||
@@ -158,7 +158,7 @@ MERGE SINK
|
|||||||
|--04:EXCHANGE [BROADCAST]
|
|--04:EXCHANGE [BROADCAST]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
| 01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
||||||
| HDFS partitions=1/1 files=20 size=22.90KB
|
| HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
| Iceberg snapshot id: 8270633197658268308
|
| Iceberg snapshot id: 8270633197658268308
|
||||||
| row-size=44B cardinality=20
|
| row-size=44B cardinality=20
|
||||||
|
|
|
|
||||||
@@ -202,12 +202,12 @@ MERGE SINK
|
|||||||
| row-size=80B cardinality=10
|
| row-size=80B cardinality=10
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
runtime filters: RF000 -> target.id
|
runtime filters: RF000 -> target.id
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=20
|
row-size=80B cardinality=20
|
||||||
@@ -248,12 +248,12 @@ MERGE SINK
|
|||||||
|--06:EXCHANGE [DIRECTED]
|
|--06:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
runtime filters: RF000 -> target.id
|
runtime filters: RF000 -> target.id
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=20
|
row-size=80B cardinality=20
|
||||||
@@ -291,7 +291,7 @@ MERGE SINK
|
|||||||
00:SCAN HDFS [functional_parquet.iceberg_partition_transforms_zorder target]
|
00:SCAN HDFS [functional_parquet.iceberg_partition_transforms_zorder target]
|
||||||
HDFS partitions=1/1 files=1 size=1.08KB
|
HDFS partitions=1/1 files=1 size=1.08KB
|
||||||
runtime filters: RF000 -> target.i
|
runtime filters: RF000 -> target.i
|
||||||
Iceberg snapshot id: 1652763117401970330
|
Iceberg snapshot id: 2888221093496088725
|
||||||
row-size=72B cardinality=1
|
row-size=72B cardinality=1
|
||||||
---- DISTRIBUTEDPLAN
|
---- DISTRIBUTEDPLAN
|
||||||
MERGE SINK
|
MERGE SINK
|
||||||
@@ -323,7 +323,7 @@ MERGE SINK
|
|||||||
00:SCAN HDFS [functional_parquet.iceberg_partition_transforms_zorder target]
|
00:SCAN HDFS [functional_parquet.iceberg_partition_transforms_zorder target]
|
||||||
HDFS partitions=1/1 files=1 size=1.08KB
|
HDFS partitions=1/1 files=1 size=1.08KB
|
||||||
runtime filters: RF000 -> target.i
|
runtime filters: RF000 -> target.i
|
||||||
Iceberg snapshot id: 1652763117401970330
|
Iceberg snapshot id: 2888221093496088725
|
||||||
row-size=72B cardinality=1
|
row-size=72B cardinality=1
|
||||||
====
|
====
|
||||||
# Merge into a partitioned Iceberg table using multiple WHEN MATCHED/NOT MATCHED clauses
|
# Merge into a partitioned Iceberg table using multiple WHEN MATCHED/NOT MATCHED clauses
|
||||||
@@ -379,12 +379,12 @@ MERGE SINK
|
|||||||
| row-size=80B cardinality=10
|
| row-size=80B cardinality=10
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=20
|
row-size=80B cardinality=20
|
||||||
---- DISTRIBUTEDPLAN
|
---- DISTRIBUTEDPLAN
|
||||||
@@ -439,12 +439,12 @@ MERGE SINK
|
|||||||
|--06:EXCHANGE [DIRECTED]
|
|--06:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=20
|
row-size=80B cardinality=20
|
||||||
====
|
====
|
||||||
@@ -486,12 +486,12 @@ MERGE SINK
|
|||||||
| row-size=80B cardinality=10
|
| row-size=80B cardinality=10
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
runtime filters: RF000 -> target.id
|
runtime filters: RF000 -> target.id
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=20
|
row-size=80B cardinality=20
|
||||||
@@ -542,12 +542,12 @@ MERGE SINK
|
|||||||
|--07:EXCHANGE [DIRECTED]
|
|--07:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
runtime filters: RF000 -> target.id
|
runtime filters: RF000 -> target.id
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=20
|
row-size=80B cardinality=20
|
||||||
@@ -581,13 +581,13 @@ MERGE SINK
|
|||||||
| row-size=120B cardinality=14.62K
|
| row-size=120B cardinality=14.62K
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
|--01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
||||||
| HDFS partitions=1/1 files=20 size=22.90KB
|
| HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
| Iceberg snapshot id: 8270633197658268308
|
| Iceberg snapshot id: 8270633197658268308
|
||||||
| row-size=44B cardinality=20
|
| row-size=44B cardinality=20
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partition_evolution target]
|
00:SCAN HDFS [functional_parquet.iceberg_partition_evolution target]
|
||||||
HDFS partitions=1/1 files=1460 size=2.49MB
|
HDFS partitions=1460/1460 files=1460 size=2.49MB
|
||||||
Iceberg snapshot id: 547864005421580562
|
Iceberg snapshot id: 8092015949657061222
|
||||||
row-size=76B cardinality=14.60K
|
row-size=76B cardinality=14.60K
|
||||||
---- DISTRIBUTEDPLAN
|
---- DISTRIBUTEDPLAN
|
||||||
MERGE SINK
|
MERGE SINK
|
||||||
@@ -616,15 +616,15 @@ MERGE SINK
|
|||||||
|--05:EXCHANGE [HASH(source.id)]
|
|--05:EXCHANGE [HASH(source.id)]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
| 01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
||||||
| HDFS partitions=1/1 files=20 size=22.90KB
|
| HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
| Iceberg snapshot id: 8270633197658268308
|
| Iceberg snapshot id: 8270633197658268308
|
||||||
| row-size=44B cardinality=20
|
| row-size=44B cardinality=20
|
||||||
|
|
|
|
||||||
04:EXCHANGE [HASH(target.id)]
|
04:EXCHANGE [HASH(target.id)]
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partition_evolution target]
|
00:SCAN HDFS [functional_parquet.iceberg_partition_evolution target]
|
||||||
HDFS partitions=1/1 files=1460 size=2.49MB
|
HDFS partitions=1460/1460 files=1460 size=2.49MB
|
||||||
Iceberg snapshot id: 547864005421580562
|
Iceberg snapshot id: 8092015949657061222
|
||||||
row-size=76B cardinality=14.60K
|
row-size=76B cardinality=14.60K
|
||||||
====
|
====
|
||||||
# Merge into a partitioned Iceberg table using multiple merge cases and a using table with partition evolution as source
|
# Merge into a partitioned Iceberg table using multiple merge cases and a using table with partition evolution as source
|
||||||
@@ -656,20 +656,20 @@ MERGE SINK
|
|||||||
| row-size=120B cardinality=14.61K
|
| row-size=120B cardinality=14.61K
|
||||||
|
|
|
|
||||||
|--03:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
|--03:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
||||||
| HDFS partitions=1/1 files=1460 size=2.49MB
|
| HDFS partitions=1460/1460 files=1460 size=2.49MB
|
||||||
| Iceberg snapshot id: 547864005421580562
|
| Iceberg snapshot id: 8092015949657061222
|
||||||
| row-size=40B cardinality=14.60K
|
| row-size=40B cardinality=14.60K
|
||||||
|
|
|
|
||||||
02:DELETE EVENTS ICEBERG DELETE [ICEBERG DELETE JOIN]
|
02:DELETE EVENTS ICEBERG DELETE [ICEBERG DELETE JOIN]
|
||||||
| row-size=80B cardinality=10
|
| row-size=80B cardinality=10
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=20
|
row-size=80B cardinality=20
|
||||||
---- DISTRIBUTEDPLAN
|
---- DISTRIBUTEDPLAN
|
||||||
@@ -699,8 +699,8 @@ MERGE SINK
|
|||||||
|--08:EXCHANGE [HASH(functional_parquet.iceberg_partition_evolution.id)]
|
|--08:EXCHANGE [HASH(functional_parquet.iceberg_partition_evolution.id)]
|
||||||
| |
|
| |
|
||||||
| 03:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
| 03:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
||||||
| HDFS partitions=1/1 files=1460 size=2.49MB
|
| HDFS partitions=1460/1460 files=1460 size=2.49MB
|
||||||
| Iceberg snapshot id: 547864005421580562
|
| Iceberg snapshot id: 8092015949657061222
|
||||||
| row-size=40B cardinality=14.60K
|
| row-size=40B cardinality=14.60K
|
||||||
|
|
|
|
||||||
07:EXCHANGE [HASH(target.id)]
|
07:EXCHANGE [HASH(target.id)]
|
||||||
@@ -711,12 +711,12 @@ MERGE SINK
|
|||||||
|--06:EXCHANGE [DIRECTED]
|
|--06:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=20
|
row-size=80B cardinality=20
|
||||||
====
|
====
|
||||||
@@ -759,20 +759,20 @@ MERGE SINK
|
|||||||
| row-size=120B cardinality=14.61K
|
| row-size=120B cardinality=14.61K
|
||||||
|
|
|
|
||||||
|--03:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
|--03:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
||||||
| HDFS partitions=1/1 files=1460 size=2.49MB
|
| HDFS partitions=1460/1460 files=1460 size=2.49MB
|
||||||
| Iceberg snapshot id: 547864005421580562
|
| Iceberg snapshot id: 8092015949657061222
|
||||||
| row-size=40B cardinality=14.60K
|
| row-size=40B cardinality=14.60K
|
||||||
|
|
|
|
||||||
02:DELETE EVENTS ICEBERG DELETE [ICEBERG DELETE JOIN]
|
02:DELETE EVENTS ICEBERG DELETE [ICEBERG DELETE JOIN]
|
||||||
| row-size=80B cardinality=10
|
| row-size=80B cardinality=10
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=20
|
row-size=80B cardinality=20
|
||||||
---- DISTRIBUTEDPLAN
|
---- DISTRIBUTEDPLAN
|
||||||
@@ -810,8 +810,8 @@ MERGE SINK
|
|||||||
|--08:EXCHANGE [HASH(functional_parquet.iceberg_partition_evolution.id)]
|
|--08:EXCHANGE [HASH(functional_parquet.iceberg_partition_evolution.id)]
|
||||||
| |
|
| |
|
||||||
| 03:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
| 03:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
||||||
| HDFS partitions=1/1 files=1460 size=2.49MB
|
| HDFS partitions=1460/1460 files=1460 size=2.49MB
|
||||||
| Iceberg snapshot id: 547864005421580562
|
| Iceberg snapshot id: 8092015949657061222
|
||||||
| row-size=40B cardinality=14.60K
|
| row-size=40B cardinality=14.60K
|
||||||
|
|
|
|
||||||
07:EXCHANGE [HASH(target.id)]
|
07:EXCHANGE [HASH(target.id)]
|
||||||
@@ -822,12 +822,12 @@ MERGE SINK
|
|||||||
|--06:EXCHANGE [DIRECTED]
|
|--06:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=20
|
row-size=80B cardinality=20
|
||||||
====
|
====
|
||||||
@@ -906,12 +906,12 @@ MERGE SINK
|
|||||||
| row-size=80B cardinality=10
|
| row-size=80B cardinality=10
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=20
|
row-size=80B cardinality=20
|
||||||
---- DISTRIBUTEDPLAN
|
---- DISTRIBUTEDPLAN
|
||||||
@@ -984,7 +984,7 @@ MERGE SINK
|
|||||||
| | | group by: id, NULL, NULL, b.tinyint_col, b.string_col
|
| | | group by: id, NULL, NULL, b.tinyint_col, b.string_col
|
||||||
| | | Class 3
|
| | | Class 3
|
||||||
| | | group by: NULL, NULL, NULL, b.tinyint_col, b.string_col
|
| | | group by: NULL, NULL, NULL, b.tinyint_col, b.string_col
|
||||||
| | | row-size=92B cardinality=28
|
| | | row-size=92B cardinality=32
|
||||||
| | |
|
| | |
|
||||||
| | 04:SCAN HDFS [functional.alltypestiny b]
|
| | 04:SCAN HDFS [functional.alltypestiny b]
|
||||||
| | HDFS partitions=4/4 files=4 size=460B
|
| | HDFS partitions=4/4 files=4 size=460B
|
||||||
@@ -1004,12 +1004,12 @@ MERGE SINK
|
|||||||
|--11:EXCHANGE [DIRECTED]
|
|--11:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=20
|
row-size=80B cardinality=20
|
||||||
====
|
====
|
||||||
|
|||||||
@@ -7,8 +7,8 @@ Per-Host Resources: mem-estimate=36.00MB mem-reservation=4.01MB thread-reservati
|
|||||||
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
|
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part]
|
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part]
|
||||||
HDFS partitions=1/1 files=1 size=726B
|
HDFS partitions=1/3 files=1 size=726B
|
||||||
Iceberg snapshot id: 1843610873488300188
|
Iceberg snapshot id: 8307075859906250293
|
||||||
skipped Iceberg predicates: int_col = CAST(1 AS INT)
|
skipped Iceberg predicates: int_col = CAST(1 AS INT)
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=3 size=2.37KB
|
table: rows=3 size=2.37KB
|
||||||
@@ -35,8 +35,8 @@ Per-Host Resources: mem-estimate=32.11MB mem-reservation=8.00KB thread-reservati
|
|||||||
DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
|
DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
|
||||||
| mem-estimate=116.00KB mem-reservation=0B thread-reservation=0
|
| mem-estimate=116.00KB mem-reservation=0B thread-reservation=0
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part, RANDOM]
|
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part, RANDOM]
|
||||||
HDFS partitions=1/1 files=1 size=726B
|
HDFS partitions=1/3 files=1 size=726B
|
||||||
Iceberg snapshot id: 1843610873488300188
|
Iceberg snapshot id: 8307075859906250293
|
||||||
skipped Iceberg predicates: int_col = CAST(1 AS INT)
|
skipped Iceberg predicates: int_col = CAST(1 AS INT)
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=3 size=2.37KB
|
table: rows=3 size=2.37KB
|
||||||
@@ -56,8 +56,8 @@ Per-Host Resources: mem-estimate=132.00MB mem-reservation=4.08MB thread-reservat
|
|||||||
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
|
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part]
|
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part]
|
||||||
HDFS partitions=1/1 files=1 size=542B
|
HDFS partitions=1/3 files=1 size=542B
|
||||||
Iceberg snapshot id: 1843610873488300188
|
Iceberg snapshot id: 8307075859906250293
|
||||||
skipped Iceberg predicates: int_col = CAST(2 AS INT)
|
skipped Iceberg predicates: int_col = CAST(2 AS INT)
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=3 size=2.37KB
|
table: rows=3 size=2.37KB
|
||||||
@@ -84,8 +84,8 @@ Per-Host Resources: mem-estimate=128.11MB mem-reservation=80.00KB thread-reserva
|
|||||||
DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
|
DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
|
||||||
| mem-estimate=116.00KB mem-reservation=0B thread-reservation=0
|
| mem-estimate=116.00KB mem-reservation=0B thread-reservation=0
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part, RANDOM]
|
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part, RANDOM]
|
||||||
HDFS partitions=1/1 files=1 size=542B
|
HDFS partitions=1/3 files=1 size=542B
|
||||||
Iceberg snapshot id: 1843610873488300188
|
Iceberg snapshot id: 8307075859906250293
|
||||||
skipped Iceberg predicates: int_col = CAST(2 AS INT)
|
skipped Iceberg predicates: int_col = CAST(2 AS INT)
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=3 size=2.37KB
|
table: rows=3 size=2.37KB
|
||||||
@@ -105,8 +105,8 @@ Per-Host Resources: mem-estimate=68.00MB mem-reservation=4.03MB thread-reservati
|
|||||||
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
|
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part]
|
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part]
|
||||||
HDFS partitions=1/1 files=1 size=1.14KB
|
HDFS partitions=1/3 files=1 size=1.14KB
|
||||||
Iceberg snapshot id: 1843610873488300188
|
Iceberg snapshot id: 8307075859906250293
|
||||||
skipped Iceberg predicates: int_col = CAST(3 AS INT)
|
skipped Iceberg predicates: int_col = CAST(3 AS INT)
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=3 size=2.37KB
|
table: rows=3 size=2.37KB
|
||||||
@@ -133,8 +133,8 @@ Per-Host Resources: mem-estimate=64.11MB mem-reservation=32.00KB thread-reservat
|
|||||||
DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
|
DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
|
||||||
| mem-estimate=116.00KB mem-reservation=0B thread-reservation=0
|
| mem-estimate=116.00KB mem-reservation=0B thread-reservation=0
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part, RANDOM]
|
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part, RANDOM]
|
||||||
HDFS partitions=1/1 files=1 size=1.14KB
|
HDFS partitions=1/3 files=1 size=1.14KB
|
||||||
Iceberg snapshot id: 1843610873488300188
|
Iceberg snapshot id: 8307075859906250293
|
||||||
skipped Iceberg predicates: int_col = CAST(3 AS INT)
|
skipped Iceberg predicates: int_col = CAST(3 AS INT)
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=3 size=2.37KB
|
table: rows=3 size=2.37KB
|
||||||
@@ -154,8 +154,8 @@ Per-Host Resources: mem-estimate=68.00MB mem-reservation=4.03MB thread-reservati
|
|||||||
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
|
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part]
|
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part]
|
||||||
HDFS partitions=1/1 files=2 size=1.84KB
|
HDFS partitions=2/3 files=2 size=1.84KB
|
||||||
Iceberg snapshot id: 1843610873488300188
|
Iceberg snapshot id: 8307075859906250293
|
||||||
skipped Iceberg predicates: int_col IN (CAST(1 AS INT), CAST(3 AS INT))
|
skipped Iceberg predicates: int_col IN (CAST(1 AS INT), CAST(3 AS INT))
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=3 size=2.37KB
|
table: rows=3 size=2.37KB
|
||||||
@@ -182,8 +182,8 @@ Per-Host Resources: mem-estimate=64.11MB mem-reservation=32.00KB thread-reservat
|
|||||||
DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
|
DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
|
||||||
| mem-estimate=116.00KB mem-reservation=0B thread-reservation=0
|
| mem-estimate=116.00KB mem-reservation=0B thread-reservation=0
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part, RANDOM]
|
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part, RANDOM]
|
||||||
HDFS partitions=1/1 files=2 size=1.84KB
|
HDFS partitions=2/3 files=2 size=1.84KB
|
||||||
Iceberg snapshot id: 1843610873488300188
|
Iceberg snapshot id: 8307075859906250293
|
||||||
skipped Iceberg predicates: int_col IN (CAST(1 AS INT), CAST(3 AS INT))
|
skipped Iceberg predicates: int_col IN (CAST(1 AS INT), CAST(3 AS INT))
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=3 size=2.37KB
|
table: rows=3 size=2.37KB
|
||||||
@@ -203,8 +203,8 @@ Per-Host Resources: mem-estimate=132.00MB mem-reservation=4.08MB thread-reservat
|
|||||||
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
|
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part]
|
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part]
|
||||||
HDFS partitions=1/1 files=2 size=1.67KB
|
HDFS partitions=2/3 files=2 size=1.67KB
|
||||||
Iceberg snapshot id: 1843610873488300188
|
Iceberg snapshot id: 8307075859906250293
|
||||||
skipped Iceberg predicates: int_col IN (CAST(2 AS INT), CAST(3 AS INT))
|
skipped Iceberg predicates: int_col IN (CAST(2 AS INT), CAST(3 AS INT))
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=3 size=2.37KB
|
table: rows=3 size=2.37KB
|
||||||
@@ -231,8 +231,8 @@ Per-Host Resources: mem-estimate=128.11MB mem-reservation=80.00KB thread-reserva
|
|||||||
DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
|
DATASTREAM SINK [FRAGMENT=F01, EXCHANGE=01, UNPARTITIONED]
|
||||||
| mem-estimate=116.00KB mem-reservation=0B thread-reservation=0
|
| mem-estimate=116.00KB mem-reservation=0B thread-reservation=0
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part, RANDOM]
|
00:SCAN HDFS [functional_parquet.iceberg_mixed_file_format_part, RANDOM]
|
||||||
HDFS partitions=1/1 files=2 size=1.67KB
|
HDFS partitions=2/3 files=2 size=1.67KB
|
||||||
Iceberg snapshot id: 1843610873488300188
|
Iceberg snapshot id: 8307075859906250293
|
||||||
skipped Iceberg predicates: int_col IN (CAST(2 AS INT), CAST(3 AS INT))
|
skipped Iceberg predicates: int_col IN (CAST(2 AS INT), CAST(3 AS INT))
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=3 size=2.37KB
|
table: rows=3 size=2.37KB
|
||||||
|
|||||||
@@ -71,7 +71,7 @@ WRITE TO HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes, OVERW
|
|||||||
| in pipelines: 00(GETNEXT), 01(OPEN)
|
| in pipelines: 00(GETNEXT), 01(OPEN)
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=10 size=9.47KB
|
| table: rows=10 size=9.47KB
|
||||||
@@ -82,7 +82,7 @@ WRITE TO HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes, OVERW
|
|||||||
| in pipelines: 01(GETNEXT)
|
| in pipelines: 01(GETNEXT)
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=20 size=12.95KB
|
table: rows=20 size=12.95KB
|
||||||
@@ -125,7 +125,7 @@ Per-Host Resources: mem-estimate=96.53MB mem-reservation=48.22KB thread-reservat
|
|||||||
| F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
|
| F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
|
||||||
| Per-Host Resources: mem-estimate=34.44MB mem-reservation=16.00KB thread-reservation=2
|
| Per-Host Resources: mem-estimate=34.44MB mem-reservation=16.00KB thread-reservation=2
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete, RANDOM]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete, RANDOM]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=10 size=9.47KB
|
| table: rows=10 size=9.47KB
|
||||||
@@ -136,7 +136,7 @@ Per-Host Resources: mem-estimate=96.53MB mem-reservation=48.22KB thread-reservat
|
|||||||
| in pipelines: 01(GETNEXT)
|
| in pipelines: 01(GETNEXT)
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes, RANDOM]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes, RANDOM]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=20 size=12.95KB
|
table: rows=20 size=12.95KB
|
||||||
|
|||||||
@@ -6,7 +6,7 @@ select user from iceberg_partitioned where action = "download" and user = "Lisa"
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=6 size=6.97KB
|
HDFS partitions=1/3 files=6 size=6.97KB
|
||||||
predicates: `user` = 'Lisa', action = 'download'
|
predicates: `user` = 'Lisa', action = 'download'
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
row-size=24B cardinality=1
|
row-size=24B cardinality=1
|
||||||
@@ -18,7 +18,7 @@ select * from iceberg_partitioned where action = "download";
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=6 size=6.97KB
|
HDFS partitions=1/3 files=6 size=6.97KB
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
row-size=44B cardinality=6
|
row-size=44B cardinality=6
|
||||||
@@ -28,7 +28,7 @@ select * from iceberg_partitioned where action = "download" and event_time < "20
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=6 size=6.97KB
|
HDFS partitions=1/3 files=6 size=6.97KB
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: action = 'download', event_time < TIMESTAMP '2022-01-01 00:00:00'
|
skipped Iceberg predicates: action = 'download', event_time < TIMESTAMP '2022-01-01 00:00:00'
|
||||||
row-size=44B cardinality=1
|
row-size=44B cardinality=1
|
||||||
|
|||||||
@@ -6,7 +6,7 @@ SELECT id, int_col, string_col from iceberg_partition_evolution where year = 201
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
00:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
||||||
HDFS partitions=1/1 files=730 size=1.25MB
|
HDFS partitions=730/1460 files=730 size=1.25MB
|
||||||
Iceberg snapshot id: 188637238055891670
|
Iceberg snapshot id: 188637238055891670
|
||||||
skipped Iceberg predicates: `year` = 2010
|
skipped Iceberg predicates: `year` = 2010
|
||||||
row-size=20B cardinality=7.30K
|
row-size=20B cardinality=7.30K
|
||||||
@@ -18,7 +18,7 @@ SELECT id, int_col, string_col from iceberg_partition_evolution where month = 1;
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
00:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
||||||
HDFS partitions=1/1 files=124 size=216.63KB
|
HDFS partitions=124/1460 files=124 size=216.63KB
|
||||||
predicates: `month` = 1
|
predicates: `month` = 1
|
||||||
Iceberg snapshot id: 188637238055891670
|
Iceberg snapshot id: 188637238055891670
|
||||||
row-size=24B cardinality=1.24K
|
row-size=24B cardinality=1.24K
|
||||||
@@ -31,7 +31,7 @@ SELECT id, int_col, string_col from iceberg_partition_evolution where year = 201
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
00:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
||||||
HDFS partitions=1/1 files=730 size=1.25MB
|
HDFS partitions=730/1460 files=730 size=1.25MB
|
||||||
predicates: power(id, 3) > 1000
|
predicates: power(id, 3) > 1000
|
||||||
Iceberg snapshot id: 188637238055891670
|
Iceberg snapshot id: 188637238055891670
|
||||||
skipped Iceberg predicates: `year` = 2010
|
skipped Iceberg predicates: `year` = 2010
|
||||||
@@ -44,7 +44,7 @@ SELECT id, int_col, string_col from iceberg_partition_evolution where year = 201
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
00:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
||||||
HDFS partitions=1/1 files=730 size=1.25MB
|
HDFS partitions=730/1460 files=730 size=1.25MB
|
||||||
predicates: id > 1000
|
predicates: id > 1000
|
||||||
Iceberg snapshot id: 188637238055891670
|
Iceberg snapshot id: 188637238055891670
|
||||||
skipped Iceberg predicates: `year` = 2010
|
skipped Iceberg predicates: `year` = 2010
|
||||||
@@ -59,7 +59,7 @@ WHERE year = 2010 AND date_string_col='061610';
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
00:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
||||||
HDFS partitions=1/1 files=2 size=3.49KB
|
HDFS partitions=2/1460 files=2 size=3.49KB
|
||||||
predicates: date_string_col = '061610'
|
predicates: date_string_col = '061610'
|
||||||
Iceberg snapshot id: 188637238055891670
|
Iceberg snapshot id: 188637238055891670
|
||||||
skipped Iceberg predicates: `year` = 2010
|
skipped Iceberg predicates: `year` = 2010
|
||||||
@@ -76,7 +76,7 @@ PLAN-ROOT SINK
|
|||||||
| row-size=8B cardinality=1
|
| row-size=8B cardinality=1
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=6 size=6.85KB
|
HDFS partitions=1/3 files=6 size=6.85KB
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: action = 'click'
|
skipped Iceberg predicates: action = 'click'
|
||||||
row-size=8B cardinality=6
|
row-size=8B cardinality=6
|
||||||
@@ -89,7 +89,7 @@ select user from iceberg_partitioned where action = 'click' and user = null
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=6 size=6.85KB
|
HDFS partitions=1/3 files=6 size=6.85KB
|
||||||
predicates: `user` = NULL
|
predicates: `user` = NULL
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: action = 'click'
|
skipped Iceberg predicates: action = 'click'
|
||||||
@@ -100,7 +100,7 @@ PLAN-ROOT SINK
|
|||||||
01:EXCHANGE [UNPARTITIONED]
|
01:EXCHANGE [UNPARTITIONED]
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=6 size=6.85KB
|
HDFS partitions=1/3 files=6 size=6.85KB
|
||||||
predicates: `user` = NULL
|
predicates: `user` = NULL
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: action = 'click'
|
skipped Iceberg predicates: action = 'click'
|
||||||
@@ -112,7 +112,7 @@ select * from iceberg_partitioned where action = NULL and event_time < "2022-01-
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=20 size=22.90KB
|
HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
predicates: action = NULL
|
predicates: action = NULL
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: event_time < TIMESTAMP '2022-01-01 00:00:00'
|
skipped Iceberg predicates: event_time < TIMESTAMP '2022-01-01 00:00:00'
|
||||||
@@ -125,7 +125,7 @@ select * from iceberg_partitioned where action like "d%" and event_time < "2022-
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=9 size=10.33KB
|
HDFS partitions=3/3 files=9 size=10.33KB
|
||||||
predicates: id < 10, action LIKE 'd%'
|
predicates: id < 10, action LIKE 'd%'
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: event_time < TIMESTAMP '2022-01-01 00:00:00'
|
skipped Iceberg predicates: event_time < TIMESTAMP '2022-01-01 00:00:00'
|
||||||
@@ -137,7 +137,7 @@ select * from iceberg_partitioned where action like "d%" and (event_time < "2020
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=10 size=11.42KB
|
HDFS partitions=3/3 files=10 size=11.42KB
|
||||||
predicates: (event_time < TIMESTAMP '2020-01-01 00:00:00' OR id > 10), action LIKE 'd%'
|
predicates: (event_time < TIMESTAMP '2020-01-01 00:00:00' OR id > 10), action LIKE 'd%'
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
row-size=44B cardinality=1
|
row-size=44B cardinality=1
|
||||||
@@ -149,7 +149,7 @@ select * from iceberg_partition_evolution where month = 12 and year = 2010
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
00:SCAN HDFS [functional_parquet.iceberg_partition_evolution]
|
||||||
HDFS partitions=1/1 files=62 size=108.30KB
|
HDFS partitions=62/1460 files=62 size=108.30KB
|
||||||
predicates: `month` = 12
|
predicates: `month` = 12
|
||||||
Iceberg snapshot id: 188637238055891670
|
Iceberg snapshot id: 188637238055891670
|
||||||
skipped Iceberg predicates: `year` = 2010
|
skipped Iceberg predicates: `year` = 2010
|
||||||
@@ -163,7 +163,7 @@ select * from iceberg_partitioned where action in ('click', 'view') and (id > 5
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=12 size=13.65KB
|
HDFS partitions=2/3 files=12 size=13.65KB
|
||||||
predicates: action IN ('click', 'view'), (id > 5 OR (id < 2))
|
predicates: action IN ('click', 'view'), (id > 5 OR (id < 2))
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
row-size=44B cardinality=1
|
row-size=44B cardinality=1
|
||||||
@@ -175,7 +175,7 @@ select * from iceberg_partitioned where action is not null and id < 10
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=9 size=10.33KB
|
HDFS partitions=3/3 files=9 size=10.33KB
|
||||||
predicates: id < 10
|
predicates: id < 10
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: action IS NOT NULL
|
skipped Iceberg predicates: action IS NOT NULL
|
||||||
|
|||||||
@@ -187,12 +187,12 @@ BUFFERED DELETE FROM ICEBERG [functional_parquet.iceberg_v2_partitioned_position
|
|||||||
| row-size=40B cardinality=1
|
| row-size=40B cardinality=1
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
predicates: id = 20
|
predicates: id = 20
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=40B cardinality=2
|
row-size=40B cardinality=2
|
||||||
@@ -207,12 +207,12 @@ BUFFERED DELETE FROM ICEBERG [functional_parquet.iceberg_v2_partitioned_position
|
|||||||
|--03:EXCHANGE [DIRECTED]
|
|--03:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
predicates: id = 20
|
predicates: id = 20
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=40B cardinality=2
|
row-size=40B cardinality=2
|
||||||
@@ -225,12 +225,12 @@ BUFFERED DELETE FROM ICEBERG [functional_parquet.iceberg_v2_partitioned_position
|
|||||||
| row-size=36B cardinality=3
|
| row-size=36B cardinality=3
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.15KB
|
| HDFS partitions=1/3 files=1 size=3.15KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=3
|
| row-size=204B cardinality=3
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.15KB
|
HDFS partitions=1/3 files=1 size=1.15KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'click'
|
skipped Iceberg predicates: action = 'click'
|
||||||
row-size=36B cardinality=6
|
row-size=36B cardinality=6
|
||||||
@@ -243,12 +243,12 @@ BUFFERED DELETE FROM ICEBERG [functional_parquet.iceberg_v2_partitioned_position
|
|||||||
|--03:EXCHANGE [DIRECTED]
|
|--03:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.15KB
|
| HDFS partitions=1/3 files=1 size=3.15KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=3
|
| row-size=204B cardinality=3
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.15KB
|
HDFS partitions=1/3 files=1 size=1.15KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'click'
|
skipped Iceberg predicates: action = 'click'
|
||||||
row-size=36B cardinality=6
|
row-size=36B cardinality=6
|
||||||
@@ -261,12 +261,12 @@ BUFFERED DELETE FROM ICEBERG [functional_parquet.iceberg_v2_partitioned_position
|
|||||||
| row-size=48B cardinality=1
|
| row-size=48B cardinality=1
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
predicates: `user` LIKE 'A%'
|
predicates: `user` LIKE 'A%'
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=48B cardinality=2
|
row-size=48B cardinality=2
|
||||||
@@ -281,12 +281,12 @@ BUFFERED DELETE FROM ICEBERG [functional_parquet.iceberg_v2_partitioned_position
|
|||||||
|--03:EXCHANGE [DIRECTED]
|
|--03:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
predicates: `user` LIKE 'A%'
|
predicates: `user` LIKE 'A%'
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=48B cardinality=2
|
row-size=48B cardinality=2
|
||||||
@@ -322,12 +322,12 @@ BUFFERED DELETE FROM ICEBERG [functional_parquet.iceberg_v2_partitioned_position
|
|||||||
| row-size=40B cardinality=10
|
| row-size=40B cardinality=10
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
runtime filters: RF000 -> id
|
runtime filters: RF000 -> id
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=40B cardinality=20
|
row-size=40B cardinality=20
|
||||||
@@ -374,12 +374,12 @@ BUFFERED DELETE FROM ICEBERG [functional_parquet.iceberg_v2_partitioned_position
|
|||||||
|--08:EXCHANGE [DIRECTED]
|
|--08:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
runtime filters: RF000 -> id
|
runtime filters: RF000 -> id
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=40B cardinality=20
|
row-size=40B cardinality=20
|
||||||
@@ -392,12 +392,12 @@ BUFFERED DELETE FROM ICEBERG [functional_parquet.iceberg_v2_partitioned_position
|
|||||||
| row-size=40B cardinality=1
|
| row-size=40B cardinality=1
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
predicates: FILE__POSITION = id
|
predicates: FILE__POSITION = id
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=40B cardinality=2
|
row-size=40B cardinality=2
|
||||||
@@ -412,12 +412,12 @@ BUFFERED DELETE FROM ICEBERG [functional_parquet.iceberg_v2_partitioned_position
|
|||||||
|--03:EXCHANGE [DIRECTED]
|
|--03:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
predicates: FILE__POSITION = id
|
predicates: FILE__POSITION = id
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=40B cardinality=2
|
row-size=40B cardinality=2
|
||||||
|
|||||||
@@ -365,12 +365,12 @@ PLAN-ROOT SINK
|
|||||||
| row-size=20B cardinality=3
|
| row-size=20B cardinality=3
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=2.63KB
|
| HDFS partitions=1/unknown files=1 size=2.63KB
|
||||||
| Iceberg snapshot id: 752781918366351945
|
| Iceberg snapshot id: 752781918366351945
|
||||||
| row-size=267B cardinality=1
|
| row-size=267B cardinality=1
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
||||||
HDFS partitions=1/1 files=1 size=625B
|
HDFS partitions=1/unknown files=1 size=625B
|
||||||
Iceberg snapshot id: 752781918366351945
|
Iceberg snapshot id: 752781918366351945
|
||||||
row-size=20B cardinality=3
|
row-size=20B cardinality=3
|
||||||
---- DISTRIBUTEDPLAN
|
---- DISTRIBUTEDPLAN
|
||||||
@@ -392,12 +392,12 @@ PLAN-ROOT SINK
|
|||||||
|--04:EXCHANGE [BROADCAST]
|
|--04:EXCHANGE [BROADCAST]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=2.63KB
|
| HDFS partitions=1/unknown files=1 size=2.63KB
|
||||||
| Iceberg snapshot id: 752781918366351945
|
| Iceberg snapshot id: 752781918366351945
|
||||||
| row-size=267B cardinality=1
|
| row-size=267B cardinality=1
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
||||||
HDFS partitions=1/1 files=1 size=625B
|
HDFS partitions=1/unknown files=1 size=625B
|
||||||
Iceberg snapshot id: 752781918366351945
|
Iceberg snapshot id: 752781918366351945
|
||||||
row-size=20B cardinality=3
|
row-size=20B cardinality=3
|
||||||
====
|
====
|
||||||
@@ -513,12 +513,12 @@ PLAN-ROOT SINK
|
|||||||
| row-size=64B cardinality=20
|
| row-size=64B cardinality=20
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=64B cardinality=20
|
row-size=64B cardinality=20
|
||||||
---- DISTRIBUTEDPLAN
|
---- DISTRIBUTEDPLAN
|
||||||
@@ -532,14 +532,14 @@ PLAN-ROOT SINK
|
|||||||
|--04:EXCHANGE [HASH(functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete.pos,functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete.file_path)]
|
|--04:EXCHANGE [HASH(functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete.pos,functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete.file_path)]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
03:EXCHANGE [HASH(functional_parquet.iceberg_v2_partitioned_position_deletes.file__position,functional_parquet.iceberg_v2_partitioned_position_deletes.input__file__name)]
|
03:EXCHANGE [HASH(functional_parquet.iceberg_v2_partitioned_position_deletes.file__position,functional_parquet.iceberg_v2_partitioned_position_deletes.input__file__name)]
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=64B cardinality=20
|
row-size=64B cardinality=20
|
||||||
====
|
====
|
||||||
@@ -615,7 +615,7 @@ PLAN-ROOT SINK
|
|||||||
| row-size=16B cardinality=10
|
| row-size=16B cardinality=10
|
||||||
|
|
|
|
||||||
|--06:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
|--06:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
||||||
| HDFS partitions=1/1 files=1 size=625B
|
| HDFS partitions=1/unknown files=1 size=625B
|
||||||
| Iceberg snapshot id: 4363979609026842966
|
| Iceberg snapshot id: 4363979609026842966
|
||||||
| row-size=16B cardinality=3
|
| row-size=16B cardinality=3
|
||||||
|
|
|
|
||||||
@@ -656,7 +656,7 @@ PLAN-ROOT SINK
|
|||||||
|--12:EXCHANGE [HASH(functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.i,functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.s)]
|
|--12:EXCHANGE [HASH(functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.i,functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.s)]
|
||||||
| |
|
| |
|
||||||
| 06:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
| 06:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
||||||
| HDFS partitions=1/1 files=1 size=625B
|
| HDFS partitions=1/unknown files=1 size=625B
|
||||||
| Iceberg snapshot id: 4363979609026842966
|
| Iceberg snapshot id: 4363979609026842966
|
||||||
| row-size=16B cardinality=3
|
| row-size=16B cardinality=3
|
||||||
|
|
|
|
||||||
@@ -899,12 +899,12 @@ PLAN-ROOT SINK
|
|||||||
| row-size=64B cardinality=1
|
| row-size=64B cardinality=1
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=207B cardinality=2
|
| row-size=207B cardinality=2
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
@@ -920,12 +920,12 @@ PLAN-ROOT SINK
|
|||||||
|--03:EXCHANGE [BROADCAST]
|
|--03:EXCHANGE [BROADCAST]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=207B cardinality=2
|
| row-size=207B cardinality=2
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
@@ -940,12 +940,12 @@ PLAN-ROOT SINK
|
|||||||
| row-size=64B cardinality=1
|
| row-size=64B cardinality=1
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=207B cardinality=2
|
| row-size=207B cardinality=2
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
predicates: `user` = 'Lisa'
|
predicates: `user` = 'Lisa'
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
@@ -961,12 +961,12 @@ PLAN-ROOT SINK
|
|||||||
|--03:EXCHANGE [BROADCAST]
|
|--03:EXCHANGE [BROADCAST]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=207B cardinality=2
|
| row-size=207B cardinality=2
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
predicates: `user` = 'Lisa'
|
predicates: `user` = 'Lisa'
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
@@ -977,7 +977,7 @@ select event_time, action from iceberg_partitioned where (action = 'click' or ac
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=14 size=15.93KB
|
HDFS partitions=2/3 files=14 size=15.93KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: action IN ('click', 'view')
|
skipped Iceberg predicates: action IN ('click', 'view')
|
||||||
@@ -988,7 +988,7 @@ PLAN-ROOT SINK
|
|||||||
01:EXCHANGE [UNPARTITIONED]
|
01:EXCHANGE [UNPARTITIONED]
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=14 size=15.93KB
|
HDFS partitions=2/3 files=14 size=15.93KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: action IN ('click', 'view')
|
skipped Iceberg predicates: action IN ('click', 'view')
|
||||||
@@ -999,7 +999,7 @@ select event_time, action from iceberg_partitioned where action in ('click', 'vi
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=14 size=15.93KB
|
HDFS partitions=2/3 files=14 size=15.93KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: action IN ('click', 'view')
|
skipped Iceberg predicates: action IN ('click', 'view')
|
||||||
@@ -1010,7 +1010,7 @@ PLAN-ROOT SINK
|
|||||||
01:EXCHANGE [UNPARTITIONED]
|
01:EXCHANGE [UNPARTITIONED]
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=14 size=15.93KB
|
HDFS partitions=2/3 files=14 size=15.93KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: action IN ('click', 'view')
|
skipped Iceberg predicates: action IN ('click', 'view')
|
||||||
@@ -1021,7 +1021,7 @@ select event_time, action from iceberg_partitioned where (event_time='2020-01-01
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=6 size=6.85KB
|
HDFS partitions=1/3 files=6 size=6.85KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action = 'click')
|
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action = 'click')
|
||||||
@@ -1032,7 +1032,7 @@ PLAN-ROOT SINK
|
|||||||
01:EXCHANGE [UNPARTITIONED]
|
01:EXCHANGE [UNPARTITIONED]
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=6 size=6.85KB
|
HDFS partitions=1/3 files=6 size=6.85KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action = 'click')
|
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action = 'click')
|
||||||
@@ -1043,7 +1043,7 @@ select event_time, action from iceberg_partitioned where (event_time='2020-01-01
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=14 size=15.93KB
|
HDFS partitions=2/3 files=14 size=15.93KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action = 'click' OR action = 'view')
|
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action = 'click' OR action = 'view')
|
||||||
@@ -1054,7 +1054,7 @@ PLAN-ROOT SINK
|
|||||||
01:EXCHANGE [UNPARTITIONED]
|
01:EXCHANGE [UNPARTITIONED]
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=14 size=15.93KB
|
HDFS partitions=2/3 files=14 size=15.93KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action = 'click' OR action = 'view')
|
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action = 'click' OR action = 'view')
|
||||||
@@ -1065,7 +1065,7 @@ select event_time, action from iceberg_partitioned where (event_time='2020-01-01
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=14 size=15.93KB
|
HDFS partitions=2/3 files=14 size=15.93KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action IN ('click', 'view'))
|
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action IN ('click', 'view'))
|
||||||
@@ -1076,7 +1076,7 @@ PLAN-ROOT SINK
|
|||||||
01:EXCHANGE [UNPARTITIONED]
|
01:EXCHANGE [UNPARTITIONED]
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=14 size=15.93KB
|
HDFS partitions=2/3 files=14 size=15.93KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action IN ('click', 'view'))
|
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action IN ('click', 'view'))
|
||||||
@@ -1087,7 +1087,7 @@ select event_time, action from iceberg_partitioned where (event_time='2020-01-01
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=20 size=22.90KB
|
HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action > 'a')
|
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action > 'a')
|
||||||
@@ -1098,7 +1098,7 @@ PLAN-ROOT SINK
|
|||||||
01:EXCHANGE [UNPARTITIONED]
|
01:EXCHANGE [UNPARTITIONED]
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=20 size=22.90KB
|
HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action > 'a')
|
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action > 'a')
|
||||||
@@ -1110,7 +1110,7 @@ select event_time, action from iceberg_partitioned where event_time='2020-01-01
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=0 size=0B
|
HDFS partitions=0/3 files=0 size=0B
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: event_time = TIMESTAMP '2020-01-01 11:00:00'
|
skipped Iceberg predicates: event_time = TIMESTAMP '2020-01-01 11:00:00'
|
||||||
row-size=28B cardinality=0
|
row-size=28B cardinality=0
|
||||||
@@ -1120,7 +1120,7 @@ PLAN-ROOT SINK
|
|||||||
01:EXCHANGE [UNPARTITIONED]
|
01:EXCHANGE [UNPARTITIONED]
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=0 size=0B
|
HDFS partitions=0/3 files=0 size=0B
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: event_time = TIMESTAMP '2020-01-01 11:00:00'
|
skipped Iceberg predicates: event_time = TIMESTAMP '2020-01-01 11:00:00'
|
||||||
row-size=28B cardinality=0
|
row-size=28B cardinality=0
|
||||||
@@ -1134,12 +1134,12 @@ PLAN-ROOT SINK
|
|||||||
| row-size=64B cardinality=6
|
| row-size=64B cardinality=6
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=207B cardinality=2
|
| row-size=207B cardinality=2
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
row-size=64B cardinality=6
|
row-size=64B cardinality=6
|
||||||
@@ -1154,12 +1154,12 @@ PLAN-ROOT SINK
|
|||||||
|--03:EXCHANGE [BROADCAST]
|
|--03:EXCHANGE [BROADCAST]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=207B cardinality=2
|
| row-size=207B cardinality=2
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
row-size=64B cardinality=6
|
row-size=64B cardinality=6
|
||||||
|
|||||||
@@ -734,7 +734,7 @@ PLAN-ROOT SINK
|
|||||||
| in pipelines: 00(GETNEXT), 01(OPEN)
|
| in pipelines: 00(GETNEXT), 01(OPEN)
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=2.63KB
|
| HDFS partitions=1/unknown files=1 size=2.63KB
|
||||||
| Iceberg snapshot id: 752781918366351945
|
| Iceberg snapshot id: 752781918366351945
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=1 size=2.63KB
|
| table: rows=1 size=2.63KB
|
||||||
@@ -745,7 +745,7 @@ PLAN-ROOT SINK
|
|||||||
| in pipelines: 01(GETNEXT)
|
| in pipelines: 01(GETNEXT)
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
||||||
HDFS partitions=1/1 files=1 size=625B
|
HDFS partitions=1/unknown files=1 size=625B
|
||||||
Iceberg snapshot id: 752781918366351945
|
Iceberg snapshot id: 752781918366351945
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=10 size=7.77KB
|
table: rows=10 size=7.77KB
|
||||||
@@ -794,7 +794,7 @@ Per-Host Resources: mem-estimate=32.02MB mem-reservation=16.76KB thread-reservat
|
|||||||
| F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
| F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
||||||
| Per-Host Resources: mem-estimate=33.06MB mem-reservation=16.00KB thread-reservation=2
|
| Per-Host Resources: mem-estimate=33.06MB mem-reservation=16.00KB thread-reservation=2
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete, RANDOM]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete, RANDOM]
|
||||||
| HDFS partitions=1/1 files=1 size=2.63KB
|
| HDFS partitions=1/unknown files=1 size=2.63KB
|
||||||
| Iceberg snapshot id: 752781918366351945
|
| Iceberg snapshot id: 752781918366351945
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=1 size=2.63KB
|
| table: rows=1 size=2.63KB
|
||||||
@@ -805,7 +805,7 @@ Per-Host Resources: mem-estimate=32.02MB mem-reservation=16.76KB thread-reservat
|
|||||||
| in pipelines: 01(GETNEXT)
|
| in pipelines: 01(GETNEXT)
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files, RANDOM]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files, RANDOM]
|
||||||
HDFS partitions=1/1 files=1 size=625B
|
HDFS partitions=1/unknown files=1 size=625B
|
||||||
Iceberg snapshot id: 752781918366351945
|
Iceberg snapshot id: 752781918366351945
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=10 size=7.77KB
|
table: rows=10 size=7.77KB
|
||||||
@@ -836,7 +836,7 @@ PLAN-ROOT SINK
|
|||||||
| in pipelines: 00(GETNEXT), 01(OPEN)
|
| in pipelines: 00(GETNEXT), 01(OPEN)
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=2.63KB
|
| HDFS partitions=1/unknown files=1 size=2.63KB
|
||||||
| Iceberg snapshot id: 752781918366351945
|
| Iceberg snapshot id: 752781918366351945
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=1 size=2.63KB
|
| table: rows=1 size=2.63KB
|
||||||
@@ -847,7 +847,7 @@ PLAN-ROOT SINK
|
|||||||
| in pipelines: 01(GETNEXT)
|
| in pipelines: 01(GETNEXT)
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
||||||
HDFS partitions=1/1 files=1 size=625B
|
HDFS partitions=1/unknown files=1 size=625B
|
||||||
Iceberg snapshot id: 752781918366351945
|
Iceberg snapshot id: 752781918366351945
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=10 size=7.77KB
|
table: rows=10 size=7.77KB
|
||||||
@@ -896,7 +896,7 @@ Per-Host Resources: mem-estimate=32.02MB mem-reservation=16.76KB thread-reservat
|
|||||||
| F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
| F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
||||||
| Per-Host Resources: mem-estimate=33.06MB mem-reservation=16.00KB thread-reservation=2
|
| Per-Host Resources: mem-estimate=33.06MB mem-reservation=16.00KB thread-reservation=2
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete, RANDOM]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete, RANDOM]
|
||||||
| HDFS partitions=1/1 files=1 size=2.63KB
|
| HDFS partitions=1/unknown files=1 size=2.63KB
|
||||||
| Iceberg snapshot id: 752781918366351945
|
| Iceberg snapshot id: 752781918366351945
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=1 size=2.63KB
|
| table: rows=1 size=2.63KB
|
||||||
@@ -907,7 +907,7 @@ Per-Host Resources: mem-estimate=32.02MB mem-reservation=16.76KB thread-reservat
|
|||||||
| in pipelines: 01(GETNEXT)
|
| in pipelines: 01(GETNEXT)
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files, RANDOM]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files, RANDOM]
|
||||||
HDFS partitions=1/1 files=1 size=625B
|
HDFS partitions=1/unknown files=1 size=625B
|
||||||
Iceberg snapshot id: 752781918366351945
|
Iceberg snapshot id: 752781918366351945
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=10 size=7.77KB
|
table: rows=10 size=7.77KB
|
||||||
@@ -1152,7 +1152,7 @@ PLAN-ROOT SINK
|
|||||||
| in pipelines: 00(GETNEXT), 01(OPEN)
|
| in pipelines: 00(GETNEXT), 01(OPEN)
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=10 size=9.47KB
|
| table: rows=10 size=9.47KB
|
||||||
@@ -1163,7 +1163,7 @@ PLAN-ROOT SINK
|
|||||||
| in pipelines: 01(GETNEXT)
|
| in pipelines: 01(GETNEXT)
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=20 size=12.95KB
|
table: rows=20 size=12.95KB
|
||||||
@@ -1200,7 +1200,7 @@ Per-Host Resources: mem-estimate=96.27MB mem-reservation=48.71KB thread-reservat
|
|||||||
| F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
|
| F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
|
||||||
| Per-Host Resources: mem-estimate=34.44MB mem-reservation=16.00KB thread-reservation=2
|
| Per-Host Resources: mem-estimate=34.44MB mem-reservation=16.00KB thread-reservation=2
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete, RANDOM]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete, RANDOM]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=10 size=9.47KB
|
| table: rows=10 size=9.47KB
|
||||||
@@ -1211,7 +1211,7 @@ Per-Host Resources: mem-estimate=96.27MB mem-reservation=48.71KB thread-reservat
|
|||||||
| in pipelines: 01(GETNEXT)
|
| in pipelines: 01(GETNEXT)
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes, RANDOM]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes, RANDOM]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=20 size=12.95KB
|
table: rows=20 size=12.95KB
|
||||||
@@ -1369,7 +1369,7 @@ PLAN-ROOT SINK
|
|||||||
| in pipelines: 05(GETNEXT), 06(OPEN)
|
| in pipelines: 05(GETNEXT), 06(OPEN)
|
||||||
|
|
|
|
||||||
|--06:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
|--06:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
||||||
| HDFS partitions=1/1 files=1 size=625B
|
| HDFS partitions=1/unknown files=1 size=625B
|
||||||
| Iceberg snapshot id: 4363979609026842966
|
| Iceberg snapshot id: 4363979609026842966
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=10 size=7.77KB
|
| table: rows=10 size=7.77KB
|
||||||
@@ -1457,7 +1457,7 @@ Per-Host Resources: mem-estimate=130.02MB mem-reservation=35.94MB thread-reserva
|
|||||||
| F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
| F05:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
||||||
| Per-Host Resources: mem-estimate=32.16MB mem-reservation=16.00KB thread-reservation=2
|
| Per-Host Resources: mem-estimate=32.16MB mem-reservation=16.00KB thread-reservation=2
|
||||||
| 06:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files, RANDOM]
|
| 06:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files, RANDOM]
|
||||||
| HDFS partitions=1/1 files=1 size=625B
|
| HDFS partitions=1/unknown files=1 size=625B
|
||||||
| Iceberg snapshot id: 4363979609026842966
|
| Iceberg snapshot id: 4363979609026842966
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=10 size=7.77KB
|
| table: rows=10 size=7.77KB
|
||||||
@@ -1947,7 +1947,7 @@ PLAN-ROOT SINK
|
|||||||
| in pipelines: 00(GETNEXT), 01(OPEN)
|
| in pipelines: 00(GETNEXT), 01(OPEN)
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=2 size=3.18KB
|
| table: rows=2 size=3.18KB
|
||||||
@@ -1958,7 +1958,7 @@ PLAN-ROOT SINK
|
|||||||
| in pipelines: 01(GETNEXT)
|
| in pipelines: 01(GETNEXT)
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
predicates: id > CAST(0 AS INT)
|
predicates: id > CAST(0 AS INT)
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
@@ -1999,7 +1999,7 @@ Per-Host Resources: mem-estimate=96.27MB mem-reservation=48.71KB thread-reservat
|
|||||||
| F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
| F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
||||||
| Per-Host Resources: mem-estimate=32.82MB mem-reservation=16.00KB thread-reservation=2
|
| Per-Host Resources: mem-estimate=32.82MB mem-reservation=16.00KB thread-reservation=2
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete, RANDOM]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete, RANDOM]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=2 size=3.18KB
|
| table: rows=2 size=3.18KB
|
||||||
@@ -2010,7 +2010,7 @@ Per-Host Resources: mem-estimate=96.27MB mem-reservation=48.71KB thread-reservat
|
|||||||
| in pipelines: 01(GETNEXT)
|
| in pipelines: 01(GETNEXT)
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes, RANDOM]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes, RANDOM]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
predicates: id > CAST(0 AS INT)
|
predicates: id > CAST(0 AS INT)
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
@@ -2040,7 +2040,7 @@ PLAN-ROOT SINK
|
|||||||
| in pipelines: 00(GETNEXT), 01(OPEN)
|
| in pipelines: 00(GETNEXT), 01(OPEN)
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=2 size=3.18KB
|
| table: rows=2 size=3.18KB
|
||||||
@@ -2051,7 +2051,7 @@ PLAN-ROOT SINK
|
|||||||
| in pipelines: 01(GETNEXT)
|
| in pipelines: 01(GETNEXT)
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
predicates: `user` = 'Lisa'
|
predicates: `user` = 'Lisa'
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
@@ -2092,7 +2092,7 @@ Per-Host Resources: mem-estimate=96.27MB mem-reservation=48.71KB thread-reservat
|
|||||||
| F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
| F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
||||||
| Per-Host Resources: mem-estimate=32.82MB mem-reservation=16.00KB thread-reservation=2
|
| Per-Host Resources: mem-estimate=32.82MB mem-reservation=16.00KB thread-reservation=2
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete, RANDOM]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete, RANDOM]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=2 size=3.18KB
|
| table: rows=2 size=3.18KB
|
||||||
@@ -2103,7 +2103,7 @@ Per-Host Resources: mem-estimate=96.27MB mem-reservation=48.71KB thread-reservat
|
|||||||
| in pipelines: 01(GETNEXT)
|
| in pipelines: 01(GETNEXT)
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes, RANDOM]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes, RANDOM]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
predicates: `user` = 'Lisa'
|
predicates: `user` = 'Lisa'
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
@@ -2133,7 +2133,7 @@ PLAN-ROOT SINK
|
|||||||
| in pipelines: 00(GETNEXT), 01(OPEN)
|
| in pipelines: 00(GETNEXT), 01(OPEN)
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=2 size=3.18KB
|
| table: rows=2 size=3.18KB
|
||||||
@@ -2144,7 +2144,7 @@ PLAN-ROOT SINK
|
|||||||
| in pipelines: 01(GETNEXT)
|
| in pipelines: 01(GETNEXT)
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
stored statistics:
|
stored statistics:
|
||||||
@@ -2182,7 +2182,7 @@ Per-Host Resources: mem-estimate=96.27MB mem-reservation=48.71KB thread-reservat
|
|||||||
| F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
| F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
||||||
| Per-Host Resources: mem-estimate=32.82MB mem-reservation=16.00KB thread-reservation=2
|
| Per-Host Resources: mem-estimate=32.82MB mem-reservation=16.00KB thread-reservation=2
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete, RANDOM]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete, RANDOM]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=2 size=3.18KB
|
| table: rows=2 size=3.18KB
|
||||||
@@ -2193,7 +2193,7 @@ Per-Host Resources: mem-estimate=96.27MB mem-reservation=48.71KB thread-reservat
|
|||||||
| in pipelines: 01(GETNEXT)
|
| in pipelines: 01(GETNEXT)
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes, RANDOM]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes, RANDOM]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
stored statistics:
|
stored statistics:
|
||||||
@@ -2867,7 +2867,7 @@ PLAN-ROOT SINK
|
|||||||
| | in pipelines: 00(GETNEXT), 01(OPEN)
|
| | in pipelines: 00(GETNEXT), 01(OPEN)
|
||||||
| |
|
| |
|
||||||
| |--01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_partitioned-equality-delete-01]
|
| |--01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_partitioned-equality-delete-01]
|
||||||
| | HDFS partitions=1/1 files=2 size=1.33KB
|
| | HDFS partitions=2/2 files=2 size=1.33KB
|
||||||
| | Iceberg snapshot id: 4821964189199835313
|
| | Iceberg snapshot id: 4821964189199835313
|
||||||
| | stored statistics:
|
| | stored statistics:
|
||||||
| | table: rows=3 size=1.33KB
|
| | table: rows=3 size=1.33KB
|
||||||
@@ -2878,7 +2878,7 @@ PLAN-ROOT SINK
|
|||||||
| | in pipelines: 01(GETNEXT)
|
| | in pipelines: 01(GETNEXT)
|
||||||
| |
|
| |
|
||||||
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned]
|
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned]
|
||||||
| HDFS partitions=1/1 files=2 size=1.76KB
|
| HDFS partitions=2/2 files=2 size=1.76KB
|
||||||
| Iceberg snapshot id: 4821964189199835313
|
| Iceberg snapshot id: 4821964189199835313
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=8 size=4.81KB
|
| table: rows=8 size=4.81KB
|
||||||
@@ -2889,7 +2889,7 @@ PLAN-ROOT SINK
|
|||||||
| in pipelines: 00(GETNEXT)
|
| in pipelines: 00(GETNEXT)
|
||||||
|
|
|
|
||||||
03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned]
|
03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned]
|
||||||
HDFS partitions=1/1 files=2 size=1.71KB
|
HDFS partitions=2/2 files=2 size=1.71KB
|
||||||
Iceberg snapshot id: 4821964189199835313
|
Iceberg snapshot id: 4821964189199835313
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=8 size=4.81KB
|
table: rows=8 size=4.81KB
|
||||||
@@ -2933,7 +2933,7 @@ Per-Host Resources: mem-estimate=64.12MB mem-reservation=1.94MB thread-reservati
|
|||||||
| | F02:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
|
| | F02:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
|
||||||
| | Per-Host Resources: mem-estimate=48.22MB mem-reservation=24.00KB thread-reservation=2
|
| | Per-Host Resources: mem-estimate=48.22MB mem-reservation=24.00KB thread-reservation=2
|
||||||
| | 01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_partitioned-equality-delete-01, RANDOM]
|
| | 01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_partitioned-equality-delete-01, RANDOM]
|
||||||
| | HDFS partitions=1/1 files=2 size=1.33KB
|
| | HDFS partitions=2/2 files=2 size=1.33KB
|
||||||
| | Iceberg snapshot id: 4821964189199835313
|
| | Iceberg snapshot id: 4821964189199835313
|
||||||
| | stored statistics:
|
| | stored statistics:
|
||||||
| | table: rows=3 size=1.33KB
|
| | table: rows=3 size=1.33KB
|
||||||
@@ -2951,7 +2951,7 @@ Per-Host Resources: mem-estimate=64.12MB mem-reservation=1.94MB thread-reservati
|
|||||||
| F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
|
| F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
|
||||||
| Per-Host Resources: mem-estimate=64.25MB mem-reservation=32.00KB thread-reservation=2
|
| Per-Host Resources: mem-estimate=64.25MB mem-reservation=32.00KB thread-reservation=2
|
||||||
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned, RANDOM]
|
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned, RANDOM]
|
||||||
| HDFS partitions=1/1 files=2 size=1.76KB
|
| HDFS partitions=2/2 files=2 size=1.76KB
|
||||||
| Iceberg snapshot id: 4821964189199835313
|
| Iceberg snapshot id: 4821964189199835313
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=8 size=4.81KB
|
| table: rows=8 size=4.81KB
|
||||||
@@ -2962,7 +2962,7 @@ Per-Host Resources: mem-estimate=64.12MB mem-reservation=1.94MB thread-reservati
|
|||||||
| in pipelines: 00(GETNEXT)
|
| in pipelines: 00(GETNEXT)
|
||||||
|
|
|
|
||||||
03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned, RANDOM]
|
03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned, RANDOM]
|
||||||
HDFS partitions=1/1 files=2 size=1.71KB
|
HDFS partitions=2/2 files=2 size=1.71KB
|
||||||
Iceberg snapshot id: 4821964189199835313
|
Iceberg snapshot id: 4821964189199835313
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=8 size=4.81KB
|
table: rows=8 size=4.81KB
|
||||||
@@ -3205,7 +3205,7 @@ PLAN-ROOT SINK
|
|||||||
| | in pipelines: 00(GETNEXT), 03(OPEN)
|
| | in pipelines: 00(GETNEXT), 03(OPEN)
|
||||||
| |
|
| |
|
||||||
| |--03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-03 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03]
|
| |--03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-03 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03]
|
||||||
| | HDFS partitions=1/1 files=1 size=444B
|
| | HDFS partitions=1/unknown files=1 size=444B
|
||||||
| | Iceberg snapshot id: 5777805847908928861
|
| | Iceberg snapshot id: 5777805847908928861
|
||||||
| | stored statistics:
|
| | stored statistics:
|
||||||
| | table: rows=1 size=444B
|
| | table: rows=1 size=444B
|
||||||
@@ -3223,7 +3223,7 @@ PLAN-ROOT SINK
|
|||||||
| | in pipelines: 00(GETNEXT), 01(OPEN)
|
| | in pipelines: 00(GETNEXT), 01(OPEN)
|
||||||
| |
|
| |
|
||||||
| |--01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01]
|
| |--01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01]
|
||||||
| | HDFS partitions=1/1 files=1 size=473B
|
| | HDFS partitions=1/unknown files=1 size=473B
|
||||||
| | Iceberg snapshot id: 5777805847908928861
|
| | Iceberg snapshot id: 5777805847908928861
|
||||||
| | stored statistics:
|
| | stored statistics:
|
||||||
| | table: rows=2 size=473B
|
| | table: rows=2 size=473B
|
||||||
@@ -3234,7 +3234,7 @@ PLAN-ROOT SINK
|
|||||||
| | in pipelines: 01(GETNEXT)
|
| | in pipelines: 01(GETNEXT)
|
||||||
| |
|
| |
|
||||||
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
|
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
|
||||||
| HDFS partitions=1/1 files=2 size=1.33KB
|
| HDFS partitions=1/unknown files=2 size=1.33KB
|
||||||
| Iceberg snapshot id: 5777805847908928861
|
| Iceberg snapshot id: 5777805847908928861
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=10 size=4.29KB
|
| table: rows=10 size=4.29KB
|
||||||
@@ -3245,7 +3245,7 @@ PLAN-ROOT SINK
|
|||||||
| in pipelines: 00(GETNEXT)
|
| in pipelines: 00(GETNEXT)
|
||||||
|
|
|
|
||||||
05:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
|
05:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
|
||||||
HDFS partitions=1/1 files=1 size=673B
|
HDFS partitions=1/unknown files=1 size=673B
|
||||||
Iceberg snapshot id: 5777805847908928861
|
Iceberg snapshot id: 5777805847908928861
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=10 size=4.29KB
|
table: rows=10 size=4.29KB
|
||||||
@@ -3289,7 +3289,7 @@ Per-Host Resources: mem-estimate=52.02MB mem-reservation=3.90MB thread-reservati
|
|||||||
| | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
| | F03:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
||||||
| | Per-Host Resources: mem-estimate=32.06MB mem-reservation=16.00KB thread-reservation=2
|
| | Per-Host Resources: mem-estimate=32.06MB mem-reservation=16.00KB thread-reservation=2
|
||||||
| | 03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-03 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03, RANDOM]
|
| | 03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-03 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03, RANDOM]
|
||||||
| | HDFS partitions=1/1 files=1 size=444B
|
| | HDFS partitions=1/unknown files=1 size=444B
|
||||||
| | Iceberg snapshot id: 5777805847908928861
|
| | Iceberg snapshot id: 5777805847908928861
|
||||||
| | stored statistics:
|
| | stored statistics:
|
||||||
| | table: rows=1 size=444B
|
| | table: rows=1 size=444B
|
||||||
@@ -3314,7 +3314,7 @@ Per-Host Resources: mem-estimate=52.02MB mem-reservation=3.90MB thread-reservati
|
|||||||
| | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
| | F02:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
|
||||||
| | Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2
|
| | Per-Host Resources: mem-estimate=32.09MB mem-reservation=16.00KB thread-reservation=2
|
||||||
| | 01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01, RANDOM]
|
| | 01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01, RANDOM]
|
||||||
| | HDFS partitions=1/1 files=1 size=473B
|
| | HDFS partitions=1/unknown files=1 size=473B
|
||||||
| | Iceberg snapshot id: 5777805847908928861
|
| | Iceberg snapshot id: 5777805847908928861
|
||||||
| | stored statistics:
|
| | stored statistics:
|
||||||
| | table: rows=2 size=473B
|
| | table: rows=2 size=473B
|
||||||
@@ -3325,7 +3325,7 @@ Per-Host Resources: mem-estimate=52.02MB mem-reservation=3.90MB thread-reservati
|
|||||||
| | in pipelines: 01(GETNEXT)
|
| | in pipelines: 01(GETNEXT)
|
||||||
| |
|
| |
|
||||||
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids, RANDOM]
|
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids, RANDOM]
|
||||||
| HDFS partitions=1/1 files=2 size=1.33KB
|
| HDFS partitions=1/unknown files=2 size=1.33KB
|
||||||
| Iceberg snapshot id: 5777805847908928861
|
| Iceberg snapshot id: 5777805847908928861
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=10 size=4.29KB
|
| table: rows=10 size=4.29KB
|
||||||
@@ -3336,7 +3336,7 @@ Per-Host Resources: mem-estimate=52.02MB mem-reservation=3.90MB thread-reservati
|
|||||||
| in pipelines: 00(GETNEXT)
|
| in pipelines: 00(GETNEXT)
|
||||||
|
|
|
|
||||||
05:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids, RANDOM]
|
05:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids, RANDOM]
|
||||||
HDFS partitions=1/1 files=1 size=673B
|
HDFS partitions=1/unknown files=1 size=673B
|
||||||
Iceberg snapshot id: 5777805847908928861
|
Iceberg snapshot id: 5777805847908928861
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=10 size=4.29KB
|
table: rows=10 size=4.29KB
|
||||||
@@ -3576,7 +3576,7 @@ PLAN-ROOT SINK
|
|||||||
| | in pipelines: 00(GETNEXT), 01(OPEN)
|
| | in pipelines: 00(GETNEXT), 01(OPEN)
|
||||||
| |
|
| |
|
||||||
| |--01:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_equality_delete_schema_evolution-equality-delete-01]
|
| |--01:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_equality_delete_schema_evolution-equality-delete-01]
|
||||||
| | HDFS partitions=1/1 files=3 size=1.97KB
|
| | HDFS partitions=2/3 files=3 size=1.97KB
|
||||||
| | Iceberg snapshot id: 5816823095034839884
|
| | Iceberg snapshot id: 5816823095034839884
|
||||||
| | stored statistics:
|
| | stored statistics:
|
||||||
| | table: rows=3 size=1.97KB
|
| | table: rows=3 size=1.97KB
|
||||||
@@ -3587,7 +3587,7 @@ PLAN-ROOT SINK
|
|||||||
| | in pipelines: 01(GETNEXT)
|
| | in pipelines: 01(GETNEXT)
|
||||||
| |
|
| |
|
||||||
| 00:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution]
|
| 00:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution]
|
||||||
| HDFS partitions=1/1 files=2 size=1.63KB
|
| HDFS partitions=2/3 files=2 size=1.63KB
|
||||||
| Iceberg snapshot id: 5816823095034839884
|
| Iceberg snapshot id: 5816823095034839884
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=6 size=5.50KB
|
| table: rows=6 size=5.50KB
|
||||||
@@ -3598,7 +3598,7 @@ PLAN-ROOT SINK
|
|||||||
| in pipelines: 00(GETNEXT)
|
| in pipelines: 00(GETNEXT)
|
||||||
|
|
|
|
||||||
03:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution]
|
03:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution]
|
||||||
HDFS partitions=1/1 files=2 size=1.90KB
|
HDFS partitions=2/3 files=2 size=1.90KB
|
||||||
Iceberg snapshot id: 5816823095034839884
|
Iceberg snapshot id: 5816823095034839884
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=6 size=5.50KB
|
table: rows=6 size=5.50KB
|
||||||
@@ -3642,7 +3642,7 @@ Per-Host Resources: mem-estimate=82.09MB mem-reservation=1.98MB thread-reservati
|
|||||||
| | F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
|
| | F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
|
||||||
| | Per-Host Resources: mem-estimate=48.08MB mem-reservation=24.00KB thread-reservation=2
|
| | Per-Host Resources: mem-estimate=48.08MB mem-reservation=24.00KB thread-reservation=2
|
||||||
| | 01:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_equality_delete_schema_evolution-equality-delete-01, RANDOM]
|
| | 01:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_equality_delete_schema_evolution-equality-delete-01, RANDOM]
|
||||||
| | HDFS partitions=1/1 files=3 size=1.97KB
|
| | HDFS partitions=2/3 files=3 size=1.97KB
|
||||||
| | Iceberg snapshot id: 5816823095034839884
|
| | Iceberg snapshot id: 5816823095034839884
|
||||||
| | stored statistics:
|
| | stored statistics:
|
||||||
| | table: rows=3 size=1.97KB
|
| | table: rows=3 size=1.97KB
|
||||||
@@ -3653,7 +3653,7 @@ Per-Host Resources: mem-estimate=82.09MB mem-reservation=1.98MB thread-reservati
|
|||||||
| | in pipelines: 01(GETNEXT)
|
| | in pipelines: 01(GETNEXT)
|
||||||
| |
|
| |
|
||||||
| 00:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution, RANDOM]
|
| 00:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution, RANDOM]
|
||||||
| HDFS partitions=1/1 files=2 size=1.63KB
|
| HDFS partitions=2/3 files=2 size=1.63KB
|
||||||
| Iceberg snapshot id: 5816823095034839884
|
| Iceberg snapshot id: 5816823095034839884
|
||||||
| stored statistics:
|
| stored statistics:
|
||||||
| table: rows=6 size=5.50KB
|
| table: rows=6 size=5.50KB
|
||||||
@@ -3664,7 +3664,7 @@ Per-Host Resources: mem-estimate=82.09MB mem-reservation=1.98MB thread-reservati
|
|||||||
| in pipelines: 00(GETNEXT)
|
| in pipelines: 00(GETNEXT)
|
||||||
|
|
|
|
||||||
03:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution, RANDOM]
|
03:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution, RANDOM]
|
||||||
HDFS partitions=1/1 files=2 size=1.90KB
|
HDFS partitions=2/3 files=2 size=1.90KB
|
||||||
Iceberg snapshot id: 5816823095034839884
|
Iceberg snapshot id: 5816823095034839884
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=6 size=5.50KB
|
table: rows=6 size=5.50KB
|
||||||
|
|||||||
@@ -361,12 +361,12 @@ PLAN-ROOT SINK
|
|||||||
| row-size=20B cardinality=2
|
| row-size=20B cardinality=2
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=2.63KB
|
| HDFS partitions=1/unknown files=1 size=2.63KB
|
||||||
| Iceberg snapshot id: 752781918366351945
|
| Iceberg snapshot id: 752781918366351945
|
||||||
| row-size=267B cardinality=1
|
| row-size=267B cardinality=1
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
||||||
HDFS partitions=1/1 files=1 size=625B
|
HDFS partitions=1/unknown files=1 size=625B
|
||||||
Iceberg snapshot id: 752781918366351945
|
Iceberg snapshot id: 752781918366351945
|
||||||
row-size=20B cardinality=3
|
row-size=20B cardinality=3
|
||||||
---- DISTRIBUTEDPLAN
|
---- DISTRIBUTEDPLAN
|
||||||
@@ -388,12 +388,12 @@ PLAN-ROOT SINK
|
|||||||
|--04:EXCHANGE [DIRECTED]
|
|--04:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=2.63KB
|
| HDFS partitions=1/unknown files=1 size=2.63KB
|
||||||
| Iceberg snapshot id: 752781918366351945
|
| Iceberg snapshot id: 752781918366351945
|
||||||
| row-size=267B cardinality=1
|
| row-size=267B cardinality=1
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
||||||
HDFS partitions=1/1 files=1 size=625B
|
HDFS partitions=1/unknown files=1 size=625B
|
||||||
Iceberg snapshot id: 752781918366351945
|
Iceberg snapshot id: 752781918366351945
|
||||||
row-size=20B cardinality=3
|
row-size=20B cardinality=3
|
||||||
====
|
====
|
||||||
@@ -409,12 +409,12 @@ PLAN-ROOT SINK
|
|||||||
| row-size=20B cardinality=2
|
| row-size=20B cardinality=2
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=2.63KB
|
| HDFS partitions=1/unknown files=1 size=2.63KB
|
||||||
| Iceberg snapshot id: 752781918366351945
|
| Iceberg snapshot id: 752781918366351945
|
||||||
| row-size=267B cardinality=1
|
| row-size=267B cardinality=1
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
||||||
HDFS partitions=1/1 files=1 size=625B
|
HDFS partitions=1/unknown files=1 size=625B
|
||||||
Iceberg snapshot id: 752781918366351945
|
Iceberg snapshot id: 752781918366351945
|
||||||
row-size=20B cardinality=3
|
row-size=20B cardinality=3
|
||||||
---- DISTRIBUTEDPLAN
|
---- DISTRIBUTEDPLAN
|
||||||
@@ -436,12 +436,12 @@ PLAN-ROOT SINK
|
|||||||
|--04:EXCHANGE [DIRECTED]
|
|--04:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-POSITION-DELETE-01 functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=2.63KB
|
| HDFS partitions=1/unknown files=1 size=2.63KB
|
||||||
| Iceberg snapshot id: 752781918366351945
|
| Iceberg snapshot id: 752781918366351945
|
||||||
| row-size=267B cardinality=1
|
| row-size=267B cardinality=1
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
||||||
HDFS partitions=1/1 files=1 size=625B
|
HDFS partitions=1/unknown files=1 size=625B
|
||||||
Iceberg snapshot id: 752781918366351945
|
Iceberg snapshot id: 752781918366351945
|
||||||
row-size=20B cardinality=3
|
row-size=20B cardinality=3
|
||||||
====
|
====
|
||||||
@@ -555,12 +555,12 @@ PLAN-ROOT SINK
|
|||||||
| row-size=64B cardinality=10
|
| row-size=64B cardinality=10
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=64B cardinality=20
|
row-size=64B cardinality=20
|
||||||
---- DISTRIBUTEDPLAN
|
---- DISTRIBUTEDPLAN
|
||||||
@@ -574,12 +574,12 @@ PLAN-ROOT SINK
|
|||||||
|--03:EXCHANGE [DIRECTED]
|
|--03:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=64B cardinality=20
|
row-size=64B cardinality=20
|
||||||
====
|
====
|
||||||
@@ -653,7 +653,7 @@ PLAN-ROOT SINK
|
|||||||
| row-size=16B cardinality=6
|
| row-size=16B cardinality=6
|
||||||
|
|
|
|
||||||
|--06:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
|--06:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
||||||
| HDFS partitions=1/1 files=1 size=625B
|
| HDFS partitions=1/unknown files=1 size=625B
|
||||||
| Iceberg snapshot id: 4363979609026842966
|
| Iceberg snapshot id: 4363979609026842966
|
||||||
| row-size=16B cardinality=3
|
| row-size=16B cardinality=3
|
||||||
|
|
|
|
||||||
@@ -694,7 +694,7 @@ PLAN-ROOT SINK
|
|||||||
|--11:EXCHANGE [HASH(functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.i,functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.s)]
|
|--11:EXCHANGE [HASH(functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.i,functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files.s)]
|
||||||
| |
|
| |
|
||||||
| 06:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
| 06:SCAN HDFS [functional_parquet.iceberg_v2_positional_not_all_data_files_have_delete_files]
|
||||||
| HDFS partitions=1/1 files=1 size=625B
|
| HDFS partitions=1/unknown files=1 size=625B
|
||||||
| Iceberg snapshot id: 4363979609026842966
|
| Iceberg snapshot id: 4363979609026842966
|
||||||
| row-size=16B cardinality=3
|
| row-size=16B cardinality=3
|
||||||
|
|
|
|
||||||
@@ -931,12 +931,12 @@ PLAN-ROOT SINK
|
|||||||
| row-size=64B cardinality=1
|
| row-size=64B cardinality=1
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=207B cardinality=2
|
| row-size=207B cardinality=2
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
@@ -952,12 +952,12 @@ PLAN-ROOT SINK
|
|||||||
|--03:EXCHANGE [DIRECTED]
|
|--03:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=207B cardinality=2
|
| row-size=207B cardinality=2
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
@@ -972,12 +972,12 @@ PLAN-ROOT SINK
|
|||||||
| row-size=64B cardinality=1
|
| row-size=64B cardinality=1
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=207B cardinality=2
|
| row-size=207B cardinality=2
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
predicates: `user` = 'Lisa'
|
predicates: `user` = 'Lisa'
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
@@ -993,12 +993,12 @@ PLAN-ROOT SINK
|
|||||||
|--03:EXCHANGE [DIRECTED]
|
|--03:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=207B cardinality=2
|
| row-size=207B cardinality=2
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
predicates: `user` = 'Lisa'
|
predicates: `user` = 'Lisa'
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
@@ -1009,7 +1009,7 @@ select event_time, action from iceberg_partitioned where (action = 'click' or ac
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=14 size=15.93KB
|
HDFS partitions=2/3 files=14 size=15.93KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: action IN ('click', 'view')
|
skipped Iceberg predicates: action IN ('click', 'view')
|
||||||
@@ -1020,7 +1020,7 @@ PLAN-ROOT SINK
|
|||||||
01:EXCHANGE [UNPARTITIONED]
|
01:EXCHANGE [UNPARTITIONED]
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=14 size=15.93KB
|
HDFS partitions=2/3 files=14 size=15.93KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: action IN ('click', 'view')
|
skipped Iceberg predicates: action IN ('click', 'view')
|
||||||
@@ -1031,7 +1031,7 @@ select event_time, action from iceberg_partitioned where action in ('click', 'vi
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=14 size=15.93KB
|
HDFS partitions=2/3 files=14 size=15.93KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: action IN ('click', 'view')
|
skipped Iceberg predicates: action IN ('click', 'view')
|
||||||
@@ -1042,7 +1042,7 @@ PLAN-ROOT SINK
|
|||||||
01:EXCHANGE [UNPARTITIONED]
|
01:EXCHANGE [UNPARTITIONED]
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=14 size=15.93KB
|
HDFS partitions=2/3 files=14 size=15.93KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: action IN ('click', 'view')
|
skipped Iceberg predicates: action IN ('click', 'view')
|
||||||
@@ -1053,7 +1053,7 @@ select event_time, action from iceberg_partitioned where (event_time='2020-01-01
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=6 size=6.85KB
|
HDFS partitions=1/3 files=6 size=6.85KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action = 'click')
|
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action = 'click')
|
||||||
@@ -1064,7 +1064,7 @@ PLAN-ROOT SINK
|
|||||||
01:EXCHANGE [UNPARTITIONED]
|
01:EXCHANGE [UNPARTITIONED]
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=6 size=6.85KB
|
HDFS partitions=1/3 files=6 size=6.85KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action = 'click')
|
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action = 'click')
|
||||||
@@ -1075,7 +1075,7 @@ select event_time, action from iceberg_partitioned where (event_time='2020-01-01
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=14 size=15.93KB
|
HDFS partitions=2/3 files=14 size=15.93KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action = 'click' OR action = 'view')
|
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action = 'click' OR action = 'view')
|
||||||
@@ -1086,7 +1086,7 @@ PLAN-ROOT SINK
|
|||||||
01:EXCHANGE [UNPARTITIONED]
|
01:EXCHANGE [UNPARTITIONED]
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=14 size=15.93KB
|
HDFS partitions=2/3 files=14 size=15.93KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action = 'click' OR action = 'view')
|
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action = 'click' OR action = 'view')
|
||||||
@@ -1097,7 +1097,7 @@ select event_time, action from iceberg_partitioned where (event_time='2020-01-01
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=14 size=15.93KB
|
HDFS partitions=2/3 files=14 size=15.93KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action IN ('click', 'view'))
|
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action IN ('click', 'view'))
|
||||||
@@ -1108,7 +1108,7 @@ PLAN-ROOT SINK
|
|||||||
01:EXCHANGE [UNPARTITIONED]
|
01:EXCHANGE [UNPARTITIONED]
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=14 size=15.93KB
|
HDFS partitions=2/3 files=14 size=15.93KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action IN ('click', 'view'))
|
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action IN ('click', 'view'))
|
||||||
@@ -1119,7 +1119,7 @@ select event_time, action from iceberg_partitioned where (event_time='2020-01-01
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=20 size=22.90KB
|
HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action > 'a')
|
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action > 'a')
|
||||||
@@ -1130,7 +1130,7 @@ PLAN-ROOT SINK
|
|||||||
01:EXCHANGE [UNPARTITIONED]
|
01:EXCHANGE [UNPARTITIONED]
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=20 size=22.90KB
|
HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
predicates: id > 0
|
predicates: id > 0
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action > 'a')
|
skipped Iceberg predicates: (event_time = TIMESTAMP '2020-01-01 11:00:00' OR action > 'a')
|
||||||
@@ -1142,7 +1142,7 @@ select event_time, action from iceberg_partitioned where event_time='2020-01-01
|
|||||||
PLAN-ROOT SINK
|
PLAN-ROOT SINK
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=0 size=0B
|
HDFS partitions=0/3 files=0 size=0B
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: event_time = TIMESTAMP '2020-01-01 11:00:00'
|
skipped Iceberg predicates: event_time = TIMESTAMP '2020-01-01 11:00:00'
|
||||||
row-size=28B cardinality=0
|
row-size=28B cardinality=0
|
||||||
@@ -1152,7 +1152,7 @@ PLAN-ROOT SINK
|
|||||||
01:EXCHANGE [UNPARTITIONED]
|
01:EXCHANGE [UNPARTITIONED]
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=0 size=0B
|
HDFS partitions=0/3 files=0 size=0B
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: event_time = TIMESTAMP '2020-01-01 11:00:00'
|
skipped Iceberg predicates: event_time = TIMESTAMP '2020-01-01 11:00:00'
|
||||||
row-size=28B cardinality=0
|
row-size=28B cardinality=0
|
||||||
@@ -1166,12 +1166,12 @@ PLAN-ROOT SINK
|
|||||||
| row-size=64B cardinality=4
|
| row-size=64B cardinality=4
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=207B cardinality=2
|
| row-size=207B cardinality=2
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
row-size=64B cardinality=6
|
row-size=64B cardinality=6
|
||||||
@@ -1186,12 +1186,12 @@ PLAN-ROOT SINK
|
|||||||
|--03:EXCHANGE [DIRECTED]
|
|--03:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.18KB
|
| HDFS partitions=1/3 files=1 size=3.18KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=207B cardinality=2
|
| row-size=207B cardinality=2
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.17KB
|
HDFS partitions=1/3 files=1 size=1.17KB
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'download'
|
skipped Iceberg predicates: action = 'download'
|
||||||
row-size=64B cardinality=6
|
row-size=64B cardinality=6
|
||||||
@@ -1520,17 +1520,17 @@ PLAN-ROOT SINK
|
|||||||
| | row-size=28B cardinality=5
|
| | row-size=28B cardinality=5
|
||||||
| |
|
| |
|
||||||
| |--01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_partitioned-equality-delete-01]
|
| |--01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_partitioned-equality-delete-01]
|
||||||
| | HDFS partitions=1/1 files=2 size=1.33KB
|
| | HDFS partitions=2/2 files=2 size=1.33KB
|
||||||
| | Iceberg snapshot id: 4821964189199835313
|
| | Iceberg snapshot id: 4821964189199835313
|
||||||
| | row-size=24B cardinality=3
|
| | row-size=24B cardinality=3
|
||||||
| |
|
| |
|
||||||
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned]
|
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned]
|
||||||
| HDFS partitions=1/1 files=2 size=1.76KB
|
| HDFS partitions=2/2 files=2 size=1.76KB
|
||||||
| Iceberg snapshot id: 4821964189199835313
|
| Iceberg snapshot id: 4821964189199835313
|
||||||
| row-size=28B cardinality=5
|
| row-size=28B cardinality=5
|
||||||
|
|
|
|
||||||
03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned]
|
03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned]
|
||||||
HDFS partitions=1/1 files=2 size=1.71KB
|
HDFS partitions=2/2 files=2 size=1.71KB
|
||||||
Iceberg snapshot id: 4821964189199835313
|
Iceberg snapshot id: 4821964189199835313
|
||||||
row-size=28B cardinality=3
|
row-size=28B cardinality=3
|
||||||
---- DISTRIBUTEDPLAN
|
---- DISTRIBUTEDPLAN
|
||||||
@@ -1550,19 +1550,19 @@ PLAN-ROOT SINK
|
|||||||
| |--06:EXCHANGE [HASH(functional_parquet.iceberg_v2_delete_equality_partitioned-equality-delete-01.d,functional_parquet.iceberg_v2_delete_equality_partitioned-equality-delete-01.s)]
|
| |--06:EXCHANGE [HASH(functional_parquet.iceberg_v2_delete_equality_partitioned-equality-delete-01.d,functional_parquet.iceberg_v2_delete_equality_partitioned-equality-delete-01.s)]
|
||||||
| | |
|
| | |
|
||||||
| | 01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_partitioned-equality-delete-01]
|
| | 01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_partitioned-equality-delete-01]
|
||||||
| | HDFS partitions=1/1 files=2 size=1.33KB
|
| | HDFS partitions=2/2 files=2 size=1.33KB
|
||||||
| | Iceberg snapshot id: 4821964189199835313
|
| | Iceberg snapshot id: 4821964189199835313
|
||||||
| | row-size=24B cardinality=3
|
| | row-size=24B cardinality=3
|
||||||
| |
|
| |
|
||||||
| 05:EXCHANGE [HASH(functional_parquet.iceberg_v2_delete_equality_partitioned.d,functional_parquet.iceberg_v2_delete_equality_partitioned.s)]
|
| 05:EXCHANGE [HASH(functional_parquet.iceberg_v2_delete_equality_partitioned.d,functional_parquet.iceberg_v2_delete_equality_partitioned.s)]
|
||||||
| |
|
| |
|
||||||
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned]
|
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned]
|
||||||
| HDFS partitions=1/1 files=2 size=1.76KB
|
| HDFS partitions=2/2 files=2 size=1.76KB
|
||||||
| Iceberg snapshot id: 4821964189199835313
|
| Iceberg snapshot id: 4821964189199835313
|
||||||
| row-size=28B cardinality=5
|
| row-size=28B cardinality=5
|
||||||
|
|
|
|
||||||
03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned]
|
03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_partitioned]
|
||||||
HDFS partitions=1/1 files=2 size=1.71KB
|
HDFS partitions=2/2 files=2 size=1.71KB
|
||||||
Iceberg snapshot id: 4821964189199835313
|
Iceberg snapshot id: 4821964189199835313
|
||||||
row-size=28B cardinality=3
|
row-size=28B cardinality=3
|
||||||
====
|
====
|
||||||
@@ -1687,7 +1687,7 @@ PLAN-ROOT SINK
|
|||||||
| | row-size=24B cardinality=4
|
| | row-size=24B cardinality=4
|
||||||
| |
|
| |
|
||||||
| |--03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-03 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03]
|
| |--03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-03 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03]
|
||||||
| | HDFS partitions=1/1 files=1 size=444B
|
| | HDFS partitions=1/unknown files=1 size=444B
|
||||||
| | Iceberg snapshot id: 5777805847908928861
|
| | Iceberg snapshot id: 5777805847908928861
|
||||||
| | row-size=12B cardinality=1
|
| | row-size=12B cardinality=1
|
||||||
| |
|
| |
|
||||||
@@ -1697,17 +1697,17 @@ PLAN-ROOT SINK
|
|||||||
| | row-size=24B cardinality=4
|
| | row-size=24B cardinality=4
|
||||||
| |
|
| |
|
||||||
| |--01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01]
|
| |--01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01]
|
||||||
| | HDFS partitions=1/1 files=1 size=473B
|
| | HDFS partitions=1/unknown files=1 size=473B
|
||||||
| | Iceberg snapshot id: 5777805847908928861
|
| | Iceberg snapshot id: 5777805847908928861
|
||||||
| | row-size=20B cardinality=2
|
| | row-size=20B cardinality=2
|
||||||
| |
|
| |
|
||||||
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
|
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
|
||||||
| HDFS partitions=1/1 files=2 size=1.33KB
|
| HDFS partitions=1/unknown files=2 size=1.33KB
|
||||||
| Iceberg snapshot id: 5777805847908928861
|
| Iceberg snapshot id: 5777805847908928861
|
||||||
| row-size=24B cardinality=4
|
| row-size=24B cardinality=4
|
||||||
|
|
|
|
||||||
05:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
|
05:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
|
||||||
HDFS partitions=1/1 files=1 size=673B
|
HDFS partitions=1/unknown files=1 size=673B
|
||||||
Iceberg snapshot id: 5777805847908928861
|
Iceberg snapshot id: 5777805847908928861
|
||||||
row-size=24B cardinality=4
|
row-size=24B cardinality=4
|
||||||
---- DISTRIBUTEDPLAN
|
---- DISTRIBUTEDPLAN
|
||||||
@@ -1727,7 +1727,7 @@ PLAN-ROOT SINK
|
|||||||
| |--08:EXCHANGE [BROADCAST]
|
| |--08:EXCHANGE [BROADCAST]
|
||||||
| | |
|
| | |
|
||||||
| | 03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-03 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03]
|
| | 03:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-03 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-03]
|
||||||
| | HDFS partitions=1/1 files=1 size=444B
|
| | HDFS partitions=1/unknown files=1 size=444B
|
||||||
| | Iceberg snapshot id: 5777805847908928861
|
| | Iceberg snapshot id: 5777805847908928861
|
||||||
| | row-size=12B cardinality=1
|
| | row-size=12B cardinality=1
|
||||||
| |
|
| |
|
||||||
@@ -1739,17 +1739,17 @@ PLAN-ROOT SINK
|
|||||||
| |--07:EXCHANGE [BROADCAST]
|
| |--07:EXCHANGE [BROADCAST]
|
||||||
| | |
|
| | |
|
||||||
| | 01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01]
|
| | 01:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_delete_equality_multi_eq_ids-equality-delete-01]
|
||||||
| | HDFS partitions=1/1 files=1 size=473B
|
| | HDFS partitions=1/unknown files=1 size=473B
|
||||||
| | Iceberg snapshot id: 5777805847908928861
|
| | Iceberg snapshot id: 5777805847908928861
|
||||||
| | row-size=20B cardinality=2
|
| | row-size=20B cardinality=2
|
||||||
| |
|
| |
|
||||||
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
|
| 00:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
|
||||||
| HDFS partitions=1/1 files=2 size=1.33KB
|
| HDFS partitions=1/unknown files=2 size=1.33KB
|
||||||
| Iceberg snapshot id: 5777805847908928861
|
| Iceberg snapshot id: 5777805847908928861
|
||||||
| row-size=24B cardinality=4
|
| row-size=24B cardinality=4
|
||||||
|
|
|
|
||||||
05:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
|
05:SCAN HDFS [functional_parquet.iceberg_v2_delete_equality_multi_eq_ids]
|
||||||
HDFS partitions=1/1 files=1 size=673B
|
HDFS partitions=1/unknown files=1 size=673B
|
||||||
Iceberg snapshot id: 5777805847908928861
|
Iceberg snapshot id: 5777805847908928861
|
||||||
row-size=24B cardinality=4
|
row-size=24B cardinality=4
|
||||||
====
|
====
|
||||||
@@ -1897,17 +1897,17 @@ PLAN-ROOT SINK
|
|||||||
| | row-size=32B cardinality=4
|
| | row-size=32B cardinality=4
|
||||||
| |
|
| |
|
||||||
| |--01:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_equality_delete_schema_evolution-equality-delete-01]
|
| |--01:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_equality_delete_schema_evolution-equality-delete-01]
|
||||||
| | HDFS partitions=1/1 files=3 size=1.97KB
|
| | HDFS partitions=2/3 files=3 size=1.97KB
|
||||||
| | Iceberg snapshot id: 5816823095034839884
|
| | Iceberg snapshot id: 5816823095034839884
|
||||||
| | row-size=16B cardinality=3
|
| | row-size=16B cardinality=3
|
||||||
| |
|
| |
|
||||||
| 00:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution]
|
| 00:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution]
|
||||||
| HDFS partitions=1/1 files=2 size=1.63KB
|
| HDFS partitions=2/3 files=2 size=1.63KB
|
||||||
| Iceberg snapshot id: 5816823095034839884
|
| Iceberg snapshot id: 5816823095034839884
|
||||||
| row-size=32B cardinality=4
|
| row-size=32B cardinality=4
|
||||||
|
|
|
|
||||||
03:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution]
|
03:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution]
|
||||||
HDFS partitions=1/1 files=2 size=1.90KB
|
HDFS partitions=2/3 files=2 size=1.90KB
|
||||||
Iceberg snapshot id: 5816823095034839884
|
Iceberg snapshot id: 5816823095034839884
|
||||||
row-size=32B cardinality=2
|
row-size=32B cardinality=2
|
||||||
---- DISTRIBUTEDPLAN
|
---- DISTRIBUTEDPLAN
|
||||||
@@ -1927,17 +1927,17 @@ PLAN-ROOT SINK
|
|||||||
| |--05:EXCHANGE [BROADCAST]
|
| |--05:EXCHANGE [BROADCAST]
|
||||||
| | |
|
| | |
|
||||||
| | 01:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_equality_delete_schema_evolution-equality-delete-01]
|
| | 01:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution-EQUALITY-DELETE-01 functional_parquet.iceberg_v2_equality_delete_schema_evolution-equality-delete-01]
|
||||||
| | HDFS partitions=1/1 files=3 size=1.97KB
|
| | HDFS partitions=2/3 files=3 size=1.97KB
|
||||||
| | Iceberg snapshot id: 5816823095034839884
|
| | Iceberg snapshot id: 5816823095034839884
|
||||||
| | row-size=16B cardinality=3
|
| | row-size=16B cardinality=3
|
||||||
| |
|
| |
|
||||||
| 00:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution]
|
| 00:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution]
|
||||||
| HDFS partitions=1/1 files=2 size=1.63KB
|
| HDFS partitions=2/3 files=2 size=1.63KB
|
||||||
| Iceberg snapshot id: 5816823095034839884
|
| Iceberg snapshot id: 5816823095034839884
|
||||||
| row-size=32B cardinality=4
|
| row-size=32B cardinality=4
|
||||||
|
|
|
|
||||||
03:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution]
|
03:SCAN HDFS [functional_parquet.iceberg_v2_equality_delete_schema_evolution]
|
||||||
HDFS partitions=1/1 files=2 size=1.90KB
|
HDFS partitions=2/3 files=2 size=1.90KB
|
||||||
Iceberg snapshot id: 5816823095034839884
|
Iceberg snapshot id: 5816823095034839884
|
||||||
row-size=32B cardinality=2
|
row-size=32B cardinality=2
|
||||||
====
|
====
|
||||||
|
|||||||
@@ -154,12 +154,12 @@ MULTI DATA SINK
|
|||||||
| row-size=80B cardinality=1
|
| row-size=80B cardinality=1
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
predicates: id IS DISTINCT FROM length(action)
|
predicates: id IS DISTINCT FROM length(action)
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=2
|
row-size=80B cardinality=2
|
||||||
@@ -180,12 +180,12 @@ MULTI DATA SINK
|
|||||||
|--03:EXCHANGE [DIRECTED]
|
|--03:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
predicates: id IS DISTINCT FROM length(action)
|
predicates: id IS DISTINCT FROM length(action)
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=2
|
row-size=80B cardinality=2
|
||||||
@@ -204,12 +204,12 @@ MULTI DATA SINK
|
|||||||
| row-size=80B cardinality=1
|
| row-size=80B cardinality=1
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
predicates: id IS DISTINCT FROM length(action), `user` LIKE 'impala'
|
predicates: id IS DISTINCT FROM length(action), `user` LIKE 'impala'
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=2
|
row-size=80B cardinality=2
|
||||||
@@ -230,12 +230,12 @@ MULTI DATA SINK
|
|||||||
|--03:EXCHANGE [DIRECTED]
|
|--03:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
predicates: id IS DISTINCT FROM length(action), `user` LIKE 'impala'
|
predicates: id IS DISTINCT FROM length(action), `user` LIKE 'impala'
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=2
|
row-size=80B cardinality=2
|
||||||
@@ -254,12 +254,12 @@ MULTI DATA SINK
|
|||||||
| row-size=80B cardinality=1
|
| row-size=80B cardinality=1
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.15KB
|
| HDFS partitions=1/3 files=1 size=3.15KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=3
|
| row-size=204B cardinality=3
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.15KB
|
HDFS partitions=1/3 files=1 size=1.15KB
|
||||||
predicates: id IS DISTINCT FROM 5
|
predicates: id IS DISTINCT FROM 5
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'click'
|
skipped Iceberg predicates: action = 'click'
|
||||||
@@ -279,12 +279,12 @@ MULTI DATA SINK
|
|||||||
|--03:EXCHANGE [DIRECTED]
|
|--03:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 functional_parquet.iceberg_v2_partitioned_position_deletes-position-delete]
|
||||||
| HDFS partitions=1/1 files=1 size=3.15KB
|
| HDFS partitions=1/3 files=1 size=3.15KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=3
|
| row-size=204B cardinality=3
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes]
|
||||||
HDFS partitions=1/1 files=1 size=1.15KB
|
HDFS partitions=1/3 files=1 size=1.15KB
|
||||||
predicates: id IS DISTINCT FROM 5
|
predicates: id IS DISTINCT FROM 5
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
skipped Iceberg predicates: action = 'click'
|
skipped Iceberg predicates: action = 'click'
|
||||||
@@ -332,12 +332,12 @@ MULTI DATA SINK
|
|||||||
| row-size=80B cardinality=10
|
| row-size=80B cardinality=10
|
||||||
|
|
|
|
||||||
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
|--01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
runtime filters: RF000 -> target.id
|
runtime filters: RF000 -> target.id
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=20
|
row-size=80B cardinality=20
|
||||||
@@ -390,12 +390,12 @@ MULTI DATA SINK
|
|||||||
|--09:EXCHANGE [DIRECTED]
|
|--09:EXCHANGE [DIRECTED]
|
||||||
| |
|
| |
|
||||||
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
| 01:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes-POSITION-DELETE-01 target-position-delete]
|
||||||
| HDFS partitions=1/1 files=3 size=9.47KB
|
| HDFS partitions=3/3 files=3 size=9.47KB
|
||||||
| Iceberg snapshot id: 8885697082976537578
|
| Iceberg snapshot id: 8885697082976537578
|
||||||
| row-size=204B cardinality=10
|
| row-size=204B cardinality=10
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
00:SCAN HDFS [functional_parquet.iceberg_v2_partitioned_position_deletes target]
|
||||||
HDFS partitions=1/1 files=3 size=3.48KB
|
HDFS partitions=3/3 files=3 size=3.48KB
|
||||||
runtime filters: RF000 -> target.id
|
runtime filters: RF000 -> target.id
|
||||||
Iceberg snapshot id: 8885697082976537578
|
Iceberg snapshot id: 8885697082976537578
|
||||||
row-size=80B cardinality=20
|
row-size=80B cardinality=20
|
||||||
@@ -454,7 +454,7 @@ MULTI DATA SINK
|
|||||||
| row-size=72B cardinality=1
|
| row-size=72B cardinality=1
|
||||||
|
|
|
|
||||||
01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
||||||
HDFS partitions=1/1 files=20 size=22.90KB
|
HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
runtime filters: RF000 -> source.id
|
runtime filters: RF000 -> source.id
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
row-size=16B cardinality=20
|
row-size=16B cardinality=20
|
||||||
@@ -483,7 +483,7 @@ MULTI DATA SINK
|
|||||||
| row-size=72B cardinality=1
|
| row-size=72B cardinality=1
|
||||||
|
|
|
|
||||||
01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
||||||
HDFS partitions=1/1 files=20 size=22.90KB
|
HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
runtime filters: RF000 -> source.id
|
runtime filters: RF000 -> source.id
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
row-size=16B cardinality=20
|
row-size=16B cardinality=20
|
||||||
@@ -512,7 +512,7 @@ MULTI DATA SINK
|
|||||||
| row-size=72B cardinality=1
|
| row-size=72B cardinality=1
|
||||||
|
|
|
|
||||||
01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
||||||
HDFS partitions=1/1 files=20 size=22.90KB
|
HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
runtime filters: RF000 -> source.id
|
runtime filters: RF000 -> source.id
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
row-size=16B cardinality=20
|
row-size=16B cardinality=20
|
||||||
@@ -541,7 +541,7 @@ MULTI DATA SINK
|
|||||||
| row-size=72B cardinality=1
|
| row-size=72B cardinality=1
|
||||||
|
|
|
|
||||||
01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
01:SCAN HDFS [functional_parquet.iceberg_partitioned source]
|
||||||
HDFS partitions=1/1 files=20 size=22.90KB
|
HDFS partitions=3/3 files=20 size=22.90KB
|
||||||
runtime filters: RF000 -> source.id
|
runtime filters: RF000 -> source.id
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
row-size=16B cardinality=20
|
row-size=16B cardinality=20
|
||||||
|
|||||||
@@ -54,7 +54,7 @@ PLAN-ROOT SINK
|
|||||||
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
|
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=10 size=11.46KB
|
HDFS partitions=3/3 files=10 size=11.46KB
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
stored statistics:
|
stored statistics:
|
||||||
table: rows=20 size=22.90KB
|
table: rows=20 size=22.90KB
|
||||||
@@ -76,7 +76,7 @@ PLAN-ROOT SINK
|
|||||||
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
|
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
|
||||||
|
|
|
|
||||||
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
00:SCAN HDFS [functional_parquet.iceberg_partitioned]
|
||||||
HDFS partitions=1/1 files=4 size=4.57KB
|
HDFS partitions=1/3 files=4 size=4.57KB
|
||||||
predicates: id > CAST(0 AS INT)
|
predicates: id > CAST(0 AS INT)
|
||||||
Iceberg snapshot id: 8270633197658268308
|
Iceberg snapshot id: 8270633197658268308
|
||||||
skipped Iceberg predicates: action = 'click'
|
skipped Iceberg predicates: action = 'click'
|
||||||
|
|||||||
107
testdata/workloads/functional-query/queries/QueryTest/iceberg-partitions.test
vendored
Normal file
107
testdata/workloads/functional-query/queries/QueryTest/iceberg-partitions.test
vendored
Normal file
@@ -0,0 +1,107 @@
|
|||||||
|
====
|
||||||
|
---- QUERY
|
||||||
|
CREATE TABLE ice_num_partitions (
|
||||||
|
id INT,
|
||||||
|
name STRING,
|
||||||
|
value DOUBLE,
|
||||||
|
ts TIMESTAMP,
|
||||||
|
active BOOLEAN
|
||||||
|
)
|
||||||
|
stored by iceberg
|
||||||
|
TBLPROPERTIES ('format-version'='2');
|
||||||
|
====
|
||||||
|
---- QUERY
|
||||||
|
INSERT INTO ice_num_partitions VALUES
|
||||||
|
(1, 'Alice', 10.5, '2023-01-15 10:00:00', TRUE),
|
||||||
|
(2, 'Bob', 20.0, '2023-01-15 11:00:00', FALSE),
|
||||||
|
(3, 'Charlie', 30.2, '2023-01-16 12:00:00', TRUE);
|
||||||
|
====
|
||||||
|
---- QUERY
|
||||||
|
SELECT id, name, value, ts, active FROM ice_num_partitions ORDER BY id;
|
||||||
|
---- TYPES
|
||||||
|
INT, STRING, DOUBLE, TIMESTAMP, BOOLEAN
|
||||||
|
---- RESULTS
|
||||||
|
1,'Alice',10.5,2023-01-15 10:00:00,true
|
||||||
|
2,'Bob',20.0,2023-01-15 11:00:00,false
|
||||||
|
3,'Charlie',30.2,2023-01-16 12:00:00,true
|
||||||
|
---- RUNTIME_PROFILE
|
||||||
|
row_regex: .* partitions=1/1 files=1 size=.*B
|
||||||
|
====
|
||||||
|
---- QUERY
|
||||||
|
ALTER TABLE ice_num_partitions SET PARTITION SPEC(year(ts), bucket(4, id));
|
||||||
|
====
|
||||||
|
---- QUERY
|
||||||
|
INSERT INTO ice_num_partitions VALUES
|
||||||
|
(4, 'David', 40.8, '2024-02-20 13:00:00', FALSE),
|
||||||
|
(5, 'Eve', 50.1, '2024-03-01 14:00:00', TRUE),
|
||||||
|
(6, 'Frank', 60.9, '2024-02-20 15:00:00', TRUE);
|
||||||
|
====
|
||||||
|
---- QUERY
|
||||||
|
SELECT * FROM ice_num_partitions;
|
||||||
|
---- RUNTIME_PROFILE
|
||||||
|
row_regex: .* partitions=4/4 files=4 size=.*B
|
||||||
|
====
|
||||||
|
---- QUERY
|
||||||
|
SELECT id, name FROM ice_num_partitions where ts between '2024-02-20 12:00:00' and '2024-02-20 18:00:00';
|
||||||
|
---- TYPES
|
||||||
|
INT, STRING
|
||||||
|
---- RESULTS
|
||||||
|
6,'Frank'
|
||||||
|
4,'David'
|
||||||
|
---- RUNTIME_PROFILE
|
||||||
|
row_regex: .* partitions=2/4 files=2 size=.*B
|
||||||
|
====
|
||||||
|
---- QUERY
|
||||||
|
ALTER TABLE ice_num_partitions SET PARTITION SPEC(void(ts), bucket(4, id));
|
||||||
|
====
|
||||||
|
---- QUERY
|
||||||
|
INSERT INTO ice_num_partitions VALUES
|
||||||
|
(7, 'Grace', 70.3, '2025-04-05 16:00:00', FALSE),
|
||||||
|
(8, 'Heidi', 80.6, '2025-04-10 17:00:00', TRUE);
|
||||||
|
SELECT * FROM ice_num_partitions;
|
||||||
|
---- RUNTIME_PROFILE
|
||||||
|
row_regex: .* partitions=5/5 files=5 size=.*B
|
||||||
|
====
|
||||||
|
---- QUERY
|
||||||
|
ALTER TABLE ice_num_partitions DROP PARTITION(bucket(4, id)=1);
|
||||||
|
---- RESULTS
|
||||||
|
'Dropped 1 partition(s)'
|
||||||
|
---- TYPES
|
||||||
|
string
|
||||||
|
====
|
||||||
|
---- QUERY
|
||||||
|
# Test that the dropped partition is not in the results.
|
||||||
|
SELECT id, name, value, ts, active FROM ice_num_partitions ORDER BY id;
|
||||||
|
---- TYPES
|
||||||
|
INT, STRING, DOUBLE, TIMESTAMP, BOOLEAN
|
||||||
|
---- RESULTS
|
||||||
|
1,'Alice',10.5,2023-01-15 10:00:00,true
|
||||||
|
2,'Bob',20.0,2023-01-15 11:00:00,false
|
||||||
|
3,'Charlie',30.2,2023-01-16 12:00:00,true
|
||||||
|
4,'David',40.8,2024-02-20 13:00:00,false
|
||||||
|
5,'Eve',50.1,2024-03-01 14:00:00,true
|
||||||
|
7,'Grace',70.3,2025-04-05 16:00:00,false
|
||||||
|
8,'Heidi',80.6,2025-04-10 17:00:00,true
|
||||||
|
---- RUNTIME_PROFILE
|
||||||
|
row_regex: .* partitions=4/4 files=4 size=.*B
|
||||||
|
====
|
||||||
|
---- QUERY
|
||||||
|
show partitions ice_num_partitions;
|
||||||
|
---- TYPES
|
||||||
|
STRING, BIGINT, BIGINT
|
||||||
|
---- RESULTS
|
||||||
|
'{"id_bucket_4":"3","ts_null":null}',2,1
|
||||||
|
'{"ts_year":"54","id_bucket_4":"2"}',1,1
|
||||||
|
'{"ts_year":"54","id_bucket_4":"3"}',1,1
|
||||||
|
'{}',3,1
|
||||||
|
====
|
||||||
|
---- QUERY
|
||||||
|
# The 'ts' column was removed from the partition spec, so no partitions can be pruned based on the predicate.
|
||||||
|
SELECT id, name FROM ice_num_partitions where ts between '2024-02-20 13:00:00' and '2024-02-20 18:00:00';
|
||||||
|
---- TYPES
|
||||||
|
INT, STRING
|
||||||
|
---- RESULTS
|
||||||
|
4,'David'
|
||||||
|
---- RUNTIME_PROFILE
|
||||||
|
row_regex: .* partitions=4/4 files=4 size=.*B
|
||||||
|
====
|
||||||
@@ -2231,6 +2231,19 @@ class TestIcebergV2Table(IcebergTestSuite):
|
|||||||
self.run_test_case('QueryTest/iceberg-predicate-push-down-hint', vector,
|
self.run_test_case('QueryTest/iceberg-predicate-push-down-hint', vector,
|
||||||
use_db=unique_database)
|
use_db=unique_database)
|
||||||
|
|
||||||
|
def test_partitions(self, vector, unique_database):
|
||||||
|
self.run_test_case('QueryTest/iceberg-partitions', vector, unique_database)
|
||||||
|
tbl_name = unique_database + ".ice_num_partitions"
|
||||||
|
snapshots = get_snapshots(self.client, tbl_name, expected_result_size=4)
|
||||||
|
second_snapshot = snapshots[1]
|
||||||
|
time_travel_data = self.execute_query(
|
||||||
|
"SELECT * FROM {0} for system_version as of {1};".format(
|
||||||
|
tbl_name, second_snapshot.get_snapshot_id()))
|
||||||
|
assert "partitions=4/unknown" in time_travel_data.runtime_profile
|
||||||
|
selective_time_travel_data = self.execute_query(
|
||||||
|
"SELECT * FROM {0} for system_version as of {1} WHERE id < 5;".format(
|
||||||
|
tbl_name, second_snapshot.get_snapshot_id()))
|
||||||
|
assert "partitions=2/unknown" in selective_time_travel_data.runtime_profile
|
||||||
|
|
||||||
# Tests to exercise the DIRECTED distribution mode for V2 Iceberg tables. Note, that most
|
# Tests to exercise the DIRECTED distribution mode for V2 Iceberg tables. Note, that most
|
||||||
# of the test coverage is in TestIcebergV2Table.test_read_position_deletes but since it
|
# of the test coverage is in TestIcebergV2Table.test_read_position_deletes but since it
|
||||||
|
|||||||
Reference in New Issue
Block a user