From 9f31a39d1aa4551d081d89124658b24ea54b6f63 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Mon, 9 Dec 2024 02:30:36 +0800 Subject: [PATCH 01/14] [enhance](auth)When authorization includes create, not check if resources exist (#45125) ### What problem does this PR solve? Issue Number: close #xxx Related PR: #39597 Problem Summary: When authorization includes create, not check if resources exist --- .../main/java/org/apache/doris/mysql/privilege/Auth.java | 8 ++++++-- .../suites/auth_p0/test_grant_nonexist_table.groovy | 4 +++- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java index 8a13c6dd125548c..a1fb57d01cafa41 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/Auth.java @@ -686,7 +686,7 @@ private void grantInternal(UserIdentity userIdent, String role, TablePattern tbl writeLock(); try { if (!isReplay) { - checkTablePatternExist(tblPattern); + checkTablePatternExist(tblPattern, privs); } if (role == null) { if (!doesUserExist(userIdent)) { @@ -706,8 +706,12 @@ private void grantInternal(UserIdentity userIdent, String role, TablePattern tbl } } - private void checkTablePatternExist(TablePattern tablePattern) throws DdlException { + private void checkTablePatternExist(TablePattern tablePattern, PrivBitSet privs) throws DdlException { Objects.requireNonNull(tablePattern, "tablePattern can not be null"); + Objects.requireNonNull(privs, "privs can not be null"); + if (privs.containsPrivs(Privilege.CREATE_PRIV)) { + return; + } PrivLevel privLevel = tablePattern.getPrivLevel(); if (privLevel == PrivLevel.GLOBAL) { return; diff --git a/regression-test/suites/auth_p0/test_grant_nonexist_table.groovy b/regression-test/suites/auth_p0/test_grant_nonexist_table.groovy index 36e75707be72529..74d211e50102018 100644 --- a/regression-test/suites/auth_p0/test_grant_nonexist_table.groovy +++ b/regression-test/suites/auth_p0/test_grant_nonexist_table.groovy @@ -39,7 +39,9 @@ suite("test_grant_nonexist_table","p0,auth") { sql """grant select_priv on internal.${dbName}.non_exist_table to ${user}""" exception "table" } - + // contain create_triv should not check name, Same behavior as MySQL + sql """grant create_priv on internal.${dbName}.non_exist_table to ${user}""" + sql """grant create_priv,select_priv on internal.${dbName}.non_exist_table to ${user}""" try_sql("DROP USER ${user}") } From 4cf908c43c60abc6652c3bd180ae9397d236718d Mon Sep 17 00:00:00 2001 From: daidai Date: Mon, 9 Dec 2024 08:26:06 +0800 Subject: [PATCH 02/14] [enchement](mc)Optimize reading of maxcompute partition tables. (#45148) ### What problem does this PR solve? Problem Summary: Optimize reading of maxcompute partition tables: 1. Introduce batch mode to generate splits for Maxcompute partition tables to optimize scenarios with a large number of partitions. Control it through the variable `num_partitions_in_batch_mode`. 2. Introduce catalog parameter `mc.split_cross_partition`. The parameter is true, which is more friendly to reading partition tables, and false, which is more friendly to debug. 3. Add `-Darrow.enable_null_check_for_get=false` to be jvm to improve the efficiency of mc arrow data conversion. --- conf/be.conf | 2 +- .../maxcompute/MaxComputeExternalCatalog.java | 8 +- .../maxcompute/source/MaxComputeScanNode.java | 276 ++-- .../property/constants/MCProperties.java | 7 + .../test_max_compute_partition_prune.out | 1364 +++++++++++++++++ .../test_max_compute_partition_prune.groovy | 356 ++--- 6 files changed, 1747 insertions(+), 266 deletions(-) diff --git a/conf/be.conf b/conf/be.conf index ae9524313d860af..ddbc385522e848d 100644 --- a/conf/be.conf +++ b/conf/be.conf @@ -24,7 +24,7 @@ LOG_DIR="${DORIS_HOME}/log/" JAVA_OPTS="-Dfile.encoding=UTF-8 -Xmx2048m -DlogPath=$LOG_DIR/jni.log -Xloggc:$LOG_DIR/be.gc.log.$CUR_DATE -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=50M -Djavax.security.auth.useSubjectCredsOnly=false -Dsun.security.krb5.debug=true -Dsun.java.command=DorisBE -XX:-CriticalJNINatives" # For jdk 17, this JAVA_OPTS will be used as default JVM options -JAVA_OPTS_FOR_JDK_17="-Dfile.encoding=UTF-8 -Djol.skipHotspotSAAttach=true -Xmx2048m -DlogPath=$LOG_DIR/jni.log -Xlog:gc*:$LOG_DIR/be.gc.log.$CUR_DATE:time,uptime:filecount=10,filesize=50M -Djavax.security.auth.useSubjectCredsOnly=false -Dsun.security.krb5.debug=true -Dsun.java.command=DorisBE -XX:-CriticalJNINatives -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED --add-opens=java.management/sun.management=ALL-UNNAMED" +JAVA_OPTS_FOR_JDK_17="-Dfile.encoding=UTF-8 -Djol.skipHotspotSAAttach=true -Xmx2048m -DlogPath=$LOG_DIR/jni.log -Xlog:gc*:$LOG_DIR/be.gc.log.$CUR_DATE:time,uptime:filecount=10,filesize=50M -Djavax.security.auth.useSubjectCredsOnly=false -Dsun.security.krb5.debug=true -Dsun.java.command=DorisBE -XX:-CriticalJNINatives -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED --add-opens=java.management/sun.management=ALL-UNNAMED -Darrow.enable_null_check_for_get=false" # Set your own JAVA_HOME # JAVA_HOME=/path/to/jdk/ diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalCatalog.java index 06c1e55dcf6f4b2..27e4976aedd85e2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalCatalog.java @@ -164,22 +164,24 @@ protected void initLocalObjectsImpl() { defaultProject = props.get(MCProperties.PROJECT); quota = props.getOrDefault(MCProperties.QUOTA, MCProperties.DEFAULT_QUOTA); + boolean splitCrossPartition = + Boolean.parseBoolean(props.getOrDefault(MCProperties.SPLIT_CROSS_PARTITION, + MCProperties.DEFAULT_SPLIT_CROSS_PARTITION)); splitStrategy = props.getOrDefault(MCProperties.SPLIT_STRATEGY, MCProperties.DEFAULT_SPLIT_STRATEGY); if (splitStrategy.equals(MCProperties.SPLIT_BY_BYTE_SIZE_STRATEGY)) { splitByteSize = Long.parseLong(props.getOrDefault(MCProperties.SPLIT_BYTE_SIZE, MCProperties.DEFAULT_SPLIT_BYTE_SIZE)); - splitOptions = SplitOptions.newBuilder() .SplitByByteSize(splitByteSize) - .withCrossPartition(false) + .withCrossPartition(splitCrossPartition) .build(); } else { splitRowCount = Long.parseLong(props.getOrDefault(MCProperties.SPLIT_ROW_COUNT, MCProperties.DEFAULT_SPLIT_ROW_COUNT)); splitOptions = SplitOptions.newBuilder() .SplitByRowOffset() - .withCrossPartition(false) + .withCrossPartition(splitCrossPartition) .build(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java index 4ad971a5c647890..9fa22a0fffaab8b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java @@ -29,6 +29,7 @@ import org.apache.doris.analysis.SlotRef; import org.apache.doris.analysis.TupleDescriptor; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.TableIf; import org.apache.doris.common.AnalysisException; @@ -43,6 +44,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan.SelectedPartitions; import org.apache.doris.nereids.util.DateUtils; import org.apache.doris.planner.PlanNodeId; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.spi.Split; import org.apache.doris.statistics.StatisticalType; import org.apache.doris.thrift.TFileFormatType; @@ -79,15 +81,18 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; public class MaxComputeScanNode extends FileQueryScanNode { private final MaxComputeExternalTable table; - private TableBatchReadSession tableBatchReadSession; private Predicate filterPredicate; - private static final LocationPath ROW_OFFSET_PATH = new LocationPath("/row_offset", Maps.newHashMap()); - private static final LocationPath BYTE_SIZE_PATH = new LocationPath("/byte_size", Maps.newHashMap()); + List requiredPartitionColumns = new ArrayList<>(); + List orderedRequiredDataColumns = new ArrayList<>(); private int connectTimeout; private int readTimeout; @@ -96,6 +101,10 @@ public class MaxComputeScanNode extends FileQueryScanNode { @Setter private SelectedPartitions selectedPartitions = null; + private static final LocationPath ROW_OFFSET_PATH = new LocationPath("/row_offset", Maps.newHashMap()); + private static final LocationPath BYTE_SIZE_PATH = new LocationPath("/byte_size", Maps.newHashMap()); + + // For new planner public MaxComputeScanNode(PlanNodeId id, TupleDescriptor desc, SelectedPartitions selectedPartitions, boolean needCheckColumnPriv) { @@ -143,33 +152,17 @@ private void setScanParams(TFileRangeDesc rangeDesc, MaxComputeSplit maxComputeS rangeDesc.setSize(maxComputeSplit.getLength()); } - // Return false if no need to read any partition data. - // Return true if need to read partition data. - boolean createTableBatchReadSession() throws UserException { - List requiredPartitionColumns = new ArrayList<>(); - List orderedRequiredDataColumns = new ArrayList<>(); - - List requiredPartitionSpecs = new ArrayList<>(); - //if requiredPartitionSpecs is empty, get all partition data. - if (!table.getPartitionColumns().isEmpty() && selectedPartitions != SelectedPartitions.NOT_PRUNED) { - this.totalPartitionNum = selectedPartitions.totalPartitionNum; - this.selectedPartitionNum = selectedPartitions.selectedPartitions.size(); - - if (selectedPartitions.selectedPartitions.isEmpty()) { - //no need read any partition data. - return false; - } - selectedPartitions.selectedPartitions.forEach( - (key, value) -> requiredPartitionSpecs.add(new PartitionSpec(key)) - ); - } + private void createRequiredColumns() { Set requiredSlots = desc.getSlots().stream().map(e -> e.getColumn().getName()).collect(Collectors.toSet()); Set partitionColumns = table.getPartitionColumns().stream().map(Column::getName).collect(Collectors.toSet()); + requiredPartitionColumns.clear(); + orderedRequiredDataColumns.clear(); + for (Column column : table.getColumns()) { String columnName = column.getName(); if (!requiredSlots.contains(columnName)) { @@ -181,32 +174,118 @@ boolean createTableBatchReadSession() throws UserException { orderedRequiredDataColumns.add(columnName); } } + } + /** + * For no partition table: request requiredPartitionSpecs is empty + * For partition table: if requiredPartitionSpecs is empty, get all partition data. + */ + TableBatchReadSession createTableBatchReadSession(List requiredPartitionSpecs) throws IOException { + MaxComputeExternalCatalog mcCatalog = (MaxComputeExternalCatalog) table.getCatalog(); + readTimeout = mcCatalog.getReadTimeout(); + connectTimeout = mcCatalog.getConnectTimeout(); + retryTimes = mcCatalog.getRetryTimes(); + + TableReadSessionBuilder scanBuilder = new TableReadSessionBuilder(); + return scanBuilder.identifier(TableIdentifier.of(table.getDbName(), table.getName())) + .withSettings(mcCatalog.getSettings()) + .withSplitOptions(mcCatalog.getSplitOption()) + .requiredPartitionColumns(requiredPartitionColumns) + .requiredDataColumns(orderedRequiredDataColumns) + .withFilterPredicate(filterPredicate) + .requiredPartitions(requiredPartitionSpecs) + .withArrowOptions( + ArrowOptions.newBuilder() + .withDatetimeUnit(TimestampUnit.MILLI) + .withTimestampUnit(TimestampUnit.NANO) + .build() + ).buildBatchReadSession(); + } - MaxComputeExternalCatalog mcCatalog = (MaxComputeExternalCatalog) table.getCatalog(); + @Override + public boolean isBatchMode() { + if (table.getPartitionColumns().isEmpty()) { + return false; + } - try { - TableReadSessionBuilder scanBuilder = new TableReadSessionBuilder(); - tableBatchReadSession = - scanBuilder.identifier(TableIdentifier.of(table.getDbName(), table.getName())) - .withSettings(mcCatalog.getSettings()) - .withSplitOptions(mcCatalog.getSplitOption()) - .requiredPartitionColumns(requiredPartitionColumns) - .requiredPartitions(requiredPartitionSpecs) - .requiredDataColumns(orderedRequiredDataColumns) - .withArrowOptions( - ArrowOptions.newBuilder() - .withDatetimeUnit(TimestampUnit.MILLI) - .withTimestampUnit(TimestampUnit.NANO) - .build() - ) - .withFilterPredicate(filterPredicate) - .buildBatchReadSession(); - } catch (java.io.IOException e) { - throw new RuntimeException(e); + com.aliyun.odps.Table odpsTable = table.getOdpsTable(); + if (desc.getSlots().isEmpty() || odpsTable.getFileNum() <= 0) { + return false; } - return true; + + int numPartitions = ConnectContext.get().getSessionVariable().getNumPartitionsInBatchMode(); + return numPartitions > 0 + && selectedPartitions != SelectedPartitions.NOT_PRUNED + && selectedPartitions.selectedPartitions.size() >= numPartitions; + } + + @Override + public int numApproximateSplits() { + return selectedPartitions.selectedPartitions.size(); + } + + @Override + public void startSplit() { + this.totalPartitionNum = selectedPartitions.totalPartitionNum; + this.selectedPartitionNum = selectedPartitions.selectedPartitions.size(); + + if (selectedPartitions.selectedPartitions.isEmpty()) { + //no need read any partition data. + return; + } + + createRequiredColumns(); + List requiredPartitionSpecs = new ArrayList<>(); + selectedPartitions.selectedPartitions.forEach( + (key, value) -> requiredPartitionSpecs.add(new PartitionSpec(key)) + ); + + + int batchNumPartitions = ConnectContext.get().getSessionVariable().getNumPartitionsInBatchMode(); + + Executor scheduleExecutor = Env.getCurrentEnv().getExtMetaCacheMgr().getScheduleExecutor(); + AtomicReference batchException = new AtomicReference<>(null); + AtomicInteger numFinishedPartitions = new AtomicInteger(0); + + CompletableFuture.runAsync(() -> { + for (int beginIndex = 0; beginIndex < requiredPartitionSpecs.size(); beginIndex += batchNumPartitions) { + int endIndex = Math.min(beginIndex + batchNumPartitions, requiredPartitionSpecs.size()); + if (batchException.get() != null || splitAssignment.isStop()) { + break; + } + List requiredBatchPartitionSpecs = requiredPartitionSpecs.subList(beginIndex, endIndex); + int curBatchSize = endIndex - beginIndex; + + try { + CompletableFuture.runAsync(() -> { + try { + TableBatchReadSession tableBatchReadSession = + createTableBatchReadSession(requiredBatchPartitionSpecs); + List batchSplit = getSplitByTableSession(tableBatchReadSession); + + splitAssignment.addToQueue(batchSplit); + } catch (IOException e) { + batchException.set(new UserException(e.getMessage(), e)); + } finally { + if (batchException.get() != null) { + splitAssignment.setException(batchException.get()); + } + + if (numFinishedPartitions.addAndGet(curBatchSize) == requiredPartitionSpecs.size()) { + splitAssignment.finishSchedule(); + } + } + }, scheduleExecutor); + } catch (Exception e) { + batchException.set(new UserException(e.getMessage(), e)); + } + + if (batchException.get() != null) { + splitAssignment.setException(batchException.get()); + } + } + }); } @Override @@ -467,67 +546,84 @@ protected Map getLocationProperties() throws UserException { return new HashMap<>(); } - @Override - public List getSplits() throws UserException { + List getSplitByTableSession(TableBatchReadSession tableBatchReadSession) throws java.io.IOException { List result = new ArrayList<>(); - com.aliyun.odps.Table odpsTable = table.getOdpsTable(); - if (desc.getSlots().isEmpty() || odpsTable.getFileNum() <= 0) { - return result; - } + String scanSessionSerialize = serializeSession(tableBatchReadSession); + InputSplitAssigner assigner = tableBatchReadSession.getInputSplitAssigner(); + long modificationTime = table.getOdpsTable().getLastDataModifiedTime().getTime(); - if (!createTableBatchReadSession()) { - return result; - } + MaxComputeExternalCatalog mcCatalog = (MaxComputeExternalCatalog) table.getCatalog(); - try { - String scanSessionSerialize = serializeSession(tableBatchReadSession); - InputSplitAssigner assigner = tableBatchReadSession.getInputSplitAssigner(); - long modificationTime = table.getOdpsTable().getLastDataModifiedTime().getTime(); + if (mcCatalog.getSplitStrategy().equals(MCProperties.SPLIT_BY_BYTE_SIZE_STRATEGY)) { - MaxComputeExternalCatalog mcCatalog = (MaxComputeExternalCatalog) table.getCatalog(); + for (com.aliyun.odps.table.read.split.InputSplit split : assigner.getAllSplits()) { + MaxComputeSplit maxComputeSplit = + new MaxComputeSplit(BYTE_SIZE_PATH, + ((IndexedInputSplit) split).getSplitIndex(), -1, + mcCatalog.getSplitByteSize(), + modificationTime, null, + Collections.emptyList()); - readTimeout = mcCatalog.getReadTimeout(); - connectTimeout = mcCatalog.getConnectTimeout(); - retryTimes = mcCatalog.getRetryTimes(); - if (mcCatalog.getSplitStrategy().equals(MCProperties.SPLIT_BY_BYTE_SIZE_STRATEGY)) { + maxComputeSplit.scanSerialize = scanSessionSerialize; + maxComputeSplit.splitType = SplitType.BYTE_SIZE; + maxComputeSplit.sessionId = split.getSessionId(); - for (com.aliyun.odps.table.read.split.InputSplit split : assigner.getAllSplits()) { - MaxComputeSplit maxComputeSplit = - new MaxComputeSplit(BYTE_SIZE_PATH, - ((IndexedInputSplit) split).getSplitIndex(), -1, - mcCatalog.getSplitByteSize(), - modificationTime, null, - Collections.emptyList()); + result.add(maxComputeSplit); + } + } else { + long totalRowCount = assigner.getTotalRowCount(); + long recordsPerSplit = mcCatalog.getSplitRowCount(); + for (long offset = 0; offset < totalRowCount; offset += recordsPerSplit) { + recordsPerSplit = Math.min(recordsPerSplit, totalRowCount - offset); + com.aliyun.odps.table.read.split.InputSplit split = + assigner.getSplitByRowOffset(offset, recordsPerSplit); - maxComputeSplit.scanSerialize = scanSessionSerialize; - maxComputeSplit.splitType = SplitType.BYTE_SIZE; - maxComputeSplit.sessionId = split.getSessionId(); + MaxComputeSplit maxComputeSplit = + new MaxComputeSplit(ROW_OFFSET_PATH, + offset, recordsPerSplit, totalRowCount, modificationTime, null, + Collections.emptyList()); - result.add(maxComputeSplit); - } - } else { - long totalRowCount = assigner.getTotalRowCount(); + maxComputeSplit.scanSerialize = scanSessionSerialize; + maxComputeSplit.splitType = SplitType.ROW_OFFSET; + maxComputeSplit.sessionId = split.getSessionId(); + + result.add(maxComputeSplit); + } + } + return result; + } - long recordsPerSplit = mcCatalog.getSplitRowCount(); - for (long offset = 0; offset < totalRowCount; offset += recordsPerSplit) { - recordsPerSplit = Math.min(recordsPerSplit, totalRowCount - offset); - com.aliyun.odps.table.read.split.InputSplit split = - assigner.getSplitByRowOffset(offset, recordsPerSplit); - MaxComputeSplit maxComputeSplit = - new MaxComputeSplit(ROW_OFFSET_PATH, - offset, recordsPerSplit, totalRowCount, modificationTime, null, - Collections.emptyList()); + @Override + public List getSplits() throws UserException { + List result = new ArrayList<>(); + com.aliyun.odps.Table odpsTable = table.getOdpsTable(); + if (desc.getSlots().isEmpty() || odpsTable.getFileNum() <= 0) { + return result; + } - maxComputeSplit.scanSerialize = scanSessionSerialize; - maxComputeSplit.splitType = SplitType.ROW_OFFSET; - maxComputeSplit.sessionId = split.getSessionId(); + createRequiredColumns(); - result.add(maxComputeSplit); - } + List requiredPartitionSpecs = new ArrayList<>(); + //if requiredPartitionSpecs is empty, get all partition data. + if (!table.getPartitionColumns().isEmpty() && selectedPartitions != SelectedPartitions.NOT_PRUNED) { + this.totalPartitionNum = selectedPartitions.totalPartitionNum; + this.selectedPartitionNum = selectedPartitions.selectedPartitions.size(); + + if (selectedPartitions.selectedPartitions.isEmpty()) { + //no need read any partition data. + return result; } + selectedPartitions.selectedPartitions.forEach( + (key, value) -> requiredPartitionSpecs.add(new PartitionSpec(key)) + ); + } + + try { + TableBatchReadSession tableBatchReadSession = createTableBatchReadSession(requiredPartitionSpecs); + result = getSplitByTableSession(tableBatchReadSession); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/MCProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/MCProperties.java index efbd01c14777de2..3957ffc91bb4b48 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/MCProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/MCProperties.java @@ -64,6 +64,13 @@ public class MCProperties extends BaseProperties { public static final String DEFAULT_READ_TIMEOUT = "120"; // 120s public static final String DEFAULT_RETRY_COUNT = "4"; // 4 times + //withCrossPartition(true): + // Very friendly to scenarios where there are many partitions but each partition is very small. + //withCrossPartition(false): + // Very debug friendly. + public static final String SPLIT_CROSS_PARTITION = "mc.split_cross_partition"; + public static final String DEFAULT_SPLIT_CROSS_PARTITION = "true"; + public static CloudCredential getCredential(Map props) { return getCloudCredential(props, ACCESS_KEY, SECRET_KEY, SESSION_TOKEN); } diff --git a/regression-test/data/external_table_p2/maxcompute/test_max_compute_partition_prune.out b/regression-test/data/external_table_p2/maxcompute/test_max_compute_partition_prune.out index 8f443829c4c3943..d6c23e6b5abf523 100644 --- a/regression-test/data/external_table_p2/maxcompute/test_max_compute_partition_prune.out +++ b/regression-test/data/external_table_p2/maxcompute/test_max_compute_partition_prune.out @@ -123,3 +123,1367 @@ -- !three_partition_11_0 -- +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + +-- !one_partition_1_1 -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 + +-- !one_partition_2_1 -- +4 David 2025 +5 Eva 2025 + +-- !one_partition_3_all -- +1 Alice 2024 +2 Bob 2024 +3 Charlie 2024 +4 David 2025 +5 Eva 2025 + +-- !one_partition_4_all -- +5 Eva 2025 + +-- !one_partition_5_1 -- +3 Charlie 2024 + +-- !two_partition_1_1 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 + +-- !two_partition_2_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_3_2 -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 + +-- !two_partition_4_all -- +1 Alice US 1 +2 Bob US 1 +3 Charlie US 1 +4 David US 2 +5 Eva US 2 +6 Frank EU 1 +7 Grace EU 1 +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !two_partition_5_1 -- + +-- !two_partition_6_1 -- +8 Hannah EU 2 +9 Ivy EU 2 +10 Jack EU 2 + +-- !three_partition_1_1 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 + +-- !three_partition_2_1 -- +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 + +-- !three_partition_3_3 -- +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_4_2 -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +6 Frank US 2025 Q1 + +-- !three_partition_5_all -- +1 Alice US 2024 Q1 +2 Bob US 2024 Q1 +3 Charlie US 2024 Q1 +4 David US 2024 Q2 +5 Eva US 2024 Q2 +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_6_1 -- +8 Hannah EU 2024 Q1 +9 Ivy EU 2024 Q1 + +-- !three_partition_7_7 -- +6 Frank US 2025 Q1 +7 Grace US 2025 Q2 +10 Jack EU 2025 Q2 +11 Leo EU 2025 Q2 +12 Mia EU 2025 Q3 +13 Nina AS 2025 Q1 +14 Oscar AS 2025 Q2 +15 Paul AS 2025 Q3 + +-- !three_partition_8_2 -- +7 Grace US 2025 Q2 + +-- !one_partition_6_0 -- + +-- !two_partition_7_0 -- + +-- !two_partition_8_0 -- + +-- !three_partition_9_0 -- + +-- !three_partition_10_0 -- + +-- !three_partition_11_0 -- + diff --git a/regression-test/suites/external_table_p2/maxcompute/test_max_compute_partition_prune.groovy b/regression-test/suites/external_table_p2/maxcompute/test_max_compute_partition_prune.groovy index e34569117a167fe..83ebbe2fb29d1a3 100644 --- a/regression-test/suites/external_table_p2/maxcompute/test_max_compute_partition_prune.groovy +++ b/regression-test/suites/external_table_p2/maxcompute/test_max_compute_partition_prune.groovy @@ -104,179 +104,191 @@ suite("test_max_compute_partition_prune", "p2,external,maxcompute,external_remot String mc_db = "mc_datalake" String mc_catalog_name = "test_max_compute_partition_prune" - sql """drop catalog if exists ${mc_catalog_name};""" - sql """ - create catalog if not exists ${mc_catalog_name} properties ( - "type" = "max_compute", - "mc.default.project" = "${mc_db}", - "mc.access_key" = "${ak}", - "mc.secret_key" = "${sk}", - "mc.endpoint" = "http://service.cn-beijing-vpc.maxcompute.aliyun-inc.com/api" - ); - """ - sql """ switch ${mc_catalog_name} """ - sql """ use ${mc_db}""" - - qt_one_partition_1_1 one_partition_1_1 - explain { - sql("${one_partition_1_1}") - contains "partition=1/2" - } - - qt_one_partition_2_1 one_partition_2_1 - explain { - sql("${one_partition_2_1}") - contains "partition=1/2" - } - - qt_one_partition_3_all one_partition_3_all - explain { - sql("${one_partition_3_all}") - contains "partition=2/2" - } - - qt_one_partition_4_all one_partition_4_all - explain { - sql("${one_partition_4_all}") - contains "partition=2/2" - } - - qt_one_partition_5_1 one_partition_5_1 - explain { - sql("${one_partition_5_1}") - contains "partition=1/2" - } - - - qt_two_partition_1_1 two_partition_1_1 - explain { - sql("${two_partition_1_1}") - contains "partition=1/4" - } - - qt_two_partition_2_1 two_partition_2_1 - explain { - sql("${two_partition_2_1}") - contains "partition=1/4" - } - - qt_two_partition_3_2 two_partition_3_2 - explain { - sql("${two_partition_3_2}") - contains "partition=2/4" - } - - qt_two_partition_4_all two_partition_4_all - explain { - sql("${two_partition_4_all}") - contains "partition=4/4" - } - - qt_two_partition_5_1 two_partition_5_1 - explain { - sql("${two_partition_5_1}") - contains "partition=1/4" - } - - qt_two_partition_6_1 two_partition_6_1 - explain { - sql("${two_partition_6_1}") - contains "partition=1/4" - } - - - - qt_three_partition_1_1 three_partition_1_1 - explain { - sql("${three_partition_1_1}") - contains "partition=1/10" - } - qt_three_partition_2_1 three_partition_2_1 - explain { - sql("${three_partition_2_1}") - contains "partition=1/10" + for (String enable_profile : ["true","false"] ) { + sql """set enable_profile = ${enable_profile} """; + + for (String num_partitions : ["1","10","100"] ) { + sql "set num_partitions_in_batch_mode = ${num_partitions} " + + for (String cross_partition : ["true","false"] ) { + + sql """drop catalog if exists ${mc_catalog_name};""" + sql """ + create catalog if not exists ${mc_catalog_name} properties ( + "type" = "max_compute", + "mc.default.project" = "${mc_db}", + "mc.access_key" = "${ak}", + "mc.secret_key" = "${sk}", + "mc.endpoint" = "http://service.cn-beijing-vpc.maxcompute.aliyun-inc.com/api", + "mc.split_cross_partition" = "${cross_partition}" + ); + """ + sql """ switch ${mc_catalog_name} """ + sql """ use ${mc_db}""" + + qt_one_partition_1_1 one_partition_1_1 + explain { + sql("${one_partition_1_1}") + contains "partition=1/2" + } + + qt_one_partition_2_1 one_partition_2_1 + explain { + sql("${one_partition_2_1}") + contains "partition=1/2" + } + + qt_one_partition_3_all one_partition_3_all + explain { + sql("${one_partition_3_all}") + contains "partition=2/2" + } + + qt_one_partition_4_all one_partition_4_all + explain { + sql("${one_partition_4_all}") + contains "partition=2/2" + } + + qt_one_partition_5_1 one_partition_5_1 + explain { + sql("${one_partition_5_1}") + contains "partition=1/2" + } + + + qt_two_partition_1_1 two_partition_1_1 + explain { + sql("${two_partition_1_1}") + contains "partition=1/4" + } + + qt_two_partition_2_1 two_partition_2_1 + explain { + sql("${two_partition_2_1}") + contains "partition=1/4" + } + + qt_two_partition_3_2 two_partition_3_2 + explain { + sql("${two_partition_3_2}") + contains "partition=2/4" + } + + qt_two_partition_4_all two_partition_4_all + explain { + sql("${two_partition_4_all}") + contains "partition=4/4" + } + + qt_two_partition_5_1 two_partition_5_1 + explain { + sql("${two_partition_5_1}") + contains "partition=1/4" + } + + qt_two_partition_6_1 two_partition_6_1 + explain { + sql("${two_partition_6_1}") + contains "partition=1/4" + } + + + + qt_three_partition_1_1 three_partition_1_1 + explain { + sql("${three_partition_1_1}") + contains "partition=1/10" + } + + qt_three_partition_2_1 three_partition_2_1 + explain { + sql("${three_partition_2_1}") + contains "partition=1/10" + } + + qt_three_partition_3_3 three_partition_3_3 + explain { + sql("${three_partition_3_3}") + contains "partition=3/10" + } + + qt_three_partition_4_2 three_partition_4_2 + explain { + sql("${three_partition_4_2}") + contains "partition=2/10" + } + + qt_three_partition_5_all three_partition_5_all + explain { + sql("${three_partition_5_all}") + contains "partition=10/10" + } + + qt_three_partition_6_1 three_partition_6_1 + explain { + sql("${three_partition_6_1}") + contains "partition=1/10" + } + + qt_three_partition_7_7 three_partition_7_7 + explain { + sql("${three_partition_7_7}") + contains "partition=7/10" + } + + qt_three_partition_8_2 three_partition_8_2 + explain { + sql("${three_partition_8_2}") + contains "partition=2/10" + } + + + // 0 partitions + def one_partition_6_0 = """SELECT * FROM one_partition_tb WHERE part1 = 2023 ORDER BY id;""" + qt_one_partition_6_0 one_partition_6_0 + explain { + sql("${one_partition_6_0}") + contains "partition=0/2" + } + + def two_partition_7_0 = """SELECT * FROM two_partition_tb WHERE part1 = 'CN' AND part2 = 1 ORDER BY id;""" + qt_two_partition_7_0 two_partition_7_0 + explain { + sql("${two_partition_7_0}") + contains "partition=0/4" + } + + def two_partition_8_0 = """SELECT * FROM two_partition_tb WHERE part1 = 'US' AND part2 = 3 ORDER BY id;""" + qt_two_partition_8_0 two_partition_8_0 + explain { + sql("${two_partition_8_0}") + contains "partition=0/4" + } + + def three_partition_9_0 = """SELECT * FROM three_partition_tb WHERE part1 = 'US' AND part2 = 2023 AND part3 = 'Q1' ORDER BY id;""" + qt_three_partition_9_0 three_partition_9_0 + explain { + sql("${three_partition_9_0}") + contains "partition=0/10" + } + + def three_partition_10_0 = """SELECT * FROM three_partition_tb WHERE part1 = 'EU' AND part2 = 2024 AND part3 = 'Q4' ORDER BY id;""" + qt_three_partition_10_0 three_partition_10_0 + explain { + sql("${three_partition_10_0}") + contains "partition=0/10" + } + + def three_partition_11_0 = """SELECT * FROM three_partition_tb WHERE part1 = 'AS' AND part2 = 2025 AND part3 = 'Q4' ORDER BY id;""" + qt_three_partition_11_0 three_partition_11_0 + explain { + sql("${three_partition_11_0}") + contains "partition=0/10" + } + } + } } - - qt_three_partition_3_3 three_partition_3_3 - explain { - sql("${three_partition_3_3}") - contains "partition=3/10" - } - - qt_three_partition_4_2 three_partition_4_2 - explain { - sql("${three_partition_4_2}") - contains "partition=2/10" - } - - qt_three_partition_5_all three_partition_5_all - explain { - sql("${three_partition_5_all}") - contains "partition=10/10" - } - - qt_three_partition_6_1 three_partition_6_1 - explain { - sql("${three_partition_6_1}") - contains "partition=1/10" - } - - qt_three_partition_7_7 three_partition_7_7 - explain { - sql("${three_partition_7_7}") - contains "partition=7/10" - } - - qt_three_partition_8_2 three_partition_8_2 - explain { - sql("${three_partition_8_2}") - contains "partition=2/10" - } - - - // 0 partitions - def one_partition_6_0 = """SELECT * FROM one_partition_tb WHERE part1 = 2023 ORDER BY id;""" - qt_one_partition_6_0 one_partition_6_0 - explain { - sql("${one_partition_6_0}") - contains "partition=0/2" - } - - def two_partition_7_0 = """SELECT * FROM two_partition_tb WHERE part1 = 'CN' AND part2 = 1 ORDER BY id;""" - qt_two_partition_7_0 two_partition_7_0 - explain { - sql("${two_partition_7_0}") - contains "partition=0/4" - } - - def two_partition_8_0 = """SELECT * FROM two_partition_tb WHERE part1 = 'US' AND part2 = 3 ORDER BY id;""" - qt_two_partition_8_0 two_partition_8_0 - explain { - sql("${two_partition_8_0}") - contains "partition=0/4" - } - - def three_partition_9_0 = """SELECT * FROM three_partition_tb WHERE part1 = 'US' AND part2 = 2023 AND part3 = 'Q1' ORDER BY id;""" - qt_three_partition_9_0 three_partition_9_0 - explain { - sql("${three_partition_9_0}") - contains "partition=0/10" - } - - def three_partition_10_0 = """SELECT * FROM three_partition_tb WHERE part1 = 'EU' AND part2 = 2024 AND part3 = 'Q4' ORDER BY id;""" - qt_three_partition_10_0 three_partition_10_0 - explain { - sql("${three_partition_10_0}") - contains "partition=0/10" - } - - def three_partition_11_0 = """SELECT * FROM three_partition_tb WHERE part1 = 'AS' AND part2 = 2025 AND part3 = 'Q4' ORDER BY id;""" - qt_three_partition_11_0 three_partition_11_0 - explain { - sql("${three_partition_11_0}") - contains "partition=0/10" - } - } } \ No newline at end of file From 2b1264c5799cc11396800796e7d6bce8555b161b Mon Sep 17 00:00:00 2001 From: Rijesh Kunhi Parambattu <147430310+rijeshkp@users.noreply.github.com> Date: Mon, 9 Dec 2024 07:48:43 +0530 Subject: [PATCH 03/14] =?UTF-8?q?=20[Enhancement]=20(nereids)implement=20a?= =?UTF-8?q?dminCheckTabletsCommand=20in=20nereids=E2=80=A6=20(#44701)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Issue Number: close #42843 Support ADMIN CHECK tabletlist command in nereids --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 17 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/AdminCheckTabletsCommand.java | 109 ++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../test_database_management_auth.groovy | 8 +- .../test_nereids_admin_check_tablet.groovy | 57 +++++++++ 7 files changed, 194 insertions(+), 5 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AdminCheckTabletsCommand.java create mode 100644 regression-test/suites/nereids_p0/admin/test_nereids_admin_check_tablet.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index f229a1b2f4a7b1e..c3fe7bb145ba723 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -491,6 +491,7 @@ supportedAdminStatement | ADMIN DIAGNOSE TABLET tabletId=INTEGER_VALUE #adminDiagnoseTablet | ADMIN SHOW REPLICA STATUS FROM baseTableRef (WHERE STATUS EQ|NEQ STRING_LITERAL)? #adminShowReplicaStatus | ADMIN COMPACT TABLE baseTableRef (WHERE TYPE EQ STRING_LITERAL)? #adminCompactTable + | ADMIN CHECK tabletList properties=propertyClause? #adminCheckTablets ; supportedRecoverStatement @@ -508,7 +509,6 @@ unsupportedAdminStatement | ADMIN CANCEL REPAIR TABLE baseTableRef #adminCancelRepairTable | ADMIN SET (FRONTEND | (ALL FRONTENDS)) CONFIG (LEFT_PAREN propertyItemList RIGHT_PAREN)? ALL? #adminSetFrontendConfig - | ADMIN CHECK tabletList properties=propertyClause? #adminCheckTablets | ADMIN REBALANCE DISK (ON LEFT_PAREN backends+=STRING_LITERAL (COMMA backends+=STRING_LITERAL) RIGHT_PAREN)? #adminRebalanceDisk | ADMIN CANCEL REBALANCE DISK (ON LEFT_PAREN backends+=STRING_LITERAL diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 71ea972c4ab2a21..15aeca952a25565 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -50,6 +50,7 @@ import org.apache.doris.mtmv.MTMVRefreshTriggerInfo; import org.apache.doris.nereids.DorisParser; import org.apache.doris.nereids.DorisParser.AddConstraintContext; +import org.apache.doris.nereids.DorisParser.AdminCheckTabletsContext; import org.apache.doris.nereids.DorisParser.AdminCompactTableContext; import org.apache.doris.nereids.DorisParser.AdminDiagnoseTabletContext; import org.apache.doris.nereids.DorisParser.AdminShowReplicaDistributionContext; @@ -470,6 +471,7 @@ import org.apache.doris.nereids.trees.plans.algebra.Aggregate; import org.apache.doris.nereids.trees.plans.algebra.SetOperation.Qualifier; import org.apache.doris.nereids.trees.plans.commands.AddConstraintCommand; +import org.apache.doris.nereids.trees.plans.commands.AdminCheckTabletsCommand; import org.apache.doris.nereids.trees.plans.commands.AdminCompactTableCommand; import org.apache.doris.nereids.trees.plans.commands.AdminShowReplicaStatusCommand; import org.apache.doris.nereids.trees.plans.commands.AlterMTMVCommand; @@ -4950,4 +4952,19 @@ public LogicalPlan visitShowCollation(ShowCollationContext ctx) { } return new ShowCollationCommand(wild); } + + @Override + public LogicalPlan visitAdminCheckTablets(AdminCheckTabletsContext ctx) { + List tabletIdLists = new ArrayList<>(); + if (ctx.tabletList() != null) { + ctx.tabletList().tabletIdList.stream().forEach(tabletToken -> { + tabletIdLists.add(Long.parseLong(tabletToken.getText())); + }); + } + Map properties = ctx.properties != null + ? Maps.newHashMap(visitPropertyClause(ctx.properties)) + : Maps.newHashMap(); + return new AdminCheckTabletsCommand(tabletIdLists, properties); + } } + diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index bc8ccf83093b258..8198a8600a60c99 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -132,6 +132,7 @@ public enum PlanType { PHYSICAL_WINDOW, // commands + ADMIN_CHECK_TABLETS_COMMAND, CREATE_POLICY_COMMAND, CREATE_TABLE_COMMAND, CREATE_SQL_BLOCK_RULE_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AdminCheckTabletsCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AdminCheckTabletsCommand.java new file mode 100644 index 000000000000000..bba555d77a2e920 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AdminCheckTabletsCommand.java @@ -0,0 +1,109 @@ +// 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.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.util.PropertyAnalyzer; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import org.apache.commons.collections.CollectionUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * drop user command + */ +public class AdminCheckTabletsCommand extends Command implements ForwardNoSync { + private static final Logger LOG = LogManager.getLogger(AdminCheckTabletsCommand.class); + private final List tabletIds; + private final Map properties; + + /** + * Check Type in Properties. + * At present only CONSISTENCY is supported + */ + public enum CheckType { + CONSISTENCY; // check the consistency of replicas of tablet + public static CheckType getTypeFromString(String str) throws AnalysisException { + try { + return CheckType.valueOf(str.toUpperCase()); + } catch (Exception e) { + throw new AnalysisException("Unknown check type: " + str); + } + } + } + + /** + * constructor + */ + // ADMIN CHECK TABLET (id1, id2, ...) PROPERTIES ("type" = "check_consistency"); + public AdminCheckTabletsCommand(List tabletIds, Map properties) { + super(PlanType.ADMIN_CHECK_TABLETS_COMMAND); + this.tabletIds = tabletIds; + this.properties = properties; + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + + if (CollectionUtils.isEmpty(tabletIds)) { + throw new AnalysisException("Tablet id list is empty"); + } + + String typeStr = PropertyAnalyzer.analyzeType(properties); + if (typeStr == null) { + throw new AnalysisException("Should specify 'type' property"); + } + CheckType checkType = CheckType.getTypeFromString(typeStr); + + if (properties != null && !properties.isEmpty()) { + throw new AnalysisException("Unknown properties: " + properties.keySet()); + } + + if (Objects.requireNonNull(checkType) == CheckType.CONSISTENCY) { + Env.getCurrentEnv().getConsistencyChecker().addTabletsToCheck(tabletIds); + } + } + + @Override + protected void checkSupportedInCloudMode(ConnectContext ctx) throws DdlException { + LOG.info("AdminCheckTabletsCommand not supported in cloud mode"); + throw new DdlException("Unsupported operation"); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitAdminCheckTabletsCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 77846ff3ad5328c..dea3a004e816bef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.trees.plans.visitor; import org.apache.doris.nereids.trees.plans.commands.AddConstraintCommand; +import org.apache.doris.nereids.trees.plans.commands.AdminCheckTabletsCommand; import org.apache.doris.nereids.trees.plans.commands.AdminCompactTableCommand; import org.apache.doris.nereids.trees.plans.commands.AdminShowReplicaStatusCommand; import org.apache.doris.nereids.trees.plans.commands.AlterJobStatusCommand; @@ -595,4 +596,8 @@ default R visitCreateRoutineLoadCommand(CreateRoutineLoadCommand createRoutineLo default R visitShowProcessListCommand(ShowProcessListCommand showProcessListCommand, C context) { return visitCommand(showProcessListCommand, context); } + + default R visitAdminCheckTabletsCommand(AdminCheckTabletsCommand adminCheckTabletsCommand, C context) { + return visitCommand(adminCheckTabletsCommand, context); + } } diff --git a/regression-test/suites/auth_call/test_database_management_auth.groovy b/regression-test/suites/auth_call/test_database_management_auth.groovy index 0e1805e20d6840b..79114c18e789b50 100644 --- a/regression-test/suites/auth_call/test_database_management_auth.groovy +++ b/regression-test/suites/auth_call/test_database_management_auth.groovy @@ -23,14 +23,14 @@ suite("test_database_management_auth","p0,auth_call") { String user = 'test_database_management_auth_user' String pwd = 'C123_567p' String dbName = 'test_database_management_auth_db' - def String show_dis_error_msg = "denied" + def String error_in_cloud = "denied" //cloud-mode if (isCloudMode()) { def clusters = sql " SHOW CLUSTERS; " assertTrue(!clusters.isEmpty()) def validCluster = clusters[0][0] sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}"""; - show_dis_error_msg = "Unsupported" + error_in_cloud = "Unsupported" } try_sql("DROP USER ${user}") @@ -79,7 +79,7 @@ suite("test_database_management_auth","p0,auth_call") { } test { sql """SHOW REPLICA DISTRIBUTION FROM tbl;""" - exception "${show_dis_error_msg}" + exception "${error_in_cloud}" } test { sql """SHOW REPLICA STATUS FROM db1.tbl1;""" @@ -95,7 +95,7 @@ suite("test_database_management_auth","p0,auth_call") { } test { sql """ADMIN CHECK TABLET (10000, 10001) PROPERTIES("type" = "consistency");""" - exception "denied" + exception "${error_in_cloud}" } test { sql """SHOW TABLET DIAGNOSIS 0;""" diff --git a/regression-test/suites/nereids_p0/admin/test_nereids_admin_check_tablet.groovy b/regression-test/suites/nereids_p0/admin/test_nereids_admin_check_tablet.groovy new file mode 100644 index 000000000000000..8812e09ea9f6f3d --- /dev/null +++ b/regression-test/suites/nereids_p0/admin/test_nereids_admin_check_tablet.groovy @@ -0,0 +1,57 @@ +// 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. + + +suite("test_nereids_admin_check_tablet") { + def table = "test_nereids_admin_check_tablet" + // create table and insert data + sql """ drop table if exists ${table} force""" + sql """ + create table ${table} ( + `id` int(11), + `name` varchar(128), + `da` date + ) + engine=olap + duplicate key(id) + partition by range(da)( + PARTITION p3 VALUES LESS THAN ('2023-01-01'), + PARTITION p4 VALUES LESS THAN ('2024-01-01'), + PARTITION p5 VALUES LESS THAN ('2025-01-01') + ) + distributed by hash(id) buckets 2 + properties( + "replication_num"="1", + "light_schema_change"="true" + ); + """ + + def tabletId + def tablets = sql_return_maparray "SHOW TABLETS FROM ${table}" + + for (def tablet : tablets) { + //get any partition ID. + tabletId = tablet.TabletId; + break; + } + + logger.info("${tabletId}"); + + checkNereidsExecute("ADMIN CHECK TABLET (${tabletId}) PROPERTIES('type' = 'consistency')"); + +} + From f3a30d182150a2ca2a9d31610d8100592dec3ecb Mon Sep 17 00:00:00 2001 From: minghong Date: Mon, 9 Dec 2024 10:44:31 +0800 Subject: [PATCH 04/14] [fix](regression)turn runtime filter off in invalid_stats case (#45147) ### What problem does this PR solve? set "runtime_filter_mode=off" to avoid interference of runtime filter in the case "invalid_stats.groovy" --- .../suites/nereids_p0/stats/invalid_stats/invalid_stats.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression-test/suites/nereids_p0/stats/invalid_stats/invalid_stats.groovy b/regression-test/suites/nereids_p0/stats/invalid_stats/invalid_stats.groovy index 51f1a37f118a42d..3bda4bc743200a7 100644 --- a/regression-test/suites/nereids_p0/stats/invalid_stats/invalid_stats.groovy +++ b/regression-test/suites/nereids_p0/stats/invalid_stats/invalid_stats.groovy @@ -17,11 +17,11 @@ suite("invalid_stats") { multi_sql """ - set global enable_auto_analyze=false; SET enable_nereids_planner=true; SET enable_fallback_to_original_planner=false; set disable_nereids_rules=PRUNE_EMPTY_PARTITION; set ignore_shape_nodes=PhysicalProject; + set runtime_filter_mode=off; drop table if exists region; CREATE TABLE region ( From a25d457fa8e3a70eb6244ca068ab7bcd84237956 Mon Sep 17 00:00:00 2001 From: shee <13843187+qzsee@users.noreply.github.com> Date: Mon, 9 Dec 2024 11:00:40 +0800 Subject: [PATCH 05/14] [opt](lambda) let lambda expression support refer outer slot (#44365) ### What problem does this PR solve? ```sql CREATE TABLE `array_test` ( `id` int(11) NULL COMMENT "", `c_array` ARRAY NULL COMMENT "" ) ENGINE=OLAP DUPLICATE KEY(`id`) COMMENT "OLAP" DISTRIBUTED BY HASH(`id`) BUCKETS 1 PROPERTIES ( "replication_allocation" = "tag.location.default: 1", "in_memory" = "false", "storage_format" = "V2" ); ``` this sql is ok ```sql select *, array_map(x -> x+1, c_array) from array_test ``` but lambda expressions in `array_map` do not support field arguments for example: ```sql select *, array_map(x -> x+id, c_array) from array_test ``` This pr solves two problems 1. support column arg in lambda expression 2. prevent high memory usage due to large block data Related PR: #xxx let lambda expression support refer outer slot Co-authored-by: garenshi --- .../rowset/segment_v2/segment_iterator.cpp | 10 +- .../exprs/lambda_function/lambda_function.h | 9 +- .../lambda_function/varray_map_function.cpp | 215 +++++++++++++++--- be/src/vec/exprs/vcolumn_ref.h | 11 +- be/src/vec/exprs/vexpr.h | 3 + be/src/vec/exprs/vlambda_function_call_expr.h | 1 + .../apache/doris/nereids/analyzer/Scope.java | 11 +- .../rules/analysis/BindExpression.java | 2 +- .../rules/analysis/ExpressionAnalyzer.java | 5 +- .../rules/analysis/SubExprAnalyzer.java | 11 +- .../test_array_map_function_with_column.out | 41 ++++ .../test_array_map_function.groovy | 3 +- ...test_array_map_function_with_column.groovy | 70 ++++++ .../datetime_functions/test_convert_tz.groovy | 3 +- 14 files changed, 335 insertions(+), 60 deletions(-) create mode 100644 regression-test/data/query_p0/sql_functions/array_functions/test_array_map_function_with_column.out create mode 100644 regression-test/suites/query_p0/sql_functions/array_functions/test_array_map_function_with_column.groovy diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp index f5f46e938650e12..abdf9116756f0ed 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp +++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp @@ -2175,11 +2175,11 @@ Status SegmentIterator::_next_batch_internal(vectorized::Block* block) { if (block->rows() == 0) { vectorized::MutableColumnPtr col0 = std::move(*block->get_by_position(0).column).mutate(); - auto res_column = vectorized::ColumnString::create(); - res_column->insert_data("", 0); - auto col_const = vectorized::ColumnConst::create(std::move(res_column), - selected_size); - block->replace_by_position(0, std::move(col_const)); + auto tmp_indicator_col = + block->get_by_position(0) + .type->create_column_const_with_default_value( + selected_size); + block->replace_by_position(0, std::move(tmp_indicator_col)); _output_index_result_column_for_expr(_sel_rowid_idx.data(), selected_size, block); block->shrink_char_type_column_suffix_zero(_char_type_idx_no_0); diff --git a/be/src/vec/exprs/lambda_function/lambda_function.h b/be/src/vec/exprs/lambda_function/lambda_function.h index 184b4c2cc38abfc..073029546882711 100644 --- a/be/src/vec/exprs/lambda_function/lambda_function.h +++ b/be/src/vec/exprs/lambda_function/lambda_function.h @@ -17,7 +17,7 @@ #pragma once -#include +#include #include "common/status.h" #include "vec/core/block.h" @@ -31,9 +31,16 @@ class LambdaFunction { virtual std::string get_name() const = 0; + virtual doris::Status prepare(RuntimeState* state) { + batch_size = state->batch_size(); + return Status::OK(); + } + virtual doris::Status execute(VExprContext* context, doris::vectorized::Block* block, int* result_column_id, const DataTypePtr& result_type, const VExprSPtrs& children) = 0; + + int batch_size; }; using LambdaFunctionPtr = std::shared_ptr; diff --git a/be/src/vec/exprs/lambda_function/varray_map_function.cpp b/be/src/vec/exprs/lambda_function/varray_map_function.cpp index f80cffa166eac44..f7a864b92d3fb41 100644 --- a/be/src/vec/exprs/lambda_function/varray_map_function.cpp +++ b/be/src/vec/exprs/lambda_function/varray_map_function.cpp @@ -15,9 +15,13 @@ // specific language governing permissions and limitations // under the License. +#include +#include +#include + #include #include -#include +#include #include "common/status.h" #include "vec/aggregate_functions/aggregate_function.h" @@ -39,9 +43,28 @@ namespace doris::vectorized { #include "common/compile_check_begin.h" - class VExprContext; +// extend a block with all required parameters +struct LambdaArgs { + // the lambda function need the column ids of all the slots + std::vector output_slot_ref_indexs; + // which line is extended to the original block + int64_t current_row_idx = 0; + // when a block is filled, the array may be truncated, recording where it was truncated + int64_t current_offset_in_array = 0; + // the beginning position of the array + size_t array_start = 0; + // the size of the array + int64_t cur_size = 0; + // offset of column array + const ColumnArray::Offsets64* offsets_ptr = nullptr; + // expend data of repeat times + int current_repeat_times = 0; + // whether the current row of the original block has been extended + bool current_row_eos = false; +}; + class ArrayMapFunction : public LambdaFunction { ENABLE_FACTORY_CREATOR(ArrayMapFunction); @@ -57,8 +80,33 @@ class ArrayMapFunction : public LambdaFunction { doris::Status execute(VExprContext* context, doris::vectorized::Block* block, int* result_column_id, const DataTypePtr& result_type, const VExprSPtrs& children) override { - ///* array_map(lambda,arg1,arg2,.....) */// + LambdaArgs args; + // collect used slot ref in lambda function body + _collect_slot_ref_column_id(children[0], args); + + int gap = 0; + if (!args.output_slot_ref_indexs.empty()) { + auto max_id = std::max_element(args.output_slot_ref_indexs.begin(), + args.output_slot_ref_indexs.end()); + gap = *max_id + 1; + _set_column_ref_column_id(children[0], gap); + } + std::vector names(gap); + DataTypes data_types(gap); + + for (int i = 0; i < gap; ++i) { + if (_contains_column_id(args, i)) { + names[i] = block->get_by_position(i).name; + data_types[i] = block->get_by_position(i).type; + } else { + // padding some mock data + names[i] = "temp"; + data_types[i] = std::make_shared(); + } + } + + ///* array_map(lambda,arg1,arg2,.....) */// //1. child[1:end]->execute(src_block) doris::vectorized::ColumnNumbers arguments(children.size() - 1); for (int i = 1; i < children.size(); ++i) { @@ -77,14 +125,13 @@ class ArrayMapFunction : public LambdaFunction { MutableColumnPtr array_column_offset; size_t nested_array_column_rows = 0; ColumnPtr first_array_offsets = nullptr; - //2. get the result column from executed expr, and the needed is nested column of array - Block lambda_block; + std::vector lambda_datas(arguments.size()); + for (int i = 0; i < arguments.size(); ++i) { const auto& array_column_type_name = block->get_by_position(arguments[i]); auto column_array = array_column_type_name.column->convert_to_full_column_if_const(); auto type_array = array_column_type_name.type; - if (type_array->is_nullable()) { // get the nullmap of nullable column const auto& column_array_nullmap = @@ -113,6 +160,7 @@ class ArrayMapFunction : public LambdaFunction { const auto& off_data = assert_cast( col_array.get_offsets_column()); array_column_offset = off_data.clone_resized(col_array.get_offsets_column().size()); + args.offsets_ptr = &col_array.get_offsets(); } else { // select array_map((x,y)->x+y,c_array1,[0,1,2,3]) from array_test2; // c_array1: [0,1,2,3,4,5,6,7,8,9] @@ -131,57 +179,164 @@ class ArrayMapFunction : public LambdaFunction { nested_array_column_rows, i + 1, col_array.get_data_ptr()->size()); } } - - // insert the data column to the new block - ColumnWithTypeAndName data_column {col_array.get_data_ptr(), col_type.get_nested_type(), - "R" + array_column_type_name.name}; - lambda_block.insert(std::move(data_column)); + lambda_datas[i] = col_array.get_data_ptr(); + names.push_back("R" + array_column_type_name.name); + data_types.push_back(col_type.get_nested_type()); } - //3. child[0]->execute(new_block) - RETURN_IF_ERROR(children[0]->execute(context, &lambda_block, result_column_id)); + ColumnPtr result_col = nullptr; + DataTypePtr res_type; + std::string res_name; + + //process first row + args.array_start = (*args.offsets_ptr)[args.current_row_idx - 1]; + args.cur_size = (*args.offsets_ptr)[args.current_row_idx] - args.array_start; + + while (args.current_row_idx < block->rows()) { + Block lambda_block; + for (int i = 0; i < names.size(); i++) { + ColumnWithTypeAndName data_column; + if (_contains_column_id(args, i) || i >= gap) { + data_column = ColumnWithTypeAndName(data_types[i], names[i]); + } else { + data_column = ColumnWithTypeAndName( + data_types[i]->create_column_const_with_default_value(0), data_types[i], + names[i]); + } + lambda_block.insert(std::move(data_column)); + } + + MutableColumns columns = lambda_block.mutate_columns(); + while (columns[gap]->size() < batch_size) { + long max_step = batch_size - columns[gap]->size(); + long current_step = + std::min(max_step, (long)(args.cur_size - args.current_offset_in_array)); + size_t pos = args.array_start + args.current_offset_in_array; + for (int i = 0; i < arguments.size(); ++i) { + columns[gap + i]->insert_range_from(*lambda_datas[i], pos, current_step); + } + args.current_offset_in_array += current_step; + args.current_repeat_times += current_step; + if (args.current_offset_in_array >= args.cur_size) { + args.current_row_eos = true; + } + _extend_data(columns, block, args, gap); + if (args.current_row_eos) { + args.current_row_idx++; + args.current_offset_in_array = 0; + if (args.current_row_idx >= block->rows()) { + break; + } + args.current_row_eos = false; + args.array_start = (*args.offsets_ptr)[args.current_row_idx - 1]; + args.cur_size = (*args.offsets_ptr)[args.current_row_idx] - args.array_start; + } + } + + lambda_block.set_columns(std::move(columns)); - auto res_col = lambda_block.get_by_position(*result_column_id) - .column->convert_to_full_column_if_const(); - auto res_type = lambda_block.get_by_position(*result_column_id).type; - auto res_name = lambda_block.get_by_position(*result_column_id).name; + //3. child[0]->execute(new_block) + RETURN_IF_ERROR(children[0]->execute(context, &lambda_block, result_column_id)); + + auto res_col = lambda_block.get_by_position(*result_column_id) + .column->convert_to_full_column_if_const(); + res_type = lambda_block.get_by_position(*result_column_id).type; + res_name = lambda_block.get_by_position(*result_column_id).name; + if (!result_col) { + result_col = std::move(res_col); + } else { + MutableColumnPtr column = (*std::move(result_col)).mutate(); + column->insert_range_from(*res_col, 0, res_col->size()); + } + } //4. get the result column after execution, reassemble it into a new array column, and return. ColumnWithTypeAndName result_arr; if (result_type->is_nullable()) { if (res_type->is_nullable()) { - result_arr = {ColumnNullable::create( - ColumnArray::create(res_col, std::move(array_column_offset)), - std::move(outside_null_map)), - result_type, res_name}; + result_arr = { + ColumnNullable::create( + ColumnArray::create(result_col, std::move(array_column_offset)), + std::move(outside_null_map)), + result_type, res_name}; } else { // deal with eg: select array_map(x -> x is null, [null, 1, 2]); // need to create the nested column null map for column array - auto nested_null_map = ColumnUInt8::create(res_col->size(), 0); + auto nested_null_map = ColumnUInt8::create(result_col->size(), 0); result_arr = { ColumnNullable::create( - ColumnArray::create( - ColumnNullable::create(res_col, std::move(nested_null_map)), - std::move(array_column_offset)), + ColumnArray::create(ColumnNullable::create( + result_col, std::move(nested_null_map)), + std::move(array_column_offset)), std::move(outside_null_map)), result_type, res_name}; } } else { if (res_type->is_nullable()) { - result_arr = {ColumnArray::create(res_col, std::move(array_column_offset)), + result_arr = {ColumnArray::create(result_col, std::move(array_column_offset)), result_type, res_name}; } else { - auto nested_null_map = ColumnUInt8::create(res_col->size(), 0); - result_arr = {ColumnArray::create( - ColumnNullable::create(res_col, std::move(nested_null_map)), - std::move(array_column_offset)), + auto nested_null_map = ColumnUInt8::create(result_col->size(), 0); + result_arr = {ColumnArray::create(ColumnNullable::create( + result_col, std::move(nested_null_map)), + std::move(array_column_offset)), result_type, res_name}; } } block->insert(std::move(result_arr)); *result_column_id = block->columns() - 1; + return Status::OK(); } + +private: + bool _contains_column_id(LambdaArgs& args, int id) { + const auto it = std::find(args.output_slot_ref_indexs.begin(), + args.output_slot_ref_indexs.end(), id); + return it != args.output_slot_ref_indexs.end(); + } + + void _set_column_ref_column_id(VExprSPtr expr, int gap) { + for (const auto& child : expr->children()) { + if (child->is_column_ref()) { + auto* ref = static_cast(child.get()); + ref->set_gap(gap); + } else { + _set_column_ref_column_id(child, gap); + } + } + } + + void _collect_slot_ref_column_id(VExprSPtr expr, LambdaArgs& args) { + for (const auto& child : expr->children()) { + if (child->is_slot_ref()) { + const auto* ref = static_cast(child.get()); + args.output_slot_ref_indexs.push_back(ref->column_id()); + } else { + _collect_slot_ref_column_id(child, args); + } + } + } + + void _extend_data(std::vector& columns, Block* block, LambdaArgs& args, + int size) { + if (!args.current_repeat_times || !size) { + return; + } + for (int i = 0; i < size; i++) { + if (_contains_column_id(args, i)) { + auto src_column = + block->get_by_position(i).column->convert_to_full_column_if_const(); + columns[i]->insert_many_from(*src_column, args.current_row_idx, + args.current_repeat_times); + } else { + // must be column const + DCHECK(is_column_const(*columns[i])); + columns[i]->resize(columns[i]->size() + args.current_repeat_times); + } + } + args.current_repeat_times = 0; + } }; void register_function_array_map(doris::vectorized::LambdaFunctionFactory& factory) { diff --git a/be/src/vec/exprs/vcolumn_ref.h b/be/src/vec/exprs/vcolumn_ref.h index a763797880e3615..d58e1375291457d 100644 --- a/be/src/vec/exprs/vcolumn_ref.h +++ b/be/src/vec/exprs/vcolumn_ref.h @@ -16,6 +16,8 @@ // under the License. #pragma once +#include + #include "runtime/descriptors.h" #include "runtime/runtime_state.h" #include "vec/exprs/vexpr.h" @@ -57,7 +59,7 @@ class VColumnRef final : public VExpr { Status execute(VExprContext* context, Block* block, int* result_column_id) override { DCHECK(_open_finished || _getting_const_col); - *result_column_id = _column_id; + *result_column_id = _column_id + _gap; return Status::OK(); } @@ -67,6 +69,12 @@ class VColumnRef final : public VExpr { const std::string& expr_name() const override { return _column_name; } + void set_gap(int gap) { + if (_gap == 0) { + _gap = gap; + } + } + std::string debug_string() const override { std::stringstream out; out << "VColumnRef(slot_id: " << _column_id << ",column_name: " << _column_name @@ -76,6 +84,7 @@ class VColumnRef final : public VExpr { private: int _column_id; + std::atomic _gap = 0; std::string _column_name; }; } // namespace vectorized diff --git a/be/src/vec/exprs/vexpr.h b/be/src/vec/exprs/vexpr.h index 3456fb431a48cb0..2580cf7ddc88f9c 100644 --- a/be/src/vec/exprs/vexpr.h +++ b/be/src/vec/exprs/vexpr.h @@ -148,6 +148,9 @@ class VExpr { TypeDescriptor type() { return _type; } bool is_slot_ref() const { return _node_type == TExprNodeType::SLOT_REF; } + + bool is_column_ref() const { return _node_type == TExprNodeType::COLUMN_REF; } + virtual bool is_literal() const { return false; } TExprNodeType::type node_type() const { return _node_type; } diff --git a/be/src/vec/exprs/vlambda_function_call_expr.h b/be/src/vec/exprs/vlambda_function_call_expr.h index 44d22b1f9ebbea1..7b0ea1dcb5a75d9 100644 --- a/be/src/vec/exprs/vlambda_function_call_expr.h +++ b/be/src/vec/exprs/vlambda_function_call_expr.h @@ -50,6 +50,7 @@ class VLambdaFunctionCallExpr : public VExpr { return Status::InternalError("Lambda Function {} is not implemented.", _fn.name.function_name); } + RETURN_IF_ERROR(_lambda_function->prepare(state)); _prepare_finished = true; return Status::OK(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/Scope.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/Scope.java index dbcbea7c104b5ad..7c77c680ff287b7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/Scope.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/Scope.java @@ -18,7 +18,6 @@ package org.apache.doris.nereids.analyzer; import org.apache.doris.nereids.trees.expressions.Slot; -import org.apache.doris.nereids.trees.expressions.SubqueryExpr; import org.apache.doris.nereids.util.Utils; import com.google.common.base.Suppliers; @@ -62,20 +61,18 @@ public class Scope { private final Optional outerScope; private final List slots; - private final Optional ownerSubquery; private final Set correlatedSlots; private final boolean buildNameToSlot; private final Supplier> nameToSlot; public Scope(List slots) { - this(Optional.empty(), slots, Optional.empty()); + this(Optional.empty(), slots); } /** Scope */ - public Scope(Optional outerScope, List slots, Optional subqueryExpr) { + public Scope(Optional outerScope, List slots) { this.outerScope = Objects.requireNonNull(outerScope, "outerScope can not be null"); this.slots = Utils.fastToImmutableList(Objects.requireNonNull(slots, "slots can not be null")); - this.ownerSubquery = Objects.requireNonNull(subqueryExpr, "subqueryExpr can not be null"); this.correlatedSlots = Sets.newLinkedHashSet(); this.buildNameToSlot = slots.size() > 500; this.nameToSlot = buildNameToSlot ? Suppliers.memoize(this::buildNameToSlot) : null; @@ -89,10 +86,6 @@ public Optional getOuterScope() { return outerScope; } - public Optional getSubquery() { - return ownerSubquery; - } - public Set getCorrelatedSlots() { return correlatedSlots; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java index cde659b91021ad8..1e481542baec125 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java @@ -1229,7 +1229,7 @@ private E checkBoundExceptLambda(E expression, Plan plan) private Scope toScope(CascadesContext cascadesContext, List slots) { Optional outerScope = cascadesContext.getOuterScope(); if (outerScope.isPresent()) { - return new Scope(outerScope, slots, outerScope.get().getSubquery()); + return new Scope(outerScope, slots); } else { return new Scope(slots); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java index adc68ac6ecac1b5..6abbbda447c8ec1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java @@ -833,8 +833,9 @@ private UnboundFunction bindHighOrderFunction(UnboundFunction unboundFunction, E .map(ArrayItemReference::toSlot) .collect(ImmutableList.toImmutableList()); - ExpressionAnalyzer lambdaAnalyzer = new ExpressionAnalyzer(currentPlan, new Scope(boundedSlots), - context == null ? null : context.cascadesContext, true, false) { + ExpressionAnalyzer lambdaAnalyzer = new ExpressionAnalyzer(currentPlan, new Scope(Optional.of(getScope()), + boundedSlots), context == null ? null : context.cascadesContext, + true, true) { @Override protected void couldNotFoundColumn(UnboundSlot unboundSlot, String tableName) { throw new AnalysisException("Unknown lambda slot '" diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubExprAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubExprAnalyzer.java index 7b0ed45708251da..9a70ce24afb8efb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubExprAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubExprAnalyzer.java @@ -53,7 +53,6 @@ import java.util.HashSet; import java.util.List; import java.util.Objects; -import java.util.Optional; import java.util.Set; /** @@ -205,19 +204,15 @@ private AnalyzedResult analyzeSubquery(SubqueryExpr expr) { } CascadesContext subqueryContext = CascadesContext.newContextWithCteContext( cascadesContext, expr.getQueryPlan(), cascadesContext.getCteContext()); - Scope subqueryScope = genScopeWithSubquery(expr); + // don't use `getScope()` because we only need `getScope().getOuterScope()` and `getScope().getSlots()` + // otherwise unexpected errors may occur + Scope subqueryScope = new Scope(getScope().getOuterScope(), getScope().getSlots()); subqueryContext.setOuterScope(subqueryScope); subqueryContext.newAnalyzer().analyze(); return new AnalyzedResult((LogicalPlan) subqueryContext.getRewritePlan(), subqueryScope.getCorrelatedSlots()); } - private Scope genScopeWithSubquery(SubqueryExpr expr) { - return new Scope(getScope().getOuterScope(), - getScope().getSlots(), - Optional.ofNullable(expr)); - } - public Scope getScope() { return scope; } diff --git a/regression-test/data/query_p0/sql_functions/array_functions/test_array_map_function_with_column.out b/regression-test/data/query_p0/sql_functions/array_functions/test_array_map_function_with_column.out new file mode 100644 index 000000000000000..24d7b53e2b34f1f --- /dev/null +++ b/regression-test/data/query_p0/sql_functions/array_functions/test_array_map_function_with_column.out @@ -0,0 +1,41 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_1 -- +\N \N \N \N \N +1 2 [1, 2, 3, 4] [null, 2, 3] [1, 1, 1, 1] +2 3 [6, 7, null, 9] [4, null, 6] [1, 1, null, 1] +3 4 \N [4, 5, 6] \N + +-- !select_2 -- +\N \N \N \N \N +1 2 [1, 2, 3, 4] [null, 2, 3] [1, 1, 1, 1] +2 3 [6, 7, null, 9] [4, null, 6] [1, 1, null, 1] +3 4 \N [4, 5, 6] \N + +-- !select_3 -- +\N \N \N \N \N +1 2 [1, 2, 3, 4] [null, 2, 3] [1, 1, 1, 1] +2 3 [6, 7, null, 9] [4, null, 6] [1, 1, null, 1] +3 4 \N [4, 5, 6] \N + +-- !select_4 -- +\N \N \N \N \N +1 2 [1, 2, 3, 4] [null, 2, 3] [1, 1, 1, 1] +2 3 [6, 7, null, 9] [4, null, 6] [1, 1, null, 1] +3 4 \N [4, 5, 6] \N + +-- !select_5 -- +\N \N \N \N \N +1 2 [1, 2, 3, 4] [null, 2, 3] [1, 1, 1, 1] +2 3 [6, 7, null, 9] [4, null, 6] [1, 1, null, 1] +3 4 \N [4, 5, 6] \N + +-- !select_6 -- +\N \N \N \N \N +4 5 [6, 7, null, 9] [4, 5, 6, 7] [0, 0, null, 0] +5 6 [10, 11, 12, 13] [8, 9, null, 11] [0, 0, null, 0] +6 7 \N \N \N + +-- !select_7 -- +4 5 [6, 7, null, 9] [4, 5, 6, 7] [0, 0, null, 0] +5 6 [10, 11, 12, 13] [8, 9, null, 11] [0, 0, null, 0] + diff --git a/regression-test/suites/query_p0/sql_functions/array_functions/test_array_map_function.groovy b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_map_function.groovy index 311b188e4335500..17de93942bfc61c 100644 --- a/regression-test/suites/query_p0/sql_functions/array_functions/test_array_map_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_map_function.groovy @@ -75,8 +75,7 @@ suite("test_array_map_function") { test { sql"""select c_array1,array_max(array_map(x->countequal(c_array1,x),c_array1)) from array_test2;""" check{result, exception, startTime, endTime -> - assertTrue(exception != null) - logger.info(exception.message) + assertTrue(exception == null) } } diff --git a/regression-test/suites/query_p0/sql_functions/array_functions/test_array_map_function_with_column.groovy b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_map_function_with_column.groovy new file mode 100644 index 000000000000000..024e4b57a34572e --- /dev/null +++ b/regression-test/suites/query_p0/sql_functions/array_functions/test_array_map_function_with_column.groovy @@ -0,0 +1,70 @@ +// 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. + +suite("test_array_map_function_with_column") { + + def tableName = "array_test_with_column" + sql "DROP TABLE IF EXISTS ${tableName}" + sql """ + CREATE TABLE IF NOT EXISTS ${tableName} ( + `k1` int(11) NULL COMMENT "", + `k2` int(11) NULL COMMENT "", + `c_array1` ARRAY NULL COMMENT "", + `c_array2` ARRAY NULL COMMENT "" + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`k1`,`k2`) BUCKETS 3 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + + sql """INSERT INTO ${tableName} values + (1, 2, [1,2,3,4], [null,2,3]), + (2, 3, [6,7,null,9], [4,null,6]), + (3, 4, NULL, [4, 5, 6]), + (NULL, NULL, NULL, NULL); + """ + + qt_select_1 "select *,array_map(x->x+k1+k2 > k1*k2,c_array1) from ${tableName} order by k1;" + + sql "set batch_size = 1;" + qt_select_2 "select *,array_map(x->x+k1+k2 > k1*k2,c_array1) from ${tableName} order by k1;" + + sql "set batch_size = 4;" + qt_select_3 "select *,array_map(x->x+k1+k2 > k1*k2,c_array1) from ${tableName} order by k1;" + + sql "set batch_size = 6;" + qt_select_4 "select *,array_map(x->x+k1+k2 > k1*k2,c_array1) from ${tableName} order by k1;" + + sql "set batch_size = 8;" + qt_select_5 "select *,array_map(x->x+k1+k2 > k1*k2,c_array1) from ${tableName} order by k1;" + + sql "truncate table ${tableName};" + + sql """INSERT INTO ${tableName} values + (4, 5, [6,7,null,9], [4,5,6,7]), + (5, 6, [10,11,12,13], [8,9,null,11]), + (6, 7, NULL, NULL), + (NULL, NULL, NULL, NULL); + """ + qt_select_6 "select *,array_map((x,y)->x+k1+k2 > y+k1*k2,c_array1,c_array2) from ${tableName} order by k1;" + + qt_select_7 "select *,array_map((x,y)->x+k1+k2 > y+k1*k2,c_array1,c_array2) from ${tableName} where array_count((x,y) -> k1*x>y+k2, c_array1, c_array2) > 1 order by k1;" + + sql "DROP TABLE IF EXISTS ${tableName}" +} diff --git a/regression-test/suites/query_p0/sql_functions/datetime_functions/test_convert_tz.groovy b/regression-test/suites/query_p0/sql_functions/datetime_functions/test_convert_tz.groovy index 2959c7f136b8e5c..e382fe9ae70b848 100644 --- a/regression-test/suites/query_p0/sql_functions/datetime_functions/test_convert_tz.groovy +++ b/regression-test/suites/query_p0/sql_functions/datetime_functions/test_convert_tz.groovy @@ -16,6 +16,7 @@ // under the License. suite("test_convert_tz") { + sql "drop table if exists `cvt_tz`" sql """ CREATE TABLE `cvt_tz` ( `rowid` int NULL, @@ -34,4 +35,4 @@ suite("test_convert_tz") { order_qt_sql1 """ select convert_tz(dt, '+00:00', IF(property_value IS NULL, '+00:00', property_value)) from cvt_tz """ -} \ No newline at end of file +} From 45168f74c8aeac40464fecab3722706192bc7ab3 Mon Sep 17 00:00:00 2001 From: Uniqueyou <134280716+wyxxxcat@users.noreply.github.com> Date: Mon, 9 Dec 2024 11:16:02 +0800 Subject: [PATCH 06/14] [fix](sql) Fix error for show create table column comment (#44958) The quotes in the comment should be escaped. --- .../main/java/org/apache/doris/catalog/Column.java | 2 +- .../doris/analysis/ShowCreateTableStmtTest.java | 14 ++++++++++++-- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java index 916a213027f2f6e..0ae6a4f8bdb5eb1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java @@ -1019,7 +1019,7 @@ public String toSql(boolean isUniqueTable, boolean isCompatible) { sb.append(" ON UPDATE ").append(defaultValue).append(""); } if (StringUtils.isNotBlank(comment)) { - sb.append(" COMMENT '").append(getComment(true)).append("'"); + sb.append(" COMMENT \"").append(getComment(true)).append("\""); } return sb.toString(); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateTableStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateTableStmtTest.java index 353fbad9fabfc82..0faf4a8f34e8a8f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateTableStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowCreateTableStmtTest.java @@ -32,7 +32,7 @@ protected void runBeforeAll() throws Exception { createDatabase("test"); useDatabase("test"); createTable("create table table1\n" - + "(k1 int comment 'test column k1', k2 int comment 'test column k2') comment 'test table1' " + + "(k1 int comment 'test column k1', k2 int comment 'test column k2', `timestamp` DATE NOT NULL COMMENT '[''0000-01-01'', ''9999-12-31'']') comment 'test table1' " + "PARTITION BY RANGE(`k1`)\n" + "(\n" + " PARTITION `p01` VALUES LESS THAN (\"10\"),\n" @@ -48,10 +48,20 @@ public void testNormal() throws Exception { String sql = "show create table table1"; ShowResultSet showResultSet = showCreateTable(sql); String showSql = showResultSet.getResultRows().get(0).get(1); - Assertions.assertTrue(showSql.contains("`k1` int NULL COMMENT 'test column k1'")); + Assertions.assertTrue(showSql.contains("`k1` int NULL COMMENT \"test column k1\"")); Assertions.assertTrue(showSql.contains("COMMENT 'test table1'")); } + @Test + public void testColumnComment() throws Exception { + String sql = "show create table table1"; + ShowResultSet showResultSet = showCreateTable(sql); + String showSql = showResultSet.getResultRows().get(0).get(1); + Assertions.assertTrue(showSql.contains("`k1` int NULL COMMENT \"test column k1\"")); + Assertions.assertTrue(showSql.contains("`k2` int NULL COMMENT \"test column k2\"")); + Assertions.assertTrue(showSql.contains("`timestamp` date NOT NULL COMMENT \"['0000-01-01', '9999-12-31']\"")); + } + @Test public void testBrief() throws Exception { String sql = "show brief create table table1"; From 84f9a6339857e50ed9e5c84607ae3e1794f87ae7 Mon Sep 17 00:00:00 2001 From: lihangyu Date: Mon, 9 Dec 2024 11:22:18 +0800 Subject: [PATCH 07/14] [fix](variant) use lock to protect schema from threading unsafe (#45026) --- be/src/cloud/cloud_tablet.cpp | 1 + be/src/olap/base_tablet.h | 5 ++++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/be/src/cloud/cloud_tablet.cpp b/be/src/cloud/cloud_tablet.cpp index d545f827bc4e218..93c7128756738c7 100644 --- a/be/src/cloud/cloud_tablet.cpp +++ b/be/src/cloud/cloud_tablet.cpp @@ -220,6 +220,7 @@ Status CloudTablet::sync_if_not_running() { } TabletSchemaSPtr CloudTablet::merged_tablet_schema() const { + std::shared_lock rlock(_meta_lock); return _merged_tablet_schema; } diff --git a/be/src/olap/base_tablet.h b/be/src/olap/base_tablet.h index bd46cdbbe14854a..c6de447200f87c0 100644 --- a/be/src/olap/base_tablet.h +++ b/be/src/olap/base_tablet.h @@ -279,7 +279,10 @@ class BaseTablet { int64_t limit); // Return the merged schema of all rowsets - virtual TabletSchemaSPtr merged_tablet_schema() const { return _max_version_schema; } + virtual TabletSchemaSPtr merged_tablet_schema() const { + std::shared_lock rlock(_meta_lock); + return _max_version_schema; + } void traverse_rowsets(std::function visitor, bool include_stale = false) { From f4ca6da97ccf6f8195873863020c135524708530 Mon Sep 17 00:00:00 2001 From: zhengyu Date: Mon, 9 Dec 2024 12:17:10 +0800 Subject: [PATCH 08/14] [fix](cloud) fix CHECK failed when transmit from non-TTL to normal (#45057) we need not transmit non-TTL cache to normal, just exclude them. --- be/src/io/cache/block_file_cache.cpp | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/be/src/io/cache/block_file_cache.cpp b/be/src/io/cache/block_file_cache.cpp index 528ace90e3e7d70..2a59a5158e46c2c 100644 --- a/be/src/io/cache/block_file_cache.cpp +++ b/be/src/io/cache/block_file_cache.cpp @@ -1093,15 +1093,16 @@ bool BlockFileCache::remove_if_ttl_file_unlock(const UInt128Wrapper& file_key, b _key_to_time.find(file_key) != _key_to_time.end()) { if (!remove_directly) { for (auto& [_, cell] : _files[file_key]) { - if (cell.file_block->cache_type() == FileCacheType::TTL) { - Status st = cell.file_block->update_expiration_time(0); - if (!st.ok()) { - LOG_WARNING("Failed to update expiration time to 0").error(st); - } + if (cell.file_block->cache_type() != FileCacheType::TTL) { + continue; + } + Status st = cell.file_block->update_expiration_time(0); + if (!st.ok()) { + LOG_WARNING("Failed to update expiration time to 0").error(st); } if (cell.file_block->cache_type() == FileCacheType::NORMAL) continue; - auto st = cell.file_block->change_cache_type_between_ttl_and_others( + st = cell.file_block->change_cache_type_between_ttl_and_others( FileCacheType::NORMAL); if (st.ok()) { if (cell.queue_iterator) { From 2bc011f33a7c350832c6fc266d0dfbd81a3a64b6 Mon Sep 17 00:00:00 2001 From: Mryange Date: Mon, 9 Dec 2024 12:40:18 +0800 Subject: [PATCH 09/14] [feature](exchange) enable shared exchange sink buffer to reduce RPC concurrency. (#44850) ### What problem does this PR solve? In the past, each exchange sink had its own sink buffer. If the query concurrency is n, there would be n * n RPCs running concurrently in a typical shuffle scenario (each sender instance can send data to all downstream instances). Here, we introduce support for shared sink buffers. This does not reduce the total number of RPCs but can limit the number of concurrent RPCs. --- be/src/pipeline/exec/exchange_sink_buffer.cpp | 144 +++++++------ be/src/pipeline/exec/exchange_sink_buffer.h | 115 +++++++--- .../pipeline/exec/exchange_sink_operator.cpp | 85 ++++++-- be/src/pipeline/exec/exchange_sink_operator.h | 34 ++- be/src/pipeline/pipeline_fragment_context.cpp | 24 ++- be/src/runtime/runtime_state.h | 5 + be/src/vec/sink/vdata_stream_sender.h | 10 +- be/test/vec/exec/exchange_sink_test.cpp | 196 ++++++++++++++++++ be/test/vec/exec/exchange_sink_test.h | 180 ++++++++++++++++ .../org/apache/doris/qe/SessionVariable.java | 7 + gensrc/thrift/PaloInternalService.thrift | 2 +- 11 files changed, 683 insertions(+), 119 deletions(-) create mode 100644 be/test/vec/exec/exchange_sink_test.cpp create mode 100644 be/test/vec/exec/exchange_sink_test.h diff --git a/be/src/pipeline/exec/exchange_sink_buffer.cpp b/be/src/pipeline/exec/exchange_sink_buffer.cpp index 6e6108d13a919f8..65e7698737076ef 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.cpp +++ b/be/src/pipeline/exec/exchange_sink_buffer.cpp @@ -87,19 +87,22 @@ void BroadcastPBlockHolderMemLimiter::release(const BroadcastPBlockHolder& holde } // namespace vectorized namespace pipeline { -ExchangeSinkBuffer::ExchangeSinkBuffer(PUniqueId query_id, PlanNodeId dest_node_id, int send_id, - int be_number, RuntimeState* state, - ExchangeSinkLocalState* parent) +ExchangeSinkBuffer::ExchangeSinkBuffer(PUniqueId query_id, PlanNodeId dest_node_id, + RuntimeState* state, + const std::vector& sender_ins_ids) : HasTaskExecutionCtx(state), _queue_capacity(0), - _is_finishing(false), + _is_failed(false), _query_id(std::move(query_id)), _dest_node_id(dest_node_id), - _sender_id(send_id), - _be_number(be_number), _state(state), _context(state->get_query_ctx()), - _parent(parent) {} + _exchange_sink_num(sender_ins_ids.size()) { + for (auto sender_ins_id : sender_ins_ids) { + _queue_deps.emplace(sender_ins_id, nullptr); + _parents.emplace(sender_ins_id, nullptr); + } +} void ExchangeSinkBuffer::close() { // Could not clear the queue here, because there maybe a running rpc want to @@ -110,8 +113,8 @@ void ExchangeSinkBuffer::close() { //_instance_to_request.clear(); } -void ExchangeSinkBuffer::register_sink(TUniqueId fragment_instance_id) { - if (_is_finishing) { +void ExchangeSinkBuffer::construct_request(TUniqueId fragment_instance_id) { + if (_is_failed) { return; } auto low_id = fragment_instance_id.lo; @@ -129,22 +132,27 @@ void ExchangeSinkBuffer::register_sink(TUniqueId fragment_instance_id) { finst_id.set_hi(fragment_instance_id.hi); finst_id.set_lo(fragment_instance_id.lo); _rpc_channel_is_idle[low_id] = true; - _instance_to_receiver_eof[low_id] = false; + _rpc_channel_is_turn_off[low_id] = false; _instance_to_rpc_stats_vec.emplace_back(std::make_shared(low_id)); _instance_to_rpc_stats[low_id] = _instance_to_rpc_stats_vec.back().get(); - _construct_request(low_id, finst_id); + _instance_to_request[low_id] = std::make_shared(); + _instance_to_request[low_id]->mutable_finst_id()->CopyFrom(finst_id); + _instance_to_request[low_id]->mutable_query_id()->CopyFrom(_query_id); + + _instance_to_request[low_id]->set_node_id(_dest_node_id); + _running_sink_count[low_id] = _exchange_sink_num; } Status ExchangeSinkBuffer::add_block(TransmitInfo&& request) { - if (_is_finishing) { + if (_is_failed) { return Status::OK(); } - auto ins_id = request.channel->_fragment_instance_id.lo; + auto ins_id = request.channel->dest_ins_id(); if (!_instance_to_package_queue_mutex.contains(ins_id)) { return Status::InternalError("fragment_instance_id {} not do register_sink", print_id(request.channel->_fragment_instance_id)); } - if (_is_receiver_eof(ins_id)) { + if (_rpc_channel_is_turn_off[ins_id]) { return Status::EndOfFile("receiver eof"); } bool send_now = false; @@ -158,12 +166,15 @@ Status ExchangeSinkBuffer::add_block(TransmitInfo&& request) { if (request.block) { RETURN_IF_ERROR( BeExecVersionManager::check_be_exec_version(request.block->be_exec_version())); - COUNTER_UPDATE(_parent->memory_used_counter(), request.block->ByteSizeLong()); + COUNTER_UPDATE(request.channel->_parent->memory_used_counter(), + request.block->ByteSizeLong()); } _instance_to_package_queue[ins_id].emplace(std::move(request)); _total_queue_size++; - if (_queue_dependency && _total_queue_size > _queue_capacity) { - _queue_dependency->block(); + if (_total_queue_size > _queue_capacity) { + for (auto& [_, dep] : _queue_deps) { + dep->block(); + } } } if (send_now) { @@ -174,15 +185,15 @@ Status ExchangeSinkBuffer::add_block(TransmitInfo&& request) { } Status ExchangeSinkBuffer::add_block(BroadcastTransmitInfo&& request) { - if (_is_finishing) { + if (_is_failed) { return Status::OK(); } - auto ins_id = request.channel->_fragment_instance_id.lo; + auto ins_id = request.channel->dest_ins_id(); if (!_instance_to_package_queue_mutex.contains(ins_id)) { return Status::InternalError("fragment_instance_id {} not do register_sink", print_id(request.channel->_fragment_instance_id)); } - if (_is_receiver_eof(ins_id)) { + if (_rpc_channel_is_turn_off[ins_id]) { return Status::EndOfFile("receiver eof"); } bool send_now = false; @@ -209,16 +220,17 @@ Status ExchangeSinkBuffer::add_block(BroadcastTransmitInfo&& request) { Status ExchangeSinkBuffer::_send_rpc(InstanceLoId id) { std::unique_lock lock(*_instance_to_package_queue_mutex[id]); - DCHECK(_rpc_channel_is_idle[id] == false); - std::queue>& q = _instance_to_package_queue[id]; std::queue>& broadcast_q = _instance_to_broadcast_package_queue[id]; - if (_is_finishing) { + if (_is_failed) { _turn_off_channel(id, lock); return Status::OK(); } + if (_rpc_channel_is_turn_off[id]) { + return Status::OK(); + } if (!q.empty()) { // If we have data to shuffle which is not broadcasted @@ -226,6 +238,8 @@ Status ExchangeSinkBuffer::_send_rpc(InstanceLoId id) { auto& brpc_request = _instance_to_request[id]; brpc_request->set_eos(request.eos); brpc_request->set_packet_seq(_instance_to_seq[id]++); + brpc_request->set_sender_id(request.channel->_parent->sender_id()); + brpc_request->set_be_number(request.channel->_parent->be_number()); if (request.block && !request.block->column_metas().empty()) { brpc_request->set_allocated_block(request.block.get()); } @@ -271,14 +285,16 @@ Status ExchangeSinkBuffer::_send_rpc(InstanceLoId id) { } else if (!s.ok()) { _failed(id, fmt::format("exchange req success but status isn't ok: {}", s.to_string())); + return; } else if (eos) { _ended(id); - } else { - s = _send_rpc(id); - if (!s) { - _failed(id, fmt::format("exchange req success but status isn't ok: {}", - s.to_string())); - } + } + // The eos here only indicates that the current exchange sink has reached eos. + // However, the queue still contains data from other exchange sinks, so RPCs need to continue being sent. + s = _send_rpc(id); + if (!s) { + _failed(id, + fmt::format("exchange req success but status isn't ok: {}", s.to_string())); } }); { @@ -296,13 +312,16 @@ Status ExchangeSinkBuffer::_send_rpc(InstanceLoId id) { } } if (request.block) { - COUNTER_UPDATE(_parent->memory_used_counter(), -request.block->ByteSizeLong()); + COUNTER_UPDATE(request.channel->_parent->memory_used_counter(), + -request.block->ByteSizeLong()); static_cast(brpc_request->release_block()); } q.pop(); _total_queue_size--; - if (_queue_dependency && _total_queue_size <= _queue_capacity) { - _queue_dependency->set_ready(); + if (_total_queue_size <= _queue_capacity) { + for (auto& [_, dep] : _queue_deps) { + dep->set_ready(); + } } } else if (!broadcast_q.empty()) { // If we have data to shuffle which is broadcasted @@ -310,6 +329,8 @@ Status ExchangeSinkBuffer::_send_rpc(InstanceLoId id) { auto& brpc_request = _instance_to_request[id]; brpc_request->set_eos(request.eos); brpc_request->set_packet_seq(_instance_to_seq[id]++); + brpc_request->set_sender_id(request.channel->_parent->sender_id()); + brpc_request->set_be_number(request.channel->_parent->be_number()); if (request.block_holder->get_block() && !request.block_holder->get_block()->column_metas().empty()) { brpc_request->set_allocated_block(request.block_holder->get_block()); @@ -352,14 +373,17 @@ Status ExchangeSinkBuffer::_send_rpc(InstanceLoId id) { } else if (!s.ok()) { _failed(id, fmt::format("exchange req success but status isn't ok: {}", s.to_string())); + return; } else if (eos) { _ended(id); - } else { - s = _send_rpc(id); - if (!s) { - _failed(id, fmt::format("exchange req success but status isn't ok: {}", - s.to_string())); - } + } + + // The eos here only indicates that the current exchange sink has reached eos. + // However, the queue still contains data from other exchange sinks, so RPCs need to continue being sent. + s = _send_rpc(id); + if (!s) { + _failed(id, + fmt::format("exchange req success but status isn't ok: {}", s.to_string())); } }); { @@ -387,16 +411,6 @@ Status ExchangeSinkBuffer::_send_rpc(InstanceLoId id) { return Status::OK(); } -void ExchangeSinkBuffer::_construct_request(InstanceLoId id, PUniqueId finst_id) { - _instance_to_request[id] = std::make_shared(); - _instance_to_request[id]->mutable_finst_id()->CopyFrom(finst_id); - _instance_to_request[id]->mutable_query_id()->CopyFrom(_query_id); - - _instance_to_request[id]->set_node_id(_dest_node_id); - _instance_to_request[id]->set_sender_id(_sender_id); - _instance_to_request[id]->set_be_number(_be_number); -} - void ExchangeSinkBuffer::_ended(InstanceLoId id) { if (!_instance_to_package_queue_mutex.template contains(id)) { std::stringstream ss; @@ -411,24 +425,29 @@ void ExchangeSinkBuffer::_ended(InstanceLoId id) { __builtin_unreachable(); } else { std::unique_lock lock(*_instance_to_package_queue_mutex[id]); - _turn_off_channel(id, lock); + _running_sink_count[id]--; + if (_running_sink_count[id] == 0) { + _turn_off_channel(id, lock); + } } } void ExchangeSinkBuffer::_failed(InstanceLoId id, const std::string& err) { - _is_finishing = true; + _is_failed = true; _context->cancel(Status::Cancelled(err)); } void ExchangeSinkBuffer::_set_receiver_eof(InstanceLoId id) { std::unique_lock lock(*_instance_to_package_queue_mutex[id]); - _instance_to_receiver_eof[id] = true; + // When the receiving side reaches eof, it means the receiver has finished early. + // The remaining data in the current rpc_channel does not need to be sent, + // and the rpc_channel should be turned off immediately. _turn_off_channel(id, lock); std::queue>& broadcast_q = _instance_to_broadcast_package_queue[id]; for (; !broadcast_q.empty(); broadcast_q.pop()) { if (broadcast_q.front().block_holder->get_block()) { - COUNTER_UPDATE(_parent->memory_used_counter(), + COUNTER_UPDATE(broadcast_q.front().channel->_parent->memory_used_counter(), -broadcast_q.front().block_holder->get_block()->ByteSizeLong()); } } @@ -440,7 +459,8 @@ void ExchangeSinkBuffer::_set_receiver_eof(InstanceLoId id) { std::queue>& q = _instance_to_package_queue[id]; for (; !q.empty(); q.pop()) { if (q.front().block) { - COUNTER_UPDATE(_parent->memory_used_counter(), -q.front().block->ByteSizeLong()); + COUNTER_UPDATE(q.front().channel->_parent->memory_used_counter(), + -q.front().block->ByteSizeLong()); } } @@ -450,22 +470,22 @@ void ExchangeSinkBuffer::_set_receiver_eof(InstanceLoId id) { } } -bool ExchangeSinkBuffer::_is_receiver_eof(InstanceLoId id) { - std::unique_lock lock(*_instance_to_package_queue_mutex[id]); - return _instance_to_receiver_eof[id]; -} - // The unused parameter `with_lock` is to ensure that the function is called when the lock is held. void ExchangeSinkBuffer::_turn_off_channel(InstanceLoId id, std::unique_lock& /*with_lock*/) { if (!_rpc_channel_is_idle[id]) { _rpc_channel_is_idle[id] = true; } - _instance_to_receiver_eof[id] = true; - + // Ensure that each RPC is turned off only once. + if (_rpc_channel_is_turn_off[id]) { + return; + } + _rpc_channel_is_turn_off[id] = true; auto weak_task_ctx = weak_task_exec_ctx(); if (auto pip_ctx = weak_task_ctx.lock()) { - _parent->on_channel_finished(id); + for (auto& [_, parent] : _parents) { + parent->on_channel_finished(id); + } } } @@ -509,7 +529,7 @@ void ExchangeSinkBuffer::update_profile(RuntimeProfile* profile) { auto* _max_rpc_timer = ADD_TIMER_WITH_LEVEL(profile, "RpcMaxTime", 1); auto* _min_rpc_timer = ADD_TIMER(profile, "RpcMinTime"); auto* _sum_rpc_timer = ADD_TIMER(profile, "RpcSumTime"); - auto* _count_rpc = ADD_COUNTER_WITH_LEVEL(profile, "RpcCount", TUnit::UNIT, 1); + auto* _count_rpc = ADD_COUNTER(profile, "RpcCount", TUnit::UNIT); auto* _avg_rpc_timer = ADD_TIMER(profile, "RpcAvgTime"); int64_t max_rpc_time = 0, min_rpc_time = 0; diff --git a/be/src/pipeline/exec/exchange_sink_buffer.h b/be/src/pipeline/exec/exchange_sink_buffer.h index 22a1452f8d545c8..b2eb32414feca21 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.h +++ b/be/src/pipeline/exec/exchange_sink_buffer.h @@ -169,13 +169,61 @@ class ExchangeSendCallback : public ::doris::DummyBrpcCallback { bool _eos; }; -// Each ExchangeSinkOperator have one ExchangeSinkBuffer -class ExchangeSinkBuffer final : public HasTaskExecutionCtx { +// ExchangeSinkBuffer can either be shared among multiple ExchangeSinkLocalState instances +// or be individually owned by each ExchangeSinkLocalState. +// The following describes the scenario where ExchangeSinkBuffer is shared among multiple ExchangeSinkLocalState instances. +// Of course, individual ownership can be seen as a special case where only one ExchangeSinkLocalState shares the buffer. + +// A sink buffer contains multiple rpc_channels. +// Each rpc_channel corresponds to a target instance on the receiving side. +// Data is sent using a ping-pong mode within each rpc_channel, +// meaning that at most one RPC can exist in a single rpc_channel at a time. +// The next RPC can only be sent after the previous one has completed. +// +// Each exchange sink sends data to all target instances on the receiving side. +// If the concurrency is 3, a single rpc_channel will be used simultaneously by three exchange sinks. + +/* + +-----------+ +-----------+ +-----------+ + |dest ins id| |dest ins id| |dest ins id| + | | | | | | + +----+------+ +-----+-----+ +------+----+ + | | | + | | | + +----------------+ +----------------+ +----------------+ + | | | | | | + sink buffer -------- | rpc_channel | | rpc_channel | | rpc_channel | + | | | | | | + +-------+--------+ +----------------+ +----------------+ + | | | + |------------------------+----------------------+ + | | | + | | | + +-----------------+ +-------+---------+ +-------+---------+ + | | | | | | + | exchange sink | | exchange sink | | exchange sink | + | | | | | | + +-----------------+ +-----------------+ +-----------------+ +*/ + +#ifdef BE_TEST +void transmit_blockv2(PBackendService_Stub& stub, + std::unique_ptr>> + closure); +#endif +class ExchangeSinkBuffer : public HasTaskExecutionCtx { public: - ExchangeSinkBuffer(PUniqueId query_id, PlanNodeId dest_node_id, int send_id, int be_number, - RuntimeState* state, ExchangeSinkLocalState* parent); + ExchangeSinkBuffer(PUniqueId query_id, PlanNodeId dest_node_id, RuntimeState* state, + const std::vector& sender_ins_ids); + +#ifdef BE_TEST + ExchangeSinkBuffer(RuntimeState* state, int64_t sinknum) + : HasTaskExecutionCtx(state), _exchange_sink_num(sinknum) {}; +#endif ~ExchangeSinkBuffer() override = default; - void register_sink(TUniqueId); + + void construct_request(TUniqueId); Status add_block(TransmitInfo&& request); Status add_block(BroadcastTransmitInfo&& request); @@ -183,17 +231,18 @@ class ExchangeSinkBuffer final : public HasTaskExecutionCtx { void update_rpc_time(InstanceLoId id, int64_t start_rpc_time, int64_t receive_rpc_time); void update_profile(RuntimeProfile* profile); - void set_dependency(std::shared_ptr queue_dependency, - std::shared_ptr finish_dependency) { - _queue_dependency = queue_dependency; - _finish_dependency = finish_dependency; - } - - void set_broadcast_dependency(std::shared_ptr broadcast_dependency) { - _broadcast_dependency = broadcast_dependency; + void set_dependency(InstanceLoId sender_ins_id, std::shared_ptr queue_dependency, + ExchangeSinkLocalState* local_state) { + DCHECK(_queue_deps.contains(sender_ins_id)); + DCHECK(_parents.contains(sender_ins_id)); + _queue_deps[sender_ins_id] = queue_dependency; + _parents[sender_ins_id] = local_state; } - +#ifdef BE_TEST +public: +#else private: +#endif friend class ExchangeSinkLocalState; phmap::flat_hash_map> @@ -214,7 +263,10 @@ class ExchangeSinkBuffer final : public HasTaskExecutionCtx { // One channel is corresponding to a downstream instance. phmap::flat_hash_map _rpc_channel_is_idle; - phmap::flat_hash_map _instance_to_receiver_eof; + // There could be multiple situations that cause an rpc_channel to be turned off, + // such as receiving the eof, manual cancellation by the user, or all sinks reaching eos. + // Therefore, it is necessary to prevent an rpc_channel from being turned off multiple times. + phmap::flat_hash_map _rpc_channel_is_turn_off; struct RpcInstanceStatistics { RpcInstanceStatistics(InstanceLoId id) : inst_lo_id(id) {} InstanceLoId inst_lo_id; @@ -226,32 +278,43 @@ class ExchangeSinkBuffer final : public HasTaskExecutionCtx { std::vector> _instance_to_rpc_stats_vec; phmap::flat_hash_map _instance_to_rpc_stats; - std::atomic _is_finishing; + // It is set to true only when an RPC fails. Currently, we do not have an error retry mechanism. + // If an RPC error occurs, the query will be canceled. + std::atomic _is_failed; PUniqueId _query_id; PlanNodeId _dest_node_id; - // Sender instance id, unique within a fragment. StreamSender save the variable - int _sender_id; - int _be_number; std::atomic _rpc_count = 0; RuntimeState* _state = nullptr; QueryContext* _context = nullptr; Status _send_rpc(InstanceLoId); - // must hold the _instance_to_package_queue_mutex[id] mutex to opera - void _construct_request(InstanceLoId id, PUniqueId); + +#ifndef BE_TEST inline void _ended(InstanceLoId id); inline void _failed(InstanceLoId id, const std::string& err); inline void _set_receiver_eof(InstanceLoId id); - inline bool _is_receiver_eof(InstanceLoId id); inline void _turn_off_channel(InstanceLoId id, std::unique_lock& with_lock); + +#else + virtual void _ended(InstanceLoId id); + virtual void _failed(InstanceLoId id, const std::string& err); + virtual void _set_receiver_eof(InstanceLoId id); + virtual void _turn_off_channel(InstanceLoId id, std::unique_lock& with_lock); +#endif + void get_max_min_rpc_time(int64_t* max_time, int64_t* min_time); int64_t get_sum_rpc_time(); std::atomic _total_queue_size = 0; - std::shared_ptr _queue_dependency = nullptr; - std::shared_ptr _finish_dependency = nullptr; - std::shared_ptr _broadcast_dependency = nullptr; - ExchangeSinkLocalState* _parent = nullptr; + + // _running_sink_count is used to track how many sinks have not finished yet. + // It is only decremented when eos is reached. + phmap::flat_hash_map _running_sink_count; + // _queue_deps is used for memory control. + phmap::flat_hash_map> _queue_deps; + // The ExchangeSinkLocalState in _parents is only used in _turn_off_channel. + phmap::flat_hash_map _parents; + const int64_t _exchange_sink_num; }; } // namespace pipeline diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index dfa6df392b74ba9..04b9653e9c82996 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -32,6 +32,8 @@ #include "pipeline/exec/operator.h" #include "pipeline/exec/sort_source_operator.h" #include "pipeline/local_exchange/local_exchange_sink_operator.h" +#include "pipeline/local_exchange/local_exchange_source_operator.h" +#include "pipeline/pipeline_fragment_context.h" #include "util/runtime_profile.h" #include "util/uid_util.h" #include "vec/columns/column_const.h" @@ -100,6 +102,24 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf fmt::format("WaitForLocalExchangeBuffer{}", i), TUnit ::TIME_NS, timer_name, 1)); } _wait_broadcast_buffer_timer = ADD_CHILD_TIMER(_profile, "WaitForBroadcastBuffer", timer_name); + + size_t local_size = 0; + for (int i = 0; i < channels.size(); ++i) { + if (channels[i]->is_local()) { + local_size++; + _last_local_channel_idx = i; + } + } + only_local_exchange = local_size == channels.size(); + + if (!only_local_exchange) { + _sink_buffer = p.get_sink_buffer(state->fragment_instance_id().lo); + register_channels(_sink_buffer.get()); + _queue_dependency = Dependency::create_shared(_parent->operator_id(), _parent->node_id(), + "ExchangeSinkQueueDependency", true); + _sink_buffer->set_dependency(state->fragment_instance_id().lo, _queue_dependency, this); + } + return Status::OK(); } @@ -149,20 +169,10 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) { id.set_hi(_state->query_id().hi); id.set_lo(_state->query_id().lo); - if (!only_local_exchange) { - _sink_buffer = std::make_unique(id, p._dest_node_id, _sender_id, - _state->be_number(), state, this); - register_channels(_sink_buffer.get()); - _queue_dependency = Dependency::create_shared(_parent->operator_id(), _parent->node_id(), - "ExchangeSinkQueueDependency", true); - _sink_buffer->set_dependency(_queue_dependency, _finish_dependency); - } - if ((_part_type == TPartitionType::UNPARTITIONED || channels.size() == 1) && !only_local_exchange) { _broadcast_dependency = Dependency::create_shared( _parent->operator_id(), _parent->node_id(), "BroadcastDependency", true); - _sink_buffer->set_broadcast_dependency(_broadcast_dependency); _broadcast_pb_mem_limiter = vectorized::BroadcastPBlockHolderMemLimiter::create_shared(_broadcast_dependency); } else if (local_size > 0) { @@ -301,7 +311,8 @@ segment_v2::CompressionTypePB ExchangeSinkLocalState::compression_type() const { ExchangeSinkOperatorX::ExchangeSinkOperatorX( RuntimeState* state, const RowDescriptor& row_desc, int operator_id, - const TDataStreamSink& sink, const std::vector& destinations) + const TDataStreamSink& sink, const std::vector& destinations, + const std::vector& fragment_instance_ids) : DataSinkOperatorX(operator_id, sink.dest_node_id), _texprs(sink.output_partition.partition_exprs), _row_desc(row_desc), @@ -315,7 +326,8 @@ ExchangeSinkOperatorX::ExchangeSinkOperatorX( _tablet_sink_tuple_id(sink.tablet_sink_tuple_id), _tablet_sink_txn_id(sink.tablet_sink_txn_id), _t_tablet_sink_exprs(&sink.tablet_sink_exprs), - _enable_local_merge_sort(state->enable_local_merge_sort()) { + _enable_local_merge_sort(state->enable_local_merge_sort()), + _fragment_instance_ids(fragment_instance_ids) { DCHECK_GT(destinations.size(), 0); DCHECK(sink.output_partition.type == TPartitionType::UNPARTITIONED || sink.output_partition.type == TPartitionType::HASH_PARTITIONED || @@ -360,6 +372,11 @@ Status ExchangeSinkOperatorX::open(RuntimeState* state) { } RETURN_IF_ERROR(vectorized::VExpr::open(_tablet_sink_expr_ctxs, state)); } + std::vector ins_ids; + for (auto fragment_instance_id : _fragment_instance_ids) { + ins_ids.push_back(fragment_instance_id.lo); + } + _sink_buffer = _create_buffer(ins_ids); return Status::OK(); } @@ -620,7 +637,7 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block void ExchangeSinkLocalState::register_channels(pipeline::ExchangeSinkBuffer* buffer) { for (auto& channel : channels) { - channel->register_exchange_buffer(buffer); + channel->set_exchange_buffer(buffer); } } @@ -669,8 +686,8 @@ std::string ExchangeSinkLocalState::debug_string(int indentation_level) const { fmt::format_to(debug_string_buffer, ", Sink Buffer: (_is_finishing = {}, blocks in queue: {}, queue capacity: " "{}, queue dep: {}), _reach_limit: {}, working channels: {}", - _sink_buffer->_is_finishing.load(), _sink_buffer->_total_queue_size, - _sink_buffer->_queue_capacity, (void*)_sink_buffer->_queue_dependency.get(), + _sink_buffer->_is_failed.load(), _sink_buffer->_total_queue_size, + _sink_buffer->_queue_capacity, (void*)_queue_dependency.get(), _reach_limit.load(), _working_channels_count.load()); } return fmt::to_string(debug_string_buffer); @@ -724,4 +741,42 @@ DataDistribution ExchangeSinkOperatorX::required_data_distribution() const { return DataSinkOperatorX::required_data_distribution(); } +std::shared_ptr ExchangeSinkOperatorX::_create_buffer( + const std::vector& sender_ins_ids) { + PUniqueId id; + id.set_hi(_state->query_id().hi); + id.set_lo(_state->query_id().lo); + auto sink_buffer = + std::make_unique(id, _dest_node_id, state(), sender_ins_ids); + for (const auto& _dest : _dests) { + sink_buffer->construct_request(_dest.fragment_instance_id); + } + return sink_buffer; +} + +// For a normal shuffle scenario, if the concurrency is n, +// there can be up to n * n RPCs in the current fragment. +// Therefore, a shared sink buffer is used here to limit the number of concurrent RPCs. +// (Note: This does not reduce the total number of RPCs.) +// In a merge sort scenario, there are only n RPCs, so a shared sink buffer is not needed. +/// TODO: Modify this to let FE handle the judgment instead of BE. +std::shared_ptr ExchangeSinkOperatorX::get_sink_buffer( + InstanceLoId sender_ins_id) { + if (!_child) { + throw doris::Exception(ErrorCode::INTERNAL_ERROR, + "ExchangeSinkOperatorX did not correctly set the child."); + } + // When the child is SortSourceOperatorX or LocalExchangeSourceOperatorX, + // it is an order-by scenario. + // In this case, there is only one target instance, and no n * n RPC concurrency will occur. + // Therefore, sharing a sink buffer is not necessary. + if (std::dynamic_pointer_cast(_child) || + std::dynamic_pointer_cast(_child)) { + return _create_buffer({sender_ins_id}); + } + if (_state->enable_shared_exchange_sink_buffer()) { + return _sink_buffer; + } + return _create_buffer({sender_ins_id}); +} } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h index 91ee1bd27a63e79..8d094b43f613f0d 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.h +++ b/be/src/pipeline/exec/exchange_sink_operator.h @@ -61,6 +61,14 @@ class ExchangeSinkLocalState final : public PipelineXSinkLocalState<> { parent->get_name() + "_FINISH_DEPENDENCY", false); } +#ifdef BE_TEST + ExchangeSinkLocalState(RuntimeState* state) : Base(nullptr, state) { + _profile = state->obj_pool()->add(new RuntimeProfile("mock")); + _memory_used_counter = + _profile->AddHighWaterMarkCounter("MemoryUsage", TUnit::BYTES, "", 1); + } +#endif + std::vector dependencies() const override { std::vector dep_vec; if (_queue_dependency) { @@ -88,7 +96,12 @@ class ExchangeSinkLocalState final : public PipelineXSinkLocalState<> { bool is_finished() const override { return _reach_limit.load(); } void set_reach_limit() { _reach_limit = true; }; + // sender_id indicates which instance within a fragment, while be_number indicates which instance + // across all fragments. For example, with 3 BEs and 8 instances, the range of sender_id would be 0 to 24, + // and the range of be_number would be from n + 0 to n + 24. + // Since be_number is a required field, it still needs to be set for compatibility with older code. [[nodiscard]] int sender_id() const { return _sender_id; } + [[nodiscard]] int be_number() const { return _state->be_number(); } std::string name_suffix() override; segment_v2::CompressionTypePB compression_type() const; @@ -112,7 +125,7 @@ class ExchangeSinkLocalState final : public PipelineXSinkLocalState<> { friend class vectorized::Channel; friend class vectorized::BlockSerializer; - std::unique_ptr _sink_buffer = nullptr; + std::shared_ptr _sink_buffer = nullptr; RuntimeProfile::Counter* _serialize_batch_timer = nullptr; RuntimeProfile::Counter* _compress_timer = nullptr; RuntimeProfile::Counter* _bytes_sent_counter = nullptr; @@ -197,7 +210,8 @@ class ExchangeSinkOperatorX final : public DataSinkOperatorX& destinations); + const std::vector& destinations, + const std::vector& fragment_instance_ids); Status init(const TDataSink& tsink) override; RuntimeState* state() { return _state; } @@ -209,6 +223,14 @@ class ExchangeSinkOperatorX final : public DataSinkOperatorX get_sink_buffer(InstanceLoId sender_ins_id); + private: friend class ExchangeSinkLocalState; @@ -225,6 +247,13 @@ class ExchangeSinkOperatorX final : public DataSinkOperatorX>& channel2rows, vectorized::Block* block, bool eos); + + // Use ExchangeSinkOperatorX to create a sink buffer. + // The sink buffer can be shared among multiple ExchangeSinkLocalState instances, + // or each ExchangeSinkLocalState can have its own sink buffer. + std::shared_ptr _create_buffer( + const std::vector& sender_ins_ids); + std::shared_ptr _sink_buffer = nullptr; RuntimeState* _state = nullptr; const std::vector _texprs; @@ -264,6 +293,7 @@ class ExchangeSinkOperatorX final : public DataSinkOperatorX& _fragment_instance_ids; }; } // namespace pipeline diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index 8ceb63eb99324c6..8ab0f1d151568d9 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -260,7 +260,7 @@ Status PipelineFragmentContext::prepare(const doris::TPipelineFragmentParams& re _runtime_state = RuntimeState::create_unique( request.query_id, request.fragment_id, request.query_options, _query_ctx->query_globals, _exec_env, _query_ctx.get()); - + _runtime_state->set_task_execution_context(shared_from_this()); SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(_runtime_state->query_mem_tracker()); if (request.__isset.backend_id) { _runtime_state->set_backend_id(request.backend_id); @@ -296,6 +296,14 @@ Status PipelineFragmentContext::prepare(const doris::TPipelineFragmentParams& re if (local_params.__isset.topn_filter_descs) { _query_ctx->init_runtime_predicates(local_params.topn_filter_descs); } + + // init fragment_instance_ids + const auto target_size = request.local_params.size(); + _fragment_instance_ids.resize(target_size); + for (size_t i = 0; i < request.local_params.size(); i++) { + auto fragment_instance_id = request.local_params[i].fragment_instance_id; + _fragment_instance_ids[i] = fragment_instance_id; + } } { @@ -353,7 +361,6 @@ Status PipelineFragmentContext::_build_pipeline_tasks(const doris::TPipelineFrag _total_tasks = 0; const auto target_size = request.local_params.size(); _tasks.resize(target_size); - _fragment_instance_ids.resize(target_size); _runtime_filter_states.resize(target_size); _task_runtime_states.resize(_pipelines.size()); for (size_t pip_idx = 0; pip_idx < _pipelines.size(); pip_idx++) { @@ -365,8 +372,6 @@ Status PipelineFragmentContext::_build_pipeline_tasks(const doris::TPipelineFrag auto pre_and_submit = [&](int i, PipelineFragmentContext* ctx) { const auto& local_params = request.local_params[i]; auto fragment_instance_id = local_params.fragment_instance_id; - _fragment_instance_ids[i] = fragment_instance_id; - _runtime_filter_states[i] = RuntimeFilterParamsContext::create(_query_ctx.get()); std::unique_ptr runtime_filter_mgr = std::make_unique( request.query_id, _runtime_filter_states[i], _query_ctx->query_mem_tracker, false); @@ -1007,7 +1012,8 @@ Status PipelineFragmentContext::_create_data_sink(ObjectPool* pool, const TDataS return Status::InternalError("Missing data stream sink."); } _sink.reset(new ExchangeSinkOperatorX(state, row_desc, next_sink_operator_id(), - thrift_sink.stream_sink, params.destinations)); + thrift_sink.stream_sink, params.destinations, + _fragment_instance_ids)); break; } case TDataSinkType::RESULT_SINK: { @@ -1134,10 +1140,10 @@ Status PipelineFragmentContext::_create_data_sink(ObjectPool* pool, const TDataS // 2. create and set sink operator of data stream sender for new pipeline DataSinkOperatorPtr sink_op; - sink_op.reset( - new ExchangeSinkOperatorX(state, *_row_desc, next_sink_operator_id(), - thrift_sink.multi_cast_stream_sink.sinks[i], - thrift_sink.multi_cast_stream_sink.destinations[i])); + sink_op.reset(new ExchangeSinkOperatorX( + state, *_row_desc, next_sink_operator_id(), + thrift_sink.multi_cast_stream_sink.sinks[i], + thrift_sink.multi_cast_stream_sink.destinations[i], _fragment_instance_ids)); RETURN_IF_ERROR(new_pipeline->set_sink(sink_op)); { diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index a49567109a3b31b..1e7c1e579f77355 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -587,6 +587,11 @@ class RuntimeState { _query_options.enable_local_merge_sort; } + bool enable_shared_exchange_sink_buffer() const { + return _query_options.__isset.enable_shared_exchange_sink_buffer && + _query_options.enable_shared_exchange_sink_buffer; + } + int64_t min_revocable_mem() const { if (_query_options.__isset.min_revocable_mem) { return std::max(_query_options.min_revocable_mem, (int64_t)1); diff --git a/be/src/vec/sink/vdata_stream_sender.h b/be/src/vec/sink/vdata_stream_sender.h index 4999602fdf49a78..16ea49e443c6306 100644 --- a/be/src/vec/sink/vdata_stream_sender.h +++ b/be/src/vec/sink/vdata_stream_sender.h @@ -76,6 +76,9 @@ namespace vectorized { class BlockSerializer { public: BlockSerializer(pipeline::ExchangeSinkLocalState* parent, bool is_local = true); +#ifdef BE_TEST + BlockSerializer() : _batch_size(0) {}; +#endif Status next_serialized_block(Block* src, PBlock* dest, size_t num_receivers, bool* serialized, bool eos, const std::vector* rows = nullptr); Status serialize_block(PBlock* dest, size_t num_receivers = 1); @@ -165,10 +168,9 @@ class Channel { return Status::OK(); } - void register_exchange_buffer(pipeline::ExchangeSinkBuffer* buffer) { - _buffer = buffer; - _buffer->register_sink(_fragment_instance_id); - } + void set_exchange_buffer(pipeline::ExchangeSinkBuffer* buffer) { _buffer = buffer; } + + InstanceLoId dest_ins_id() const { return _fragment_instance_id.lo; } std::shared_ptr> get_send_callback( InstanceLoId id, bool eos) { diff --git a/be/test/vec/exec/exchange_sink_test.cpp b/be/test/vec/exec/exchange_sink_test.cpp new file mode 100644 index 000000000000000..9576ed71ee2ada9 --- /dev/null +++ b/be/test/vec/exec/exchange_sink_test.cpp @@ -0,0 +1,196 @@ +// 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. + +#include "exchange_sink_test.h" + +#include + +#include +#include + +#include "pipeline/exec/exchange_sink_buffer.h" + +namespace doris::vectorized { +using namespace pipeline; +TEST_F(ExchangeSInkTest, test_normal_end) { + { + auto state = create_runtime_state(); + auto buffer = create_buffer(state); + + auto sink1 = create_sink(state, buffer); + auto sink2 = create_sink(state, buffer); + auto sink3 = create_sink(state, buffer); + + EXPECT_EQ(sink1.add_block(dest_ins_id_1, true), Status::OK()); + EXPECT_EQ(sink1.add_block(dest_ins_id_2, true), Status::OK()); + EXPECT_EQ(sink1.add_block(dest_ins_id_3, true), Status::OK()); + + EXPECT_EQ(sink2.add_block(dest_ins_id_1, true), Status::OK()); + EXPECT_EQ(sink2.add_block(dest_ins_id_2, true), Status::OK()); + EXPECT_EQ(sink2.add_block(dest_ins_id_3, true), Status::OK()); + + EXPECT_EQ(sink3.add_block(dest_ins_id_1, true), Status::OK()); + EXPECT_EQ(sink3.add_block(dest_ins_id_2, true), Status::OK()); + EXPECT_EQ(sink3.add_block(dest_ins_id_3, true), Status::OK()); + + for (auto [id, count] : buffer->_running_sink_count) { + EXPECT_EQ(count, 3) << "id : " << id; + } + + for (auto [id, is_turn_off] : buffer->_rpc_channel_is_turn_off) { + EXPECT_EQ(is_turn_off, false) << "id : " << id; + } + + pop_block(dest_ins_id_1, PopState::accept); + pop_block(dest_ins_id_1, PopState::accept); + pop_block(dest_ins_id_1, PopState::accept); + + pop_block(dest_ins_id_2, PopState::accept); + pop_block(dest_ins_id_2, PopState::accept); + pop_block(dest_ins_id_2, PopState::accept); + + pop_block(dest_ins_id_3, PopState::accept); + pop_block(dest_ins_id_3, PopState::accept); + pop_block(dest_ins_id_3, PopState::accept); + + for (auto [id, count] : buffer->_running_sink_count) { + EXPECT_EQ(count, 0) << "id : " << id; + } + + for (auto [id, is_turn_off] : buffer->_rpc_channel_is_turn_off) { + EXPECT_EQ(is_turn_off, true) << "id : " << id; + } + clear_all_done(); + } +} + +TEST_F(ExchangeSInkTest, test_eof_end) { + { + auto state = create_runtime_state(); + auto buffer = create_buffer(state); + + auto sink1 = create_sink(state, buffer); + auto sink2 = create_sink(state, buffer); + auto sink3 = create_sink(state, buffer); + + EXPECT_EQ(sink1.add_block(dest_ins_id_1, false), Status::OK()); + EXPECT_EQ(sink1.add_block(dest_ins_id_2, false), Status::OK()); + EXPECT_EQ(sink1.add_block(dest_ins_id_3, false), Status::OK()); + + EXPECT_EQ(sink2.add_block(dest_ins_id_1, true), Status::OK()); + EXPECT_EQ(sink2.add_block(dest_ins_id_2, true), Status::OK()); + EXPECT_EQ(sink2.add_block(dest_ins_id_3, true), Status::OK()); + + EXPECT_EQ(sink3.add_block(dest_ins_id_1, false), Status::OK()); + EXPECT_EQ(sink3.add_block(dest_ins_id_2, true), Status::OK()); + EXPECT_EQ(sink3.add_block(dest_ins_id_3, false), Status::OK()); + + for (auto [id, count] : buffer->_running_sink_count) { + EXPECT_EQ(count, 3) << "id : " << id; + } + + for (auto [id, is_turn_off] : buffer->_rpc_channel_is_turn_off) { + EXPECT_EQ(is_turn_off, false) << "id : " << id; + } + + pop_block(dest_ins_id_1, PopState::eof); + EXPECT_EQ(buffer->_rpc_channel_is_turn_off[dest_ins_id_1], true); + EXPECT_TRUE(buffer->_instance_to_package_queue[dest_ins_id_1].empty()); + + pop_block(dest_ins_id_2, PopState::accept); + pop_block(dest_ins_id_2, PopState::accept); + pop_block(dest_ins_id_2, PopState::accept); + + pop_block(dest_ins_id_3, PopState::accept); + pop_block(dest_ins_id_3, PopState::accept); + pop_block(dest_ins_id_3, PopState::accept); + + EXPECT_EQ(buffer->_rpc_channel_is_turn_off[dest_ins_id_1], true); + EXPECT_EQ(buffer->_rpc_channel_is_turn_off[dest_ins_id_2], false) << "not all eos"; + EXPECT_EQ(buffer->_rpc_channel_is_turn_off[dest_ins_id_3], false) << " not all eos"; + + EXPECT_TRUE(sink1.add_block(dest_ins_id_1, true).is()); + EXPECT_EQ(sink1.add_block(dest_ins_id_2, true), Status::OK()); + EXPECT_EQ(sink1.add_block(dest_ins_id_3, true), Status::OK()); + pop_block(dest_ins_id_2, PopState::accept); + pop_block(dest_ins_id_3, PopState::accept); + + EXPECT_EQ(buffer->_rpc_channel_is_turn_off[dest_ins_id_1], true); + EXPECT_EQ(buffer->_rpc_channel_is_turn_off[dest_ins_id_2], true); + EXPECT_EQ(buffer->_rpc_channel_is_turn_off[dest_ins_id_3], false); + EXPECT_EQ(buffer->_running_sink_count[dest_ins_id_3], 1); + + clear_all_done(); + } +} + +TEST_F(ExchangeSInkTest, test_error_end) { + { + auto state = create_runtime_state(); + auto buffer = create_buffer(state); + + auto sink1 = create_sink(state, buffer); + auto sink2 = create_sink(state, buffer); + auto sink3 = create_sink(state, buffer); + + EXPECT_EQ(sink1.add_block(dest_ins_id_1, false), Status::OK()); + EXPECT_EQ(sink1.add_block(dest_ins_id_2, false), Status::OK()); + EXPECT_EQ(sink1.add_block(dest_ins_id_3, false), Status::OK()); + + EXPECT_EQ(sink2.add_block(dest_ins_id_1, false), Status::OK()); + EXPECT_EQ(sink2.add_block(dest_ins_id_2, false), Status::OK()); + EXPECT_EQ(sink2.add_block(dest_ins_id_3, false), Status::OK()); + + EXPECT_EQ(sink3.add_block(dest_ins_id_1, false), Status::OK()); + EXPECT_EQ(sink3.add_block(dest_ins_id_2, false), Status::OK()); + EXPECT_EQ(sink3.add_block(dest_ins_id_3, false), Status::OK()); + + for (auto [id, count] : buffer->_running_sink_count) { + EXPECT_EQ(count, 3) << "id : " << id; + } + + for (auto [id, is_turn_off] : buffer->_rpc_channel_is_turn_off) { + EXPECT_EQ(is_turn_off, false) << "id : " << id; + } + + pop_block(dest_ins_id_2, PopState::error); + + auto orgin_queue_1_size = done_map[dest_ins_id_1].size(); + auto orgin_queue_2_size = done_map[dest_ins_id_2].size(); + auto orgin_queue_3_size = done_map[dest_ins_id_3].size(); + + EXPECT_EQ(sink1.add_block(dest_ins_id_1, false), Status::OK()); + EXPECT_EQ(sink1.add_block(dest_ins_id_2, false), Status::OK()); + EXPECT_EQ(sink1.add_block(dest_ins_id_3, false), Status::OK()); + + EXPECT_EQ(sink2.add_block(dest_ins_id_1, false), Status::OK()); + EXPECT_EQ(sink2.add_block(dest_ins_id_2, false), Status::OK()); + EXPECT_EQ(sink2.add_block(dest_ins_id_3, false), Status::OK()); + + EXPECT_EQ(sink3.add_block(dest_ins_id_1, false), Status::OK()); + EXPECT_EQ(sink3.add_block(dest_ins_id_2, false), Status::OK()); + EXPECT_EQ(sink3.add_block(dest_ins_id_3, false), Status::OK()); + + EXPECT_EQ(orgin_queue_1_size, done_map[dest_ins_id_1].size()); + EXPECT_EQ(orgin_queue_2_size, done_map[dest_ins_id_2].size()); + EXPECT_EQ(orgin_queue_3_size, done_map[dest_ins_id_3].size()); + + clear_all_done(); + } +} + +} // namespace doris::vectorized diff --git a/be/test/vec/exec/exchange_sink_test.h b/be/test/vec/exec/exchange_sink_test.h new file mode 100644 index 000000000000000..253d7b267f91ed3 --- /dev/null +++ b/be/test/vec/exec/exchange_sink_test.h @@ -0,0 +1,180 @@ +// 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. + +#include + +#include +#include + +#include "common/status.h" +#include "pipeline/exec/exchange_sink_buffer.h" +#include "pipeline/exec/exchange_sink_operator.h" +#include "runtime/runtime_state.h" +#include "udf/udf.h" +#include "vec/sink/writer/vhive_utils.h" + +namespace doris::pipeline { + +std::map>*>> + + done_map; + +void add_request(int64_t id, auto* done) { + done_map[id].push(done); +} + +void clear_all_done() { + for (auto& [id, dones] : done_map) { + while (!dones.empty()) { + dones.front()->Run(); + dones.pop(); + } + } +} + +enum PopState : int { + eof, + error, + accept, +}; + +void pop_block(int64_t id, PopState state) { + if (done_map[id].empty()) { + return; + } + auto* done = done_map[id].front(); + done_map[id].pop(); + switch (state) { + case PopState::eof: { + Status st = Status::EndOfFile("Mock eof"); + st.to_protobuf(done->response_->mutable_status()); + done->Run(); + break; + } + case error: { + done->cntl_->SetFailed("Mock error"); + done->Run(); + break; + } + case accept: { + done->Run(); + break; + } + } +} +void transmit_blockv2(PBackendService_Stub& stub, + std::unique_ptr>> + closure) { + std::cout << "mock transmit_blockv2 dest ins id :" << closure->request_->finst_id().lo() + << "\n"; + add_request(closure->request_->finst_id().lo(), closure.release()); +} +}; // namespace doris::pipeline + +namespace doris::vectorized { + +using namespace pipeline; +class ExchangeSInkTest : public testing::Test { +public: + ExchangeSInkTest() = default; + ~ExchangeSInkTest() override = default; +}; + +class MockContext : public TaskExecutionContext {}; + +std::shared_ptr _mock_context = std::make_shared(); + +auto create_runtime_state() { + auto state = RuntimeState::create_shared(); + + state->set_task_execution_context(_mock_context); + return state; +} +constexpr int64_t recvr_fragment_id = 2; +constexpr int64_t sender_fragment_id = 2; + +TUniqueId create_TUniqueId(int64_t hi, int64_t lo) { + TUniqueId t {}; + t.hi = hi; + t.lo = lo; + return t; +} + +const auto dest_fragment_ins_id_1 = create_TUniqueId(recvr_fragment_id, 1); +const auto dest_fragment_ins_id_2 = create_TUniqueId(recvr_fragment_id, 2); +const auto dest_fragment_ins_id_3 = create_TUniqueId(recvr_fragment_id, 3); +const auto dest_ins_id_1 = dest_fragment_ins_id_1.lo; +const auto dest_ins_id_2 = dest_fragment_ins_id_2.lo; +const auto dest_ins_id_3 = dest_fragment_ins_id_3.lo; + +class MockSinkBuffer : public ExchangeSinkBuffer { +public: + MockSinkBuffer(RuntimeState* state, int64_t sinknum) : ExchangeSinkBuffer(state, sinknum) {}; + void _failed(InstanceLoId id, const std::string& err) override { + _is_failed = true; + std::cout << "_failed\n"; + } +}; + +struct SinkWithChannel { + std::shared_ptr sink; + std::shared_ptr buffer; + std::map> channels; + Status add_block(int64_t id, bool eos) { + auto channel = channels[id]; + TransmitInfo transmitInfo {.channel = channel.get(), + .block = std::make_unique(), + .eos = eos, + .exec_status = Status::OK()}; + return buffer->add_block(std::move(transmitInfo)); + } +}; + +auto create_buffer(std::shared_ptr state) { + auto sink_buffer = std::make_shared(state.get(), 3); + + sink_buffer->construct_request(dest_fragment_ins_id_1); + sink_buffer->construct_request(dest_fragment_ins_id_2); + sink_buffer->construct_request(dest_fragment_ins_id_3); + return sink_buffer; +} + +auto create_sink(std::shared_ptr state, std::shared_ptr sink_buffer) { + SinkWithChannel sink_with_channel; + sink_with_channel.sink = ExchangeSinkLocalState::create_shared(state.get()); + sink_with_channel.buffer = sink_buffer; + { + auto channel = std::make_shared( + sink_with_channel.sink.get(), TNetworkAddress {}, dest_fragment_ins_id_1, 0); + sink_with_channel.channels[channel->dest_ins_id()] = channel; + } + { + auto channel = std::make_shared( + sink_with_channel.sink.get(), TNetworkAddress {}, dest_fragment_ins_id_2, 0); + sink_with_channel.channels[channel->dest_ins_id()] = channel; + } + { + auto channel = std::make_shared( + sink_with_channel.sink.get(), TNetworkAddress {}, dest_fragment_ins_id_3, 0); + sink_with_channel.channels[channel->dest_ins_id()] = channel; + } + return sink_with_channel; +} + +} // namespace doris::vectorized diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index d07a8f022d70929..380c758e5751c86 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -284,6 +284,8 @@ public class SessionVariable implements Serializable, Writable { public static final String ENABLE_LOCAL_MERGE_SORT = "enable_local_merge_sort"; + public static final String ENABLE_SHARED_EXCHANGE_SINK_BUFFER = "enable_shared_exchange_sink_buffer"; + public static final String ENABLE_AGG_STATE = "enable_agg_state"; public static final String ENABLE_RPC_OPT_FOR_PIPELINE = "enable_rpc_opt_for_pipeline"; @@ -1135,6 +1137,9 @@ public enum IgnoreSplitType { @VariableMgr.VarAttr(name = ENABLE_LOCAL_MERGE_SORT) private boolean enableLocalMergeSort = true; + @VariableMgr.VarAttr(name = ENABLE_SHARED_EXCHANGE_SINK_BUFFER, fuzzy = true) + private boolean enableSharedExchangeSinkBuffer = true; + @VariableMgr.VarAttr(name = ENABLE_AGG_STATE, fuzzy = false, varType = VariableAnnotation.EXPERIMENTAL, needForward = true) public boolean enableAggState = false; @@ -2370,6 +2375,7 @@ public void initFuzzyModeVariables() { this.parallelPrepareThreshold = random.nextInt(32) + 1; this.enableCommonExprPushdown = random.nextBoolean(); this.enableLocalExchange = random.nextBoolean(); + this.enableSharedExchangeSinkBuffer = random.nextBoolean(); this.useSerialExchange = random.nextBoolean(); // This will cause be dead loop, disable it first // this.disableJoinReorder = random.nextBoolean(); @@ -3965,6 +3971,7 @@ public TQueryOptions toThrift() { tResult.setDataQueueMaxBlocks(dataQueueMaxBlocks); tResult.setEnableLocalMergeSort(enableLocalMergeSort); + tResult.setEnableSharedExchangeSinkBuffer(enableSharedExchangeSinkBuffer); tResult.setEnableParallelResultSink(enableParallelResultSink); tResult.setEnableParallelOutfile(enableParallelOutfile); tResult.setEnableShortCircuitQueryAccessColumnStore(enableShortCircuitQueryAcessColumnStore); diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index 745cb8f21fb6274..85abbf9b66ddba4 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -359,7 +359,7 @@ struct TQueryOptions { 141: optional bool ignore_runtime_filter_error = false; 142: optional bool enable_fixed_len_to_uint32_v2 = false; - + 143: optional bool enable_shared_exchange_sink_buffer = true; // For cloud, to control if the content would be written into file cache // In write path, to control if the content would be written into file cache. // In read path, read from file cache or remote storage when execute query. From 107ac9f9908d07694800878c063ce76e37ace3e0 Mon Sep 17 00:00:00 2001 From: minghong Date: Mon, 9 Dec 2024 14:41:41 +0800 Subject: [PATCH 10/14] [fix](nereids) common-sub-expression opt: avoid computing alias repeatedly (#45043) ### What problem does this PR solve? in previous PR#33087, we extract common sub expression, and set multi-layer projects to save computing effort. project: f1(h(a)), f2(h(a)) => multi-layer: [ L0: a L1: a, h(a) as x L2: a, h(a) as x, f1(x), f2(x) ] "h(a) as x " is computed at layer L1 and layer L2. this PR avoids the duplicated computing in L2 by set L2 as L2: a, x, f1(x), f2(x) --- .../post/CommonSubExpressionOpt.java | 16 ++++- .../suites/nereids_rules_p0/cse/cse.groovy | 72 +++++++++++++++++++ 2 files changed, 85 insertions(+), 3 deletions(-) create mode 100644 regression-test/suites/nereids_rules_p0/cse/cse.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/CommonSubExpressionOpt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/CommonSubExpressionOpt.java index fca84167994d08f..72ae4183c3b1477 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/CommonSubExpressionOpt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/CommonSubExpressionOpt.java @@ -29,7 +29,8 @@ import com.google.common.collect.Lists; -import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -67,8 +68,10 @@ private List> computeMultiLayerProjections( for (Expression expr : projects) { expr.accept(collector, null); } - Map aliasMap = new HashMap<>(); + // use linkedHashMap to make projects order stable + Map aliasMap = new LinkedHashMap<>(); if (!collector.commonExprByDepth.isEmpty()) { + Set previousAlias = new HashSet<>(); for (int i = 1; i <= collector.commonExprByDepth.size(); i++) { List layer = Lists.newArrayList(); layer.addAll(inputSlots); @@ -87,7 +90,14 @@ private List> computeMultiLayerProjections( aliasMap.put(expr, rewritten instanceof Alias ? (Alias) rewritten : new Alias(rewritten)); } }); - layer.addAll(aliasMap.values()); + for (Alias alias : aliasMap.values()) { + if (previousAlias.contains(alias)) { + layer.add(alias.toSlot()); + } else { + layer.add(alias); + previousAlias.add(alias); + } + } multiLayers.add(layer); } // final layer diff --git a/regression-test/suites/nereids_rules_p0/cse/cse.groovy b/regression-test/suites/nereids_rules_p0/cse/cse.groovy new file mode 100644 index 000000000000000..4e35737674de694 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/cse/cse.groovy @@ -0,0 +1,72 @@ +// 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. +suite("cse") { + sql """ + drop table if exists cse; + CREATE TABLE `cse` ( + `k1` int NOT NULL, + d datev2, + i int + ) ENGINE=OLAP + DUPLICATE KEY(`k1`) + DISTRIBUTED BY HASH(`k1`) BUCKETS 3 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "min_load_replica_num" = "-1", + "is_being_synced" = "false", + "storage_medium" = "hdd", + "storage_format" = "V2", + "inverted_index_storage_format" = "V2", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false", + "group_commit_interval_ms" = "10000", + "group_commit_data_bytes" = "134217728" + ); + + insert into cse values (1, '20240101', 100); + """ + + explain { + sql """ + physical plan + select sum( + case when k1 between i and + cast(from_unixtime( + unix_timestamp(date_add( + from_unixtime(unix_timestamp(cast(d as string), 'yyyyMMdd')), + INTERVAL 5 DAY)), 'yyyyMMdd') as int) + THEN 1 + ELSE 0 + end) as c1, + sum( + case when k1 between i and + cast(from_unixtime( + unix_timestamp(date_add( + from_unixtime(unix_timestamp(cast(d as string), 'yyyyMMdd')), + INTERVAL 2 DAY)), 'yyyyMMdd') as int) + THEN 1 + ELSE 0 + end) as c2 + from cse + group by d; + """ + contains("l1([k1#0, d#1, i#2, (k1 >= i)#9, cast(d as TEXT)#10, unix_timestamp(cast(d as TEXT)#10, '%Y%m%d') AS `unix_timestamp(cast(d as TEXT), '%Y%m%d')`#11])") + } + +} + From 5f0a2392af643fc35b109d4884070bf28f794e93 Mon Sep 17 00:00:00 2001 From: Xinyi Zou Date: Mon, 9 Dec 2024 14:52:52 +0800 Subject: [PATCH 11/14] [fix](memory) Fix init segment map check memory exceeds limit (#44092) ### What problem does this PR solve? Problem Summary: `RowIdConversion::init_segment_map` may use a lot of memory in compaction. however, this part of memory is not controlled, the process may OOM crash. If process has less than 10M available memory before generating `rowid_map` for each segment in `init_segment_map`, it will terminate. ![image](https://github.com/user-attachments/assets/652fadc9-bca0-4edd-a03e-04f2546cdf76) --- be/src/olap/rowid_conversion.h | 30 +++++++++++++++-------- be/src/olap/rowset/beta_rowset_reader.cpp | 3 ++- be/test/olap/rowid_conversion_test.cpp | 6 +++-- 3 files changed, 26 insertions(+), 13 deletions(-) diff --git a/be/src/olap/rowid_conversion.h b/be/src/olap/rowid_conversion.h index 01a2cea0d5e3088..8f9d96a136aab4d 100644 --- a/be/src/olap/rowid_conversion.h +++ b/be/src/olap/rowid_conversion.h @@ -37,21 +37,33 @@ class RowIdConversion { ~RowIdConversion() { RELEASE_THREAD_MEM_TRACKER(_seg_rowid_map_mem_used); } // resize segment rowid map to its rows num - void init_segment_map(const RowsetId& src_rowset_id, const std::vector& num_rows) { - size_t delta_std_pair_cap = 0; + Status init_segment_map(const RowsetId& src_rowset_id, const std::vector& num_rows) { for (size_t i = 0; i < num_rows.size(); i++) { + constexpr size_t RESERVED_MEMORY = 10 * 1024 * 1024; // 10M + if (doris::GlobalMemoryArbitrator::is_exceed_hard_mem_limit(RESERVED_MEMORY)) { + return Status::MemoryLimitExceeded(fmt::format( + "RowIdConversion init_segment_map failed, memory exceed limit, {}, " + "consuming " + "tracker:<{}>, peak used {}, current used {}.", + doris::GlobalMemoryArbitrator::process_limit_exceeded_errmsg_str(), + doris::thread_context()->thread_mem_tracker()->label(), + doris::thread_context()->thread_mem_tracker()->peak_consumption(), + doris::thread_context()->thread_mem_tracker()->consumption())); + } + uint32_t id = _segments_rowid_map.size(); _segment_to_id_map.emplace(std::pair {src_rowset_id, i}, id); _id_to_segment_map.emplace_back(src_rowset_id, i); std::vector> vec( num_rows[i], std::pair(UINT32_MAX, UINT32_MAX)); - delta_std_pair_cap += vec.capacity(); + + //NOTE: manually count _segments_rowid_map's memory here, because _segments_rowid_map could be used by indexCompaction. + // indexCompaction is a thridparty code, it's too complex to modify it. + // refer compact_column. + track_mem_usage(vec.capacity()); _segments_rowid_map.emplace_back(std::move(vec)); } - //NOTE: manually count _segments_rowid_map's memory here, because _segments_rowid_map could be used by indexCompaction. - // indexCompaction is a thridparty code, it's too complex to modify it. - // refer compact_column. - track_mem_usage(delta_std_pair_cap); + return Status::OK(); } // set dst rowset id @@ -124,9 +136,7 @@ class RowIdConversion { size_t new_size = _std_pair_cap * sizeof(std::pair) + _segments_rowid_map.capacity() * sizeof(std::vector>); - - RELEASE_THREAD_MEM_TRACKER(_seg_rowid_map_mem_used); - CONSUME_THREAD_MEM_TRACKER(new_size); + CONSUME_THREAD_MEM_TRACKER(new_size - _seg_rowid_map_mem_used); _seg_rowid_map_mem_used = new_size; } diff --git a/be/src/olap/rowset/beta_rowset_reader.cpp b/be/src/olap/rowset/beta_rowset_reader.cpp index 042893f1374374e..47cf9b820e8562d 100644 --- a/be/src/olap/rowset/beta_rowset_reader.cpp +++ b/be/src/olap/rowset/beta_rowset_reader.cpp @@ -239,7 +239,8 @@ Status BetaRowsetReader::get_segment_iterators(RowsetReaderContext* read_context // init segment rowid map for rowid conversion std::vector segment_num_rows; RETURN_IF_ERROR(get_segment_num_rows(&segment_num_rows)); - _read_context->rowid_conversion->init_segment_map(rowset()->rowset_id(), segment_num_rows); + RETURN_IF_ERROR(_read_context->rowid_conversion->init_segment_map(rowset()->rowset_id(), + segment_num_rows)); } auto [seg_start, seg_end] = _segment_offsets; diff --git a/be/test/olap/rowid_conversion_test.cpp b/be/test/olap/rowid_conversion_test.cpp index d48d4150ad3d0df..df56cd0559e4d87 100644 --- a/be/test/olap/rowid_conversion_test.cpp +++ b/be/test/olap/rowid_conversion_test.cpp @@ -490,10 +490,12 @@ TEST_F(TestRowIdConversion, Basic) { RowIdConversion rowid_conversion; src_rowset.init(0); std::vector rs0_segment_num_rows = {4, 3}; - rowid_conversion.init_segment_map(src_rowset, rs0_segment_num_rows); + auto st = rowid_conversion.init_segment_map(src_rowset, rs0_segment_num_rows); + EXPECT_EQ(st.ok(), true); src_rowset.init(1); std::vector rs1_segment_num_rows = {4}; - rowid_conversion.init_segment_map(src_rowset, rs1_segment_num_rows); + st = rowid_conversion.init_segment_map(src_rowset, rs1_segment_num_rows); + EXPECT_EQ(st.ok(), true); rowid_conversion.set_dst_rowset_id(dst_rowset); std::vector dst_segment_num_rows = {4, 3, 4}; From 9e49bdf012f7fdbc7bfa20606e701141ab2d4fba Mon Sep 17 00:00:00 2001 From: shee <13843187+qzsee@users.noreply.github.com> Date: Mon, 9 Dec 2024 14:54:21 +0800 Subject: [PATCH 12/14] [Audit](fix) fix audit log lost for npe (#44145) ### What problem does this PR solve? ``` private static void logAuditLogImpl(ConnectContext ctx, String origStmt, StatementBase parsedStmt, org.apache.doris.proto.Data.PQueryStatistics statistics, boolean printFuzzyVariables) { //........ // When we execute a statement, we execute the executeQuery method. In this method, we parse the sql using the new optimizer first, and if the parsing fails, we execute handleQueryException to record the audit log. At this time, there is no executor set, resulting in the npe if (!Env.getCurrentEnv().isMaster()) { if (ctx.executor != null && ctx.executor.isForwardToMaster()) { auditEventBuilder.setState(ctx.executor.getProxyStatus()); int proxyStatusCode = ctx.executor.getProxyStatusCode(); if (proxyStatusCode != 0) { auditEventBuilder.setErrorCode(proxyStatusCode); auditEventBuilder.setErrorMessage(ctx.executor.getProxyErrMsg()); } } } if (ctx.getCommand() == MysqlCommand.COM_STMT_PREPARE && ctx.getState().getErrorCode() == null) { auditEventBuilder.setState(String.valueOf(MysqlStateType.OK)); } Env.getCurrentEnv().getWorkloadRuntimeStatusMgr().submitFinishQueryToAudit(auditEventBuilder.build()); } ``` Co-authored-by: garenshi --- .../src/main/java/org/apache/doris/qe/AuditLogHelper.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java b/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java index 903b72898c95127..f29c617c15861e7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java @@ -286,7 +286,7 @@ private static void logAuditLogImpl(ConnectContext ctx, String origStmt, Stateme auditEventBuilder.setStmtType(getStmtType(parsedStmt)); if (!Env.getCurrentEnv().isMaster()) { - if (ctx.executor.isForwardToMaster()) { + if (ctx.executor != null && ctx.executor.isForwardToMaster()) { auditEventBuilder.setState(ctx.executor.getProxyStatus()); int proxyStatusCode = ctx.executor.getProxyStatusCode(); if (proxyStatusCode != 0) { From 90cbfc5a4e4419ea294d11463fdd0f7cae63c294 Mon Sep 17 00:00:00 2001 From: Rijesh Kunhi Parambattu <147430310+rijeshkp@users.noreply.github.com> Date: Mon, 9 Dec 2024 12:30:23 +0530 Subject: [PATCH 13/14] [Enhancement] (nereids)implement showDataSkewCommand in nereids #42755 (#44704) Issue Number: close #42755 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../apache/doris/catalog/MetadataViewer.java | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 8 ++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/ShowDataSkewCommand.java | 109 ++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../show/test_nereids_show_data_skew.groovy | 50 ++++++++ 7 files changed, 175 insertions(+), 2 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDataSkewCommand.java create mode 100644 regression-test/suites/nereids_p0/show/test_nereids_show_data_skew.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index c3fe7bb145ba723..04ef55d82c70651 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -265,6 +265,7 @@ supportedShowStatement | SHOW WHITELIST #showWhitelist | SHOW TABLETS BELONG tabletIds+=INTEGER_VALUE (COMMA tabletIds+=INTEGER_VALUE)* #showTabletsBelong + | SHOW DATA SKEW FROM baseTableRef #showDataSkew ; supportedLoadStatement @@ -331,7 +332,6 @@ unsupportedShowStatement | SHOW ALTER TABLE (ROLLUP | (MATERIALIZED VIEW) | COLUMN) ((FROM | IN) database=multipartIdentifier)? wildWhere? sortClause? limitClause? #showAlterTable - | SHOW DATA SKEW FROM baseTableRef #showDataSkew | SHOW DATA (ALL)? (FROM tableName=multipartIdentifier)? sortClause? propertyClause? #showData | SHOW TEMPORARY? PARTITIONS FROM tableName=multipartIdentifier diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java index 501d49e4add8d9b..61853bea3d48b7b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java @@ -401,7 +401,7 @@ public static List> getDataSkew(ShowDataSkewStmt stmt) throws DdlEx return getDataSkew(stmt.getDbName(), stmt.getTblName(), stmt.getPartitionNames()); } - private static List> getDataSkew(String dbName, String tblName, PartitionNames partitionNames) + public static List> getDataSkew(String dbName, String tblName, PartitionNames partitionNames) throws DdlException { DecimalFormat df = new DecimalFormat("00.00 %"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 15aeca952a25565..d5fbd193b4468a4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -246,6 +246,7 @@ import org.apache.doris.nereids.DorisParser.ShowCreateRepositoryContext; import org.apache.doris.nereids.DorisParser.ShowCreateTableContext; import org.apache.doris.nereids.DorisParser.ShowCreateViewContext; +import org.apache.doris.nereids.DorisParser.ShowDataSkewContext; import org.apache.doris.nereids.DorisParser.ShowDatabaseIdContext; import org.apache.doris.nereids.DorisParser.ShowDeleteContext; import org.apache.doris.nereids.DorisParser.ShowDiagnoseTabletContext; @@ -548,6 +549,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowCreateRepositoryCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateViewCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowDataSkewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDatabaseIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDeleteCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDiagnoseTabletCommand; @@ -4966,5 +4968,11 @@ public LogicalPlan visitAdminCheckTablets(AdminCheckTabletsContext ctx) { : Maps.newHashMap(); return new AdminCheckTabletsCommand(tabletIdLists, properties); } + + @Override + public LogicalPlan visitShowDataSkew(ShowDataSkewContext ctx) { + TableRefInfo tableRefInfo = visitBaseTableRefContext(ctx.baseTableRef()); + return new ShowDataSkewCommand(tableRefInfo); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 8198a8600a60c99..44d67d7478274ac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -212,6 +212,7 @@ public enum PlanType { SHOW_CREATE_TABLE_COMMAND, SHOW_CREATE_VIEW_COMMAND, SHOW_DATABASE_ID_COMMAND, + SHOW_DATA_SKEW_COMMAND, SHOW_DELETE_COMMAND, SHOW_DIAGNOSE_TABLET_COMMAND, SHOW_DYNAMIC_PARTITION_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDataSkewCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDataSkewCommand.java new file mode 100644 index 000000000000000..993f7db225e8023 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDataSkewCommand.java @@ -0,0 +1,109 @@ +// 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.nereids.trees.plans.commands; + +import org.apache.doris.analysis.PartitionNames; +import org.apache.doris.analysis.RedirectStatus; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MetadataViewer; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.util.Util; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.commands.info.TableRefInfo; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * show table id command + */ +public class ShowDataSkewCommand extends ShowCommand { + public static final ImmutableList TITLE_NAMES = new ImmutableList.Builder() + .add("PartitionName").add("BucketIdx").add("AvgRowCount").add("AvgDataSize") + .add("Graph").add("Percent") + .build(); + + private final TableRefInfo tableRefInfo; + + /** + * constructor + */ + public ShowDataSkewCommand(TableRefInfo tableRefInfo) { + super(PlanType.SHOW_DATA_SKEW_COMMAND); + this.tableRefInfo = tableRefInfo; + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + tableRefInfo.analyze(ctx); + // disallow external catalog + Util.prohibitExternalCatalog(tableRefInfo.getTableNameInfo().getCtl(), this.getClass().getSimpleName()); + if (!Env.getCurrentEnv().getAccessManager() + .checkTblPriv(ctx, tableRefInfo.getTableNameInfo().getCtl(), + tableRefInfo.getTableNameInfo().getDb(), tableRefInfo.getTableNameInfo().getTbl(), + PrivPredicate.SHOW)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "SHOW DATA SKEW", + ctx.getQualifiedUser(), + ctx.getRemoteIP(), + tableRefInfo.getTableNameInfo().getDb() + "." + tableRefInfo.getTableNameInfo().getTbl()); + } + try { + tableRefInfo.analyze(ctx); + Util.prohibitExternalCatalog(tableRefInfo.getTableNameInfo().getCtl(), this.getClass().getSimpleName()); + + PartitionNames partitionNames = (tableRefInfo.getPartitionNamesInfo() != null) + ? tableRefInfo.getPartitionNamesInfo().translateToLegacyPartitionNames() : null; + List> results = MetadataViewer.getDataSkew(tableRefInfo.getTableNameInfo().getDb(), + tableRefInfo.getTableNameInfo().getTbl(), partitionNames); + return new ShowResultSet(getMetaData(), results); + } catch (DdlException e) { + throw new AnalysisException(e.getMessage()); + } + + } + + public ShowResultSetMetaData getMetaData() { + ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); + for (String title : TITLE_NAMES) { + builder.addColumn(new Column(title, ScalarType.createVarchar(30))); + } + return builder.build(); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowDataSkewCommand(this, context); + } + + @Override + public RedirectStatus toRedirectStatus() { + return RedirectStatus.FORWARD_NO_SYNC; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index dea3a004e816bef..81296b0b9b8204b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -92,6 +92,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowCreateRepositoryCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateViewCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowDataSkewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDatabaseIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDeleteCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDiagnoseTabletCommand; @@ -600,4 +601,8 @@ default R visitShowProcessListCommand(ShowProcessListCommand showProcessListComm default R visitAdminCheckTabletsCommand(AdminCheckTabletsCommand adminCheckTabletsCommand, C context) { return visitCommand(adminCheckTabletsCommand, context); } + + default R visitShowDataSkewCommand(ShowDataSkewCommand showDataSkewCommand, C context) { + return visitCommand(showDataSkewCommand, context); + } } diff --git a/regression-test/suites/nereids_p0/show/test_nereids_show_data_skew.groovy b/regression-test/suites/nereids_p0/show/test_nereids_show_data_skew.groovy new file mode 100644 index 000000000000000..5e316f22afcf170 --- /dev/null +++ b/regression-test/suites/nereids_p0/show/test_nereids_show_data_skew.groovy @@ -0,0 +1,50 @@ +// 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. + + +suite("test_nereids_show_data_skew") { + def table = "test_nereids_show_data_skew" + String dbName = "${table}_db" + sql "CREATE DATABASE IF NOT EXISTS ${dbName}" + sql "use ${dbName}" + // create table and insert data + sql """ drop table if exists ${table} force""" + sql """ + CREATE TABLE ${table} ( + id int, + name string, + pdate DATETIME) + PARTITION BY RANGE(pdate) ( + FROM ("2023-04-16") TO ("2023-04-20") INTERVAL 1 DAY + ) DISTRIBUTED BY HASH(id) BUCKETS 5 + properties("replication_num" = "1"); + """ + + checkNereidsExecute("show data skew from ${dbName}.test_nereids_show_data_skew;") + def result = sql """show data skew from ${dbName}.test_nereids_show_data_skew;""" + assertTrue(result.size() == 20) + + checkNereidsExecute("show data skew from ${dbName}.test_nereids_show_data_skew partition(p_20230416);") + def result2 = sql """show data skew from ${dbName}.test_nereids_show_data_skew partition(p_20230416);""" + assertTrue(result2.size() == 5) + + checkNereidsExecute("show data skew from ${dbName}.test_nereids_show_data_skew partition(p_20230416, p_20230418);") + def result3 = sql """show data skew from ${dbName}.test_nereids_show_data_skew partition(p_20230416, p_20230418);""" + assertTrue(result3.size() == 10) + + +} From 9c4c0b378f1f73973a449ac5c44d82b5b1192fb8 Mon Sep 17 00:00:00 2001 From: Sridhar R Manikarnike Date: Mon, 9 Dec 2024 12:31:00 +0530 Subject: [PATCH 14/14] [Enhancement] (nereids)implement CreateWorkloadGroupCommand in nereids (#44970) Issue Number: close #42589 --- .../org/apache/doris/nereids/DorisParser.g4 | 6 +- .../nereids/parser/LogicalPlanBuilder.java | 11 ++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../commands/CreateWorkloadGroupCommand.java | 100 ++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../workloadgroup/WorkloadGroupMgr.java | 10 +- .../test_nereids_workload_test.groovy | 6 +- 7 files changed, 130 insertions(+), 9 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateWorkloadGroupCommand.java diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 04ef55d82c70651..eda5dda16984c98 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -183,7 +183,9 @@ supportedCreateStatement | CREATE (EXTERNAL)? TABLE (IF NOT EXISTS)? name=multipartIdentifier LIKE existedTable=multipartIdentifier (WITH ROLLUP (rollupNames=identifierList)?)? #createTableLike - | CREATE ROLE (IF NOT EXISTS)? name=identifier (COMMENT STRING_LITERAL)? #createRole + | CREATE ROLE (IF NOT EXISTS)? name=identifier (COMMENT STRING_LITERAL)? #createRole + | CREATE WORKLOAD GROUP (IF NOT EXISTS)? + name=identifierOrText properties=propertyClause? #createWorkloadGroup | CREATE ROW POLICY (IF NOT EXISTS)? name=identifier ON table=multipartIdentifier AS type=(RESTRICTIVE | PERMISSIVE) @@ -764,8 +766,6 @@ unsupportedCreateStatement name=identifierOrText properties=propertyClause? #createResource | CREATE STORAGE VAULT (IF NOT EXISTS)? name=identifierOrText properties=propertyClause? #createStorageVault - | CREATE WORKLOAD GROUP (IF NOT EXISTS)? - name=identifierOrText properties=propertyClause? #createWorkloadGroup | CREATE WORKLOAD POLICY (IF NOT EXISTS)? name=identifierOrText (CONDITIONS LEFT_PAREN workloadPolicyConditions RIGHT_PAREN)? (ACTIONS LEFT_PAREN workloadPolicyActions RIGHT_PAREN)? diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index d5fbd193b4468a4..ff5bed55ebc8576 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -104,6 +104,7 @@ import org.apache.doris.nereids.DorisParser.CreateTableContext; import org.apache.doris.nereids.DorisParser.CreateTableLikeContext; import org.apache.doris.nereids.DorisParser.CreateViewContext; +import org.apache.doris.nereids.DorisParser.CreateWorkloadGroupContext; import org.apache.doris.nereids.DorisParser.CteContext; import org.apache.doris.nereids.DorisParser.DataTypeWithNullableContext; import org.apache.doris.nereids.DorisParser.DateCeilContext; @@ -502,6 +503,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableLikeCommand; import org.apache.doris.nereids.trees.plans.commands.CreateViewCommand; +import org.apache.doris.nereids.trees.plans.commands.CreateWorkloadGroupCommand; import org.apache.doris.nereids.trees.plans.commands.DeleteFromCommand; import org.apache.doris.nereids.trees.plans.commands.DeleteFromUsingCommand; import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand; @@ -4862,6 +4864,15 @@ public LogicalPlan visitDropEncryptkey(DropEncryptkeyContext ctx) { return new DropEncryptkeyCommand(new EncryptKeyName(nameParts), ctx.EXISTS() != null); } + @Override + public LogicalPlan visitCreateWorkloadGroup(CreateWorkloadGroupContext ctx) { + String workloadGroupName = stripQuotes(ctx.name.getText()); + boolean ifNotExists = ctx.EXISTS() != null; + Map properties = ctx.propertyClause() != null + ? Maps.newHashMap(visitPropertyClause(ctx.propertyClause())) : Maps.newHashMap(); + return new CreateWorkloadGroupCommand(workloadGroupName, ifNotExists, properties); + } + @Override public LogicalPlan visitDropFile(DropFileContext ctx) { String dbName = null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 44d67d7478274ac..fac113564f857e5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -247,6 +247,7 @@ public enum PlanType { RECOVER_PARTITION_COMMAND, REPLAY_COMMAND, CREATE_ENCRYPTKEY_COMMAND, + CREATE_WORKLOAD_GROUP_COMMAND, CREATE_FILE_COMMAND, CREATE_ROUTINE_LOAD_COMMAND } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateWorkloadGroupCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateWorkloadGroupCommand.java new file mode 100644 index 000000000000000..ad4db8cbfdfae82 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateWorkloadGroupCommand.java @@ -0,0 +1,100 @@ +// 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.nereids.trees.plans.commands; + +import org.apache.doris.analysis.StmtType; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.FeNameFormat; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.resource.workloadgroup.WorkloadGroup; +import org.apache.doris.resource.workloadgroup.WorkloadGroupMgr; + +import org.apache.commons.lang3.StringUtils; + +import java.util.Map; + +/** + * Create workload group command + */ +public class CreateWorkloadGroupCommand extends Command implements ForwardWithSync { + private final boolean ifNotExists; + private final String workloadGroupName; + private final Map properties; + + /** + * Constructor for CreateWorkloadGroupCommand + */ + public CreateWorkloadGroupCommand(String workloadGroupName, boolean ifNotExists, Map properties) { + super(PlanType.CREATE_WORKLOAD_GROUP_COMMAND); + this.workloadGroupName = workloadGroupName; + this.ifNotExists = ifNotExists; + this.properties = properties; + } + + private void validate(ConnectContext ctx) throws AnalysisException { + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + + // check name + FeNameFormat.checkWorkloadGroupName(workloadGroupName); + + if (properties == null || properties.isEmpty()) { + throw new AnalysisException("Workload Group properties can't be empty"); + } + + if (properties.containsKey(WorkloadGroup.INTERNAL_TYPE)) { + throw new AnalysisException(WorkloadGroup.INTERNAL_TYPE + " can not be create or modified "); + } + + String tagStr = properties.get(WorkloadGroup.TAG); + if (!StringUtils.isEmpty(tagStr) && (WorkloadGroupMgr.DEFAULT_GROUP_NAME.equals(workloadGroupName) + || WorkloadGroupMgr.INTERNAL_GROUP_NAME.equals(workloadGroupName))) { + throw new AnalysisException( + WorkloadGroupMgr.INTERNAL_GROUP_NAME + " and " + WorkloadGroupMgr.DEFAULT_GROUP_NAME + + " group can not set tag"); + } + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + validate(ctx); + // Create workload group + WorkloadGroup workloadGroup = WorkloadGroup.create(workloadGroupName, properties); + WorkloadGroupMgr workloadGroupMgr = Env.getCurrentEnv().getWorkloadGroupMgr(); + workloadGroupMgr.createWorkloadGroup(workloadGroup, ifNotExists); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCreateWorkloadGroupCommand(this, context); + } + + @Override + public StmtType stmtType() { + return StmtType.CREATE; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 81296b0b9b8204b..05ee8548cfa93d2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -47,6 +47,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableLikeCommand; import org.apache.doris.nereids.trees.plans.commands.CreateViewCommand; +import org.apache.doris.nereids.trees.plans.commands.CreateWorkloadGroupCommand; import org.apache.doris.nereids.trees.plans.commands.DeleteFromCommand; import org.apache.doris.nereids.trees.plans.commands.DeleteFromUsingCommand; import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand; @@ -562,6 +563,10 @@ default R visitShowTableIdCommand(ShowTableIdCommand showTableIdCommand, C conte return visitCommand(showTableIdCommand, context); } + default R visitCreateWorkloadGroupCommand(CreateWorkloadGroupCommand createWorkloadGroupCommand, C context) { + return visitCommand(createWorkloadGroupCommand, context); + } + default R visitShowEncryptKeysCommand(ShowEncryptKeysCommand showEncryptKeysCommand, C context) { return visitCommand(showEncryptKeysCommand, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java index 22cbcf4a0a15023..94695cc8d5b8ac1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java @@ -357,13 +357,12 @@ private String getWorkloadGroupNameAndCheckPriv(ConnectContext context) throws A return groupName; } - public void createWorkloadGroup(CreateWorkloadGroupStmt stmt) throws DdlException { - WorkloadGroup workloadGroup = WorkloadGroup.create(stmt.getWorkloadGroupName(), stmt.getProperties()); + public void createWorkloadGroup(WorkloadGroup workloadGroup, boolean isIfNotExists) throws DdlException { String workloadGroupName = workloadGroup.getName(); writeLock(); try { if (nameToWorkloadGroup.containsKey(workloadGroupName)) { - if (stmt.isIfNotExists()) { + if (isIfNotExists) { return; } throw new DdlException("workload group " + workloadGroupName + " already exist"); @@ -382,6 +381,11 @@ public void createWorkloadGroup(CreateWorkloadGroupStmt stmt) throws DdlExceptio LOG.info("Create workload group success: {}", workloadGroup); } + public void createWorkloadGroup(CreateWorkloadGroupStmt stmt) throws DdlException { + WorkloadGroup workloadGroup = WorkloadGroup.create(stmt.getWorkloadGroupName(), stmt.getProperties()); + createWorkloadGroup(workloadGroup, stmt.isIfNotExists()); + } + public void createInternalWorkloadGroup() { Map properties = Maps.newHashMap(); // 100 is cgroup v2 default cpu_share value diff --git a/regression-test/suites/workload_manager_p0/test_nereids_workload_test.groovy b/regression-test/suites/workload_manager_p0/test_nereids_workload_test.groovy index 3f7c55f5c99cf91..680f5a28ca982ef 100644 --- a/regression-test/suites/workload_manager_p0/test_nereids_workload_test.groovy +++ b/regression-test/suites/workload_manager_p0/test_nereids_workload_test.groovy @@ -18,10 +18,10 @@ suite("test_nereids_workload_test") { sql "drop workload group if exists test_nereids_wg1;" sql "drop workload group if exists test_nereids_wg2;" - sql "create workload group test_nereids_wg1 properties('cpu_share'='1024');" - sql "create workload group test_nereids_wg2 properties('cpu_share'='1024');" + checkNereidsExecute("create workload group test_nereids_wg1 properties('cpu_share'='1024');") + checkNereidsExecute("create workload group test_nereids_wg2 properties('cpu_share'='1024');") qt_check_workload_check1("select NAME from information_schema.workload_groups where NAME='test_nereids_wg1';") checkNereidsExecute("drop workload group test_nereids_wg1;") qt_check_workload_check2("select NAME from information_schema.workload_groups where NAME='test_nereids_wg1';") checkNereidsExecute("drop workload group if exists test_nereids_wg2;") -} \ No newline at end of file +}