IcebergScanNode.java
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.datasource.iceberg.source;
import org.apache.doris.analysis.Expr;
import org.apache.doris.analysis.FunctionCallExpr;
import org.apache.doris.analysis.TableSnapshot;
import org.apache.doris.analysis.TupleDescriptor;
import org.apache.doris.catalog.Column;
import org.apache.doris.catalog.Env;
import org.apache.doris.catalog.TableIf;
import org.apache.doris.common.DdlException;
import org.apache.doris.common.UserException;
import org.apache.doris.common.security.authentication.PreExecutionAuthenticator;
import org.apache.doris.common.util.LocationPath;
import org.apache.doris.datasource.ExternalTable;
import org.apache.doris.datasource.FileQueryScanNode;
import org.apache.doris.datasource.TableFormatType;
import org.apache.doris.datasource.hive.HMSExternalTable;
import org.apache.doris.datasource.iceberg.IcebergExternalCatalog;
import org.apache.doris.datasource.iceberg.IcebergExternalTable;
import org.apache.doris.datasource.iceberg.IcebergUtils;
import org.apache.doris.nereids.exceptions.NotSupportedException;
import org.apache.doris.planner.PlanNodeId;
import org.apache.doris.qe.SessionVariable;
import org.apache.doris.spi.Split;
import org.apache.doris.statistics.StatisticalType;
import org.apache.doris.thrift.TExplainLevel;
import org.apache.doris.thrift.TFileFormatType;
import org.apache.doris.thrift.TFileRangeDesc;
import org.apache.doris.thrift.TIcebergDeleteFileDesc;
import org.apache.doris.thrift.TIcebergFileDesc;
import org.apache.doris.thrift.TPlanNode;
import org.apache.doris.thrift.TPushAggOp;
import org.apache.doris.thrift.TTableFormatFileDesc;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import org.apache.commons.lang3.exception.ExceptionUtils;
import org.apache.iceberg.BaseTable;
import org.apache.iceberg.DeleteFile;
import org.apache.iceberg.FileContent;
import org.apache.iceberg.FileScanTask;
import org.apache.iceberg.ManifestFile;
import org.apache.iceberg.MetadataColumns;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.StructLike;
import org.apache.iceberg.Table;
import org.apache.iceberg.TableScan;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.CloseableIterator;
import org.apache.iceberg.types.Conversions;
import org.apache.iceberg.util.TableScanUtil;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.OptionalLong;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicReference;
public class IcebergScanNode extends FileQueryScanNode {
public static final int MIN_DELETE_FILE_SUPPORT_VERSION = 2;
private static final Logger LOG = LogManager.getLogger(IcebergScanNode.class);
private IcebergSource source;
private Table icebergTable;
private List<String> pushdownIcebergPredicates = Lists.newArrayList();
// If tableLevelPushDownCount is true, means we can do count push down opt at table level.
// which means all splits have no position/equality delete files,
// so for query like "select count(*) from ice_tbl", we can get count from snapshot row count info directly.
// If tableLevelPushDownCount is false, means we can't do count push down opt at table level,
// But for part of splits which have no position/equality delete files, we can still do count push down opt.
// And for split level count push down opt, the flag is set in each split.
private boolean tableLevelPushDownCount = false;
private long countFromSnapshot;
private static final long COUNT_WITH_PARALLEL_SPLITS = 10000;
private long targetSplitSize;
private ConcurrentHashMap.KeySetView<Object, Boolean> partitionPathSet;
private boolean isPartitionedTable;
private int formatVersion;
private PreExecutionAuthenticator preExecutionAuthenticator;
private TableScan icebergTableScan;
// for test
@VisibleForTesting
public IcebergScanNode(PlanNodeId id, TupleDescriptor desc, SessionVariable sv) {
super(id, desc, "ICEBERG_SCAN_NODE", StatisticalType.ICEBERG_SCAN_NODE, false, sv);
}
/**
* External file scan node for Query iceberg table
* needCheckColumnPriv: Some of ExternalFileScanNode do not need to check column priv
* eg: s3 tvf
* These scan nodes do not have corresponding catalog/database/table info, so no need to do priv check
*/
public IcebergScanNode(PlanNodeId id, TupleDescriptor desc, boolean needCheckColumnPriv, SessionVariable sv) {
super(id, desc, "ICEBERG_SCAN_NODE", StatisticalType.ICEBERG_SCAN_NODE, needCheckColumnPriv, sv);
ExternalTable table = (ExternalTable) desc.getTable();
if (table instanceof HMSExternalTable) {
source = new IcebergHMSSource((HMSExternalTable) table, desc, columnNameToRange);
} else if (table instanceof IcebergExternalTable) {
String catalogType = ((IcebergExternalTable) table).getIcebergCatalogType();
switch (catalogType) {
case IcebergExternalCatalog.ICEBERG_HMS:
case IcebergExternalCatalog.ICEBERG_REST:
case IcebergExternalCatalog.ICEBERG_DLF:
case IcebergExternalCatalog.ICEBERG_GLUE:
case IcebergExternalCatalog.ICEBERG_HADOOP:
case IcebergExternalCatalog.ICEBERG_S3_TABLES:
source = new IcebergApiSource((IcebergExternalTable) table, desc, columnNameToRange);
break;
default:
Preconditions.checkState(false, "Unknown iceberg catalog type: " + catalogType);
break;
}
}
Preconditions.checkNotNull(source);
}
@Override
protected void doInitialize() throws UserException {
icebergTable = source.getIcebergTable();
targetSplitSize = getRealFileSplitSize(0);
partitionPathSet = ConcurrentHashMap.newKeySet();
isPartitionedTable = icebergTable.spec().isPartitioned();
formatVersion = ((BaseTable) icebergTable).operations().current().formatVersion();
preExecutionAuthenticator = source.getCatalog().getPreExecutionAuthenticator();
super.doInitialize();
}
@Override
protected void setScanParams(TFileRangeDesc rangeDesc, Split split) {
if (split instanceof IcebergSplit) {
setIcebergParams(rangeDesc, (IcebergSplit) split);
}
}
private void setIcebergParams(TFileRangeDesc rangeDesc, IcebergSplit icebergSplit) {
TTableFormatFileDesc tableFormatFileDesc = new TTableFormatFileDesc();
tableFormatFileDesc.setTableFormatType(icebergSplit.getTableFormatType().value());
if (tableLevelPushDownCount) {
tableFormatFileDesc.setTableLevelRowCount(icebergSplit.getTableLevelRowCount());
}
TIcebergFileDesc fileDesc = new TIcebergFileDesc();
fileDesc.setFormatVersion(formatVersion);
fileDesc.setOriginalFilePath(icebergSplit.getOriginalPath());
if (formatVersion < MIN_DELETE_FILE_SUPPORT_VERSION) {
fileDesc.setContent(FileContent.DATA.id());
} else {
for (IcebergDeleteFileFilter filter : icebergSplit.getDeleteFileFilters()) {
TIcebergDeleteFileDesc deleteFileDesc = new TIcebergDeleteFileDesc();
String deleteFilePath = filter.getDeleteFilePath();
LocationPath locationPath = new LocationPath(deleteFilePath, icebergSplit.getConfig());
deleteFileDesc.setPath(locationPath.toStorageLocation().toString());
if (filter instanceof IcebergDeleteFileFilter.PositionDelete) {
IcebergDeleteFileFilter.PositionDelete positionDelete =
(IcebergDeleteFileFilter.PositionDelete) filter;
OptionalLong lowerBound = positionDelete.getPositionLowerBound();
OptionalLong upperBound = positionDelete.getPositionUpperBound();
if (lowerBound.isPresent()) {
deleteFileDesc.setPositionLowerBound(lowerBound.getAsLong());
}
if (upperBound.isPresent()) {
deleteFileDesc.setPositionUpperBound(upperBound.getAsLong());
}
deleteFileDesc.setContent(FileContent.POSITION_DELETES.id());
} else {
IcebergDeleteFileFilter.EqualityDelete equalityDelete =
(IcebergDeleteFileFilter.EqualityDelete) filter;
deleteFileDesc.setFieldIds(equalityDelete.getFieldIds());
deleteFileDesc.setContent(FileContent.EQUALITY_DELETES.id());
}
fileDesc.addToDeleteFiles(deleteFileDesc);
}
}
tableFormatFileDesc.setIcebergParams(fileDesc);
rangeDesc.setTableFormatParams(tableFormatFileDesc);
}
@Override
public List<Split> getSplits(int numBackends) throws UserException {
try {
return preExecutionAuthenticator.execute(() -> doGetSplits(numBackends));
} catch (Exception e) {
Optional<NotSupportedException> opt = checkNotSupportedException(e);
if (opt.isPresent()) {
throw opt.get();
} else {
throw new RuntimeException(ExceptionUtils.getRootCauseMessage(e), e);
}
}
}
@Override
public void startSplit(int numBackends) throws UserException {
try {
preExecutionAuthenticator.execute(() -> {
doStartSplit();
return null;
});
} catch (Exception e) {
throw new UserException(e.getMessage(), e);
}
}
public void doStartSplit() {
TableScan scan = createTableScan();
CompletableFuture.runAsync(() -> {
AtomicReference<CloseableIterable<FileScanTask>> taskRef = new AtomicReference<>();
try {
preExecutionAuthenticator.execute(
() -> {
CloseableIterable<FileScanTask> fileScanTasks = planFileScanTask(scan);
taskRef.set(fileScanTasks);
CloseableIterator<FileScanTask> iterator = fileScanTasks.iterator();
while (splitAssignment.needMoreSplit() && iterator.hasNext()) {
try {
splitAssignment.addToQueue(Lists.newArrayList(createIcebergSplit(iterator.next())));
} catch (UserException e) {
throw new RuntimeException(e);
}
}
}
);
splitAssignment.finishSchedule();
} catch (Exception e) {
Optional<NotSupportedException> opt = checkNotSupportedException(e);
if (opt.isPresent()) {
splitAssignment.setException(new UserException(opt.get().getMessage(), opt.get()));
} else {
splitAssignment.setException(new UserException(e.getMessage(), e));
}
} finally {
if (taskRef.get() != null) {
try {
taskRef.get().close();
} catch (IOException e) {
// ignore
}
}
}
}, Env.getCurrentEnv().getExtMetaCacheMgr().getScheduleExecutor());
}
@VisibleForTesting
public TableScan createTableScan() {
if (icebergTableScan != null) {
return icebergTableScan;
}
TableScan scan = icebergTable.newScan();
// set snapshot
Long snapshotId = getSpecifiedSnapshot();
if (snapshotId != null) {
scan = scan.useSnapshot(snapshotId);
}
// set filter
List<Expression> expressions = new ArrayList<>();
for (Expr conjunct : conjuncts) {
Expression expression = IcebergUtils.convertToIcebergExpr(conjunct, icebergTable.schema());
if (expression != null) {
expressions.add(expression);
}
}
for (Expression predicate : expressions) {
scan = scan.filter(predicate);
this.pushdownIcebergPredicates.add(predicate.toString());
}
icebergTableScan = scan.planWith(source.getCatalog().getThreadPoolWithPreAuth());
return icebergTableScan;
}
private CloseableIterable<FileScanTask> planFileScanTask(TableScan scan) {
long targetSplitSize = getRealFileSplitSize(0);
return TableScanUtil.splitFiles(scan.planFiles(), targetSplitSize);
}
private Split createIcebergSplit(FileScanTask fileScanTask) {
if (isPartitionedTable) {
StructLike structLike = fileScanTask.file().partition();
// Counts the number of partitions read
partitionPathSet.add(structLike.toString());
}
String originalPath = fileScanTask.file().path().toString();
LocationPath locationPath = new LocationPath(originalPath, source.getCatalog().getProperties());
IcebergSplit split = new IcebergSplit(
locationPath,
fileScanTask.start(),
fileScanTask.length(),
fileScanTask.file().fileSizeInBytes(),
new String[0],
formatVersion,
source.getCatalog().getProperties(),
new ArrayList<>(),
originalPath);
if (!fileScanTask.deletes().isEmpty()) {
split.setDeleteFileFilters(getDeleteFileFilters(fileScanTask));
}
split.setTableFormatType(TableFormatType.ICEBERG);
split.setTargetSplitSize(targetSplitSize);
return split;
}
private List<Split> doGetSplits(int numBackends) throws UserException {
TableScan scan = createTableScan();
List<Split> splits = new ArrayList<>();
try (CloseableIterable<FileScanTask> fileScanTasks = planFileScanTask(scan)) {
if (tableLevelPushDownCount) {
int needSplitCnt = countFromSnapshot < COUNT_WITH_PARALLEL_SPLITS
? 1 : sessionVariable.getParallelExecInstanceNum() * numBackends;
for (FileScanTask next : fileScanTasks) {
splits.add(createIcebergSplit(next));
if (splits.size() >= needSplitCnt) {
break;
}
}
setPushDownCount(countFromSnapshot);
assignCountToSplits(splits, countFromSnapshot);
return splits;
} else {
fileScanTasks.forEach(taskGrp -> splits.add(createIcebergSplit(taskGrp)));
}
} catch (IOException e) {
throw new UserException(e.getMessage(), e.getCause());
}
selectedPartitionNum = partitionPathSet.size();
return splits;
}
@Override
public boolean isBatchMode() {
TPushAggOp aggOp = getPushDownAggNoGroupingOp();
if (aggOp.equals(TPushAggOp.COUNT)) {
countFromSnapshot = getCountFromSnapshot();
if (countFromSnapshot >= 0) {
tableLevelPushDownCount = true;
return false;
}
}
if (createTableScan().snapshot() == null) {
return false;
}
if (!sessionVariable.getEnableExternalTableBatchMode()) {
return false;
}
try {
return preExecutionAuthenticator.execute(() -> {
try (CloseableIterator<ManifestFile> matchingManifest =
IcebergUtils.getMatchingManifest(
createTableScan().snapshot().dataManifests(icebergTable.io()),
icebergTable.specs(),
createTableScan().filter()).iterator()) {
int cnt = 0;
while (matchingManifest.hasNext()) {
ManifestFile next = matchingManifest.next();
cnt += next.addedFilesCount() + next.existingFilesCount();
if (cnt >= sessionVariable.getNumFilesInBatchMode()) {
return true;
}
}
}
return false;
});
} catch (Exception e) {
Optional<NotSupportedException> opt = checkNotSupportedException(e);
if (opt.isPresent()) {
throw opt.get();
} else {
throw new RuntimeException(ExceptionUtils.getRootCauseMessage(e), e);
}
}
}
public Long getSpecifiedSnapshot() {
TableSnapshot tableSnapshot = getQueryTableSnapshot();
if (tableSnapshot != null) {
return IcebergUtils.getQuerySpecSnapshot(icebergTable, tableSnapshot);
}
return null;
}
private List<IcebergDeleteFileFilter> getDeleteFileFilters(FileScanTask spitTask) {
List<IcebergDeleteFileFilter> filters = new ArrayList<>();
for (DeleteFile delete : spitTask.deletes()) {
if (delete.content() == FileContent.POSITION_DELETES) {
Optional<Long> positionLowerBound = Optional.ofNullable(delete.lowerBounds())
.map(m -> m.get(MetadataColumns.DELETE_FILE_POS.fieldId()))
.map(bytes -> Conversions.fromByteBuffer(MetadataColumns.DELETE_FILE_POS.type(), bytes));
Optional<Long> positionUpperBound = Optional.ofNullable(delete.upperBounds())
.map(m -> m.get(MetadataColumns.DELETE_FILE_POS.fieldId()))
.map(bytes -> Conversions.fromByteBuffer(MetadataColumns.DELETE_FILE_POS.type(), bytes));
filters.add(IcebergDeleteFileFilter.createPositionDelete(delete.path().toString(),
positionLowerBound.orElse(-1L), positionUpperBound.orElse(-1L),
delete.fileSizeInBytes()));
} else if (delete.content() == FileContent.EQUALITY_DELETES) {
filters.add(IcebergDeleteFileFilter.createEqualityDelete(
delete.path().toString(), delete.equalityFieldIds(), delete.fileSizeInBytes()));
} else {
throw new IllegalStateException("Unknown delete content: " + delete.content());
}
}
return filters;
}
@Override
public TFileFormatType getFileFormatType() throws UserException {
TFileFormatType type;
String icebergFormat = source.getFileFormat();
if (icebergFormat.equalsIgnoreCase("parquet")) {
type = TFileFormatType.FORMAT_PARQUET;
} else if (icebergFormat.equalsIgnoreCase("orc")) {
type = TFileFormatType.FORMAT_ORC;
} else {
throw new DdlException(String.format("Unsupported format name: %s for iceberg table.", icebergFormat));
}
return type;
}
@Override
public List<String> getPathPartitionKeys() throws UserException {
// return icebergTable.spec().fields().stream().map(PartitionField::name).map(String::toLowerCase)
// .collect(Collectors.toList());
/**First, iceberg partition columns are based on existing fields, which will be stored in the actual data file.
* Second, iceberg partition columns support Partition transforms. In this case, the path partition key is not
* equal to the column name of the partition column, so remove this code and get all the columns you want to
* read from the file.
* Related code:
* be/src/vec/exec/scan/vfile_scanner.cpp:
* VFileScanner::_init_expr_ctxes()
* if (slot_info.is_file_slot) {
* xxxx
* }
*/
return new ArrayList<>();
}
@Override
public TableIf getTargetTable() {
return source.getTargetTable();
}
@Override
public Map<String, String> getLocationProperties() throws UserException {
return source.getCatalog().getCatalogProperty().getHadoopProperties();
}
@Override
public boolean pushDownAggNoGrouping(FunctionCallExpr aggExpr) {
String aggFunctionName = aggExpr.getFnName().getFunction().toUpperCase();
return "COUNT".equals(aggFunctionName);
}
@Override
public boolean pushDownAggNoGroupingCheckCol(FunctionCallExpr aggExpr, Column col) {
return !col.isAllowNull();
}
@VisibleForTesting
public long getCountFromSnapshot() {
Long specifiedSnapshot = getSpecifiedSnapshot();
Snapshot snapshot = specifiedSnapshot == null
? icebergTable.currentSnapshot() : icebergTable.snapshot(specifiedSnapshot);
// empty table
if (snapshot == null) {
return 0;
}
// `TOTAL_POSITION_DELETES` is need to 0,
// because prevent 'dangling delete' problem after `rewrite_data_files`
// ref: https://iceberg.apache.org/docs/nightly/spark-procedures/#rewrite_position_delete_files
Map<String, String> summary = snapshot.summary();
if (!summary.get(IcebergUtils.TOTAL_EQUALITY_DELETES).equals("0")
|| !summary.get(IcebergUtils.TOTAL_POSITION_DELETES).equals("0")) {
return -1;
}
return Long.parseLong(summary.get(IcebergUtils.TOTAL_RECORDS));
}
@Override
protected void toThrift(TPlanNode planNode) {
super.toThrift(planNode);
}
@Override
public String getNodeExplainString(String prefix, TExplainLevel detailLevel) {
if (pushdownIcebergPredicates.isEmpty()) {
return super.getNodeExplainString(prefix, detailLevel);
}
StringBuilder sb = new StringBuilder();
for (String predicate : pushdownIcebergPredicates) {
sb.append(prefix).append(prefix).append(predicate).append("\n");
}
return super.getNodeExplainString(prefix, detailLevel)
+ String.format("%sicebergPredicatePushdown=\n%s\n", prefix, sb);
}
private void assignCountToSplits(List<Split> splits, long totalCount) {
int size = splits.size();
long countPerSplit = totalCount / size;
for (int i = 0; i < size - 1; i++) {
((IcebergSplit) splits.get(i)).setTableLevelRowCount(countPerSplit);
}
((IcebergSplit) splits.get(size - 1)).setTableLevelRowCount(countPerSplit + totalCount % size);
}
@Override
public int numApproximateSplits() {
return NUM_SPLITS_PER_PARTITION * partitionPathSet.size() > 0 ? partitionPathSet.size() : 1;
}
private Optional<NotSupportedException> checkNotSupportedException(Exception e) {
if (e instanceof NullPointerException) {
/*
Caused by: java.lang.NullPointerException: Type cannot be null
at org.apache.iceberg.relocated.com.google.common.base.Preconditions.checkNotNull
(Preconditions.java:921) ~[iceberg-bundled-guava-1.4.3.jar:?]
at org.apache.iceberg.types.Types$NestedField.<init>(Types.java:447) ~[iceberg-api-1.4.3.jar:?]
at org.apache.iceberg.types.Types$NestedField.optional(Types.java:416) ~[iceberg-api-1.4.3.jar:?]
at org.apache.iceberg.PartitionSpec.partitionType(PartitionSpec.java:132) ~[iceberg-api-1.4.3.jar:?]
at org.apache.iceberg.DeleteFileIndex.lambda$new$0(DeleteFileIndex.java:97) ~[iceberg-core-1.4.3.jar:?]
at org.apache.iceberg.relocated.com.google.common.collect.RegularImmutableMap.forEach
(RegularImmutableMap.java:297) ~[iceberg-bundled-guava-1.4.3.jar:?]
at org.apache.iceberg.DeleteFileIndex.<init>(DeleteFileIndex.java:97) ~[iceberg-core-1.4.3.jar:?]
at org.apache.iceberg.DeleteFileIndex.<init>(DeleteFileIndex.java:71) ~[iceberg-core-1.4.3.jar:?]
at org.apache.iceberg.DeleteFileIndex$Builder.build(DeleteFileIndex.java:578) ~[iceberg-core-1.4.3.jar:?]
at org.apache.iceberg.ManifestGroup.plan(ManifestGroup.java:183) ~[iceberg-core-1.4.3.jar:?]
at org.apache.iceberg.ManifestGroup.planFiles(ManifestGroup.java:170) ~[iceberg-core-1.4.3.jar:?]
at org.apache.iceberg.DataTableScan.doPlanFiles(DataTableScan.java:89) ~[iceberg-core-1.4.3.jar:?]
at org.apache.iceberg.SnapshotScan.planFiles(SnapshotScan.java:139) ~[iceberg-core-1.4.3.jar:?]
at org.apache.doris.datasource.iceberg.source.IcebergScanNode.doGetSplits
(IcebergScanNode.java:209) ~[doris-fe.jar:1.2-SNAPSHOT]
EXAMPLE:
CREATE TABLE iceberg_tb(col1 INT,col2 STRING) USING ICEBERG PARTITIONED BY (bucket(10,col2));
INSERT INTO iceberg_tb VALUES( ... );
ALTER TABLE iceberg_tb DROP PARTITION FIELD bucket(10,col2);
ALTER TABLE iceberg_tb DROP COLUMNS col2 STRING;
Link: https://github.com/apache/iceberg/pull/10755
*/
LOG.warn("Iceberg TableScanUtil.splitFiles throw NullPointerException. Cause : ", e);
return Optional.of(
new NotSupportedException("Unable to read Iceberg table with dropped old partition column."));
}
return Optional.empty();
}
}