From 928860d8ba86f29fd796b227d6f1fce7b04cad81 Mon Sep 17 00:00:00 2001 From: 924060929 <924060929@qq.com> Date: Fri, 12 Apr 2024 19:54:36 +0800 Subject: [PATCH 1/6] [enhancement](Nereids) Enable parse sql from sql cache (#33262) Before this pr, the query must pass through parser, analyzer, rewriter, optimizer and translator, then we can check whether this query can use sql cache, if the query is too long, or the number of join tables too big, the plan time usually >= 500ms. This pr reduce this time by skip the fashion plan path, because we can reuse the previous physical plan and query result if no any changed. In some cases we should not parse sql from sql cache, e.g. table structure changed, data changed, user policies changed, privileges changed, contains non-deterministic functions, and user variables changed. In my test case: query a view which has lots of join and union, and the tables has empty partition, the query latency is about 3ms. if not parse sql from sql cache, the plan time is about 550ms ## Features 1. use Config.sql_cache_manage_num to control how many sql cache be reused in on fe 2. if explain plan appear some plans contains `LogicalSqlCache` or `PhysicalSqlCache`, it means the query can use sql cache, like this: ```sql mysql> set enable_sql_cache=true; Query OK, 0 rows affected (0.00 sec) mysql> explain physical plan select * from test.t; +----------------------------------------------------------------------------------+ | Explain String(Nereids Planner) | +----------------------------------------------------------------------------------+ | cost = 3.135 | | PhysicalResultSink[53] ( outputExprs=[c1#0, c2#1] ) | | +--PhysicalDistribute[50]@0 ( stats=3, distributionSpec=DistributionSpecGather ) | | +--PhysicalOlapScan[t]@0 ( stats=3 ) | +----------------------------------------------------------------------------------+ 4 rows in set (0.02 sec) mysql> select * from test.t; +------+------+ | c1 | c2 | +------+------+ | 1 | 2 | | -2 | -2 | | NULL | 30 | +------+------+ 3 rows in set (0.05 sec) mysql> explain physical plan select * from test.t; +-------------------------------------------------------------------------------------------+ | Explain String(Nereids Planner) | +-------------------------------------------------------------------------------------------+ | cost = 0.0 | | PhysicalSqlCache[2] ( queryId=78511f515cda466b-95385d892d6c68d0, backend=127.0.0.1:9050 ) | | +--PhysicalResultSink[52] ( outputExprs=[c1#0, c2#1] ) | | +--PhysicalDistribute[49]@0 ( stats=3, distributionSpec=DistributionSpecGather ) | | +--PhysicalOlapScan[t]@0 ( stats=3 ) | +-------------------------------------------------------------------------------------------+ 5 rows in set (0.01 sec) ``` (cherry picked from commit 03bd2a337d4a56ea9c91673b3bd4ae518ed10f20) --- .../java/org/apache/doris/common/Config.java | 8 + .../java/org/apache/doris/common/Pair.java | 4 + .../doris/blockrule/SqlBlockRuleMgr.java | 8 +- .../java/org/apache/doris/catalog/Env.java | 10 + .../doris/common/NereidsSqlCacheManager.java | 360 ++++++++++++ .../doris/common/profile/SummaryProfile.java | 2 +- .../apache/doris/nereids/CascadesContext.java | 6 +- .../apache/doris/nereids/NereidsPlanner.java | 217 ++++--- .../apache/doris/nereids/SqlCacheContext.java | 353 +++++++++++ .../doris/nereids/StatementContext.java | 118 +++- .../doris/nereids/parser/NereidsParser.java | 113 ++++ .../rules/analysis/BindExpression.java | 16 +- .../nereids/rules/analysis/BindRelation.java | 118 ++-- .../rules/analysis/ExpressionAnalyzer.java | 73 ++- .../expression/rules/FunctionBinder.java | 4 +- .../rules/ReplaceVariableByLiteral.java | 13 +- .../rules/rewrite/CheckPrivileges.java | 12 +- .../expressions/ExpressionEvaluator.java | 13 +- .../functions/AggCombinerFunctionBuilder.java | 17 +- .../functions/BuiltinFunctionBuilder.java | 7 +- .../functions/FunctionBuilder.java | 10 +- .../functions/scalar/ConnectionId.java | 3 +- .../functions/scalar/CurrentUser.java | 3 +- .../functions/scalar/Database.java | 3 +- .../expressions/functions/scalar/User.java | 3 +- .../functions/table/TableValuedFunction.java | 4 +- .../functions/udf/AliasUdfBuilder.java | 5 +- .../functions/udf/JavaUdafBuilder.java | 21 +- .../functions/udf/JavaUdfBuilder.java | 7 +- .../expressions/literal/DateLiteral.java | 40 +- .../expressions/literal/DateTimeLiteral.java | 77 ++- .../literal/DateTimeV2Literal.java | 109 ++-- .../expressions/literal/DateV2Literal.java | 13 +- .../nereids/trees/plans/AbstractPlan.java | 30 +- .../doris/nereids/trees/plans/PlanType.java | 2 + .../nereids/trees/plans/TreeStringPlan.java | 112 ++++ .../nereids/trees/plans/algebra/SqlCache.java | 36 ++ .../plans/logical/LogicalCheckPolicy.java | 15 +- .../trees/plans/logical/LogicalSqlCache.java | 147 +++++ .../plans/physical/PhysicalSqlCache.java | 139 +++++ .../trees/plans/visitor/PlanVisitor.java | 8 + .../org/apache/doris/qe/ConnectProcessor.java | 191 ++++-- .../org/apache/doris/qe/StmtExecutor.java | 60 +- .../java/org/apache/doris/qe/cache/Cache.java | 19 +- .../apache/doris/qe/cache/CacheAnalyzer.java | 156 ++++- .../doris/qe/cache/CacheCoordinator.java | 5 +- .../org/apache/doris/qe/cache/SqlCache.java | 56 +- .../rules/analysis/FunctionRegistryTest.java | 2 +- .../regression/action/ExplainAction.groovy | 4 +- .../doris/regression/suite/Suite.groovy | 74 ++- regression-test/plugins/test_helper.groovy | 63 ++ .../cache/parse_sql_from_sql_cache.groovy | 550 ++++++++++++++++++ .../suites/query_p0/cache/sql_cache.groovy | 2 +- .../test_schema_change_duplicate.groovy | 1 - 54 files changed, 3021 insertions(+), 421 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/TreeStringPlan.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/SqlCache.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSqlCache.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSqlCache.java create mode 100644 regression-test/plugins/test_helper.groovy create mode 100644 regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index 3525d411b66288..11fa7773adf1eb 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -2008,6 +2008,14 @@ public class Config extends ConfigBase { + "the old load statement will be degraded."}) public static boolean enable_nereids_load = false; + /** + * the plan cache num which can be reused for the next query + */ + @ConfField(mutable = false, varType = VariableAnnotation.EXPERIMENTAL, description = { + "当前默认设置为 100,用来控制控制NereidsSqlCacheManager管理的sql cache数量。", + "Now default set to 100, this config is used to control the number of " + + "sql cache managed by NereidsSqlCacheManager"}) + public static int sql_cache_manage_num = 100; /** * Maximum number of events to poll in each RPC. diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Pair.java b/fe/fe-common/src/main/java/org/apache/doris/common/Pair.java index a699284676131c..3a8b1940d59759 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Pair.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Pair.java @@ -43,6 +43,10 @@ private Pair(F first, S second) { this.second = second; } + public static Pair ofSame(K same) { + return new Pair<>(same, same); + } + public static Pair of(F first, S second) { return new Pair<>(first, second); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java index ca4e68a6ae2f9c..2eba1d4fae3385 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java @@ -269,10 +269,10 @@ public void checkLimitations(Long partitionNum, Long tabletNum, Long cardinality return; } // match global rule - List globalRules = - nameToSqlBlockRuleMap.values().stream().filter(SqlBlockRule::getGlobal).collect(Collectors.toList()); - for (SqlBlockRule rule : globalRules) { - checkLimitations(rule, partitionNum, tabletNum, cardinality); + for (SqlBlockRule rule : nameToSqlBlockRuleMap.values()) { + if (rule.getGlobal()) { + checkLimitations(rule, partitionNum, tabletNum, cardinality); + } } // match user rule String[] bindSqlBlockRules = Env.getCurrentEnv().getAuth().getSqlBlockRules(user); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index 1012b9c2033d01..71bcdec4f29198 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -105,6 +105,7 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.NereidsSqlCacheManager; import org.apache.doris.common.Pair; import org.apache.doris.common.ThreadPoolManager; import org.apache.doris.common.UserException; @@ -529,6 +530,8 @@ public class Env { private DNSCache dnsCache; + private final NereidsSqlCacheManager sqlCacheManager; + public List getFrontendInfos() { List res = new ArrayList<>(); @@ -764,6 +767,9 @@ public Env(boolean isCheckpointCatalog) { this.mtmvService = new MTMVService(); this.insertOverwriteManager = new InsertOverwriteManager(); this.dnsCache = new DNSCache(); + this.sqlCacheManager = new NereidsSqlCacheManager( + Config.sql_cache_manage_num, Config.cache_last_version_interval_second + ); } public static void destroyCheckpoint() { @@ -6052,6 +6058,10 @@ public StatisticsAutoCollector getStatisticsAutoCollector() { return statisticsAutoCollector; } + public NereidsSqlCacheManager getSqlCacheManager() { + return sqlCacheManager; + } + public void alterMTMVRefreshInfo(AlterMTMVRefreshInfo info) { AlterMTMV alter = new AlterMTMV(info.getMvName(), info.getRefreshInfo(), MTMVAlterOpType.ALTER_REFRESH_INFO); this.alter.processAlterMTMV(alter, false); diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java b/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java new file mode 100644 index 00000000000000..f3643046cade04 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java @@ -0,0 +1,360 @@ +// 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.common; + +import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.View; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.metric.MetricRepo; +import org.apache.doris.mysql.privilege.DataMaskPolicy; +import org.apache.doris.mysql.privilege.RowFilterPolicy; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.SqlCacheContext; +import org.apache.doris.nereids.SqlCacheContext.FullColumnName; +import org.apache.doris.nereids.SqlCacheContext.FullTableName; +import org.apache.doris.nereids.SqlCacheContext.ScanTable; +import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.analyzer.UnboundVariable; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.rules.analysis.ExpressionAnalyzer; +import org.apache.doris.nereids.rules.analysis.UserAuthentication; +import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; +import org.apache.doris.nereids.rules.expression.rules.FoldConstantRuleOnFE; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Variable; +import org.apache.doris.nereids.trees.expressions.functions.Nondeterministic; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.RelationId; +import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalSqlCache; +import org.apache.doris.proto.InternalService; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.cache.CacheAnalyzer; +import org.apache.doris.qe.cache.SqlCache; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.google.common.collect.ImmutableList; +import org.apache.commons.collections.CollectionUtils; + +import java.time.Duration; +import java.util.List; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; + +/** NereidsSqlCacheManager */ +public class NereidsSqlCacheManager { + // key: : + // value: CacheAnalyzer + private final Cache sqlCache; + + public NereidsSqlCacheManager(int sqlCacheNum, long cacheIntervalSeconds) { + sqlCache = Caffeine.newBuilder() + .maximumSize(sqlCacheNum) + .expireAfterAccess(Duration.ofSeconds(cacheIntervalSeconds)) + // auto evict cache when jvm memory too low + .softValues() + .build(); + } + + /** tryAddCache */ + public void tryAddCache( + ConnectContext connectContext, String sql, + CacheAnalyzer analyzer, boolean currentMissParseSqlFromSqlCache) { + Optional sqlCacheContextOpt = connectContext.getStatementContext().getSqlCacheContext(); + if (!sqlCacheContextOpt.isPresent()) { + return; + } + if (!(analyzer.getCache() instanceof SqlCache)) { + return; + } + SqlCacheContext sqlCacheContext = sqlCacheContextOpt.get(); + UserIdentity currentUserIdentity = connectContext.getCurrentUserIdentity(); + String key = currentUserIdentity.toString() + ":" + sql.trim(); + if (analyzer.getCache() instanceof SqlCache + && (currentMissParseSqlFromSqlCache || sqlCache.getIfPresent(key) == null)) { + SqlCache cache = (SqlCache) analyzer.getCache(); + sqlCacheContext.setCacheKeyMd5(cache.getOrComputeCacheMd5()); + sqlCacheContext.setSumOfPartitionNum(cache.getSumOfPartitionNum()); + sqlCacheContext.setLatestPartitionId(cache.getLatestId()); + sqlCacheContext.setLatestPartitionVersion(cache.getLatestVersion()); + sqlCacheContext.setLatestPartitionTime(cache.getLatestTime()); + sqlCacheContext.setCacheProxy(cache.getProxy()); + + for (ScanTable scanTable : analyzer.getScanTables()) { + sqlCacheContext.addScanTable(scanTable); + } + + sqlCache.put(key, sqlCacheContext); + } + } + + /** invalidateCache */ + public void invalidateCache(ConnectContext connectContext, String sql) { + UserIdentity currentUserIdentity = connectContext.getCurrentUserIdentity(); + String key = currentUserIdentity.toString() + ":" + sql.trim(); + sqlCache.invalidate(key); + } + + /** tryParseSql */ + public Optional tryParseSql(ConnectContext connectContext, String sql) { + UserIdentity currentUserIdentity = connectContext.getCurrentUserIdentity(); + Env env = connectContext.getEnv(); + String key = currentUserIdentity.toString() + ":" + sql.trim(); + SqlCacheContext sqlCacheContext = sqlCache.getIfPresent(key); + if (sqlCacheContext == null) { + return Optional.empty(); + } + + // LOG.info("Total size: " + GraphLayout.parseInstance(sqlCacheContext).totalSize()); + + // check table and view and their columns authority + if (privilegeChanged(connectContext, env, sqlCacheContext)) { + return invalidateCache(key); + } + if (tablesOrDataChanged(env, sqlCacheContext)) { + return invalidateCache(key); + } + if (viewsChanged(env, sqlCacheContext)) { + return invalidateCache(key); + } + if (usedVariablesChanged(sqlCacheContext)) { + return invalidateCache(key); + } + + LogicalEmptyRelation whateverPlan = new LogicalEmptyRelation(new RelationId(0), ImmutableList.of()); + if (nondeterministicFunctionChanged(whateverPlan, connectContext, sqlCacheContext)) { + return invalidateCache(key); + } + + // table structure and data not changed, now check policy + if (rowPoliciesChanged(currentUserIdentity, env, sqlCacheContext)) { + return invalidateCache(key); + } + if (dataMaskPoliciesChanged(currentUserIdentity, env, sqlCacheContext)) { + return invalidateCache(key); + } + + try { + Status status = new Status(); + InternalService.PFetchCacheResult cacheData = + SqlCache.getCacheData(sqlCacheContext.getCacheProxy(), + sqlCacheContext.getCacheKeyMd5(), sqlCacheContext.getLatestPartitionId(), + sqlCacheContext.getLatestPartitionVersion(), sqlCacheContext.getLatestPartitionTime(), + sqlCacheContext.getSumOfPartitionNum(), status); + + if (status.ok() && cacheData != null && cacheData.getStatus() == InternalService.PCacheStatus.CACHE_OK) { + List cacheValues = cacheData.getValuesList(); + String cachedPlan = sqlCacheContext.getPhysicalPlan(); + String backendAddress = SqlCache.findCacheBe(sqlCacheContext.getCacheKeyMd5()).getAddress(); + + MetricRepo.COUNTER_CACHE_HIT_SQL.increase(1L); + + LogicalSqlCache logicalSqlCache = new LogicalSqlCache( + sqlCacheContext.getQueryId(), sqlCacheContext.getColLabels(), + sqlCacheContext.getResultExprs(), cacheValues, backendAddress, cachedPlan); + return Optional.of(logicalSqlCache); + } + return Optional.empty(); + } catch (Throwable t) { + return Optional.empty(); + } + } + + private boolean tablesOrDataChanged(Env env, SqlCacheContext sqlCacheContext) { + long latestPartitionTime = sqlCacheContext.getLatestPartitionTime(); + long latestPartitionVersion = sqlCacheContext.getLatestPartitionVersion(); + + if (sqlCacheContext.hasUnsupportedTables()) { + return true; + } + + for (ScanTable scanTable : sqlCacheContext.getScanTables()) { + FullTableName fullTableName = scanTable.fullTableName; + TableIf tableIf = findTableIf(env, fullTableName); + if (!(tableIf instanceof OlapTable)) { + return true; + } + OlapTable olapTable = (OlapTable) tableIf; + long currentTableTime = olapTable.getVisibleVersionTime(); + long cacheTableTime = scanTable.latestTimestamp; + long currentTableVersion = olapTable.getVisibleVersion(); + long cacheTableVersion = scanTable.latestVersion; + // some partitions have been dropped, or delete or update or insert rows into new partition? + if (currentTableTime > cacheTableTime + || (currentTableTime == cacheTableTime && currentTableVersion > cacheTableVersion)) { + return true; + } + + for (Long scanPartitionId : scanTable.getScanPartitions()) { + Partition partition = olapTable.getPartition(scanPartitionId); + // partition == null: is this partition truncated? + if (partition == null || partition.getVisibleVersionTime() > latestPartitionTime + || (partition.getVisibleVersionTime() == latestPartitionTime + && partition.getVisibleVersion() > latestPartitionVersion)) { + return true; + } + } + } + return false; + } + + private boolean viewsChanged(Env env, SqlCacheContext sqlCacheContext) { + for (Entry cacheView : sqlCacheContext.getUsedViews().entrySet()) { + TableIf currentView = findTableIf(env, cacheView.getKey()); + if (currentView == null) { + return true; + } + + String cacheValueDdlSql = cacheView.getValue(); + if (currentView instanceof View) { + if (!((View) currentView).getInlineViewDef().equals(cacheValueDdlSql)) { + return true; + } + } else { + return true; + } + } + return false; + } + + private boolean rowPoliciesChanged(UserIdentity currentUserIdentity, Env env, SqlCacheContext sqlCacheContext) { + for (Entry> kv : sqlCacheContext.getRowPolicies().entrySet()) { + FullTableName qualifiedTable = kv.getKey(); + List cachedPolicies = kv.getValue(); + + List rowPolicies = env.getAccessManager().evalRowFilterPolicies( + currentUserIdentity, qualifiedTable.catalog, qualifiedTable.db, qualifiedTable.table); + if (!CollectionUtils.isEqualCollection(cachedPolicies, rowPolicies)) { + return true; + } + } + return false; + } + + private boolean dataMaskPoliciesChanged( + UserIdentity currentUserIdentity, Env env, SqlCacheContext sqlCacheContext) { + for (Entry> kv : sqlCacheContext.getDataMaskPolicies().entrySet()) { + FullColumnName qualifiedColumn = kv.getKey(); + Optional cachedPolicy = kv.getValue(); + + Optional dataMaskPolicy = env.getAccessManager() + .evalDataMaskPolicy(currentUserIdentity, qualifiedColumn.catalog, + qualifiedColumn.db, qualifiedColumn.table, qualifiedColumn.column); + if (!Objects.equals(cachedPolicy, dataMaskPolicy)) { + return true; + } + } + return false; + } + + private boolean privilegeChanged(ConnectContext connectContext, Env env, SqlCacheContext sqlCacheContext) { + StatementContext currentStatementContext = connectContext.getStatementContext(); + for (Entry> kv : sqlCacheContext.getCheckPrivilegeTablesOrViews().entrySet()) { + Set usedColumns = kv.getValue(); + TableIf tableIf = findTableIf(env, kv.getKey()); + if (tableIf == null) { + return true; + } + // release when close statementContext + currentStatementContext.addTableReadLock(tableIf); + try { + UserAuthentication.checkPermission(tableIf, connectContext, usedColumns); + } catch (Throwable t) { + return true; + } + } + return false; + } + + private boolean usedVariablesChanged(SqlCacheContext sqlCacheContext) { + for (Variable variable : sqlCacheContext.getUsedVariables()) { + Variable currentVariable = ExpressionAnalyzer.resolveUnboundVariable( + new UnboundVariable(variable.getName(), variable.getType())); + if (!Objects.equals(currentVariable, variable) + || variable.getRealExpression().anyMatch(Nondeterministic.class::isInstance)) { + return true; + } + } + return false; + } + + private boolean nondeterministicFunctionChanged( + Plan plan, ConnectContext connectContext, SqlCacheContext sqlCacheContext) { + if (sqlCacheContext.containsCannotProcessExpression()) { + return true; + } + + List> nondeterministicFunctions = sqlCacheContext.getFoldNondeterministicPairs(); + if (nondeterministicFunctions.isEmpty()) { + return false; + } + + CascadesContext tempCascadeContext = CascadesContext.initContext( + connectContext.getStatementContext(), plan, PhysicalProperties.ANY); + ExpressionRewriteContext rewriteContext = new ExpressionRewriteContext(tempCascadeContext); + for (Pair foldPair : nondeterministicFunctions) { + Expression nondeterministic = foldPair.first; + Expression deterministic = foldPair.second; + Expression fold = nondeterministic.accept(FoldConstantRuleOnFE.VISITOR_INSTANCE, rewriteContext); + if (!Objects.equals(deterministic, fold)) { + return true; + } + } + return false; + } + + private boolean isValidDbAndTable(TableIf tableIf, Env env) { + return getTableFromEnv(tableIf, env) != null; + } + + private TableIf getTableFromEnv(TableIf tableIf, Env env) { + Optional db = env.getInternalCatalog().getDb(tableIf.getDatabase().getId()); + if (!db.isPresent()) { + return null; + } + Optional table = db.get().getTable(tableIf.getId()); + return table.orElse(null); + } + + private Optional invalidateCache(String key) { + sqlCache.invalidate(key); + return Optional.empty(); + } + + private TableIf findTableIf(Env env, FullTableName fullTableName) { + CatalogIf> catalog = env.getCatalogMgr().getCatalog(fullTableName.catalog); + if (catalog == null) { + return null; + } + Optional> db = catalog.getDb(fullTableName.db); + if (!db.isPresent()) { + return null; + } + return db.get().getTable(fullTableName.table).orElse(null); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java b/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java index a8f64f1443545e..178a0e802d5bc1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java @@ -585,7 +585,7 @@ public Map build() { } public String getPrettyParseSqlTime() { - return getPrettyTime(parseSqlStartTime, parseSqlFinishTime, TUnit.TIME_MS); + return getPrettyTime(parseSqlFinishTime, parseSqlStartTime, TUnit.TIME_MS); } public String getPrettyNereidsAnalysisTime() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java index 60b7c0343a7001..dd569ef8f7519a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java @@ -102,7 +102,7 @@ public class CascadesContext implements ScheduleContext { private Optional currentRootRewriteJobContext; // in optimize stage, the plan will storage in the memo private Memo memo; - private final StatementContext statementContext; + private StatementContext statementContext; private final CTEContext cteContext; private final RuleSet ruleSet; @@ -265,6 +265,10 @@ public Memo getMemo() { return memo; } + public void releaseMemo() { + this.memo = null; + } + public void setTables(List tables) { this.tables = tables.stream().collect(Collectors.toMap(TableIf::getId, t -> t, (t1, t2) -> t1)); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java index 7457e4de04a4ef..a19477093c7dcf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java @@ -51,10 +51,12 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalSqlCache; import org.apache.doris.nereids.trees.plans.physical.PhysicalCatalogRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalOneRowRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; import org.apache.doris.nereids.trees.plans.physical.PhysicalResultSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalSqlCache; import org.apache.doris.planner.PlanFragment; import org.apache.doris.planner.Planner; import org.apache.doris.planner.RuntimeFilter; @@ -94,6 +96,7 @@ public class NereidsPlanner extends Planner { private PhysicalPlan physicalPlan; // The cost of optimized plan private double cost = 0; + private LogicalPlanAdapter logicalPlanAdapter; private List hooks = new ArrayList<>(); public NereidsPlanner(StatementContext statementContext) { @@ -111,7 +114,7 @@ public void plan(StatementBase queryStmt, org.apache.doris.thrift.TQueryOptions throw new RuntimeException("Wrong type of queryStmt, expected: "); } - LogicalPlanAdapter logicalPlanAdapter = (LogicalPlanAdapter) queryStmt; + logicalPlanAdapter = (LogicalPlanAdapter) queryStmt; ExplainLevel explainLevel = getExplainLevel(queryStmt.getExplainOptions()); @@ -128,32 +131,7 @@ public void plan(StatementBase queryStmt, org.apache.doris.thrift.TQueryOptions return; } physicalPlan = (PhysicalPlan) resultPlan; - PlanTranslatorContext planTranslatorContext = new PlanTranslatorContext(cascadesContext); - PhysicalPlanTranslator physicalPlanTranslator = new PhysicalPlanTranslator(planTranslatorContext, - statementContext.getConnectContext().getStatsErrorEstimator()); - if (statementContext.getConnectContext().getExecutor() != null) { - statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsTranslateTime(); - } - if (cascadesContext.getConnectContext().getSessionVariable().isEnableNereidsTrace()) { - CounterEvent.clearCounter(); - } - if (cascadesContext.getConnectContext().getSessionVariable().isPlayNereidsDump()) { - return; - } - PlanFragment root = physicalPlanTranslator.translatePlan(physicalPlan); - - scanNodeList.addAll(planTranslatorContext.getScanNodes()); - descTable = planTranslatorContext.getDescTable(); - fragments = new ArrayList<>(planTranslatorContext.getPlanFragments()); - for (int seq = 0; seq < fragments.size(); seq++) { - fragments.get(seq).setFragmentSequenceNum(seq); - } - // set output exprs - logicalPlanAdapter.setResultExprs(root.getOutputExprs()); - ArrayList columnLabelList = physicalPlan.getOutput().stream().map(NamedExpression::getName) - .collect(Collectors.toCollection(ArrayList::new)); - logicalPlanAdapter.setColLabels(columnLabelList); - logicalPlanAdapter.setViewDdlSqls(statementContext.getViewDdlSqls()); + translate(physicalPlan); } @VisibleForTesting @@ -183,84 +161,98 @@ public Plan plan(LogicalPlan plan, PhysicalProperties requireProperties, Explain */ public Plan plan(LogicalPlan plan, PhysicalProperties requireProperties, ExplainLevel explainLevel, boolean showPlanProcess) { - if (explainLevel == ExplainLevel.PARSED_PLAN || explainLevel == ExplainLevel.ALL_PLAN) { - parsedPlan = plan; - if (explainLevel == ExplainLevel.PARSED_PLAN) { - return parsedPlan; + try { + if (plan instanceof LogicalSqlCache) { + rewrittenPlan = analyzedPlan = plan; + LogicalSqlCache logicalSqlCache = (LogicalSqlCache) plan; + physicalPlan = new PhysicalSqlCache( + logicalSqlCache.getQueryId(), logicalSqlCache.getColumnLabels(), + logicalSqlCache.getResultExprs(), logicalSqlCache.getCacheValues(), + logicalSqlCache.getBackendAddress(), logicalSqlCache.getPlanBody() + ); + return physicalPlan; + } + if (explainLevel == ExplainLevel.PARSED_PLAN || explainLevel == ExplainLevel.ALL_PLAN) { + parsedPlan = plan; + if (explainLevel == ExplainLevel.PARSED_PLAN) { + return parsedPlan; + } } - } - // pre-process logical plan out of memo, e.g. process SET_VAR hint - plan = preprocess(plan); + // pre-process logical plan out of memo, e.g. process SET_VAR hint + plan = preprocess(plan); - initCascadesContext(plan, requireProperties); + initCascadesContext(plan, requireProperties); - try (Lock lock = new Lock(plan, cascadesContext)) { - // resolve column, table and function - // analyze this query - analyze(showAnalyzeProcess(explainLevel, showPlanProcess)); - // minidump of input must be serialized first, this process ensure minidump string not null - try { - MinidumpUtils.serializeInputsToDumpFile(plan, cascadesContext.getTables()); - } catch (IOException e) { - throw new RuntimeException(e); - } + try (Lock lock = new Lock(plan, cascadesContext)) { + // resolve column, table and function + // analyze this query + analyze(showAnalyzeProcess(explainLevel, showPlanProcess)); + // minidump of input must be serialized first, this process ensure minidump string not null + try { + MinidumpUtils.serializeInputsToDumpFile(plan, cascadesContext.getTables()); + } catch (IOException e) { + throw new RuntimeException(e); + } - if (statementContext.getConnectContext().getExecutor() != null) { - statementContext.getConnectContext().getExecutor().getSummaryProfile().setQueryAnalysisFinishTime(); - statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsAnalysisTime(); - } + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile().setQueryAnalysisFinishTime(); + statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsAnalysisTime(); + } - if (explainLevel == ExplainLevel.ANALYZED_PLAN || explainLevel == ExplainLevel.ALL_PLAN) { - analyzedPlan = cascadesContext.getRewritePlan(); - if (explainLevel == ExplainLevel.ANALYZED_PLAN) { - return analyzedPlan; + if (explainLevel == ExplainLevel.ANALYZED_PLAN || explainLevel == ExplainLevel.ALL_PLAN) { + analyzedPlan = cascadesContext.getRewritePlan(); + if (explainLevel == ExplainLevel.ANALYZED_PLAN) { + return analyzedPlan; + } } - } - // rule-based optimize - rewrite(showRewriteProcess(explainLevel, showPlanProcess)); - if (statementContext.getConnectContext().getExecutor() != null) { - statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsRewriteTime(); - } + // rule-based optimize + rewrite(showRewriteProcess(explainLevel, showPlanProcess)); + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsRewriteTime(); + } - if (explainLevel == ExplainLevel.REWRITTEN_PLAN || explainLevel == ExplainLevel.ALL_PLAN) { - rewrittenPlan = cascadesContext.getRewritePlan(); - if (explainLevel == ExplainLevel.REWRITTEN_PLAN) { - return rewrittenPlan; + if (explainLevel == ExplainLevel.REWRITTEN_PLAN || explainLevel == ExplainLevel.ALL_PLAN) { + rewrittenPlan = cascadesContext.getRewritePlan(); + if (explainLevel == ExplainLevel.REWRITTEN_PLAN) { + return rewrittenPlan; + } } - } - optimize(); - if (statementContext.getConnectContext().getExecutor() != null) { - statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsOptimizeTime(); - } + optimize(); + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsOptimizeTime(); + } - // print memo before choose plan. - // if chooseNthPlan failed, we could get memo to debug - if (cascadesContext.getConnectContext().getSessionVariable().dumpNereidsMemo) { - String memo = cascadesContext.getMemo().toString(); - LOG.info(ConnectContext.get().getQueryIdentifier() + "\n" + memo); - } + // print memo before choose plan. + // if chooseNthPlan failed, we could get memo to debug + if (cascadesContext.getConnectContext().getSessionVariable().dumpNereidsMemo) { + String memo = cascadesContext.getMemo().toString(); + LOG.info(ConnectContext.get().getQueryIdentifier() + "\n" + memo); + } - int nth = cascadesContext.getConnectContext().getSessionVariable().getNthOptimizedPlan(); - PhysicalPlan physicalPlan = chooseNthPlan(getRoot(), requireProperties, nth); + int nth = cascadesContext.getConnectContext().getSessionVariable().getNthOptimizedPlan(); + PhysicalPlan physicalPlan = chooseNthPlan(getRoot(), requireProperties, nth); - physicalPlan = postProcess(physicalPlan); - if (cascadesContext.getConnectContext().getSessionVariable().dumpNereidsMemo) { - String tree = physicalPlan.treeString(); - LOG.info(ConnectContext.get().getQueryIdentifier() + "\n" + tree); - } - if (explainLevel == ExplainLevel.OPTIMIZED_PLAN - || explainLevel == ExplainLevel.ALL_PLAN - || explainLevel == ExplainLevel.SHAPE_PLAN) { - optimizedPlan = physicalPlan; - } - // serialize optimized plan to dumpfile, dumpfile do not have this part means optimize failed - MinidumpUtils.serializeOutputToDumpFile(physicalPlan); - NereidsTracer.output(statementContext.getConnectContext()); + physicalPlan = postProcess(physicalPlan); + if (cascadesContext.getConnectContext().getSessionVariable().dumpNereidsMemo) { + String tree = physicalPlan.treeString(); + LOG.info(ConnectContext.get().getQueryIdentifier() + "\n" + tree); + } + if (explainLevel == ExplainLevel.OPTIMIZED_PLAN + || explainLevel == ExplainLevel.ALL_PLAN + || explainLevel == ExplainLevel.SHAPE_PLAN) { + optimizedPlan = physicalPlan; + } + // serialize optimized plan to dumpfile, dumpfile do not have this part means optimize failed + MinidumpUtils.serializeOutputToDumpFile(physicalPlan); + NereidsTracer.output(statementContext.getConnectContext()); - return physicalPlan; + return physicalPlan; + } + } finally { + statementContext.releasePlannerResources(); } } @@ -313,6 +305,47 @@ private void optimize() { } } + private void translate(PhysicalPlan resultPlan) throws UserException { + if (resultPlan instanceof PhysicalSqlCache) { + return; + } + + PlanTranslatorContext planTranslatorContext = new PlanTranslatorContext(cascadesContext); + PhysicalPlanTranslator physicalPlanTranslator = new PhysicalPlanTranslator(planTranslatorContext, + statementContext.getConnectContext().getStatsErrorEstimator()); + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsTranslateTime(); + } + if (cascadesContext.getConnectContext().getSessionVariable().isEnableNereidsTrace()) { + CounterEvent.clearCounter(); + } + if (cascadesContext.getConnectContext().getSessionVariable().isPlayNereidsDump()) { + return; + } + PlanFragment root = physicalPlanTranslator.translatePlan(physicalPlan); + + scanNodeList.addAll(planTranslatorContext.getScanNodes()); + descTable = planTranslatorContext.getDescTable(); + fragments = new ArrayList<>(planTranslatorContext.getPlanFragments()); + for (int seq = 0; seq < fragments.size(); seq++) { + fragments.get(seq).setFragmentSequenceNum(seq); + } + // set output exprs + logicalPlanAdapter.setResultExprs(root.getOutputExprs()); + ArrayList columnLabelList = physicalPlan.getOutput().stream().map(NamedExpression::getName) + .collect(Collectors.toCollection(ArrayList::new)); + logicalPlanAdapter.setColLabels(columnLabelList); + logicalPlanAdapter.setViewDdlSqls(statementContext.getViewDdlSqls()); + if (statementContext.getSqlCacheContext().isPresent()) { + SqlCacheContext sqlCacheContext = statementContext.getSqlCacheContext().get(); + sqlCacheContext.setColLabels(columnLabelList); + sqlCacheContext.setResultExprs(root.getOutputExprs()); + sqlCacheContext.setPhysicalPlan(resultPlan.treeString()); + } + + cascadesContext.releaseMemo(); + } + private PhysicalPlan postProcess(PhysicalPlan physicalPlan) { return new PlanPostProcessors(cascadesContext).process(physicalPlan); } @@ -569,6 +602,10 @@ public PhysicalPlan getPhysicalPlan() { return physicalPlan; } + public LogicalPlanAdapter getLogicalPlanAdapter() { + return logicalPlanAdapter; + } + public List getHooks() { return hooks; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java new file mode 100644 index 00000000000000..6afda4e4fe96d6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java @@ -0,0 +1,353 @@ +// 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; + +import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.common.Pair; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.mysql.privilege.DataMaskPolicy; +import org.apache.doris.mysql.privilege.RowFilterPolicy; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Variable; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.proto.Types.PUniqueId; +import org.apache.doris.qe.cache.CacheProxy; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; + +/** SqlCacheContext */ +public class SqlCacheContext { + private final UserIdentity userIdentity; + private final TUniqueId queryId; + // if contains udf/udaf/tableValuesFunction we can not process it and skip use sql cache + private volatile boolean cannotProcessExpression; + private volatile String physicalPlan; + private volatile long latestPartitionId = -1; + private volatile long latestPartitionTime = -1; + private volatile long latestPartitionVersion = -1; + private volatile long sumOfPartitionNum = -1; + private final Set usedTables = Sets.newLinkedHashSet(); + // value: ddl sql + private final Map usedViews = Maps.newLinkedHashMap(); + // value: usedColumns + private final Map> checkPrivilegeTablesOrViews = Maps.newLinkedHashMap(); + private final Map> rowPolicies = Maps.newLinkedHashMap(); + private final Map> dataMaskPolicies = Maps.newLinkedHashMap(); + private final Set usedVariables = Sets.newLinkedHashSet(); + // key: the expression which contains nondeterministic function, e.g. date(now()) + // value: the expression which already try to fold nondeterministic function, e.g. '2024-01-01' + // note that value maybe contains nondeterministic function too, when fold failed + private final List> foldNondeterministicPairs = Lists.newArrayList(); + private volatile boolean hasUnsupportedTables; + private final List scanTables = Lists.newArrayList(); + private volatile CacheProxy cacheProxy; + + private volatile List resultExprs; + private volatile List colLabels; + + private volatile PUniqueId cacheKeyMd5; + + public SqlCacheContext(UserIdentity userIdentity, TUniqueId queryId) { + this.userIdentity = Objects.requireNonNull(userIdentity, "userIdentity cannot be null"); + this.queryId = Objects.requireNonNull(queryId, "queryId cannot be null"); + } + + public String getPhysicalPlan() { + return physicalPlan; + } + + public void setPhysicalPlan(String physicalPlan) { + this.physicalPlan = physicalPlan; + } + + public void setCannotProcessExpression(boolean cannotProcessExpression) { + this.cannotProcessExpression = cannotProcessExpression; + } + + public boolean containsCannotProcessExpression() { + return cannotProcessExpression; + } + + public boolean hasUnsupportedTables() { + return hasUnsupportedTables; + } + + public void setHasUnsupportedTables(boolean hasUnsupportedTables) { + this.hasUnsupportedTables = hasUnsupportedTables; + } + + /** addUsedTable */ + public synchronized void addUsedTable(TableIf tableIf) { + if (tableIf == null) { + return; + } + DatabaseIf database = tableIf.getDatabase(); + if (database == null) { + setCannotProcessExpression(true); + return; + } + CatalogIf catalog = database.getCatalog(); + if (catalog == null) { + setCannotProcessExpression(true); + return; + } + + usedTables.add( + new FullTableName(database.getCatalog().getName(), database.getFullName(), tableIf.getName()) + ); + } + + /** addUsedView */ + public synchronized void addUsedView(TableIf tableIf, String ddlSql) { + if (tableIf == null) { + return; + } + DatabaseIf database = tableIf.getDatabase(); + if (database == null) { + setCannotProcessExpression(true); + return; + } + CatalogIf catalog = database.getCatalog(); + if (catalog == null) { + setCannotProcessExpression(true); + return; + } + + usedViews.put( + new FullTableName(database.getCatalog().getName(), database.getFullName(), tableIf.getName()), + ddlSql + ); + } + + /** addNeedCheckPrivilegeTablesOrViews */ + public synchronized void addCheckPrivilegeTablesOrViews(TableIf tableIf, Set usedColumns) { + if (tableIf == null) { + return; + } + DatabaseIf database = tableIf.getDatabase(); + if (database == null) { + setCannotProcessExpression(true); + return; + } + CatalogIf catalog = database.getCatalog(); + if (catalog == null) { + setCannotProcessExpression(true); + return; + } + FullTableName fullTableName = new FullTableName(catalog.getName(), database.getFullName(), tableIf.getName()); + Set existsColumns = checkPrivilegeTablesOrViews.get(fullTableName); + if (existsColumns == null) { + checkPrivilegeTablesOrViews.put(fullTableName, usedColumns); + } else { + ImmutableSet.Builder allUsedColumns = ImmutableSet.builderWithExpectedSize( + existsColumns.size() + usedColumns.size()); + allUsedColumns.addAll(existsColumns); + allUsedColumns.addAll(usedColumns); + checkPrivilegeTablesOrViews.put(fullTableName, allUsedColumns.build()); + } + } + + public synchronized void setRowFilterPolicy( + String catalog, String db, String table, List rowFilterPolicy) { + rowPolicies.put(new FullTableName(catalog, db, table), Utils.fastToImmutableList(rowFilterPolicy)); + } + + public synchronized Map> getRowFilterPolicies() { + return ImmutableMap.copyOf(rowPolicies); + } + + public synchronized void addDataMaskPolicy( + String catalog, String db, String table, String columnName, Optional dataMaskPolicy) { + dataMaskPolicies.put( + new FullColumnName(catalog, db, table, columnName.toLowerCase(Locale.ROOT)), dataMaskPolicy + ); + } + + public synchronized Map> getDataMaskPolicies() { + return ImmutableMap.copyOf(dataMaskPolicies); + } + + public synchronized void addUsedVariable(Variable value) { + usedVariables.add(value); + } + + public synchronized List getUsedVariables() { + return ImmutableList.copyOf(usedVariables); + } + + public synchronized void addFoldNondeterministicPair(Expression unfold, Expression fold) { + foldNondeterministicPairs.add(Pair.of(unfold, fold)); + } + + public synchronized List> getFoldNondeterministicPairs() { + return ImmutableList.copyOf(foldNondeterministicPairs); + } + + public boolean isCannotProcessExpression() { + return cannotProcessExpression; + } + + public UserIdentity getUserIdentity() { + return userIdentity; + } + + public long getLatestPartitionTime() { + return latestPartitionTime; + } + + public void setLatestPartitionTime(long latestPartitionTime) { + this.latestPartitionTime = latestPartitionTime; + } + + public long getLatestPartitionVersion() { + return latestPartitionVersion; + } + + public void setLatestPartitionVersion(long latestPartitionVersion) { + this.latestPartitionVersion = latestPartitionVersion; + } + + public long getLatestPartitionId() { + return latestPartitionId; + } + + public void setLatestPartitionId(long latestPartitionId) { + this.latestPartitionId = latestPartitionId; + } + + public long getSumOfPartitionNum() { + return sumOfPartitionNum; + } + + public void setSumOfPartitionNum(long sumOfPartitionNum) { + this.sumOfPartitionNum = sumOfPartitionNum; + } + + public CacheProxy getCacheProxy() { + return cacheProxy; + } + + public void setCacheProxy(CacheProxy cacheProxy) { + this.cacheProxy = cacheProxy; + } + + public Set getUsedTables() { + return ImmutableSet.copyOf(usedTables); + } + + public Map getUsedViews() { + return ImmutableMap.copyOf(usedViews); + } + + public synchronized Map> getCheckPrivilegeTablesOrViews() { + return ImmutableMap.copyOf(checkPrivilegeTablesOrViews); + } + + public synchronized Map> getRowPolicies() { + return ImmutableMap.copyOf(rowPolicies); + } + + public boolean isHasUnsupportedTables() { + return hasUnsupportedTables; + } + + public synchronized void addScanTable(ScanTable scanTable) { + this.scanTables.add(scanTable); + } + + public synchronized List getScanTables() { + return ImmutableList.copyOf(scanTables); + } + + public List getResultExprs() { + return resultExprs; + } + + public void setResultExprs(List resultExprs) { + this.resultExprs = ImmutableList.copyOf(resultExprs); + } + + public List getColLabels() { + return colLabels; + } + + public void setColLabels(List colLabels) { + this.colLabels = ImmutableList.copyOf(colLabels); + } + + public TUniqueId getQueryId() { + return queryId; + } + + public PUniqueId getCacheKeyMd5() { + return cacheKeyMd5; + } + + public void setCacheKeyMd5(PUniqueId cacheKeyMd5) { + this.cacheKeyMd5 = cacheKeyMd5; + } + + /** FullTableName */ + @lombok.Data + @lombok.AllArgsConstructor + public static class FullTableName { + public final String catalog; + public final String db; + public final String table; + } + + /** FullColumnName */ + @lombok.Data + @lombok.AllArgsConstructor + public static class FullColumnName { + public final String catalog; + public final String db; + public final String table; + public final String column; + } + + /** ScanTable */ + @lombok.Data + @lombok.AllArgsConstructor + public static class ScanTable { + public final FullTableName fullTableName; + public final long latestTimestamp; + public final long latestVersion; + public final List scanPartitions = Lists.newArrayList(); + + public void addScanPartition(Long partitionId) { + this.scanPartitions.add(partitionId); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java index 403c605ba75992..5baa59a35dec1b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids; import org.apache.doris.analysis.StatementBase; +import org.apache.doris.catalog.TableIf; import org.apache.doris.common.IdGenerator; import org.apache.doris.common.Pair; import org.apache.doris.nereids.hint.Hint; @@ -37,6 +38,7 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.OriginStatement; import org.apache.doris.qe.SessionVariable; +import org.apache.doris.qe.cache.CacheAnalyzer; import com.google.common.base.Stopwatch; import com.google.common.base.Supplier; @@ -44,7 +46,11 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.sparkproject.guava.base.Throwables; +import java.io.Closeable; import java.util.ArrayList; import java.util.BitSet; import java.util.Collection; @@ -53,14 +59,18 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; +import java.util.Stack; import java.util.TreeMap; +import java.util.concurrent.TimeUnit; import javax.annotation.concurrent.GuardedBy; /** * Statement context for nereids */ -public class StatementContext { +public class StatementContext implements Closeable { + private static final Logger LOG = LogManager.getLogger(StatementContext.class); private ConnectContext connectContext; @@ -101,6 +111,8 @@ public class StatementContext { private final Map rewrittenCteProducer = new HashMap<>(); private final Map rewrittenCteConsumer = new HashMap<>(); private final Set viewDdlSqlSet = Sets.newHashSet(); + private final SqlCacheContext sqlCacheContext; + private Map> checkedPrivilegedTableAndUsedColumns = Maps.newLinkedHashMap(); // collect all hash join conditions to compute node connectivity in join graph private final List joinFilters = new ArrayList<>(); @@ -122,18 +134,30 @@ public class StatementContext { private BitSet disableRules; + // table locks + private Stack plannerResources = new Stack<>(); + // for create view support in nereids // key is the start and end position of the sql substring that needs to be replaced, // and value is the new string used for replacement. private TreeMap, String> indexInSqlToString = new TreeMap<>(new Pair.PairComparator<>()); public StatementContext() { - this.connectContext = ConnectContext.get(); + this(ConnectContext.get(), null); } + /** StatementContext */ public StatementContext(ConnectContext connectContext, OriginStatement originStatement) { this.connectContext = connectContext; this.originStatement = originStatement; + if (connectContext != null && connectContext.getSessionVariable() != null + && connectContext.queryId() != null + && CacheAnalyzer.canUseSqlCache(connectContext.getSessionVariable())) { + this.sqlCacheContext = new SqlCacheContext( + connectContext.getCurrentUserIdentity(), connectContext.queryId()); + } else { + this.sqlCacheContext = null; + } } public void setConnectContext(ConnectContext connectContext) { @@ -172,6 +196,10 @@ public void setMaxContinuousJoin(int joinCount) { } } + public Optional getSqlCacheContext() { + return Optional.ofNullable(sqlCacheContext); + } + public int getMaxContinuousJoin() { return joinCount; } @@ -368,4 +396,90 @@ public TreeMap, String> getIndexInSqlToString() { public void addIndexInSqlToString(Pair pair, String replacement) { indexInSqlToString.put(pair, replacement); } + + /** addTableReadLock */ + public synchronized void addTableReadLock(TableIf tableIf) { + if (!tableIf.needReadLockWhenPlan()) { + return; + } + if (!tableIf.tryReadLock(1, TimeUnit.MINUTES)) { + close(); + throw new RuntimeException(String.format("Failed to get read lock on table: %s", tableIf.getName())); + } + + String fullTableName = tableIf.getNameWithFullQualifiers(); + String resourceName = "tableReadLock(" + fullTableName + ")"; + plannerResources.push(new CloseableResource( + resourceName, Thread.currentThread().getName(), originStatement.originStmt, tableIf::readUnlock)); + } + + /** releasePlannerResources */ + public synchronized void releasePlannerResources() { + Throwable throwable = null; + while (!plannerResources.isEmpty()) { + try { + plannerResources.pop().close(); + } catch (Throwable t) { + if (throwable == null) { + throwable = t; + } + } + } + if (throwable != null) { + Throwables.propagateIfInstanceOf(throwable, RuntimeException.class); + throw new IllegalStateException("Release resource failed", throwable); + } + } + + // CHECKSTYLE OFF + @Override + protected void finalize() throws Throwable { + if (!plannerResources.isEmpty()) { + String msg = "Resources leak: " + plannerResources; + LOG.error(msg); + throw new IllegalStateException(msg); + } + } + // CHECKSTYLE ON + + @Override + public void close() { + releasePlannerResources(); + } + + private static class CloseableResource implements Closeable { + public final String resourceName; + public final String threadName; + public final String sql; + + private final Closeable resource; + + private boolean closed; + + public CloseableResource(String resourceName, String threadName, String sql, Closeable resource) { + this.resourceName = resourceName; + this.threadName = threadName; + this.sql = sql; + this.resource = resource; + } + + @Override + public void close() { + if (!closed) { + try { + resource.close(); + } catch (Throwable t) { + Throwables.propagateIfInstanceOf(t, RuntimeException.class); + throw new IllegalStateException("Close resource failed: " + t.getMessage(), t); + } + closed = true; + } + } + + @Override + public String toString() { + return "\nResource {\n name: " + resourceName + ",\n thread: " + threadName + + ",\n sql:\n" + sql + "\n}"; + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/NereidsParser.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/NereidsParser.java index da6881a4905468..35e1a6a354ba33 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/NereidsParser.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/NereidsParser.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.parser; +import org.apache.doris.analysis.ExplainOptions; import org.apache.doris.analysis.StatementBase; import org.apache.doris.catalog.Env; import org.apache.doris.common.Pair; @@ -26,6 +27,7 @@ import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.parser.plsql.PLSqlLogicalPlanBuilder; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.types.DataType; import org.apache.doris.plugin.DialectConverterPlugin; @@ -37,14 +39,18 @@ import org.antlr.v4.runtime.CharStreams; import org.antlr.v4.runtime.CommonTokenStream; import org.antlr.v4.runtime.ParserRuleContext; +import org.antlr.v4.runtime.Token; +import org.antlr.v4.runtime.TokenSource; import org.antlr.v4.runtime.atn.PredictionMode; import org.antlr.v4.runtime.misc.ParseCancellationException; import org.apache.commons.collections.CollectionUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.BitSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.function.Function; import javax.annotation.Nullable; @@ -56,6 +62,21 @@ public class NereidsParser { private static final ParseErrorListener PARSE_ERROR_LISTENER = new ParseErrorListener(); private static final PostProcessor POST_PROCESSOR = new PostProcessor(); + private static final BitSet EXPLAIN_TOKENS = new BitSet(); + + static { + EXPLAIN_TOKENS.set(DorisLexer.EXPLAIN); + EXPLAIN_TOKENS.set(DorisLexer.PARSED); + EXPLAIN_TOKENS.set(DorisLexer.ANALYZED); + EXPLAIN_TOKENS.set(DorisLexer.LOGICAL); + EXPLAIN_TOKENS.set(DorisLexer.REWRITTEN); + EXPLAIN_TOKENS.set(DorisLexer.PHYSICAL); + EXPLAIN_TOKENS.set(DorisLexer.OPTIMIZED); + EXPLAIN_TOKENS.set(DorisLexer.PLAN); + EXPLAIN_TOKENS.set(DorisLexer.PROCESS); + + } + /** * In MySQL protocol, client could send multi-statement in a single packet. * see docs for more information. @@ -83,6 +104,98 @@ public List parseSQL(String originStr, @Nullable LogicalPlanBuild return statementBases; } + /** + * scan to token + * for example: select id from tbl return Tokens: ['select', 'id', 'from', 'tbl'] + */ + public static TokenSource scan(String sql) { + return new DorisLexer(new CaseInsensitiveStream(CharStreams.fromString(sql))); + } + + /** + * tryParseExplainPlan + * @param sql sql + * @return key: ExplainOptions, value: explain body + */ + public static Optional> tryParseExplainPlan(String sql) { + try { + TokenSource tokenSource = scan(sql); + if (expect(tokenSource, DorisLexer.EXPLAIN) == null) { + return Optional.empty(); + } + + Token token = readUntilNonComment(tokenSource); + if (token == null) { + return Optional.empty(); + } + + int tokenType = token.getType(); + ExplainLevel explainLevel = ExplainLevel.ALL_PLAN; + if (tokenType == DorisLexer.PARSED) { + explainLevel = ExplainLevel.PARSED_PLAN; + token = readUntilNonComment(tokenSource); + } else if (tokenType == DorisLexer.ANALYZED) { + explainLevel = ExplainLevel.ANALYZED_PLAN; + token = readUntilNonComment(tokenSource); + } else if (tokenType == DorisLexer.LOGICAL || tokenType == DorisLexer.REWRITTEN) { + explainLevel = ExplainLevel.REWRITTEN_PLAN; + token = readUntilNonComment(tokenSource); + } else if (tokenType == DorisLexer.PHYSICAL || tokenType == DorisLexer.OPTIMIZED) { + explainLevel = ExplainLevel.OPTIMIZED_PLAN; + token = readUntilNonComment(tokenSource); + } + + if (token == null) { + return Optional.empty(); + } + tokenType = token.getType(); + if (tokenType != DorisLexer.PLAN) { + return Optional.empty(); + } + + token = readUntilNonComment(tokenSource); + Token explainPlanBody; + boolean showPlanProcess = false; + if (token.getType() == DorisLexer.PROCESS) { + showPlanProcess = true; + explainPlanBody = readUntilNonComment(tokenSource); + } else { + explainPlanBody = token; + } + + if (explainPlanBody == null) { + return Optional.empty(); + } + ExplainOptions explainOptions = new ExplainOptions(explainLevel, showPlanProcess); + return Optional.of(Pair.of(explainOptions, sql.substring(explainPlanBody.getStartIndex()))); + } catch (Throwable t) { + return Optional.empty(); + } + } + + private static Token expect(TokenSource tokenSource, int tokenType) { + Token nextToken = readUntilNonComment(tokenSource); + if (nextToken == null) { + return null; + } + return nextToken.getType() == tokenType ? nextToken : null; + } + + private static Token readUntilNonComment(TokenSource tokenSource) { + Token token = tokenSource.nextToken(); + while (token != null) { + int tokenType = token.getType(); + if (tokenType == DorisLexer.BRACKETED_COMMENT + || tokenType == DorisLexer.SIMPLE_COMMENT + || tokenType == DorisLexer.WS) { + token = tokenSource.nextToken(); + continue; + } + break; + } + return token; + } + private List parseSQLWithDialect(String sql, SessionVariable sessionVariable) { @Nullable Dialect sqlDialect = Dialect.getByName(sessionVariable.getSqlDialect()); 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 3c73a53dcbaf57..34af516c45b05a 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 @@ -19,8 +19,10 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.FunctionRegistry; +import org.apache.doris.common.Pair; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.SqlCacheContext; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.MappingSlot; import org.apache.doris.nereids.analyzer.Scope; @@ -49,6 +51,7 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; +import org.apache.doris.nereids.trees.expressions.functions.BoundFunction; import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.functions.FunctionBuilder; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; @@ -809,11 +812,16 @@ private LogicalTVFRelation bindTableValuedFunction(MatchingContext bindResult + = functionBuilder.build(functionName, arguments); + if (!(bindResult.first instanceof TableValuedFunction)) { + throw new AnalysisException(bindResult.first.toSql() + " is not a TableValuedFunction"); } - return new LogicalTVFRelation(unboundTVFRelation.getRelationId(), (TableValuedFunction) function); + Optional sqlCacheContext = statementContext.getSqlCacheContext(); + if (sqlCacheContext.isPresent()) { + sqlCacheContext.get().setCannotProcessExpression(true); + } + return new LogicalTVFRelation(unboundTVFRelation.getRelationId(), (TableValuedFunction) bindResult.first); } private void checkSameNameSlot(List childOutputs, String subQueryAlias) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index 0aac17cff9b978..84a3021ecd1740 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java @@ -31,6 +31,7 @@ import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.nereids.CTEContext; import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.SqlCacheContext; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.Unbound; import org.apache.doris.nereids.analyzer.UnboundRelation; @@ -249,68 +250,89 @@ private LogicalPlan getLogicalPlan(TableIf table, UnboundRelation unboundRelatio }); List qualifierWithoutTableName = Lists.newArrayList(); qualifierWithoutTableName.addAll(tableQualifier.subList(0, tableQualifier.size() - 1)); - switch (table.getType()) { - case OLAP: - case MATERIALIZED_VIEW: - return makeOlapScan(table, unboundRelation, qualifierWithoutTableName); - case VIEW: - View view = (View) table; - String inlineViewDef = view.getInlineViewDef(); - Plan viewBody = parseAndAnalyzeView(inlineViewDef, cascadesContext); - LogicalView logicalView = new LogicalView<>(view, viewBody); - return new LogicalSubQueryAlias<>(tableQualifier, logicalView); - case HMS_EXTERNAL_TABLE: - HMSExternalTable hmsTable = (HMSExternalTable) table; - if (Config.enable_query_hive_views && hmsTable.isView()) { - String hiveCatalog = hmsTable.getCatalog().getName(); - String ddlSql = hmsTable.getViewText(); - Plan hiveViewPlan = parseAndAnalyzeHiveView(hiveCatalog, ddlSql, cascadesContext); - return new LogicalSubQueryAlias<>(tableQualifier, hiveViewPlan); - } - hmsTable.setScanParams(unboundRelation.getScanParams()); - return new LogicalFileScan(unboundRelation.getRelationId(), (HMSExternalTable) table, - qualifierWithoutTableName, unboundRelation.getTableSample()); - case ICEBERG_EXTERNAL_TABLE: - case PAIMON_EXTERNAL_TABLE: - case MAX_COMPUTE_EXTERNAL_TABLE: - return new LogicalFileScan(unboundRelation.getRelationId(), (ExternalTable) table, - qualifierWithoutTableName, unboundRelation.getTableSample()); - case SCHEMA: - return new LogicalSchemaScan(unboundRelation.getRelationId(), table, qualifierWithoutTableName); - case JDBC_EXTERNAL_TABLE: - case JDBC: - return new LogicalJdbcScan(unboundRelation.getRelationId(), table, qualifierWithoutTableName); - case ODBC: - return new LogicalOdbcScan(unboundRelation.getRelationId(), table, qualifierWithoutTableName); - case ES_EXTERNAL_TABLE: - return new LogicalEsScan(unboundRelation.getRelationId(), (EsExternalTable) table, - qualifierWithoutTableName); - case TEST_EXTERNAL_TABLE: - return new LogicalTestScan(unboundRelation.getRelationId(), table, qualifierWithoutTableName); - default: - try { - // TODO: support other type table, such as ELASTICSEARCH - cascadesContext.getConnectContext().getSessionVariable().enableFallbackToOriginalPlannerOnce(); - } catch (Exception e) { - // ignore + boolean isView = false; + try { + switch (table.getType()) { + case OLAP: + case MATERIALIZED_VIEW: + return makeOlapScan(table, unboundRelation, qualifierWithoutTableName); + case VIEW: + View view = (View) table; + isView = true; + String inlineViewDef = view.getInlineViewDef(); + Plan viewBody = parseAndAnalyzeView(view, inlineViewDef, cascadesContext); + LogicalView logicalView = new LogicalView<>(view, viewBody); + return new LogicalSubQueryAlias<>(tableQualifier, logicalView); + case HMS_EXTERNAL_TABLE: + HMSExternalTable hmsTable = (HMSExternalTable) table; + if (Config.enable_query_hive_views && hmsTable.isView()) { + isView = true; + String hiveCatalog = hmsTable.getCatalog().getName(); + String ddlSql = hmsTable.getViewText(); + Plan hiveViewPlan = parseAndAnalyzeHiveView(hmsTable, hiveCatalog, ddlSql, cascadesContext); + return new LogicalSubQueryAlias<>(tableQualifier, hiveViewPlan); + } + hmsTable.setScanParams(unboundRelation.getScanParams()); + return new LogicalFileScan(unboundRelation.getRelationId(), (HMSExternalTable) table, + qualifierWithoutTableName, unboundRelation.getTableSample()); + case ICEBERG_EXTERNAL_TABLE: + case PAIMON_EXTERNAL_TABLE: + case MAX_COMPUTE_EXTERNAL_TABLE: + return new LogicalFileScan(unboundRelation.getRelationId(), (ExternalTable) table, + qualifierWithoutTableName, unboundRelation.getTableSample()); + case SCHEMA: + return new LogicalSchemaScan(unboundRelation.getRelationId(), table, qualifierWithoutTableName); + case JDBC_EXTERNAL_TABLE: + case JDBC: + return new LogicalJdbcScan(unboundRelation.getRelationId(), table, qualifierWithoutTableName); + case ODBC: + return new LogicalOdbcScan(unboundRelation.getRelationId(), table, qualifierWithoutTableName); + case ES_EXTERNAL_TABLE: + return new LogicalEsScan(unboundRelation.getRelationId(), (EsExternalTable) table, + qualifierWithoutTableName); + case TEST_EXTERNAL_TABLE: + return new LogicalTestScan(unboundRelation.getRelationId(), table, qualifierWithoutTableName); + default: + try { + // TODO: support other type table, such as ELASTICSEARCH + cascadesContext.getConnectContext().getSessionVariable().enableFallbackToOriginalPlannerOnce(); + } catch (Exception e) { + // ignore + } + throw new AnalysisException("Unsupported tableType " + table.getType()); + } + } finally { + if (!isView) { + Optional sqlCacheContext = cascadesContext.getStatementContext().getSqlCacheContext(); + if (sqlCacheContext.isPresent()) { + if (table instanceof OlapTable) { + sqlCacheContext.get().addUsedTable(table); + } else { + sqlCacheContext.get().setHasUnsupportedTables(true); + } } - throw new AnalysisException("Unsupported tableType " + table.getType()); + } } } - private Plan parseAndAnalyzeHiveView(String hiveCatalog, String ddlSql, CascadesContext cascadesContext) { + private Plan parseAndAnalyzeHiveView( + HMSExternalTable table, String hiveCatalog, String ddlSql, CascadesContext cascadesContext) { ConnectContext ctx = cascadesContext.getConnectContext(); String previousCatalog = ctx.getCurrentCatalog().getName(); String previousDb = ctx.getDatabase(); ctx.changeDefaultCatalog(hiveCatalog); - Plan hiveViewPlan = parseAndAnalyzeView(ddlSql, cascadesContext); + Plan hiveViewPlan = parseAndAnalyzeView(table, ddlSql, cascadesContext); ctx.changeDefaultCatalog(previousCatalog); ctx.setDatabase(previousDb); return hiveViewPlan; } - private Plan parseAndAnalyzeView(String ddlSql, CascadesContext parentContext) { + private Plan parseAndAnalyzeView(TableIf view, String ddlSql, CascadesContext parentContext) { parentContext.getStatementContext().addViewDdlSql(ddlSql); + Optional sqlCacheContext = parentContext.getStatementContext().getSqlCacheContext(); + if (sqlCacheContext.isPresent()) { + sqlCacheContext.get().addUsedView(view, ddlSql); + } LogicalPlan parsedViewPlan = new NereidsParser().parseSingle(ddlSql); // TODO: use a good to do this, such as eliminate UnboundResultSink if (parsedViewPlan instanceof UnboundResultSink) { 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 954675a28c80a0..cc85bc01323c75 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 @@ -22,8 +22,10 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.FunctionRegistry; import org.apache.doris.common.DdlException; +import org.apache.doris.common.Pair; import org.apache.doris.common.util.Util; import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.SqlCacheContext; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.Scope; import org.apache.doris.nereids.analyzer.UnboundAlias; @@ -34,6 +36,7 @@ import org.apache.doris.nereids.analyzer.UnboundVariable.VariableType; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; +import org.apache.doris.nereids.rules.expression.rules.FoldConstantRuleOnFE; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.ArrayItemReference; import org.apache.doris.nereids.trees.expressions.BinaryArithmetic; @@ -60,12 +63,15 @@ import org.apache.doris.nereids.trees.expressions.WhenClause; import org.apache.doris.nereids.trees.expressions.functions.BoundFunction; import org.apache.doris.nereids.trees.expressions.functions.FunctionBuilder; +import org.apache.doris.nereids.trees.expressions.functions.Nondeterministic; import org.apache.doris.nereids.trees.expressions.functions.agg.Count; import org.apache.doris.nereids.trees.expressions.functions.scalar.ElementAt; import org.apache.doris.nereids.trees.expressions.functions.scalar.Lambda; import org.apache.doris.nereids.trees.expressions.functions.scalar.Nvl; import org.apache.doris.nereids.trees.expressions.functions.scalar.PushDownToProjectionFunction; import org.apache.doris.nereids.trees.expressions.functions.udf.AliasUdfBuilder; +import org.apache.doris.nereids.trees.expressions.functions.udf.JavaUdaf; +import org.apache.doris.nereids.trees.expressions.functions.udf.JavaUdf; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.literal.IntegerLikeLiteral; import org.apache.doris.nereids.trees.expressions.literal.Literal; @@ -83,6 +89,7 @@ import org.apache.doris.qe.SessionVariable; import org.apache.doris.qe.VariableMgr; import org.apache.doris.qe.VariableVarConverters; +import org.apache.doris.qe.cache.CacheAnalyzer; import com.google.common.base.Preconditions; import com.google.common.base.Strings; @@ -113,6 +120,7 @@ public class ExpressionAnalyzer extends SubExprAnalyzer buildResult = builder.build(functionName, arguments); + StatementContext statementContext = context.cascadesContext.getStatementContext(); + if (buildResult.second instanceof Nondeterministic) { + hasNondeterministic = true; + } + Optional sqlCacheContext = statementContext.getSqlCacheContext(); + if (builder instanceof AliasUdfBuilder + || buildResult.second instanceof JavaUdf || buildResult.second instanceof JavaUdaf) { + if (sqlCacheContext.isPresent()) { + sqlCacheContext.get().setCannotProcessExpression(true); + } + } if (builder instanceof AliasUdfBuilder) { + if (sqlCacheContext.isPresent()) { + sqlCacheContext.get().setCannotProcessExpression(true); + } // we do type coercion in build function in alias function, so it's ok to return directly. - return builder.build(functionName, arguments); + return buildResult.first; } else { - Expression boundFunction = TypeCoercionUtils - .processBoundFunction((BoundFunction) builder.build(functionName, arguments)); - if (boundFunction instanceof Count + Expression castFunction = TypeCoercionUtils.processBoundFunction((BoundFunction) buildResult.first); + if (castFunction instanceof Count && context.cascadesContext.getOuterScope().isPresent() && !context.cascadesContext.getOuterScope().get().getCorrelatedSlots() .isEmpty()) { @@ -339,20 +392,20 @@ public Expression visitUnboundFunction(UnboundFunction unboundFunction, Expressi // if there is no match, the row from right table is filled with nulls // but COUNT function is always not nullable. // so wrap COUNT with Nvl to ensure it's result is 0 instead of null to get the correct result - boundFunction = new Nvl(boundFunction, new BigIntLiteral(0)); + castFunction = new Nvl(castFunction, new BigIntLiteral(0)); } if (currentElementAtLevel == 1 - && PushDownToProjectionFunction.validToPushDown(boundFunction)) { + && PushDownToProjectionFunction.validToPushDown(castFunction)) { // Only rewrite the top level of PushDownToProjectionFunction, otherwise invalid slot will be generated // currentElementAtLevel == 1 means at the top of element_at function, other levels will be ignored. currentElementAtLevel = 0; - return visitElementAt((ElementAt) boundFunction, context); + return visitElementAt((ElementAt) castFunction, context); } - if (boundFunction instanceof ElementAt) { + if (castFunction instanceof ElementAt) { --currentElementAtLevel; } - return boundFunction; + return castFunction; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FunctionBinder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FunctionBinder.java index e8ed474a1be733..8e79e60abad1bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FunctionBinder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FunctionBinder.java @@ -181,10 +181,10 @@ public Expression visitUnboundFunction(UnboundFunction unboundFunction, Expressi unboundFunction.getDbName(), functionName, arguments); if (builder instanceof AliasUdfBuilder) { // we do type coercion in build function in alias function, so it's ok to return directly. - return builder.build(functionName, arguments); + return builder.build(functionName, arguments).first; } else { Expression boundFunction = TypeCoercionUtils - .processBoundFunction((BoundFunction) builder.build(functionName, arguments)); + .processBoundFunction((BoundFunction) builder.build(functionName, arguments).first); if (boundFunction instanceof Count && context.cascadesContext.getOuterScope().isPresent() && !context.cascadesContext.getOuterScope().get().getCorrelatedSlots() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ReplaceVariableByLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ReplaceVariableByLiteral.java index b4c5552706c589..74f41e17cac49d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ReplaceVariableByLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ReplaceVariableByLiteral.java @@ -17,6 +17,8 @@ package org.apache.doris.nereids.rules.expression.rules; +import org.apache.doris.nereids.SqlCacheContext; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher; import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory; import org.apache.doris.nereids.trees.expressions.Expression; @@ -25,6 +27,7 @@ import com.google.common.collect.ImmutableList; import java.util.List; +import java.util.Optional; /** * replace varaible to real expression @@ -35,7 +38,15 @@ public class ReplaceVariableByLiteral implements ExpressionPatternRuleFactory { @Override public List> buildRules() { return ImmutableList.of( - matchesType(Variable.class).then(Variable::getRealExpression) + matchesType(Variable.class).thenApply(ctx -> { + StatementContext statementContext = ctx.cascadesContext.getStatementContext(); + Variable variable = ctx.expr; + Optional sqlCacheContext = statementContext.getSqlCacheContext(); + if (sqlCacheContext.isPresent()) { + sqlCacheContext.get().addUsedVariable(variable); + } + return variable.getRealExpression(); + }) ); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java index 70a5c593ee3dc8..713a9404dc08c4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java @@ -19,6 +19,9 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.common.UserException; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.SqlCacheContext; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.rules.analysis.UserAuthentication; @@ -34,6 +37,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; /** CheckPrivileges */ @@ -84,11 +88,17 @@ private Set computeUsedColumns(Plan plan, Set requiredSlots) { } private void checkColumnPrivileges(TableIf table, Set usedColumns) { - ConnectContext connectContext = jobContext.getCascadesContext().getConnectContext(); + CascadesContext cascadesContext = jobContext.getCascadesContext(); + ConnectContext connectContext = cascadesContext.getConnectContext(); try { UserAuthentication.checkPermission(table, connectContext, usedColumns); } catch (UserException e) { throw new AnalysisException(e.getMessage(), e); } + StatementContext statementContext = cascadesContext.getStatementContext(); + Optional sqlCacheContext = statementContext.getSqlCacheContext(); + if (sqlCacheContext.isPresent()) { + sqlCacheContext.get().addCheckPrivilegeTablesOrViews(table, usedColumns); + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/ExpressionEvaluator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/ExpressionEvaluator.java index adced61c7b7508..566798ec2d4e46 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/ExpressionEvaluator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/ExpressionEvaluator.java @@ -20,7 +20,6 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.AnalysisException; import org.apache.doris.nereids.trees.expressions.functions.BoundFunction; -import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; import org.apache.doris.nereids.trees.expressions.functions.executable.DateTimeAcquire; import org.apache.doris.nereids.trees.expressions.functions.executable.DateTimeArithmetic; @@ -79,7 +78,10 @@ public Expression eval(Expression expression) { } else if (expression instanceof BoundFunction) { BoundFunction function = ((BoundFunction) expression); fnName = function.getName(); - args = function.children().stream().map(ExpressionTrait::getDataType).toArray(DataType[]::new); + args = new DataType[function.arity()]; + for (int i = 0; i < function.children().size(); i++) { + args[i] = function.child(i).getDataType(); + } } if ((Env.getCurrentEnv().isNullResultWithOneNullParamFunction(fnName))) { @@ -166,8 +168,11 @@ private void registerFEFunction(ImmutableMultimap.Builder arguments) { } private AggregateFunction buildState(String nestedName, List arguments) { - return (AggregateFunction) nestedBuilder.build(nestedName, arguments); + return (AggregateFunction) nestedBuilder.build(nestedName, arguments).first; } private AggregateFunction buildForEach(String nestedName, List arguments) { @@ -96,7 +97,7 @@ private AggregateFunction buildForEach(String nestedName, List DataType itemType = ((ArrayType) arrayType).getItemType(); return new SlotReference("mocked", itemType, (((ArrayType) arrayType).containsNull())); }).collect(Collectors.toList()); - return (AggregateFunction) nestedBuilder.build(nestedName, forEachargs); + return (AggregateFunction) nestedBuilder.build(nestedName, forEachargs).first; } private AggregateFunction buildMergeOrUnion(String nestedName, List arguments) { @@ -118,24 +119,24 @@ private AggregateFunction buildMergeOrUnion(String nestedName, List arguments) { + public Pair build(String name, List arguments) { String nestedName = getNestedName(name); if (combinatorSuffix.equals(STATE)) { AggregateFunction nestedFunction = buildState(nestedName, arguments); - return new StateCombinator((List) arguments, nestedFunction); + return Pair.of(new StateCombinator((List) arguments, nestedFunction), nestedFunction); } else if (combinatorSuffix.equals(MERGE)) { AggregateFunction nestedFunction = buildMergeOrUnion(nestedName, arguments); - return new MergeCombinator((List) arguments, nestedFunction); + return Pair.of(new MergeCombinator((List) arguments, nestedFunction), nestedFunction); } else if (combinatorSuffix.equals(UNION)) { AggregateFunction nestedFunction = buildMergeOrUnion(nestedName, arguments); - return new UnionCombinator((List) arguments, nestedFunction); + return Pair.of(new UnionCombinator((List) arguments, nestedFunction), nestedFunction); } else if (combinatorSuffix.equals(FOREACH)) { AggregateFunction nestedFunction = buildForEach(nestedName, arguments); - return new ForEachCombinator((List) arguments, nestedFunction); + return Pair.of(new ForEachCombinator((List) arguments, nestedFunction), nestedFunction); } return null; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/BuiltinFunctionBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/BuiltinFunctionBuilder.java index e2dab713332fd6..a071328ec52635 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/BuiltinFunctionBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/BuiltinFunctionBuilder.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.expressions.functions; +import org.apache.doris.common.Pair; import org.apache.doris.common.util.ReflectionUtils; import org.apache.doris.nereids.trees.expressions.Expression; @@ -86,12 +87,12 @@ private Class getConstructorArgumentType(int index) { } @Override - public BoundFunction build(String name, List arguments) { + public Pair build(String name, List arguments) { try { if (isVariableLength) { - return builderMethod.newInstance(toVariableLengthArguments(arguments)); + return Pair.ofSame(builderMethod.newInstance(toVariableLengthArguments(arguments))); } else { - return builderMethod.newInstance(arguments.toArray()); + return Pair.ofSame(builderMethod.newInstance(arguments.toArray())); } } catch (Throwable t) { String argString = arguments.stream() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/FunctionBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/FunctionBuilder.java index d1e69d3e307d6f..760edacaeabaf7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/FunctionBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/FunctionBuilder.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.expressions.functions; +import org.apache.doris.common.Pair; import org.apache.doris.nereids.trees.expressions.Expression; import com.google.common.collect.ImmutableList; @@ -32,7 +33,7 @@ public abstract class FunctionBuilder { /** check whether arguments can apply to the constructor */ public abstract boolean canApply(List arguments); - public final Expression build(String name, Object argument) { + public final Pair build(String name, Object argument) { return build(name, ImmutableList.of(argument)); } @@ -40,7 +41,10 @@ public final Expression build(String name, Object argument) { * build a BoundFunction by function name and arguments. * @param name function name which in the sql expression * @param arguments the function's argument expressions - * @return the concrete bound function instance + * @return the concrete bound function instance, + * key: the final result expression that should return, e.g. the function wrapped some cast function, + * value: the real BoundFunction */ - public abstract Expression build(String name, List arguments); + public abstract Pair build( + String name, List arguments); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConnectionId.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConnectionId.java index 7f16fa1a4acb69..6f17e247c44042 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConnectionId.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConnectionId.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.Nondeterministic; import org.apache.doris.nereids.trees.expressions.shape.LeafExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.BigIntType; @@ -32,7 +33,7 @@ * ScalarFunction 'ConnectionId'. */ public class ConnectionId extends ScalarFunction - implements LeafExpression, ExplicitlyCastableSignature, AlwaysNotNullable { + implements LeafExpression, ExplicitlyCastableSignature, AlwaysNotNullable, Nondeterministic { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/CurrentUser.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/CurrentUser.java index f36e4548ebc5c4..5f00e374716f32 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/CurrentUser.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/CurrentUser.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.Nondeterministic; import org.apache.doris.nereids.trees.expressions.shape.LeafExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.StringType; @@ -32,7 +33,7 @@ * ScalarFunction 'CurrentUser'. */ public class CurrentUser extends ScalarFunction - implements LeafExpression, ExplicitlyCastableSignature, AlwaysNotNullable { + implements LeafExpression, ExplicitlyCastableSignature, AlwaysNotNullable, Nondeterministic { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(StringType.INSTANCE).args() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Database.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Database.java index acb31f5ae4db07..c213fc6bc5c29d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Database.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.Nondeterministic; import org.apache.doris.nereids.trees.expressions.shape.LeafExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.VarcharType; @@ -32,7 +33,7 @@ * ScalarFunction 'database'. */ public class Database extends ScalarFunction - implements LeafExpression, ExplicitlyCastableSignature, AlwaysNotNullable { + implements LeafExpression, ExplicitlyCastableSignature, AlwaysNotNullable, Nondeterministic { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/User.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/User.java index 3a53c291f9db74..6cf547f8ab5ac1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/User.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/User.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.Nondeterministic; import org.apache.doris.nereids.trees.expressions.shape.LeafExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.VarcharType; @@ -32,7 +33,7 @@ * ScalarFunction 'User'. */ public class User extends ScalarFunction - implements LeafExpression, ExplicitlyCastableSignature, AlwaysNotNullable { + implements LeafExpression, ExplicitlyCastableSignature, AlwaysNotNullable, Nondeterministic { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java index e3e25481691c02..26602435651fab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java @@ -27,6 +27,7 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.BoundFunction; import org.apache.doris.nereids.trees.expressions.functions.CustomSignature; +import org.apache.doris.nereids.trees.expressions.functions.Nondeterministic; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.DataType; @@ -44,7 +45,8 @@ import java.util.stream.Collectors; /** TableValuedFunction */ -public abstract class TableValuedFunction extends BoundFunction implements UnaryExpression, CustomSignature { +public abstract class TableValuedFunction extends BoundFunction + implements UnaryExpression, CustomSignature, Nondeterministic { protected final Supplier catalogFunctionCache = Suppliers.memoize(this::toCatalogFunction); protected final Supplier tableCache = Suppliers.memoize(() -> { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/AliasUdfBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/AliasUdfBuilder.java index 733bd5fcae1164..1f15b7e6049a25 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/AliasUdfBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/AliasUdfBuilder.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.expressions.functions.udf; +import org.apache.doris.common.Pair; import org.apache.doris.common.util.ReflectionUtils; import org.apache.doris.nereids.rules.expression.rules.FunctionBinder; import org.apache.doris.nereids.trees.expressions.Expression; @@ -72,7 +73,7 @@ public boolean canApply(List arguments) { } @Override - public Expression build(String name, List arguments) { + public Pair build(String name, List arguments) { // use AliasFunction to process TypeCoercion BoundFunction boundAliasFunction = ((BoundFunction) aliasUdf.withChildren(arguments.stream() .map(Expression.class::cast).collect(Collectors.toList()))); @@ -95,7 +96,7 @@ public Expression build(String name, List arguments) { replaceMap.put(slots.get(parameter), inputs.get(i)); } - return SlotReplacer.INSTANCE.replace(boundFunction, replaceMap); + return Pair.of(SlotReplacer.INSTANCE.replace(boundFunction, replaceMap), boundAliasFunction); } private static class SlotReplacer extends DefaultExpressionRewriter> { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/JavaUdafBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/JavaUdafBuilder.java index 89073b4eb77e4d..0d8ad443f533e5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/JavaUdafBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/JavaUdafBuilder.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.expressions.functions.udf; +import org.apache.doris.common.Pair; import org.apache.doris.common.util.ReflectionUtils; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.BoundFunction; @@ -32,19 +33,19 @@ * function builder for java udaf */ public class JavaUdafBuilder extends UdfBuilder { - private final JavaUdaf udf; + private final JavaUdaf udaf; private final int arity; private final boolean isVarArgs; - public JavaUdafBuilder(JavaUdaf udf) { - this.udf = udf; - this.isVarArgs = udf.hasVarArguments(); - this.arity = udf.arity(); + public JavaUdafBuilder(JavaUdaf udaf) { + this.udaf = udaf; + this.isVarArgs = udaf.hasVarArguments(); + this.arity = udaf.arity(); } @Override public List getArgTypes() { - return Suppliers.memoize(() -> udf.getSignatures().get(0).argumentsTypes.stream() + return Suppliers.memoize(() -> udaf.getSignatures().get(0).argumentsTypes.stream() .map(DataType.class::cast) .collect(Collectors.toList())).get(); } @@ -71,7 +72,11 @@ public boolean canApply(List arguments) { } @Override - public BoundFunction build(String name, List arguments) { - return udf.withChildren(arguments.stream().map(Expression.class::cast).collect(Collectors.toList())); + public Pair build(String name, List arguments) { + return Pair.ofSame((JavaUdaf) udaf.withChildren( + arguments.stream() + .map(Expression.class::cast) + .collect(Collectors.toList())) + ); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/JavaUdfBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/JavaUdfBuilder.java index a78f8cfe4b46ae..04d4741286a4ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/JavaUdfBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/JavaUdfBuilder.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.expressions.functions.udf; +import org.apache.doris.common.Pair; import org.apache.doris.common.util.ReflectionUtils; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.BoundFunction; @@ -73,14 +74,14 @@ public boolean canApply(List arguments) { } @Override - public BoundFunction build(String name, List arguments) { + public Pair build(String name, List arguments) { List exprs = arguments.stream().map(Expression.class::cast).collect(Collectors.toList()); List argTypes = udf.getSignatures().get(0).argumentsTypes; List processedExprs = Lists.newArrayList(); for (int i = 0; i < exprs.size(); ++i) { - processedExprs.add(TypeCoercionUtils.castIfNotSameType(exprs.get(i), ((DataType) argTypes.get(i)))); + processedExprs.add(TypeCoercionUtils.castIfNotSameType(exprs.get(i), argTypes.get(i))); } - return udf.withChildren(processedExprs); + return Pair.ofSame(udf.withChildren(processedExprs)); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java index 620159135c8542..93127933ea7160 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java @@ -27,7 +27,6 @@ import org.apache.doris.nereids.types.coercion.DateLikeType; import org.apache.doris.nereids.util.DateTimeFormatterUtils; import org.apache.doris.nereids.util.DateUtils; -import org.apache.doris.nereids.util.StandardDateFormat; import com.google.common.collect.ImmutableSet; @@ -352,12 +351,36 @@ public double getDouble() { @Override public String getStringValue() { + if (0 <= year && year <= 9999 && 0 <= month && month <= 99 && 0 <= day && day <= 99) { + char[] format = new char[] {'0', '0', '0', '0', '-', '0', '0', '-', '0', '0'}; + int offset = 3; + long year = this.year; + while (year > 0) { + format[offset--] = (char) ('0' + (year % 10)); + year /= 10; + } + + offset = 6; + long month = this.month; + while (month > 0) { + format[offset--] = (char) ('0' + (month % 10)); + month /= 10; + } + + offset = 9; + long day = this.day; + while (day > 0) { + format[offset--] = (char) ('0' + (day % 10)); + day /= 10; + } + return String.valueOf(format); + } return String.format("%04d-%02d-%02d", year, month, day); } @Override public String toSql() { - return String.format("'%s'", toString()); + return "'" + getStringValue() + "'"; } @Override @@ -367,7 +390,7 @@ public R accept(ExpressionVisitor visitor, C context) { @Override public String toString() { - return String.format("%04d-%02d-%02d", year, month, day); + return getStringValue(); } @Override @@ -388,22 +411,19 @@ public long getDay() { } public Expression plusDays(long days) { - return fromJavaDateType(DateUtils.getTime(StandardDateFormat.DATE_FORMATTER, getStringValue()).plusDays(days)); + return fromJavaDateType(toJavaDateType().plusDays(days)); } public Expression plusMonths(long months) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_FORMATTER, getStringValue()).plusMonths(months)); + return fromJavaDateType(toJavaDateType().plusMonths(months)); } public Expression plusWeeks(long weeks) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_FORMATTER, getStringValue()).plusWeeks(weeks)); + return fromJavaDateType(toJavaDateType().plusWeeks(weeks)); } public Expression plusYears(long years) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_FORMATTER, getStringValue()).plusYears(years)); + return fromJavaDateType(toJavaDateType().plusYears(years)); } public LocalDateTime toJavaDateType() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeLiteral.java index 16849feb89233f..1fc446da356073 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeLiteral.java @@ -25,7 +25,6 @@ import org.apache.doris.nereids.types.DateTimeType; import org.apache.doris.nereids.types.coercion.DateLikeType; import org.apache.doris.nereids.util.DateUtils; -import org.apache.doris.nereids.util.StandardDateFormat; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -199,16 +198,63 @@ public double getDouble() { @Override public String toSql() { - return String.format("'%s'", toString()); + return "'" + getStringValue() + "'"; } @Override public String toString() { - return String.format("%04d-%02d-%02d %02d:%02d:%02d", year, month, day, hour, minute, second); + return getStringValue(); } @Override public String getStringValue() { + if (0 <= year && year <= 9999 && 0 <= month && month <= 99 && 0 <= day && day <= 99 + && 0 <= hour && hour <= 99 && 0 <= minute && minute <= 99 && 0 <= second && second <= 99) { + char[] format = new char[] { + '0', '0', '0', '0', '-', '0', '0', '-', '0', '0', ' ', '0', '0', ':', '0', '0', ':', '0', '0'}; + int offset = 3; + long year = this.year; + while (year > 0) { + format[offset--] = (char) ('0' + (year % 10)); + year /= 10; + } + + offset = 6; + long month = this.month; + while (month > 0) { + format[offset--] = (char) ('0' + (month % 10)); + month /= 10; + } + + offset = 9; + long day = this.day; + while (day > 0) { + format[offset--] = (char) ('0' + (day % 10)); + day /= 10; + } + + offset = 12; + long hour = this.hour; + while (hour > 0) { + format[offset--] = (char) ('0' + (hour % 10)); + hour /= 10; + } + + offset = 15; + long minute = this.minute; + while (minute > 0) { + format[offset--] = (char) ('0' + (minute % 10)); + minute /= 10; + } + + offset = 18; + long second = this.second; + while (second > 0) { + format[offset--] = (char) ('0' + (second % 10)); + second /= 10; + } + return String.valueOf(format); + } return String.format("%04d-%02d-%02d %02d:%02d:%02d", year, month, day, hour, minute, second); } @@ -217,39 +263,32 @@ public LiteralExpr toLegacyLiteral() { return new org.apache.doris.analysis.DateLiteral(year, month, day, hour, minute, second, Type.DATETIME); } - public Expression plusYears(long years) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER, getStringValue()).plusYears(years)); + public Expression plusDays(long days) { + return fromJavaDateType(toJavaDateType().plusDays(days)); } public Expression plusMonths(long months) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER, getStringValue()).plusMonths(months)); + return fromJavaDateType(toJavaDateType().plusMonths(months)); } public Expression plusWeeks(long weeks) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER, getStringValue()).plusWeeks(weeks)); + return fromJavaDateType(toJavaDateType().plusWeeks(weeks)); } - public Expression plusDays(long days) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER, getStringValue()).plusDays(days)); + public Expression plusYears(long years) { + return fromJavaDateType(toJavaDateType().plusYears(years)); } public Expression plusHours(long hours) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER, getStringValue()).plusHours(hours)); + return fromJavaDateType(toJavaDateType().plusHours(hours)); } public Expression plusMinutes(long minutes) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER, getStringValue()).plusMinutes(minutes)); + return fromJavaDateType(toJavaDateType().plusMinutes(minutes)); } public Expression plusSeconds(long seconds) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER, getStringValue()).plusSeconds(seconds)); + return fromJavaDateType(toJavaDateType().plusSeconds(seconds)); } public long getHour() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java index 53cfe1e1835bb8..2417d92e89addf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java @@ -110,10 +110,72 @@ public String toString() { @Override public String getStringValue() { + int scale = getDataType().getScale(); + if (scale <= 0) { + return super.getStringValue(); + } + + if (0 <= year && year <= 9999 && 0 <= month && month <= 99 && 0 <= day && day <= 99 + && 0 <= hour && hour <= 99 && 0 <= minute && minute <= 99 && 0 <= second && second <= 99 + && 0 <= microSecond && microSecond <= MAX_MICROSECOND) { + char[] format = new char[] { + '0', '0', '0', '0', '-', '0', '0', '-', '0', '0', ' ', '0', '0', ':', '0', '0', ':', '0', '0', + '.', '0', '0', '0', '0', '0', '0'}; + int offset = 3; + long year = this.year; + while (year > 0) { + format[offset--] = (char) ('0' + (year % 10)); + year /= 10; + } + + offset = 6; + long month = this.month; + while (month > 0) { + format[offset--] = (char) ('0' + (month % 10)); + month /= 10; + } + + offset = 9; + long day = this.day; + while (day > 0) { + format[offset--] = (char) ('0' + (day % 10)); + day /= 10; + } + + offset = 12; + long hour = this.hour; + while (hour > 0) { + format[offset--] = (char) ('0' + (hour % 10)); + hour /= 10; + } + + offset = 15; + long minute = this.minute; + while (minute > 0) { + format[offset--] = (char) ('0' + (minute % 10)); + minute /= 10; + } + + offset = 18; + long second = this.second; + while (second > 0) { + format[offset--] = (char) ('0' + (second % 10)); + second /= 10; + } + + offset = 19 + scale; + long microSecond = (int) (this.microSecond / Math.pow(10, DateTimeV2Type.MAX_SCALE - scale)); + while (microSecond > 0) { + format[offset--] = (char) ('0' + (microSecond % 10)); + microSecond /= 10; + } + return String.valueOf(format, 0, 20 + scale); + } + return String.format("%04d-%02d-%02d %02d:%02d:%02d" - + (getDataType().getScale() > 0 ? ".%0" + getDataType().getScale() + "d" : ""), + + (scale > 0 ? ".%0" + scale + "d" : ""), year, month, day, hour, minute, second, - (int) (microSecond / Math.pow(10, DateTimeV2Type.MAX_SCALE - getDataType().getScale()))); + (int) (microSecond / Math.pow(10, DateTimeV2Type.MAX_SCALE - scale))); } public String getMicrosecondString() { @@ -124,59 +186,36 @@ public String getMicrosecondString() { (int) (microSecond / Math.pow(10, DateTimeV2Type.MAX_SCALE - getDataType().getScale()))); } - @Override - public Expression plusYears(long years) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) - .plusYears(years), getDataType().getScale()); + public Expression plusDays(long days) { + return fromJavaDateType(toJavaDateType().plusDays(days), getDataType().getScale()); } - @Override public Expression plusMonths(long months) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) - .plusMonths(months), getDataType().getScale()); + return fromJavaDateType(toJavaDateType().plusMonths(months), getDataType().getScale()); } - @Override public Expression plusWeeks(long weeks) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) - .plusWeeks(weeks), getDataType().getScale()); + return fromJavaDateType(toJavaDateType().plusWeeks(weeks), getDataType().getScale()); } - @Override - public Expression plusDays(long days) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) - .plusDays(days), getDataType().getScale()); + public Expression plusYears(long years) { + return fromJavaDateType(toJavaDateType().plusYears(years), getDataType().getScale()); } - @Override public Expression plusHours(long hours) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) - .plusHours(hours), getDataType().getScale()); + return fromJavaDateType(toJavaDateType().plusHours(hours), getDataType().getScale()); } - @Override public Expression plusMinutes(long minutes) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) - .plusMinutes(minutes), getDataType().getScale()); + return fromJavaDateType(toJavaDateType().plusMinutes(minutes), getDataType().getScale()); } - @Override public Expression plusSeconds(long seconds) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) - .plusSeconds(seconds), getDataType().getScale()); + return fromJavaDateType(toJavaDateType().plusSeconds(seconds), getDataType().getScale()); } public Expression plusMicroSeconds(long microSeconds) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER_TO_MICRO_SECOND, getFullMicroSecondValue()) - .plusNanos(microSeconds * 1000L), getDataType().getScale()); + return fromJavaDateType(toJavaDateType().plusNanos(microSeconds * 1000L), getDataType().getScale()); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateV2Literal.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateV2Literal.java index f522265f1617fb..d51a0eccc82cb7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateV2Literal.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateV2Literal.java @@ -24,8 +24,6 @@ import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.DateTimeV2Type; import org.apache.doris.nereids.types.DateV2Type; -import org.apache.doris.nereids.util.DateUtils; -import org.apache.doris.nereids.util.StandardDateFormat; import java.time.LocalDateTime; @@ -53,22 +51,19 @@ public R accept(ExpressionVisitor visitor, C context) { } public Expression plusDays(long days) { - return fromJavaDateType(DateUtils.getTime(StandardDateFormat.DATE_FORMATTER, getStringValue()).plusDays(days)); + return fromJavaDateType(toJavaDateType().plusDays(days)); } public Expression plusMonths(long months) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_FORMATTER, getStringValue()).plusMonths(months)); + return fromJavaDateType(toJavaDateType().plusMonths(months)); } public Expression plusWeeks(long weeks) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_FORMATTER, getStringValue()).plusWeeks(weeks)); + return fromJavaDateType(toJavaDateType().plusWeeks(weeks)); } public Expression plusYears(long years) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_FORMATTER, getStringValue()).plusYears(years)); + return fromJavaDateType(toJavaDateType().plusYears(years)); } public static Expression fromJavaDateType(LocalDateTime dateTime) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java index 4747aa84898a8e..ecb04ac7e49748 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java @@ -26,6 +26,7 @@ import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; +import org.apache.doris.nereids.trees.plans.TreeStringPlan.TreeStringNode; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.util.MutableState; import org.apache.doris.nereids.util.TreeStringUtils; @@ -34,6 +35,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.json.JSONArray; import org.json.JSONObject; @@ -120,9 +122,31 @@ public boolean canBind() { public String treeString() { return TreeStringUtils.treeString(this, plan -> plan.toString(), - plan -> (List) ((Plan) plan).children(), - plan -> (List) ((Plan) plan).extraPlans(), - plan -> ((Plan) plan).displayExtraPlanFirst()); + plan -> { + if (plan instanceof TreeStringPlan) { + Optional treeStringNode = ((TreeStringPlan) plan).parseTreeStringNode(); + return treeStringNode.isPresent() ? ImmutableList.of(treeStringNode.get()) : ImmutableList.of(); + } + if (plan instanceof TreeStringNode) { + return (List) ((TreeStringNode) plan).children; + } + if (!(plan instanceof Plan)) { + return ImmutableList.of(); + } + return (List) ((Plan) plan).children(); + }, + plan -> { + if (!(plan instanceof Plan)) { + return ImmutableList.of(); + } + return (List) ((Plan) plan).extraPlans(); + }, + plan -> { + if (!(plan instanceof Plan)) { + return false; + } + return ((Plan) plan).displayExtraPlanFirst(); + }); } /** top toJson method, can be override by specific operator */ 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 daab18438b4774..165dcd5e3cab7f 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 @@ -27,6 +27,7 @@ public enum PlanType { // logical plans // logical relations + LOGICAL_SQL_CACHE, LOGICAL_BOUND_RELATION, LOGICAL_CTE_CONSUMER, LOGICAL_FILE_SCAN, @@ -83,6 +84,7 @@ public enum PlanType { // physical plans // physical relations + PHYSICAL_SQL_CACHE, PHYSICAL_CTE_CONSUMER, PHYSICAL_EMPTY_RELATION, PHYSICAL_ES_SCAN, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/TreeStringPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/TreeStringPlan.java new file mode 100644 index 00000000000000..c392b9ed9dd4b9 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/TreeStringPlan.java @@ -0,0 +1,112 @@ +// 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; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import org.apache.commons.io.LineIterator; + +import java.io.StringReader; +import java.util.List; +import java.util.Optional; +import java.util.Stack; + +/** TreeStringPlan */ +public interface TreeStringPlan { + String getChildrenTreeString(); + + /** parseTreeStringNode */ + default Optional parseTreeStringNode() { + String treeString = getChildrenTreeString(); + LineIterator lineIt = new LineIterator(new StringReader(treeString)); + + if (!lineIt.hasNext()) { + return Optional.empty(); + } + + Stack parseStack = new Stack<>(); + parseStack.push(new ParseTreeStringNodeContext(lineIt.next())); + + while (lineIt.hasNext()) { + String line = lineIt.next(); + int level = getLevel(line); + while (parseStack.size() >= level) { + ParseTreeStringNodeContext child = parseStack.pop(); + parseStack.peek().children.add( + new TreeStringNode(child.currentString, ImmutableList.copyOf(child.children)) + ); + } + parseStack.push(new ParseTreeStringNodeContext(line.substring((level - 1) * 3))); + } + + while (parseStack.size() > 1) { + ParseTreeStringNodeContext child = parseStack.pop(); + parseStack.peek().children.add( + new TreeStringNode(child.currentString, ImmutableList.copyOf(child.children)) + ); + } + + ParseTreeStringNodeContext top = parseStack.pop(); + return Optional.of(new TreeStringNode(top.currentString, ImmutableList.copyOf(top.children))); + } + + /** TreeStringNode */ + class TreeStringNode { + /** currentString */ + public final String currentString; + /** children */ + public final List children; + + public TreeStringNode(String currentString, List children) { + this.currentString = currentString; + this.children = children; + } + + @Override + public String toString() { + return currentString.trim(); + } + } + + /** ParseTreeStringNodeContext */ + class ParseTreeStringNodeContext { + /** currentString */ + public final String currentString; + /** children */ + public final List children; + + public ParseTreeStringNodeContext(String currentString) { + this.currentString = currentString; + this.children = Lists.newArrayList(); + } + } + + /** getLevel */ + static int getLevel(String currentString) { + int prefix = 0; + for (int i = 0; i < currentString.length(); i++) { + char c = currentString.charAt(i); + if (c == ' ' || c == '|' || c == '-' || c == '+') { + prefix++; + } else { + break; + } + } + return prefix / 3 + 1; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/SqlCache.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/SqlCache.java new file mode 100644 index 00000000000000..bd71fa1fd75ebe --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/SqlCache.java @@ -0,0 +1,36 @@ +// 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.algebra; + +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalSqlCache; +import org.apache.doris.qe.cache.CacheAnalyzer; + +/** SqlCache */ +public interface SqlCache { + /** computeChild */ + static

P computeChild(CacheAnalyzer cacheAnalyzer) { + NereidsPlanner nereidsPlanner = (NereidsPlanner) cacheAnalyzer.getPlanner(); + PhysicalPlan physicalPlan = nereidsPlanner.getPhysicalPlan(); + while (physicalPlan instanceof PhysicalSqlCache) { + physicalPlan = (PhysicalPlan) physicalPlan.child(0); + } + return (P) physicalPlan; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCheckPolicy.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCheckPolicy.java index bda3b1f49d570d..69b669b25f7677 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCheckPolicy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCheckPolicy.java @@ -22,6 +22,8 @@ import org.apache.doris.mysql.privilege.DataMaskPolicy; import org.apache.doris.mysql.privilege.RowFilterPolicy; import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.SqlCacheContext; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.UnboundAlias; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.memo.GroupExpression; @@ -147,6 +149,8 @@ public RelatedPolicy findPolicy(LogicalRelation logicalRelation, CascadesContext ImmutableList.Builder dataMasks = ImmutableList.builderWithExpectedSize(logicalRelation.getOutput().size()); + StatementContext statementContext = cascadesContext.getStatementContext(); + Optional sqlCacheContext = statementContext.getSqlCacheContext(); boolean hasDataMask = false; for (Slot slot : logicalRelation.getOutput()) { Optional dataMaskPolicy = accessManager.evalDataMaskPolicy( @@ -165,12 +169,19 @@ public RelatedPolicy findPolicy(LogicalRelation logicalRelation, CascadesContext } else { dataMasks.add(slot); } + if (sqlCacheContext.isPresent()) { + sqlCacheContext.get().addDataMaskPolicy(ctlName, dbName, tableName, slot.getName(), dataMaskPolicy); + } } - List policies = accessManager.evalRowFilterPolicies( + List rowPolicies = accessManager.evalRowFilterPolicies( currentUserIdentity, ctlName, dbName, tableName); + if (sqlCacheContext.isPresent()) { + sqlCacheContext.get().setRowFilterPolicy(ctlName, dbName, tableName, rowPolicies); + } + return new RelatedPolicy( - Optional.ofNullable(CollectionUtils.isEmpty(policies) ? null : mergeRowPolicy(policies)), + Optional.ofNullable(CollectionUtils.isEmpty(rowPolicies) ? null : mergeRowPolicy(rowPolicies)), hasDataMask ? Optional.of(dataMasks.build()) : Optional.empty() ); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSqlCache.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSqlCache.java new file mode 100644 index 00000000000000..c0c728d8fc9a86 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSqlCache.java @@ -0,0 +1,147 @@ +// 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.logical; + +import org.apache.doris.analysis.Expr; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.FdItem; +import org.apache.doris.nereids.properties.FunctionalDependencies.Builder; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.TreeStringPlan; +import org.apache.doris.nereids.trees.plans.algebra.SqlCache; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.proto.InternalService; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** LogicalSqlCache */ +public class LogicalSqlCache extends LogicalLeaf implements SqlCache, TreeStringPlan { + private final TUniqueId queryId; + private final List columnLabels; + private final List resultExprs; + private final List cacheValues; + private final String backendAddress; + private final String planBody; + + /** LogicalSqlCache */ + public LogicalSqlCache(TUniqueId queryId, + List columnLabels, List resultExprs, + List cacheValues, String backendAddress, String planBody) { + super(PlanType.LOGICAL_SQL_CACHE, Optional.empty(), Optional.empty()); + this.queryId = Objects.requireNonNull(queryId, "queryId can not be null"); + this.columnLabels = Objects.requireNonNull(columnLabels, "columnLabels can not be null"); + this.resultExprs = Objects.requireNonNull(resultExprs, "resultExprs can not be null"); + this.cacheValues = Objects.requireNonNull(cacheValues, "cacheValues can not be null"); + this.backendAddress = Objects.requireNonNull(backendAddress, "backendAddress can not be null"); + this.planBody = Objects.requireNonNull(planBody, "planBody can not be null"); + } + + public TUniqueId getQueryId() { + return queryId; + } + + public List getCacheValues() { + return cacheValues; + } + + public String getBackendAddress() { + return backendAddress; + } + + public List getColumnLabels() { + return columnLabels; + } + + public List getResultExprs() { + return resultExprs; + } + + public String getPlanBody() { + return planBody; + } + + @Override + public String toString() { + return Utils.toSqlString("LogicalSqlCache[" + id.asInt() + "]", + "queryId", DebugUtil.printId(queryId) + ); + } + + @Override + public Plan withChildren(List children) { + throw new UnsupportedOperationException(); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitLogicalSqlCache(this, context); + } + + @Override + public List getExpressions() { + return ImmutableList.of(); + } + + @Override + public Plan withGroupExpression(Optional groupExpression) { + throw new UnsupportedOperationException(); + } + + @Override + public Plan withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + throw new UnsupportedOperationException(); + } + + @Override + public List computeOutput() { + throw new UnsupportedOperationException(); + } + + @Override + public String getChildrenTreeString() { + return planBody; + } + + @Override + public ImmutableSet computeFdItems() { + return ImmutableSet.of(); + } + + @Override + public void computeUnique(Builder fdBuilder) { + + } + + @Override + public void computeUniform(Builder fdBuilder) { + + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSqlCache.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSqlCache.java new file mode 100644 index 00000000000000..6e223a49cde915 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSqlCache.java @@ -0,0 +1,139 @@ +// 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.physical; + +import org.apache.doris.analysis.Expr; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.FunctionalDependencies; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.TreeStringPlan; +import org.apache.doris.nereids.trees.plans.algebra.SqlCache; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.proto.InternalService; +import org.apache.doris.statistics.Statistics; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** PhysicalSqlCache */ +public class PhysicalSqlCache extends PhysicalLeaf implements SqlCache, TreeStringPlan { + private final TUniqueId queryId; + private final List columnLabels; + private final List resultExprs; + private final List cacheValues; + private final String backendAddress; + private final String planBody; + + /** PhysicalSqlCache */ + public PhysicalSqlCache(TUniqueId queryId, + List columnLabels, List resultExprs, + List cacheValues, String backendAddress, String planBody) { + super(PlanType.PHYSICAL_SQL_CACHE, Optional.empty(), + new LogicalProperties(() -> ImmutableList.of(), () -> FunctionalDependencies.EMPTY_FUNC_DEPS)); + this.queryId = Objects.requireNonNull(queryId, "queryId can not be null"); + this.columnLabels = Objects.requireNonNull(columnLabels, "colNames can not be null"); + this.resultExprs = Objects.requireNonNull(resultExprs, "resultExprs can not be null"); + this.cacheValues = Objects.requireNonNull(cacheValues, "cacheValues can not be null"); + this.backendAddress = Objects.requireNonNull(backendAddress, "backendAddress can not be null"); + this.planBody = Objects.requireNonNull(planBody, "planBody can not be null"); + } + + public TUniqueId getQueryId() { + return queryId; + } + + public List getCacheValues() { + return cacheValues; + } + + public String getBackendAddress() { + return backendAddress; + } + + public List getColumnLabels() { + return columnLabels; + } + + public List getResultExprs() { + return resultExprs; + } + + public String getPlanBody() { + return planBody; + } + + @Override + public String toString() { + return Utils.toSqlString("PhysicalSqlCache[" + id.asInt() + "]", + "queryId", DebugUtil.printId(queryId), + "backend", backendAddress + ); + } + + @Override + public Plan withChildren(List children) { + throw new UnsupportedOperationException(); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitPhysicalSqlCache(this, context); + } + + @Override + public List getExpressions() { + return ImmutableList.of(); + } + + @Override + public Plan withGroupExpression(Optional groupExpression) { + throw new UnsupportedOperationException(); + } + + @Override + public Plan withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + throw new UnsupportedOperationException(); + } + + @Override + public List computeOutput() { + throw new UnsupportedOperationException(); + } + + @Override + public PhysicalPlan withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) { + throw new UnsupportedOperationException(); + } + + @Override + public String getChildrenTreeString() { + return planBody; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java index 82c99abb808642..9a21e5b14e0195 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java @@ -45,6 +45,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalSetOperation; import org.apache.doris.nereids.trees.plans.logical.LogicalSink; import org.apache.doris.nereids.trees.plans.logical.LogicalSort; +import org.apache.doris.nereids.trees.plans.logical.LogicalSqlCache; import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; import org.apache.doris.nereids.trees.plans.logical.LogicalTopN; import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; @@ -73,6 +74,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalRepeat; import org.apache.doris.nereids.trees.plans.physical.PhysicalSetOperation; import org.apache.doris.nereids.trees.plans.physical.PhysicalSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalSqlCache; import org.apache.doris.nereids.trees.plans.physical.PhysicalStorageLayerAggregate; import org.apache.doris.nereids.trees.plans.physical.PhysicalTopN; import org.apache.doris.nereids.trees.plans.physical.PhysicalUnion; @@ -128,6 +130,9 @@ public R visitPhysicalSink(PhysicalSink physicalSink, C context) // ******************************* // Logical plans // ******************************* + public R visitLogicalSqlCache(LogicalSqlCache sqlCache, C context) { + return visit(sqlCache, context); + } public R visitLogicalAggregate(LogicalAggregate aggregate, C context) { return visit(aggregate, context); @@ -248,6 +253,9 @@ public R visitLogicalWindow(LogicalWindow window, C context) { // ******************************* // Physical plans // ******************************* + public R visitPhysicalSqlCache(PhysicalSqlCache sqlCache, C context) { + return visit(sqlCache, context); + } public R visitPhysicalHashAggregate(PhysicalHashAggregate agg, C context) { return visit(agg, context); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java index 3c993179d76b8e..2f2c3441e22575 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java @@ -17,6 +17,7 @@ package org.apache.doris.qe; +import org.apache.doris.analysis.ExplainOptions; import org.apache.doris.analysis.InsertStmt; import org.apache.doris.analysis.KillStmt; import org.apache.doris.analysis.LiteralExpr; @@ -34,6 +35,7 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.NotImplementedException; +import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.common.util.SqlParserUtils; @@ -46,6 +48,7 @@ import org.apache.doris.mysql.MysqlPacket; import org.apache.doris.mysql.MysqlSerializer; import org.apache.doris.mysql.MysqlServerStatusFlag; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.exceptions.NotSupportedException; import org.apache.doris.nereids.exceptions.ParseException; import org.apache.doris.nereids.glue.LogicalPlanAdapter; @@ -53,10 +56,14 @@ import org.apache.doris.nereids.parser.Dialect; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.stats.StatsErrorEstimator; +import org.apache.doris.nereids.trees.plans.commands.ExplainCommand; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalSqlCache; import org.apache.doris.plugin.DialectConverterPlugin; import org.apache.doris.plugin.PluginMgr; import org.apache.doris.proto.Data; import org.apache.doris.qe.QueryState.MysqlStateType; +import org.apache.doris.qe.cache.CacheAnalyzer; import org.apache.doris.thrift.TExprNode; import org.apache.doris.thrift.TMasterOpRequest; import org.apache.doris.thrift.TMasterOpResult; @@ -64,6 +71,8 @@ import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.lang3.StringUtils; @@ -77,6 +86,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.UUID; import javax.annotation.Nullable; @@ -205,33 +215,46 @@ public void executeQuery(MysqlCommand mysqlCommand, String originStmt) throws Ex String sqlHash = DigestUtils.md5Hex(convertedStmt); ctx.setSqlHash(sqlHash); + SessionVariable sessionVariable = ctx.getSessionVariable(); + boolean wantToParseSqlFromSqlCache = sessionVariable.isEnableNereidsPlanner() + && CacheAnalyzer.canUseSqlCache(sessionVariable); List stmts = null; Exception nereidsParseException = null; long parseSqlStartTime = System.currentTimeMillis(); + List cachedStmts = null; // Nereids do not support prepare and execute now, so forbid prepare command, only process query command - if (mysqlCommand == MysqlCommand.COM_QUERY && ctx.getSessionVariable().isEnableNereidsPlanner()) { - try { - stmts = new NereidsParser().parseSQL(convertedStmt, ctx.getSessionVariable()); - } catch (NotSupportedException e) { - // Parse sql failed, audit it and return - handleQueryException(e, convertedStmt, null, null); - return; - } catch (ParseException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("Nereids parse sql failed. Reason: {}. Statement: \"{}\".", - e.getMessage(), convertedStmt); + if (mysqlCommand == MysqlCommand.COM_QUERY && sessionVariable.isEnableNereidsPlanner()) { + if (wantToParseSqlFromSqlCache) { + cachedStmts = parseFromSqlCache(originStmt); + if (cachedStmts != null) { + stmts = cachedStmts; } - // ATTN: Do not set nereidsParseException in this case. - // Because ParseException means the sql is not supported by Nereids. - // It should be parsed by old parser, so not setting nereidsParseException to avoid - // suppressing the exception thrown by old parser. - } catch (Exception e) { - // TODO: We should catch all exception here until we support all query syntax. - if (LOG.isDebugEnabled()) { - LOG.debug("Nereids parse sql failed with other exception. Reason: {}. Statement: \"{}\".", - e.getMessage(), convertedStmt); + } + + if (cachedStmts == null) { + try { + stmts = new NereidsParser().parseSQL(convertedStmt, sessionVariable); + } catch (NotSupportedException e) { + // Parse sql failed, audit it and return + handleQueryException(e, convertedStmt, null, null); + return; + } catch (ParseException e) { + if (LOG.isDebugEnabled()) { + LOG.debug("Nereids parse sql failed. Reason: {}. Statement: \"{}\".", + e.getMessage(), convertedStmt); + } + // ATTN: Do not set nereidsParseException in this case. + // Because ParseException means the sql is not supported by Nereids. + // It should be parsed by old parser, so not setting nereidsParseException to avoid + // suppressing the exception thrown by old parser. + } catch (Exception e) { + // TODO: We should catch all exception here until we support all query syntax. + if (LOG.isDebugEnabled()) { + LOG.debug("Nereids parse sql failed with other exception. Reason: {}. Statement: \"{}\".", + e.getMessage(), convertedStmt); + } + nereidsParseException = e; } - nereidsParseException = e; } } @@ -266,58 +289,104 @@ public void executeQuery(MysqlCommand mysqlCommand, String originStmt) throws Ex boolean usingOrigSingleStmt = origSingleStmtList != null && origSingleStmtList.size() == stmts.size(); for (int i = 0; i < stmts.size(); ++i) { String auditStmt = usingOrigSingleStmt ? origSingleStmtList.get(i) : convertedStmt; + try { + ctx.getState().reset(); + if (i > 0) { + ctx.resetReturnRows(); + } - ctx.getState().reset(); - if (i > 0) { - ctx.resetReturnRows(); - } + StatementBase parsedStmt = stmts.get(i); + parsedStmt.setOrigStmt(new OriginStatement(convertedStmt, i)); + parsedStmt.setUserInfo(ctx.getCurrentUserIdentity()); + executor = new StmtExecutor(ctx, parsedStmt); + executor.getProfile().getSummaryProfile().setParseSqlStartTime(parseSqlStartTime); + executor.getProfile().getSummaryProfile().setParseSqlFinishTime(parseSqlFinishTime); + ctx.setExecutor(executor); - StatementBase parsedStmt = stmts.get(i); - parsedStmt.setOrigStmt(new OriginStatement(convertedStmt, i)); - parsedStmt.setUserInfo(ctx.getCurrentUserIdentity()); - executor = new StmtExecutor(ctx, parsedStmt); - executor.getProfile().getSummaryProfile().setParseSqlStartTime(parseSqlStartTime); - executor.getProfile().getSummaryProfile().setParseSqlFinishTime(parseSqlFinishTime); - ctx.setExecutor(executor); - - try { - executor.execute(); - if (connectType.equals(ConnectType.MYSQL)) { - if (i != stmts.size() - 1) { - ctx.getState().serverStatus |= MysqlServerStatusFlag.SERVER_MORE_RESULTS_EXISTS; - if (ctx.getState().getStateType() != MysqlStateType.ERR) { - finalizeCommand(); + try { + executor.execute(); + if (connectType.equals(ConnectType.MYSQL)) { + if (i != stmts.size() - 1) { + ctx.getState().serverStatus |= MysqlServerStatusFlag.SERVER_MORE_RESULTS_EXISTS; + if (ctx.getState().getStateType() != MysqlStateType.ERR) { + finalizeCommand(); + } + } + } else if (connectType.equals(ConnectType.ARROW_FLIGHT_SQL)) { + if (!ctx.isReturnResultFromLocal()) { + returnResultFromRemoteExecutor.add(executor); + } + Preconditions.checkState(ctx.getFlightSqlChannel().resultNum() <= 1); + if (ctx.getFlightSqlChannel().resultNum() == 1 && i != stmts.size() - 1) { + String errMsg = "Only be one stmt that returns the result and it is at the end. " + + "stmts.size(): " + stmts.size(); + LOG.warn(errMsg); + ctx.getState().setError(ErrorCode.ERR_ARROW_FLIGHT_SQL_MUST_ONLY_RESULT_STMT, errMsg); + ctx.getState().setErrType(QueryState.ErrType.OTHER_ERR); + break; } } - } else if (connectType.equals(ConnectType.ARROW_FLIGHT_SQL)) { - if (!ctx.isReturnResultFromLocal()) { - returnResultFromRemoteExecutor.add(executor); - } - Preconditions.checkState(ctx.getFlightSqlChannel().resultNum() <= 1); - if (ctx.getFlightSqlChannel().resultNum() == 1 && i != stmts.size() - 1) { - String errMsg = "Only be one stmt that returns the result and it is at the end. stmts.size(): " - + stmts.size(); - LOG.warn(errMsg); - ctx.getState().setError(ErrorCode.ERR_ARROW_FLIGHT_SQL_MUST_ONLY_RESULT_STMT, errMsg); - ctx.getState().setErrType(QueryState.ErrType.OTHER_ERR); + auditAfterExec(auditStmt, executor.getParsedStmt(), executor.getQueryStatisticsForAuditLog(), + true); + // execute failed, skip remaining stmts + if (ctx.getState().getStateType() == MysqlStateType.ERR) { break; } + } catch (Throwable throwable) { + handleQueryException(throwable, auditStmt, executor.getParsedStmt(), + executor.getQueryStatisticsForAuditLog()); + // execute failed, skip remaining stmts + throw throwable; } - auditAfterExec(auditStmt, executor.getParsedStmt(), executor.getQueryStatisticsForAuditLog(), - true); - // execute failed, skip remaining stmts - if (ctx.getState().getStateType() == MysqlStateType.ERR) { - break; + } finally { + StatementContext statementContext = ctx.getStatementContext(); + if (statementContext != null) { + statementContext.close(); } - } catch (Throwable throwable) { - handleQueryException(throwable, auditStmt, executor.getParsedStmt(), - executor.getQueryStatisticsForAuditLog()); - // execute failed, skip remaining stmts - throw throwable; } } } + private List parseFromSqlCache(String originStmt) { + StatementContext statementContext = new StatementContext(ctx, new OriginStatement(originStmt, 0)); + ctx.setStatementContext(statementContext); + try { + Optional> explainPlan = NereidsParser.tryParseExplainPlan(originStmt); + String cacheSqlKey = originStmt; + if (explainPlan.isPresent()) { + cacheSqlKey = explainPlan.get().second; + } + Env env = ctx.getEnv(); + Optional sqlCachePlanOpt = env.getSqlCacheManager().tryParseSql(ctx, cacheSqlKey); + if (sqlCachePlanOpt.isPresent()) { + LogicalSqlCache logicalSqlCache = sqlCachePlanOpt.get(); + LogicalPlan parsedPlan = logicalSqlCache; + if (explainPlan.isPresent()) { + ExplainOptions explainOptions = explainPlan.get().first; + parsedPlan = new ExplainCommand( + explainOptions.getExplainLevel(), + parsedPlan, + explainOptions.showPlanProcess() + ); + } + + LogicalPlanAdapter logicalPlanAdapter = new LogicalPlanAdapter(parsedPlan, statementContext); + logicalPlanAdapter.setColLabels( + Lists.newArrayList(logicalSqlCache.getColumnLabels()) + ); + logicalPlanAdapter.setResultExprs(logicalSqlCache.getResultExprs()); + logicalPlanAdapter.setOrigStmt(statementContext.getOriginStatement()); + logicalPlanAdapter.setUserInfo(ctx.getCurrentUserIdentity()); + return ImmutableList.of(logicalPlanAdapter); + } + } catch (Throwable t) { + LOG.warn("Parse from sql cache failed: " + t.getMessage(), t); + } finally { + statementContext.releasePlannerResources(); + } + return null; + } + private String convertOriginStmt(String originStmt) { String convertedStmt = originStmt; @Nullable Dialect sqlDialect = Dialect.getByName(ctx.getSessionVariable().getSqlDialect()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java index e7203054c53500..a44b95f7fbb09f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -99,6 +99,7 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.NereidsException; +import org.apache.doris.common.NereidsSqlCacheManager; import org.apache.doris.common.UserException; import org.apache.doris.common.Version; import org.apache.doris.common.profile.Profile; @@ -142,6 +143,7 @@ import org.apache.doris.nereids.trees.plans.commands.insert.InsertOverwriteTableCommand; import org.apache.doris.nereids.trees.plans.commands.insert.OlapInsertExecutor; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalSqlCache; import org.apache.doris.planner.DataSink; import org.apache.doris.planner.GroupCommitPlanner; import org.apache.doris.planner.GroupCommitScanNode; @@ -167,6 +169,7 @@ import org.apache.doris.qe.cache.Cache; import org.apache.doris.qe.cache.CacheAnalyzer; import org.apache.doris.qe.cache.CacheAnalyzer.CacheMode; +import org.apache.doris.qe.cache.SqlCache; import org.apache.doris.rewrite.ExprRewriter; import org.apache.doris.rewrite.mvrewrite.MVSelectFailedException; import org.apache.doris.rpc.BackendServiceProxy; @@ -1514,15 +1517,30 @@ private boolean sendCachedValues(MysqlChannel channel, List allViewStmtSet; + private final Set allViewStmtSet; + private String allViewExpandStmtListStr; + private Planner planner; + private List scanTables = Lists.newArrayList(); public Cache getCache() { return cache; @@ -106,9 +117,10 @@ public CacheAnalyzer(ConnectContext context, StatementBase parsedStmt, Planner p this.context = context; this.queryId = context.queryId(); this.parsedStmt = parsedStmt; - scanNodes = planner.getScanNodes(); - latestTable = new CacheTable(); - allViewStmtSet = new HashSet<>(); + this.scanNodes = planner.getScanNodes(); + this.latestTable = new CacheTable(); + this.allViewStmtSet = new HashSet<>(); + this.planner = planner; checkCacheConfig(); } @@ -117,7 +129,7 @@ public CacheAnalyzer(ConnectContext context, StatementBase parsedStmt, List(); + this.allViewStmtSet = new HashSet<>(); checkCacheConfig(); } @@ -134,36 +146,46 @@ private void checkCacheConfig() { } } + public TUniqueId getQueryId() { + return queryId; + } + public CacheMode getCacheMode() { return cacheMode; } + public Planner getPlanner() { + return planner; + } + public class CacheTable implements Comparable { public TableIf table; public long latestPartitionId; - public long latestVersion; - public long latestTime; + public long latestPartitionVersion; + public long latestPartitionTime; public long partitionNum; public long sumOfPartitionNum; public CacheTable() { table = null; latestPartitionId = 0; - latestVersion = 0; - latestTime = 0; + latestPartitionVersion = 0; + latestPartitionTime = 0; partitionNum = 0; sumOfPartitionNum = 0; } @Override public int compareTo(CacheTable table) { - return Long.compare(table.latestTime, this.latestTime); + return Long.compare(table.latestPartitionTime, this.latestPartitionTime); } public void debug() { if (LOG.isDebugEnabled()) { - LOG.debug("table {}, partition id {}, ver {}, time {}, partition num {}, sumOfPartitionNum: {}", - table.getName(), latestPartitionId, latestVersion, latestTime, partitionNum, sumOfPartitionNum); + LOG.debug("table {}, partition id {}, ver {}, time {}," + + "partition num {}, sumOfPartitionNum: {}", + table.getName(), latestPartitionId, latestPartitionVersion, latestPartitionTime, + partitionNum, sumOfPartitionNum); } } } @@ -180,6 +202,19 @@ public boolean enablePartitionCache() { return enablePartitionCache; } + public static boolean canUseCache(SessionVariable sessionVariable) { + return (sessionVariable.isEnableSqlCache() || sessionVariable.isEnablePartitionCache()) + && commonCacheCondition(sessionVariable); + } + + public static boolean canUseSqlCache(SessionVariable sessionVariable) { + return sessionVariable.isEnableSqlCache() && commonCacheCondition(sessionVariable); + } + + public static boolean commonCacheCondition(SessionVariable sessionVariable) { + return sessionVariable.getSqlSelectLimit() < 0 && sessionVariable.getDefaultOrderByLimit() < 0; + } + /** * Check cache mode with SQL and table * 1、Only Olap table @@ -226,15 +261,17 @@ private CacheMode innerCheckCacheMode(long now) { latestTable.debug(); addAllViewStmt(selectStmt); - String allViewExpandStmtListStr = StringUtils.join(allViewStmtSet, "|"); + if (allViewExpandStmtListStr == null) { + allViewExpandStmtListStr = StringUtils.join(allViewStmtSet, "|"); + } if (now == 0) { now = nowtime(); } if (enableSqlCache() - && (now - latestTable.latestTime) >= Config.cache_last_version_interval_second * 1000L) { + && (now - latestTable.latestPartitionTime) >= Config.cache_last_version_interval_second * 1000L) { if (LOG.isDebugEnabled()) { - LOG.debug("Query cache time:{},{},{}", now, latestTable.latestTime, + LOG.debug("Query cache time:{},{},{}", now, latestTable.latestPartitionTime, Config.cache_last_version_interval_second * 1000); } cache = new SqlCache(this.queryId, this.selectStmt); @@ -260,7 +297,7 @@ private CacheMode innerCheckCacheMode(long now) { //Check if selectStmt matches partition key //Only one table can be updated in Config.cache_last_version_interval_second range for (int i = 1; i < tblTimeList.size(); i++) { - if ((now - tblTimeList.get(i).latestTime) < Config.cache_last_version_interval_second * 1000L) { + if ((now - tblTimeList.get(i).latestPartitionTime) < Config.cache_last_version_interval_second * 1000L) { if (LOG.isDebugEnabled()) { LOG.debug("the time of other tables is newer than {} s, queryid {}", Config.cache_last_version_interval_second, DebugUtil.printId(queryId)); @@ -343,9 +380,9 @@ private CacheMode innerCheckCacheModeSetOperation(long now) { now = nowtime(); } if (enableSqlCache() - && (now - latestTable.latestTime) >= Config.cache_last_version_interval_second * 1000L) { + && (now - latestTable.latestPartitionTime) >= Config.cache_last_version_interval_second * 1000L) { if (LOG.isDebugEnabled()) { - LOG.debug("Query cache time:{},{},{}", now, latestTable.latestTime, + LOG.debug("Query cache time:{},{},{}", now, latestTable.latestPartitionTime, Config.cache_last_version_interval_second * 1000); } cache = new SqlCache(this.queryId, parsedStmt.toSql()); @@ -377,29 +414,43 @@ private CacheMode innerCheckCacheModeForNereids(long now) { return CacheMode.None; } latestTable = tblTimeList.get(0); - latestTable.sumOfPartitionNum = tblTimeList.stream().mapToLong(item -> item.partitionNum).sum(); + long sumOfPartitionNum = 0; + for (CacheTable cacheTable : tblTimeList) { + sumOfPartitionNum += cacheTable.partitionNum; + } + latestTable.sumOfPartitionNum = sumOfPartitionNum; latestTable.debug(); if (((LogicalPlanAdapter) parsedStmt).getStatementContext().getParsedStatement().isExplain()) { return CacheMode.NoNeed; } - allViewStmtSet.addAll(((LogicalPlanAdapter) parsedStmt).getViewDdlSqls()); - String allViewExpandStmtListStr = StringUtils.join(allViewStmtSet, "|"); + boolean isNewAllViewExpandStmtListStr = allViewExpandStmtListStr == null; + if (isNewAllViewExpandStmtListStr) { + allViewStmtSet.addAll(((LogicalPlanAdapter) parsedStmt).getViewDdlSqls()); + allViewExpandStmtListStr = StringUtils.join(allViewStmtSet, "|"); + } if (now == 0) { now = nowtime(); } if (enableSqlCache() - && (now - latestTable.latestTime) >= Config.cache_last_version_interval_second * 1000L) { + && (now - latestTable.latestPartitionTime) >= Config.cache_last_version_interval_second * 1000L) { if (LOG.isDebugEnabled()) { - LOG.debug("Query cache time :{},{},{}", now, latestTable.latestTime, + LOG.debug("Query cache time :{},{},{}", now, latestTable.latestPartitionTime, Config.cache_last_version_interval_second * 1000); } + + PUniqueId existsMd5 = null; + if (cache instanceof SqlCache) { + existsMd5 = ((SqlCache) cache).getOrComputeCacheMd5(); + } cache = new SqlCache(this.queryId, ((LogicalPlanAdapter) parsedStmt).getStatementContext() .getOriginStatement().originStmt); - ((SqlCache) cache).setCacheInfo(this.latestTable, allViewExpandStmtListStr); + SqlCache sqlCache = (SqlCache) cache; + sqlCache.setCacheInfo(this.latestTable, allViewExpandStmtListStr); + sqlCache.setCacheMd5(existsMd5); MetricRepo.COUNTER_CACHE_ADDED_SQL.increase(1L); return CacheMode.Sql; } @@ -409,8 +460,15 @@ private CacheMode innerCheckCacheModeForNereids(long now) { private List buildCacheTableList() { //Check the last version time of the table MetricRepo.COUNTER_QUERY_TABLE.increase(1L); - long olapScanNodeSize = scanNodes.stream().filter(node -> node instanceof OlapScanNode).count(); - long hiveScanNodeSize = scanNodes.stream().filter(node -> node instanceof HiveScanNode).count(); + long olapScanNodeSize = 0; + long hiveScanNodeSize = 0; + for (ScanNode scanNode : scanNodes) { + if (scanNode instanceof OlapScanNode) { + olapScanNodeSize++; + } else if (scanNode instanceof HiveScanNode) { + hiveScanNodeSize++; + } + } if (olapScanNodeSize > 0) { MetricRepo.COUNTER_QUERY_OLAP_TABLE.increase(1L); } @@ -621,12 +679,20 @@ private CacheTable buildCacheTableForOlapScanNode(OlapScanNode node) { OlapTable olapTable = node.getOlapTable(); cacheTable.partitionNum = node.getSelectedPartitionIds().size(); cacheTable.table = olapTable; + + DatabaseIf database = olapTable.getDatabase(); + CatalogIf catalog = database.getCatalog(); + ScanTable scanTable = new ScanTable( + new FullTableName(catalog.getName(), database.getFullName(), olapTable.getName()), + olapTable.getVisibleVersionTime(), olapTable.getVisibleVersion()); + scanTables.add(scanTable); for (Long partitionId : node.getSelectedPartitionIds()) { Partition partition = olapTable.getPartition(partitionId); - if (partition.getVisibleVersionTime() >= cacheTable.latestTime) { + if (partition.getVisibleVersionTime() >= cacheTable.latestPartitionTime) { cacheTable.latestPartitionId = partition.getId(); - cacheTable.latestTime = partition.getVisibleVersionTime(); - cacheTable.latestVersion = partition.getVisibleVersion(); + cacheTable.latestPartitionTime = partition.getVisibleVersionTime(); + cacheTable.latestPartitionVersion = partition.getVisibleVersion(); + scanTable.addScanPartition(partitionId); } } return cacheTable; @@ -636,7 +702,14 @@ private CacheTable buildCacheTableForHiveScanNode(HiveScanNode node) { CacheTable cacheTable = new CacheTable(); cacheTable.table = node.getTargetTable(); cacheTable.partitionNum = node.getReadPartitionNum(); - cacheTable.latestTime = cacheTable.table.getUpdateTime(); + cacheTable.latestPartitionTime = cacheTable.table.getUpdateTime(); + TableIf tableIf = cacheTable.table; + DatabaseIf database = tableIf.getDatabase(); + CatalogIf catalog = database.getCatalog(); + ScanTable scanTable = new ScanTable(new FullTableName( + catalog.getName(), database.getFullName(), tableIf.getName() + ), cacheTable.latestPartitionTime, 0); + scanTables.add(scanTable); return cacheTable; } @@ -695,4 +768,27 @@ public void updateCache() { } cache.updateCache(); } + + public List getScanTables() { + return scanTables; + } + + public CacheTable getLatestTable() { + return latestTable; + } + + public boolean isEqualViewString(List views) { + Set viewSet = Sets.newHashSet(); + for (TableIf view : views) { + if (view instanceof View) { + viewSet.add(((View) view).getInlineViewDef()); + } else if (view instanceof HMSExternalTable) { + viewSet.add(((HMSExternalTable) view).getViewText()); + } else { + return false; + } + } + + return StringUtils.equals(allViewExpandStmtListStr, StringUtils.join(viewSet, "|")); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheCoordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheCoordinator.java index 232a9400e6c42f..529454b9fa3048 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheCoordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheCoordinator.java @@ -146,12 +146,11 @@ private void clearBackend(ImmutableMap idToBackend) { } public void addBackend(Backend backend) { - if (realNodes.contains(backend.getId())) { + if (realNodes.putIfAbsent(backend.getId(), backend) != null) { return; } - realNodes.put(backend.getId(), backend); for (int i = 0; i < VIRTUAL_NODES; i++) { - String nodeName = String.valueOf(backend.getId()) + "::" + String.valueOf(i); + String nodeName = backend.getId() + "::" + i; Types.PUniqueId nodeId = CacheBeProxy.getMd5(nodeName); virtualNodes.put(nodeId.getHi(), backend); if (LOG.isDebugEnabled()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/SqlCache.java b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/SqlCache.java index 52736ab2210d3d..a7351a306d2e2d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/SqlCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/SqlCache.java @@ -22,7 +22,9 @@ import org.apache.doris.common.util.DebugUtil; import org.apache.doris.metric.MetricRepo; import org.apache.doris.proto.InternalService; +import org.apache.doris.proto.Types.PUniqueId; import org.apache.doris.qe.RowBatch; +import org.apache.doris.system.Backend; import org.apache.doris.thrift.TUniqueId; import org.apache.logging.log4j.LogManager; @@ -32,6 +34,7 @@ public class SqlCache extends Cache { private static final Logger LOG = LogManager.getLogger(SqlCache.class); private String originSql; + private PUniqueId cacheMd5; public SqlCache(TUniqueId queryId, SelectStmt selectStmt) { super(queryId, selectStmt); @@ -46,6 +49,18 @@ public SqlCache(TUniqueId queryId, String originSql) { public void setCacheInfo(CacheAnalyzer.CacheTable latestTable, String allViewExpandStmtListStr) { this.latestTable = latestTable; this.allViewExpandStmtListStr = allViewExpandStmtListStr; + this.cacheMd5 = null; + } + + public PUniqueId getOrComputeCacheMd5() { + if (cacheMd5 == null) { + cacheMd5 = CacheProxy.getMd5(getSqlWithViewStmt()); + } + return cacheMd5; + } + + public void setCacheMd5(PUniqueId cacheMd5) { + this.cacheMd5 = cacheMd5; } public String getSqlWithViewStmt() { @@ -57,27 +72,50 @@ public String getSqlWithViewStmt() { return cacheKey; } + public long getLatestId() { + return latestTable.latestPartitionId; + } + public long getLatestTime() { - return latestTable.latestTime; + return latestTable.latestPartitionTime; + } + + public long getLatestVersion() { + return latestTable.latestPartitionVersion; } public long getSumOfPartitionNum() { return latestTable.sumOfPartitionNum; } - public InternalService.PFetchCacheResult getCacheData(Status status) { + public static Backend findCacheBe(PUniqueId cacheMd5) { + return CacheCoordinator.getInstance().findBackend(cacheMd5); + } + + public static InternalService.PFetchCacheResult getCacheData(CacheProxy proxy, + PUniqueId cacheKeyMd5, long latestPartitionId, long latestPartitionVersion, + long latestPartitionTime, long sumOfPartitionNum, Status status) { InternalService.PFetchCacheRequest request = InternalService.PFetchCacheRequest.newBuilder() - .setSqlKey(CacheProxy.getMd5(getSqlWithViewStmt())) + .setSqlKey(cacheKeyMd5) .addParams(InternalService.PCacheParam.newBuilder() - .setPartitionKey(latestTable.latestPartitionId) - .setLastVersion(latestTable.latestVersion) - .setLastVersionTime(latestTable.latestTime) - .setPartitionNum(latestTable.sumOfPartitionNum)) + .setPartitionKey(latestPartitionId) + .setLastVersion(latestPartitionVersion) + .setLastVersionTime(latestPartitionTime) + .setPartitionNum(sumOfPartitionNum)) .build(); InternalService.PFetchCacheResult cacheResult = proxy.fetchCache(request, 10000, status); if (status.ok() && cacheResult != null && cacheResult.getStatus() == InternalService.PCacheStatus.CACHE_OK) { cacheResult = cacheResult.toBuilder().setAllCount(1).build(); + } + return cacheResult; + } + + public InternalService.PFetchCacheResult getCacheData(Status status) { + InternalService.PFetchCacheResult cacheResult = getCacheData(proxy, getOrComputeCacheMd5(), + latestTable.latestPartitionId, latestTable.latestPartitionVersion, + latestTable.latestPartitionTime, latestTable.sumOfPartitionNum, status); + if (status.ok() && cacheResult != null && cacheResult.getStatus() == InternalService.PCacheStatus.CACHE_OK) { MetricRepo.COUNTER_CACHE_HIT_SQL.increase(1L); hitRange = HitRange.Full; } @@ -105,7 +143,9 @@ public void updateCache() { InternalService.PUpdateCacheRequest updateRequest = rowBatchBuilder.buildSqlUpdateRequest(getSqlWithViewStmt(), latestTable.latestPartitionId, - latestTable.latestVersion, latestTable.latestTime, latestTable.sumOfPartitionNum); + latestTable.latestPartitionVersion, latestTable.latestPartitionTime, + latestTable.sumOfPartitionNum + ); if (updateRequest.getValuesCount() > 0) { CacheBeProxy proxy = new CacheBeProxy(); Status status = new Status(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FunctionRegistryTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FunctionRegistryTest.java index 851189cf0aa95f..e4ec929f2847c1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FunctionRegistryTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FunctionRegistryTest.java @@ -121,7 +121,7 @@ protected void afterRegisterBuiltinFunctions(Map> ImmutableList arguments = ImmutableList.of(Literal.of(1)); FunctionBuilder functionBuilder = functionRegistry.findFunctionBuilder("foo", arguments); - Expression function = functionBuilder.build("foo", arguments); + Expression function = functionBuilder.build("foo", arguments).first; Assertions.assertEquals(function.getClass(), ExtendFunction.class); Assertions.assertEquals(arguments, function.getArguments()); } diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/ExplainAction.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/ExplainAction.groovy index cf0c03fc3bd73b..63708ce7ac6f1b 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/ExplainAction.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/ExplainAction.groovy @@ -102,7 +102,7 @@ class ExplainAction implements SuiteAction { for (String string : containsStrings) { if (!explainString.contains(string)) { String msg = ("Explain and check failed, expect contains '${string}'," - + "but actual explain string is:\n${explainString}").toString() + + " but actual explain string is:\n${explainString}").toString() log.info(msg) def t = new IllegalStateException(msg) throw t @@ -111,7 +111,7 @@ class ExplainAction implements SuiteAction { for (String string : notContainsStrings) { if (explainString.contains(string)) { String msg = ("Explain and check failed, expect not contains '${string}'," - + "but actual explain string is:\n${explainString}").toString() + + " but actual explain string is:\n${explainString}").toString() log.info(msg) def t = new IllegalStateException(msg) throw t diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy index 51e97bb9d091ee..2ffb7f8f6d4770 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy @@ -41,6 +41,7 @@ import org.apache.doris.regression.util.JdbcUtils import org.apache.doris.regression.util.Hdfs import org.apache.doris.regression.util.SuiteUtils import org.apache.doris.regression.util.DebugPoint +import org.jetbrains.annotations.NotNull import org.junit.jupiter.api.Assertions import org.slf4j.Logger import org.slf4j.LoggerFactory @@ -48,7 +49,10 @@ import groovy.util.logging.Slf4j import java.sql.Connection import java.util.concurrent.Callable +import java.util.concurrent.ExecutorService +import java.util.concurrent.Executors import java.util.concurrent.Future +import java.util.concurrent.ThreadFactory import java.util.concurrent.atomic.AtomicBoolean import java.util.stream.Collectors import java.util.stream.LongStream @@ -163,33 +167,63 @@ class Suite implements GroovyInterceptable { return SuiteUtils.timer(actionSupplier) } + public ListenableFuture extraThread( + String threadName = null, boolean daemon = false, Closure actionSupplier) { + def executorService = Executors.newFixedThreadPool(1, new ThreadFactory() { + @Override + Thread newThread(@NotNull Runnable r) { + def thread = new Thread(r, name) + thread.setDaemon(daemon) + return thread + } + }) + + try { + def connInfo = context.threadLocalConn.get() + return MoreExecutors.listeningDecorator(executorService).submit( + buildThreadCallable(threadName, connInfo, actionSupplier) + ) + } finally { + executorService.shutdown() + } + } + public ListenableFuture thread(String threadName = null, Closure actionSupplier) { def connInfo = context.threadLocalConn.get() - return MoreExecutors.listeningDecorator(context.actionExecutors).submit((Callable) { - long startTime = System.currentTimeMillis() - def originThreadName = Thread.currentThread().name - try { - Thread.currentThread().setName(threadName == null ? originThreadName : threadName) - if (connInfo != null) { - context.connectTo(connInfo.conn.getMetaData().getURL(), connInfo.username, connInfo.password); - } - context.scriptContext.eventListeners.each { it.onThreadStarted(context) } + return MoreExecutors.listeningDecorator(context.actionExecutors).submit( + buildThreadCallable(threadName, connInfo, actionSupplier) + ) + } - return actionSupplier.call() - } catch (Throwable t) { - context.scriptContext.eventListeners.each { it.onThreadFailed(context, t) } - throw t - } finally { + private Callable buildThreadCallable(String threadName, ConnectionInfo connInfo, Closure actionSupplier) { + return new Callable() { + @Override + T call() throws Exception { + long startTime = System.currentTimeMillis() + def originThreadName = Thread.currentThread().name try { - context.closeThreadLocal() + Thread.currentThread().setName(threadName == null ? originThreadName : threadName) + if (connInfo != null) { + context.connectTo(connInfo.conn.getMetaData().getURL(), connInfo.username, connInfo.password); + } + context.scriptContext.eventListeners.each { it.onThreadStarted(context) } + + return actionSupplier.call() } catch (Throwable t) { - logger.warn("Close thread local context failed", t) + context.scriptContext.eventListeners.each { it.onThreadFailed(context, t) } + throw t + } finally { + try { + context.closeThreadLocal() + } catch (Throwable t) { + logger.warn("Close thread local context failed", t) + } + long finishTime = System.currentTimeMillis() + context.scriptContext.eventListeners.each { it.onThreadFinished(context, finishTime - startTime) } + Thread.currentThread().setName(originThreadName) } - long finishTime = System.currentTimeMillis() - context.scriptContext.eventListeners.each { it.onThreadFinished(context, finishTime - startTime) } - Thread.currentThread().setName(originThreadName) } - }) + }; } public ListenableFuture lazyCheckThread(String threadName = null, Closure actionSupplier) { diff --git a/regression-test/plugins/test_helper.groovy b/regression-test/plugins/test_helper.groovy new file mode 100644 index 00000000000000..4f7eeb3c09b380 --- /dev/null +++ b/regression-test/plugins/test_helper.groovy @@ -0,0 +1,63 @@ +// 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. + +import org.apache.doris.regression.suite.Suite + +Suite.metaClass.createTestTable = { String tableName, boolean uniqueTable = false -> + Suite suite = delegate as Suite + def sql = { String sqlStr -> + suite.sql sqlStr + } + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + + sql "drop table if exists ${tableName}" + + sql """ + create table ${tableName} + ( + id int, + value int + ) + ${uniqueTable ? "unique key(id)" : ""} + partition by range(id) + ( + partition p1 values[('1'), ('2')), + partition p2 values[('2'), ('3')), + partition p3 values[('3'), ('4')), + partition p4 values[('4'), ('5')), + partition p5 values[('5'), ('6')) + ) + distributed by hash(id) + properties( + 'replication_num'='1' + ) + """ + + sql """ + insert into ${tableName} + values (1, 1), (1, 2), + (2, 1), (2, 2), + (3, 1), (3, 2), + (4, 1), (4, 2), + (5, 1), (5, 2) + """ +} + + +logger.info("Added 'createTestTable' function to Suite") diff --git a/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy new file mode 100644 index 00000000000000..7523df43f0cbb8 --- /dev/null +++ b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy @@ -0,0 +1,550 @@ +// 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("parse_sql_from_sql_cache") { + def assertHasCache = { String sqlStr -> + explain { + sql ("physical plan ${sqlStr}") + contains("PhysicalSqlCache") + } + } + + def assertNoCache = { String sqlStr -> + explain { + sql ("physical plan ${sqlStr}") + notContains("PhysicalSqlCache") + } + } + + + sql "ADMIN SET FRONTEND CONFIG ('cache_last_version_interval_second' = '10')" + + combineFutures( + extraThread("testUsePlanCache", { + createTestTable "test_use_plan_cache" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache" + + // create sql cache + sql "select * from test_use_plan_cache" + + // use sql cache + assertHasCache "select * from test_use_plan_cache" + }), + extraThread("testAddPartitionAndInsert", { + createTestTable "test_use_plan_cache2" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache2" + sql "select * from test_use_plan_cache2" + assertHasCache "select * from test_use_plan_cache2" + + // add empty partition can use cache + sql "alter table test_use_plan_cache2 add partition p6 values[('6'),('7'))" + assertHasCache "select * from test_use_plan_cache2" + + // insert data can not use cache + sql "insert into test_use_plan_cache2 values(6, 1)" + assertNoCache "select * from test_use_plan_cache2" + }), + extraThread("testDropPartition", { + createTestTable "test_use_plan_cache3" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache3" + sql "select * from test_use_plan_cache3" + assertHasCache "select * from test_use_plan_cache3" + + // drop partition can not use cache + sql "alter table test_use_plan_cache3 drop partition p5" + assertNoCache "select * from test_use_plan_cache3" + }), + extraThread("testReplacePartition", { + createTestTable "test_use_plan_cache4" + + sql "alter table test_use_plan_cache4 add temporary partition tp1 values [('1'), ('2'))" + + streamLoad { + table "test_use_plan_cache4" + set "temporaryPartitions", "tp1" + inputIterator([[1, 3], [1, 4]].iterator()) + } + sql "sync" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache4" + sql "select * from test_use_plan_cache4" + assertHasCache "select * from test_use_plan_cache4" + + // replace partition can not use cache + sql "alter table test_use_plan_cache4 replace partition (p1) with temporary partition(tp1)" + assertNoCache "select * from test_use_plan_cache4" + }), + extraThread("testStreamLoad", { + createTestTable "test_use_plan_cache5" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache5" + sql "select * from test_use_plan_cache5" + assertHasCache "select * from test_use_plan_cache5" + + streamLoad { + table "test_use_plan_cache5" + set "partitions", "p1" + inputIterator([[1, 3], [1, 4]].iterator()) + } + + // stream load can not use cache + sql "select * from test_use_plan_cache5" + assertNoCache "select * from test_use_plan_cache5" + }), + extraThread("testUpdate",{ + createTestTable "test_use_plan_cache6", uniqueTable=true + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache6" + sql "select * from test_use_plan_cache6" + assertHasCache "select * from test_use_plan_cache6" + + sql "update test_use_plan_cache6 set value=3 where id=1" + + // update can not use cache + sql "select * from test_use_plan_cache6" + assertNoCache "select * from test_use_plan_cache6" + }), + extraThread("testDelete", { + createTestTable "test_use_plan_cache7" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache7" + sql "select * from test_use_plan_cache7" + assertHasCache "select * from test_use_plan_cache7" + + sql "delete from test_use_plan_cache7 where id = 1" + + // delete can not use cache + sql "select * from test_use_plan_cache7" + assertNoCache "select * from test_use_plan_cache7" + }), + extraThread("testDropTable", { + createTestTable "test_use_plan_cache8" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache8" + sql "select * from test_use_plan_cache8" + assertHasCache "select * from test_use_plan_cache8" + + sql "drop table test_use_plan_cache8" + + // should visible the table has bean deleted + test { + sql "select * from test_use_plan_cache8" + exception "does not exist in database" + } + }), + extraThread("testCreateAndAlterView", { + createTestTable "test_use_plan_cache9" + + sql "drop view if exists test_use_plan_cache9_view" + sql "create view test_use_plan_cache9_view as select * from test_use_plan_cache9" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache9_view" + sql "select * from test_use_plan_cache9_view" + assertHasCache "select * from test_use_plan_cache9_view" + + // alter view should not use cache + sql "alter view test_use_plan_cache9_view as select id from test_use_plan_cache9" + assertNoCache "select * from test_use_plan_cache9_view" + }), + extraThread( "testDropView", { + createTestTable "test_use_plan_cache10" + + sql "drop view if exists test_use_plan_cache10_view" + sql "create view test_use_plan_cache10_view as select * from test_use_plan_cache10" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache10_view" + sql "select * from test_use_plan_cache10_view" + assertHasCache "select * from test_use_plan_cache10_view" + + sql "drop view test_use_plan_cache10_view" + // should visible the view has bean deleted + test { + sql "select * from test_use_plan_cache10_view" + exception "does not exist in database" + } + }), + extraThread("testBaseTableChanged", { + createTestTable "test_use_plan_cache11" + + sql "drop view if exists test_use_plan_cache11_view" + sql "create view test_use_plan_cache11_view as select * from test_use_plan_cache11" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache11_view" + sql "select * from test_use_plan_cache11_view" + assertHasCache "select * from test_use_plan_cache11_view" + + sql "insert into test_use_plan_cache11 values(1, 3)" + + // base table already changed, can not use cache + assertNoCache "select * from test_use_plan_cache11_view" + }), + extraThread("testNotShareCacheBetweenUsers", { + sql "drop user if exists test_cache_user1" + sql "create user test_cache_user1 identified by 'DORIS@2024'" + def dbName = context.config.getDbNameByFile(context.file) + sql """GRANT SELECT_PRIV ON *.* TO test_cache_user1""" + + createTestTable "test_use_plan_cache12" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache12" + sql "select * from test_use_plan_cache12" + assertHasCache "select * from test_use_plan_cache12" + + + extraThread("test_cache_user1_thread", { + connect(user = "test_cache_user1", password="DORIS@2024") { + sql "use ${dbName}" + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache12" + } + }).get() + }), + extraThread("testAddRowPolicy", { + def dbName = context.config.getDbNameByFile(context.file) + sql "set enable_nereids_planner=false" + try_sql """ + DROP ROW POLICY if exists test_cache_row_policy_2 + ON ${dbName}.test_use_plan_cache13 + FOR test_cache_user2""" + sql "set enable_nereids_planner=true" + + sql "drop user if exists test_cache_user2" + sql "create user test_cache_user2 identified by 'DORIS@2024'" + sql """GRANT SELECT_PRIV ON *.* TO test_cache_user2""" + + createTestTable "test_use_plan_cache13" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + extraThread("test_cache_user2_thread", { + connect(user = "test_cache_user2", password="DORIS@2024") { + sql "use ${dbName}" + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache13" + sql "select * from test_use_plan_cache13" + assertHasCache "select * from test_use_plan_cache13" + } + }).get() + + sql "set enable_nereids_planner=false" + sql """ + CREATE ROW POLICY test_cache_row_policy_2 + ON ${dbName}.test_use_plan_cache13 + AS RESTRICTIVE TO test_cache_user2 + USING (id = 'concat(id, "**")')""" + sql "set enable_nereids_planner=true" + + // after row policy changed, the cache is invalidate + extraThread("test_cache_user2_thread2", { + connect(user = "test_cache_user2", password="DORIS@2024") { + sql "use ${dbName}" + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache13" + } + }).get() + }), + extraThread("testDropRowPolicy", { + def dbName = context.config.getDbNameByFile(context.file) + sql "set enable_nereids_planner=false" + try_sql """ + DROP ROW POLICY if exists test_cache_row_policy_3 + ON ${dbName}.test_use_plan_cache14 + FOR test_cache_user3""" + sql "set enable_nereids_planner=true" + + sql "drop user if exists test_cache_user3" + sql "create user test_cache_user3 identified by 'DORIS@2024'" + sql """GRANT SELECT_PRIV ON *.* TO test_cache_user3""" + + createTestTable "test_use_plan_cache14" + + sql "set enable_nereids_planner=false" + sql """ + CREATE ROW POLICY test_cache_row_policy_3 + ON ${dbName}.test_use_plan_cache14 + AS RESTRICTIVE TO test_cache_user3 + USING (id = 'concat(id, "**")')""" + sql "set enable_nereids_planner=true" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + extraThread("test_cache_user3_thread", { + connect(user = "test_cache_user3", password="DORIS@2024") { + sql "use ${dbName}" + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache14" + sql "select * from test_use_plan_cache14" + assertHasCache "select * from test_use_plan_cache14" + } + }).get() + + sql "set enable_nereids_planner=false" + try_sql """ + DROP ROW POLICY if exists test_cache_row_policy_3 + ON ${dbName}.test_use_plan_cache14 + FOR test_cache_user3""" + sql "set enable_nereids_planner=true" + + // after row policy changed, the cache is invalidate + extraThread("test_cache_user3_thread2", { + connect(user = "test_cache_user3", password="DORIS@2024") { + sql "use ${dbName}" + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache14" + } + }).get() + }), + extraThread("testRemovePrivilege", { + def dbName = context.config.getDbNameByFile(context.file) + + createTestTable "test_use_plan_cache15" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "drop user if exists test_cache_user4" + sql "create user test_cache_user4 identified by 'DORIS@2024'" + sql "GRANT SELECT_PRIV ON regression_test.* TO test_cache_user4" + sql "GRANT SELECT_PRIV ON ${dbName}.test_use_plan_cache15 TO test_cache_user4" + + extraThread("test_cache_user4_thread", { + connect(user = "test_cache_user4", password="DORIS@2024") { + sql "use ${dbName}" + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache15" + sql "select * from test_use_plan_cache15" + assertHasCache "select * from test_use_plan_cache15" + } + }).get() + + sql "REVOKE SELECT_PRIV ON ${dbName}.test_use_plan_cache15 FROM test_cache_user4" + + // after privileges changed, the cache is invalidate + extraThread("test_cache_user4_thread2", { + connect(user = "test_cache_user4", password="DORIS@2024") { + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + test { + sql ("select * from ${dbName}.test_use_plan_cache15") + exception "Permission denied" + } + } + }).get() + }), + extraThread("testNondeterministic", { + createTestTable "test_use_plan_cache16" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select random() from test_use_plan_cache16" + // create sql cache + sql "select random() from test_use_plan_cache16" + // can not use sql cache + assertNoCache "select random() from test_use_plan_cache16" + + + assertNoCache "select year(now()) from test_use_plan_cache16" + sql "select year(now()) from test_use_plan_cache16" + assertHasCache "select year(now()) from test_use_plan_cache16" + + + assertNoCache "select second(now()) from test_use_plan_cache16" + sql "select second(now()) from test_use_plan_cache16" + sleep(1000) + assertNoCache "select second(now()) from test_use_plan_cache16" + }), + extraThread("testUserVariable", { + // make sure if the table has been dropped, the cache should invalidate, + // so we should retry twice to check + for (i in 0..2) { + createTestTable "test_use_plan_cache17" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + sql "set @custom_variable=10" + assertNoCache "select @custom_variable from test_use_plan_cache17" + // create sql cache + sql "select @custom_variable from test_use_plan_cache17" + // can use sql cache + assertHasCache "select @custom_variable from test_use_plan_cache17" + + sql "set @custom_variable=20" + assertNoCache "select @custom_variable from test_use_plan_cache17" + } + }), + extraThread("test_udf", { + def jarPath = """${context.config.suitePath}/javaudf_p0/jars/java-udf-case-jar-with-dependencies.jar""" + try_sql("DROP FUNCTION IF EXISTS java_udf_string_test(string, int, int);") + try_sql("DROP TABLE IF EXISTS test_javaudf_string") + + sql """ DROP TABLE IF EXISTS test_javaudf_string """ + sql """ + CREATE TABLE IF NOT EXISTS test_javaudf_string ( + `user_id` INT NOT NULL COMMENT "用户id", + `char_col` CHAR NOT NULL COMMENT "", + `varchar_col` VARCHAR(10) NOT NULL COMMENT "", + `string_col` STRING NOT NULL COMMENT "" + ) + DISTRIBUTED BY HASH(user_id) PROPERTIES("replication_num" = "1"); + """ + + StringBuilder values = new StringBuilder() + int i = 1 + for (; i < 9; i ++) { + values.append(" (${i}, '${i}','abcdefg${i}','poiuytre${i}abcdefg'),\n") + } + values.append("(${i}, '${i}','abcdefg${i}','poiuytre${i}abcdefg')") + + sql "INSERT INTO test_javaudf_string VALUES ${values}" + sql "sync" + + File path = new File(jarPath) + if (!path.exists()) { + throw new IllegalStateException("""${jarPath} doesn't exist! """) + } + + sql """ CREATE FUNCTION java_udf_string_test(string, int, int) RETURNS string PROPERTIES ( + "file"="file://${jarPath}", + "symbol"="org.apache.doris.udf.StringTest", + "type"="JAVA_UDF" + ); """ + + assertNoCache "SELECT java_udf_string_test(varchar_col, 2, 3) result FROM test_javaudf_string ORDER BY result;" + sql "SELECT java_udf_string_test(varchar_col, 2, 3) result FROM test_javaudf_string ORDER BY result;" + assertNoCache "SELECT java_udf_string_test(varchar_col, 2, 3) result FROM test_javaudf_string ORDER BY result;" + }) + ).get() +} diff --git a/regression-test/suites/query_p0/cache/sql_cache.groovy b/regression-test/suites/query_p0/cache/sql_cache.groovy index cccfca4d9650b1..d4b16cb5438fd2 100644 --- a/regression-test/suites/query_p0/cache/sql_cache.groovy +++ b/regression-test/suites/query_p0/cache/sql_cache.groovy @@ -211,5 +211,5 @@ suite("sql_cache") { k1; """ - sql "ADMIN SET FRONTEND CONFIG ('cache_last_version_interval_second' = '900')" + sql "ADMIN SET FRONTEND CONFIG ('cache_last_version_interval_second' = '10')" } diff --git a/regression-test/suites/schema_change_p0/test_schema_change_duplicate.groovy b/regression-test/suites/schema_change_p0/test_schema_change_duplicate.groovy index 19e78ac05ba8ec..007bb196244eff 100644 --- a/regression-test/suites/schema_change_p0/test_schema_change_duplicate.groovy +++ b/regression-test/suites/schema_change_p0/test_schema_change_duplicate.groovy @@ -53,7 +53,6 @@ suite("test_schema_change_duplicate", "p0") { set 'column_separator', ',' file 'all_types.csv' - time 10000 // limit inflight 10s check { result, exception, startTime, endTime -> if (exception != null) { From dc2c2845334635b9d54c0ed6ac3e7985b87bb9d4 Mon Sep 17 00:00:00 2001 From: 924060929 <924060929@qq.com> Date: Thu, 18 Apr 2024 19:41:41 +0800 Subject: [PATCH 2/6] fix --- .../src/main/java/org/apache/doris/nereids/NereidsPlanner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java index a19477093c7dcf..66e409ae7a96d1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java @@ -305,7 +305,7 @@ private void optimize() { } } - private void translate(PhysicalPlan resultPlan) throws UserException { + private void translate(PhysicalPlan resultPlan) { if (resultPlan instanceof PhysicalSqlCache) { return; } From 095103e2fd0ec9f217793848bdbd37edbe24948d Mon Sep 17 00:00:00 2001 From: 924060929 <924060929@qq.com> Date: Wed, 17 Apr 2024 18:11:44 +0800 Subject: [PATCH 3/6] [fix](Nereids) fix some sql cache consistence bug between multiple frontends (#33722) fix some sql cache consistence bug between multiple frontends which introduced by [enhancement](Nereids) Enable parse sql from sql cache #33262, fix by use row policy as the part of sql cache key. support dynamic update the num of fe manage sql cache key (cherry picked from commit 90abd76f71e73702e49794d375ace4f27f834a30) --- .../org/apache/doris/catalog/ScalarType.java | 12 ++- .../java/org/apache/doris/common/Config.java | 20 +++-- .../org/apache/doris/common/ConfigBase.java | 12 +++ .../doris/common/NereidsSqlCacheManager.java | 90 +++++++++++-------- .../apache/doris/nereids/SqlCacheContext.java | 88 ++++++++++++++++-- .../doris/nereids/StatementContext.java | 7 +- .../rules/analysis/ExpressionAnalyzer.java | 8 +- .../apache/doris/nereids/util/JoinUtils.java | 17 +++- .../java/org/apache/doris/qe/Coordinator.java | 18 ++-- .../org/apache/doris/qe/SessionVariable.java | 34 +++++-- .../org/apache/doris/qe/StmtExecutor.java | 1 - .../apache/doris/qe/cache/CacheAnalyzer.java | 24 +++-- .../org/apache/doris/qe/cache/CacheProxy.java | 1 - .../doris/qe/cache/RowBatchBuilder.java | 5 +- .../org/apache/doris/qe/cache/SqlCache.java | 7 +- .../apache/doris/qe/HmsQueryCacheTest.java | 48 +++++----- .../cache/parse_sql_from_sql_cache.groovy | 63 ++++++++++++- 17 files changed, 343 insertions(+), 112 deletions(-) diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java index fbae7ae7b2f9f5..8e8a66849e2492 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java +++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java @@ -744,15 +744,19 @@ public void toThrift(TTypeDesc container) { case DECIMAL128: case DECIMAL256: case DATETIMEV2: { - Preconditions.checkArgument(precision >= scale, - String.format("given precision %d is out of scale bound %d", precision, scale)); + if (precision < scale) { + throw new IllegalArgumentException( + String.format("given precision %d is out of scale bound %d", precision, scale)); + } scalarType.setScale(scale); scalarType.setPrecision(precision); break; } case TIMEV2: { - Preconditions.checkArgument(precision >= scale, - String.format("given precision %d is out of scale bound %d", precision, scale)); + if (precision < scale) { + throw new IllegalArgumentException( + String.format("given precision %d is out of scale bound %d", precision, scale)); + } scalarType.setScale(scale); scalarType.setPrecision(precision); break; diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index 11fa7773adf1eb..5749de27c43e01 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -1300,7 +1300,11 @@ public class Config extends ConfigBase { * Minimum interval between last version when caching results, * This parameter distinguishes between offline and real-time updates */ - @ConfField(mutable = true, masterOnly = false) + @ConfField( + mutable = true, + masterOnly = false, + callbackClassString = "org.apache.doris.common.NereidsSqlCacheManager$UpdateConfig" + ) public static int cache_last_version_interval_second = 30; /** @@ -2011,10 +2015,16 @@ public class Config extends ConfigBase { /** * the plan cache num which can be reused for the next query */ - @ConfField(mutable = false, varType = VariableAnnotation.EXPERIMENTAL, description = { - "当前默认设置为 100,用来控制控制NereidsSqlCacheManager管理的sql cache数量。", - "Now default set to 100, this config is used to control the number of " - + "sql cache managed by NereidsSqlCacheManager"}) + @ConfField( + mutable = true, + varType = VariableAnnotation.EXPERIMENTAL, + callbackClassString = "org.apache.doris.common.NereidsSqlCacheManager$UpdateConfig", + description = { + "当前默认设置为 100,用来控制控制NereidsSqlCacheManager管理的sql cache数量。", + "Now default set to 100, this config is used to control the number of " + + "sql cache managed by NereidsSqlCacheManager" + } + ) public static int sql_cache_manage_num = 100; /** diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/ConfigBase.java b/fe/fe-common/src/main/java/org/apache/doris/common/ConfigBase.java index 2411ff997c7d0b..dd66fdcb0f593b 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/ConfigBase.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/ConfigBase.java @@ -56,6 +56,8 @@ public class ConfigBase { Class callback() default DefaultConfHandler.class; + String callbackClassString() default ""; + // description for this config item. // There should be 2 elements in the array. // The first element is the description in Chinese. @@ -329,6 +331,16 @@ public static synchronized void setMutableConfig(String key, String value) throw throw new ConfigException("Failed to set config '" + key + "'. err: " + e.getMessage()); } + String callbackClassString = anno.callbackClassString(); + if (!Strings.isNullOrEmpty(callbackClassString)) { + try { + ConfHandler confHandler = (ConfHandler) Class.forName(anno.callbackClassString()).newInstance(); + confHandler.handle(field, value); + } catch (Exception e) { + throw new ConfigException("Failed to set config '" + key + "'. err: " + e.getMessage()); + } + } + LOG.info("set config {} to {}", key, value); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java b/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java index f3643046cade04..ad047a1693eb7c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java @@ -18,14 +18,13 @@ package org.apache.doris.common; import org.apache.doris.analysis.UserIdentity; -import org.apache.doris.catalog.Database; import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; -import org.apache.doris.catalog.Table; import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.View; +import org.apache.doris.common.ConfigBase.DefaultConfHandler; import org.apache.doris.datasource.CatalogIf; import org.apache.doris.metric.MetricRepo; import org.apache.doris.mysql.privilege.DataMaskPolicy; @@ -50,6 +49,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalSqlCache; import org.apache.doris.proto.InternalService; +import org.apache.doris.proto.Types.PUniqueId; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.cache.CacheAnalyzer; import org.apache.doris.qe.cache.SqlCache; @@ -59,6 +59,7 @@ import com.google.common.collect.ImmutableList; import org.apache.commons.collections.CollectionUtils; +import java.lang.reflect.Field; import java.time.Duration; import java.util.List; import java.util.Map.Entry; @@ -69,11 +70,36 @@ /** NereidsSqlCacheManager */ public class NereidsSqlCacheManager { // key: : - // value: CacheAnalyzer - private final Cache sqlCache; + // value: SqlCacheContext + private volatile Cache sqlCaches; public NereidsSqlCacheManager(int sqlCacheNum, long cacheIntervalSeconds) { - sqlCache = Caffeine.newBuilder() + sqlCaches = buildSqlCaches(sqlCacheNum, cacheIntervalSeconds); + } + + public static synchronized void updateConfig() { + Env currentEnv = Env.getCurrentEnv(); + if (currentEnv == null) { + return; + } + NereidsSqlCacheManager sqlCacheManager = currentEnv.getSqlCacheManager(); + if (sqlCacheManager == null) { + return; + } + + Cache sqlCaches = buildSqlCaches( + Config.sql_cache_manage_num, + Config.cache_last_version_interval_second + ); + sqlCaches.putAll(sqlCacheManager.sqlCaches.asMap()); + sqlCacheManager.sqlCaches = sqlCaches; + } + + private static Cache buildSqlCaches(int sqlCacheNum, long cacheIntervalSeconds) { + sqlCacheNum = sqlCacheNum <= 0 ? 100 : sqlCacheNum; + cacheIntervalSeconds = cacheIntervalSeconds <= 0 ? 30 : cacheIntervalSeconds; + + return Caffeine.newBuilder() .maximumSize(sqlCacheNum) .expireAfterAccess(Duration.ofSeconds(cacheIntervalSeconds)) // auto evict cache when jvm memory too low @@ -95,10 +121,9 @@ public void tryAddCache( SqlCacheContext sqlCacheContext = sqlCacheContextOpt.get(); UserIdentity currentUserIdentity = connectContext.getCurrentUserIdentity(); String key = currentUserIdentity.toString() + ":" + sql.trim(); - if (analyzer.getCache() instanceof SqlCache - && (currentMissParseSqlFromSqlCache || sqlCache.getIfPresent(key) == null)) { + if ((currentMissParseSqlFromSqlCache || sqlCaches.getIfPresent(key) == null) + && sqlCacheContext.getOrComputeCacheKeyMd5() != null) { SqlCache cache = (SqlCache) analyzer.getCache(); - sqlCacheContext.setCacheKeyMd5(cache.getOrComputeCacheMd5()); sqlCacheContext.setSumOfPartitionNum(cache.getSumOfPartitionNum()); sqlCacheContext.setLatestPartitionId(cache.getLatestId()); sqlCacheContext.setLatestPartitionVersion(cache.getLatestVersion()); @@ -109,23 +134,16 @@ public void tryAddCache( sqlCacheContext.addScanTable(scanTable); } - sqlCache.put(key, sqlCacheContext); + sqlCaches.put(key, sqlCacheContext); } } - /** invalidateCache */ - public void invalidateCache(ConnectContext connectContext, String sql) { - UserIdentity currentUserIdentity = connectContext.getCurrentUserIdentity(); - String key = currentUserIdentity.toString() + ":" + sql.trim(); - sqlCache.invalidate(key); - } - /** tryParseSql */ public Optional tryParseSql(ConnectContext connectContext, String sql) { UserIdentity currentUserIdentity = connectContext.getCurrentUserIdentity(); Env env = connectContext.getEnv(); String key = currentUserIdentity.toString() + ":" + sql.trim(); - SqlCacheContext sqlCacheContext = sqlCache.getIfPresent(key); + SqlCacheContext sqlCacheContext = sqlCaches.getIfPresent(key); if (sqlCacheContext == null) { return Optional.empty(); } @@ -161,16 +179,17 @@ public Optional tryParseSql(ConnectContext connectContext, Stri try { Status status = new Status(); + PUniqueId cacheKeyMd5 = sqlCacheContext.getOrComputeCacheKeyMd5(); InternalService.PFetchCacheResult cacheData = SqlCache.getCacheData(sqlCacheContext.getCacheProxy(), - sqlCacheContext.getCacheKeyMd5(), sqlCacheContext.getLatestPartitionId(), + cacheKeyMd5, sqlCacheContext.getLatestPartitionId(), sqlCacheContext.getLatestPartitionVersion(), sqlCacheContext.getLatestPartitionTime(), sqlCacheContext.getSumOfPartitionNum(), status); if (status.ok() && cacheData != null && cacheData.getStatus() == InternalService.PCacheStatus.CACHE_OK) { List cacheValues = cacheData.getValuesList(); String cachedPlan = sqlCacheContext.getPhysicalPlan(); - String backendAddress = SqlCache.findCacheBe(sqlCacheContext.getCacheKeyMd5()).getAddress(); + String backendAddress = SqlCache.findCacheBe(cacheKeyMd5).getAddress(); MetricRepo.COUNTER_CACHE_HIT_SQL.increase(1L); @@ -179,9 +198,9 @@ public Optional tryParseSql(ConnectContext connectContext, Stri sqlCacheContext.getResultExprs(), cacheValues, backendAddress, cachedPlan); return Optional.of(logicalSqlCache); } - return Optional.empty(); + return invalidateCache(key); } catch (Throwable t) { - return Optional.empty(); + return invalidateCache(key); } } @@ -309,7 +328,8 @@ private boolean nondeterministicFunctionChanged( return true; } - List> nondeterministicFunctions = sqlCacheContext.getFoldNondeterministicPairs(); + List> nondeterministicFunctions + = sqlCacheContext.getFoldFullNondeterministicPairs(); if (nondeterministicFunctions.isEmpty()) { return false; } @@ -328,21 +348,8 @@ private boolean nondeterministicFunctionChanged( return false; } - private boolean isValidDbAndTable(TableIf tableIf, Env env) { - return getTableFromEnv(tableIf, env) != null; - } - - private TableIf getTableFromEnv(TableIf tableIf, Env env) { - Optional db = env.getInternalCatalog().getDb(tableIf.getDatabase().getId()); - if (!db.isPresent()) { - return null; - } - Optional

table = db.get().getTable(tableIf.getId()); - return table.orElse(null); - } - private Optional invalidateCache(String key) { - sqlCache.invalidate(key); + sqlCaches.invalidate(key); return Optional.empty(); } @@ -357,4 +364,15 @@ private TableIf findTableIf(Env env, FullTableName fullTableName) { } return db.get().getTable(fullTableName.table).orElse(null); } + + // NOTE: used in Config.sql_cache_manage_num.callbackClassString and + // Config.cache_last_version_interval_second.callbackClassString, + // don't remove it! + public static class UpdateConfig extends DefaultConfHandler { + @Override + public void handle(Field field, String confVal) throws Exception { + super.handle(field, confVal); + NereidsSqlCacheManager.updateConfig(); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java index 6afda4e4fe96d6..6a7dbd2ed9de4f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java @@ -42,6 +42,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -52,6 +53,7 @@ public class SqlCacheContext { private final TUniqueId queryId; // if contains udf/udaf/tableValuesFunction we can not process it and skip use sql cache private volatile boolean cannotProcessExpression; + private volatile String originSql; private volatile String physicalPlan; private volatile long latestPartitionId = -1; private volatile long latestPartitionTime = -1; @@ -65,9 +67,13 @@ public class SqlCacheContext { private final Map> rowPolicies = Maps.newLinkedHashMap(); private final Map> dataMaskPolicies = Maps.newLinkedHashMap(); private final Set usedVariables = Sets.newLinkedHashSet(); - // key: the expression which contains nondeterministic function, e.g. date(now()) - // value: the expression which already try to fold nondeterministic function, e.g. '2024-01-01' + // key: the expression which **contains** nondeterministic function, e.g. date_add(date_column, date(now())) + // value: the expression which already try to fold nondeterministic function, + // e.g. date_add(date_column, '2024-01-01') // note that value maybe contains nondeterministic function too, when fold failed + private final List> foldFullNondeterministicPairs = Lists.newArrayList(); + // key: the expression which **is** nondeterministic function, e.g. now() + // value: the expression which already try to fold nondeterministic function, e.g. '2024-01-01 10:01:03' private final List> foldNondeterministicPairs = Lists.newArrayList(); private volatile boolean hasUnsupportedTables; private final List scanTables = Lists.newArrayList(); @@ -206,6 +212,14 @@ public synchronized List getUsedVariables() { return ImmutableList.copyOf(usedVariables); } + public synchronized void addFoldFullNondeterministicPair(Expression unfold, Expression fold) { + foldFullNondeterministicPairs.add(Pair.of(unfold, fold)); + } + + public synchronized List> getFoldFullNondeterministicPairs() { + return ImmutableList.copyOf(foldFullNondeterministicPairs); + } + public synchronized void addFoldNondeterministicPair(Expression unfold, Expression fold) { foldNondeterministicPairs.add(Pair.of(unfold, fold)); } @@ -278,10 +292,6 @@ public synchronized Map> getRowPolicies() { return ImmutableMap.copyOf(rowPolicies); } - public boolean isHasUnsupportedTables() { - return hasUnsupportedTables; - } - public synchronized void addScanTable(ScanTable scanTable) { this.scanTables.add(scanTable); } @@ -310,12 +320,62 @@ public TUniqueId getQueryId() { return queryId; } - public PUniqueId getCacheKeyMd5() { + /** getOrComputeCacheKeyMd5 */ + public PUniqueId getOrComputeCacheKeyMd5() { + if (cacheKeyMd5 == null && originSql != null) { + synchronized (this) { + if (cacheKeyMd5 != null) { + return cacheKeyMd5; + } + + StringBuilder cacheKey = new StringBuilder(originSql); + for (Entry entry : usedViews.entrySet()) { + cacheKey.append("|") + .append(entry.getKey()) + .append("=") + .append(entry.getValue()); + } + for (Variable usedVariable : usedVariables) { + cacheKey.append("|") + .append(usedVariable.getType().name()) + .append(":") + .append(usedVariable.getName()) + .append("=") + .append(usedVariable.getRealExpression().toSql()); + } + for (Pair pair : foldNondeterministicPairs) { + cacheKey.append("|") + .append(pair.key().toSql()) + .append("=") + .append(pair.value().toSql()); + } + for (Entry> entry : rowPolicies.entrySet()) { + List policy = entry.getValue(); + if (policy.isEmpty()) { + continue; + } + cacheKey.append("|") + .append(entry.getKey()) + .append("=") + .append(policy); + } + for (Entry> entry : dataMaskPolicies.entrySet()) { + if (!entry.getValue().isPresent()) { + continue; + } + cacheKey.append("|") + .append(entry.getKey()) + .append("=") + .append(entry.getValue().map(Object::toString).orElse("")); + } + cacheKeyMd5 = CacheProxy.getMd5(cacheKey.toString()); + } + } return cacheKeyMd5; } - public void setCacheKeyMd5(PUniqueId cacheKeyMd5) { - this.cacheKeyMd5 = cacheKeyMd5; + public void setOriginSql(String originSql) { + this.originSql = originSql.trim(); } /** FullTableName */ @@ -325,6 +385,11 @@ public static class FullTableName { public final String catalog; public final String db; public final String table; + + @Override + public String toString() { + return catalog + "." + db + "." + table; + } } /** FullColumnName */ @@ -335,6 +400,11 @@ public static class FullColumnName { public final String db; public final String table; public final String column; + + @Override + public String toString() { + return catalog + "." + db + "." + table + "." + column; + } } /** ScanTable */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java index 5baa59a35dec1b..a468a0f0cb7f21 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java @@ -112,7 +112,6 @@ public class StatementContext implements Closeable { private final Map rewrittenCteConsumer = new HashMap<>(); private final Set viewDdlSqlSet = Sets.newHashSet(); private final SqlCacheContext sqlCacheContext; - private Map> checkedPrivilegedTableAndUsedColumns = Maps.newLinkedHashMap(); // collect all hash join conditions to compute node connectivity in join graph private final List joinFilters = new ArrayList<>(); @@ -155,6 +154,9 @@ public StatementContext(ConnectContext connectContext, OriginStatement originSta && CacheAnalyzer.canUseSqlCache(connectContext.getSessionVariable())) { this.sqlCacheContext = new SqlCacheContext( connectContext.getCurrentUserIdentity(), connectContext.queryId()); + if (originStatement != null) { + this.sqlCacheContext.setOriginSql(originStatement.originStmt.trim()); + } } else { this.sqlCacheContext = null; } @@ -170,6 +172,9 @@ public ConnectContext getConnectContext() { public void setOriginStatement(OriginStatement originStatement) { this.originStatement = originStatement; + if (originStatement != null && sqlCacheContext != null) { + sqlCacheContext.setOriginSql(originStatement.originStmt.trim()); + } } public OriginStatement getOriginStatement() { 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 cc85bc01323c75..691e4aca1a337d 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 @@ -154,14 +154,18 @@ public Expression analyze(Expression expression, ExpressionRewriteContext contex @Override public Expression visitBoundFunction(BoundFunction boundFunction, ExpressionRewriteContext context) { Expression fold = super.visitBoundFunction(boundFunction, context); - if (fold instanceof Nondeterministic) { + boolean unfold = fold instanceof Nondeterministic; + if (unfold) { sqlCacheContext.setCannotProcessExpression(true); } + if (boundFunction instanceof Nondeterministic && !unfold) { + sqlCacheContext.addFoldNondeterministicPair(boundFunction, fold); + } return fold; } }.rewrite(analyzeResult, context); - sqlCacheContext.addFoldNondeterministicPair(analyzeResult, foldNondeterministic); + sqlCacheContext.addFoldFullNondeterministicPair(analyzeResult, foldNondeterministic); return foldNondeterministic; } return analyzeResult; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/JoinUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/JoinUtils.java index 5c01fd4df9a87a..79462a9d8e7ebf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/JoinUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/JoinUtils.java @@ -141,11 +141,20 @@ public boolean isHashJoinCondition(EqualPredicate equal) { public static Pair, List> extractExpressionForHashTable(List leftSlots, List rightSlots, List onConditions) { JoinSlotCoverageChecker checker = new JoinSlotCoverageChecker(leftSlots, rightSlots); - Map> mapper = onConditions.stream().collect(Collectors.groupingBy( - expr -> (expr instanceof EqualPredicate) && checker.isHashJoinCondition((EqualPredicate) expr))); + + ImmutableList.Builder hashConditions = ImmutableList.builderWithExpectedSize(onConditions.size()); + ImmutableList.Builder otherConditions = ImmutableList.builderWithExpectedSize(onConditions.size()); + for (Expression expr : onConditions) { + if (expr instanceof EqualPredicate && checker.isHashJoinCondition((EqualPredicate) expr)) { + hashConditions.add(expr); + } else { + otherConditions.add(expr); + } + } + return Pair.of( - mapper.getOrDefault(true, ImmutableList.of()), - mapper.getOrDefault(false, ImmutableList.of()) + hashConditions.build(), + otherConditions.build() ); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index c591f5412c1851..87873d356402d3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -62,6 +62,7 @@ import org.apache.doris.planner.RuntimeFilterId; import org.apache.doris.planner.ScanNode; import org.apache.doris.planner.SetOperationNode; +import org.apache.doris.planner.SortNode; import org.apache.doris.planner.UnionNode; import org.apache.doris.proto.InternalService; import org.apache.doris.proto.InternalService.PExecPlanFragmentResult; @@ -998,7 +999,9 @@ private void sendPipelineCtx() throws TException, RpcException, UserException { // 4. send and wait fragments rpc // 4.1 serialize fragment // unsetFields() must be called serially. - beToPipelineExecCtxs.values().stream().forEach(ctxs -> ctxs.unsetFields()); + for (PipelineExecContexts ctxs : beToPipelineExecCtxs.values()) { + ctxs.unsetFields(); + } // serializeFragments() can be called in parallel. final AtomicLong compressedSize = new AtomicLong(0); beToPipelineExecCtxs.values().parallelStream().forEach(ctxs -> { @@ -3805,10 +3808,15 @@ Map toTPipelineParams(int backendNum) int rate = Math.min(Config.query_colocate_join_memory_limit_penalty_factor, instanceExecParams.size()); memLimit = queryOptions.getMemLimit() / rate; } - Set topnFilterSources = scanNodes.stream() - .filter(scanNode -> scanNode instanceof OlapScanNode) - .flatMap(scanNode -> ((OlapScanNode) scanNode).getTopnFilterSortNodes().stream()) - .map(sort -> sort.getId().asInt()).collect(Collectors.toSet()); + Set topnFilterSources = Sets.newLinkedHashSet(); + for (ScanNode scanNode : scanNodes) { + if (scanNode instanceof OlapScanNode) { + for (SortNode sortNode : ((OlapScanNode) scanNode).getTopnFilterSortNodes()) { + topnFilterSources.add(sortNode.getId().asInt()); + } + } + } + Map res = new HashMap(); Map instanceIdx = new HashMap(); TPlanFragment fragmentThrift = fragment.toThrift(); 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 636ac9fca45b71..6c71f621860fe6 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 @@ -42,6 +42,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -1621,16 +1622,33 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { public static final String IGNORE_RUNTIME_FILTER_IDS = "ignore_runtime_filter_ids"; public Set getIgnoredRuntimeFilterIds() { - return Arrays.stream(ignoreRuntimeFilterIds.split(",[\\s]*")) - .map(v -> { - int res = -1; + Set ids = Sets.newLinkedHashSet(); + if (ignoreRuntimeFilterIds.isEmpty()) { + return ImmutableSet.of(); + } + for (String v : ignoreRuntimeFilterIds.split(",[\\s]*")) { + int res = -1; + if (!v.isEmpty()) { + boolean isNumber = true; + for (int i = 0; i < v.length(); ++i) { + char c = v.charAt(i); + if (c < '0' || c > '9') { + isNumber = false; + break; + } + } + if (isNumber) { try { - res = Integer.valueOf(v); - } catch (Exception e) { - //ignore it + res = Integer.parseInt(v); + } catch (Throwable t) { + // ignore } - return res; - }).collect(ImmutableSet.toImmutableSet()); + } + + } + ids.add(res); + } + return ids; } public void setIgnoreRuntimeFilterIds(String ignoreRuntimeFilterIds) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java index a44b95f7fbb09f..04ee42f0198a9f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -1540,7 +1540,6 @@ private void handleCacheStmt(CacheAnalyzer cacheAnalyzer, MysqlChannel channel) } } - CacheMode mode = cacheAnalyzer.getCacheMode(); Queriable queryStmt = (Queriable) parsedStmt; boolean isSendFields = false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheAnalyzer.java index f1c5d98f16095f..161661055a4981 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheAnalyzer.java @@ -45,6 +45,8 @@ import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.hive.source.HiveScanNode; import org.apache.doris.metric.MetricRepo; +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.SqlCacheContext; import org.apache.doris.nereids.SqlCacheContext.FullTableName; import org.apache.doris.nereids.SqlCacheContext.ScanTable; import org.apache.doris.nereids.glue.LogicalPlanAdapter; @@ -70,6 +72,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Optional; import java.util.Set; /** @@ -401,7 +404,7 @@ private CacheMode innerCheckCacheModeForNereids(long now) { } return CacheMode.NoNeed; } - if (!(parsedStmt instanceof LogicalPlanAdapter) || scanNodes.size() == 0) { + if (!(parsedStmt instanceof LogicalPlanAdapter) || scanNodes.isEmpty()) { if (LOG.isDebugEnabled()) { LOG.debug("not a select stmt or no scan node. queryid {}", DebugUtil.printId(queryId)); } @@ -442,13 +445,22 @@ private CacheMode innerCheckCacheModeForNereids(long now) { Config.cache_last_version_interval_second * 1000); } + String originStmt = ((LogicalPlanAdapter) parsedStmt).getStatementContext() + .getOriginStatement().originStmt; + cache = new SqlCache(this.queryId, originStmt); + SqlCache sqlCache = (SqlCache) cache; PUniqueId existsMd5 = null; - if (cache instanceof SqlCache) { - existsMd5 = ((SqlCache) cache).getOrComputeCacheMd5(); + if (planner instanceof NereidsPlanner) { + NereidsPlanner nereidsPlanner = (NereidsPlanner) planner; + Optional sqlCacheContext = nereidsPlanner + .getCascadesContext() + .getStatementContext() + .getSqlCacheContext(); + if (sqlCacheContext.isPresent()) { + existsMd5 = sqlCacheContext.get().getOrComputeCacheKeyMd5(); + } } - cache = new SqlCache(this.queryId, ((LogicalPlanAdapter) parsedStmt).getStatementContext() - .getOriginStatement().originStmt); - SqlCache sqlCache = (SqlCache) cache; + sqlCache.setCacheInfo(this.latestTable, allViewExpandStmtListStr); sqlCache.setCacheMd5(existsMd5); MetricRepo.COUNTER_CACHE_ADDED_SQL.increase(1L); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheProxy.java b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheProxy.java index 20d6f5d0d02401..dd0e0ce6bcdb77 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheProxy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheProxy.java @@ -60,7 +60,6 @@ public abstract InternalService.PFetchCacheResult fetchCache(InternalService.PFe public abstract void clearCache(InternalService.PClearCacheRequest clearRequest); - public static Types.PUniqueId getMd5(String str) { MessageDigest msgDigest; final byte[] digest; diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/RowBatchBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/RowBatchBuilder.java index 5cab6f74bb3411..20d9f93efb494e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/RowBatchBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/RowBatchBuilder.java @@ -21,6 +21,7 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Type; import org.apache.doris.proto.InternalService; +import org.apache.doris.proto.Types.PUniqueId; import org.apache.doris.qe.RowBatch; import org.apache.doris.thrift.TResultBatch; @@ -117,10 +118,10 @@ public void clear() { } public InternalService.PUpdateCacheRequest buildSqlUpdateRequest( - String sql, long partitionKey, long lastVersion, long lastestTime, long partitionNum) { + PUniqueId cacheKeyMd5, long partitionKey, long lastVersion, long lastestTime, long partitionNum) { if (updateRequest == null) { updateRequest = InternalService.PUpdateCacheRequest.newBuilder() - .setSqlKey(CacheProxy.getMd5(sql)) + .setSqlKey(cacheKeyMd5) .setCacheType(InternalService.CacheType.SQL_CACHE).build(); } updateRequest = updateRequest.toBuilder() diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/SqlCache.java b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/SqlCache.java index a7351a306d2e2d..efe4ed5949f6d6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/SqlCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/SqlCache.java @@ -141,9 +141,12 @@ public void updateCache() { return; } + PUniqueId cacheKeyMd5 = getOrComputeCacheMd5(); InternalService.PUpdateCacheRequest updateRequest = - rowBatchBuilder.buildSqlUpdateRequest(getSqlWithViewStmt(), latestTable.latestPartitionId, - latestTable.latestPartitionVersion, latestTable.latestPartitionTime, + rowBatchBuilder.buildSqlUpdateRequest(cacheKeyMd5, + latestTable.latestPartitionId, + latestTable.latestPartitionVersion, + latestTable.latestPartitionTime, latestTable.sumOfPartitionNum ); if (updateRequest.getValuesCount() > 0) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/HmsQueryCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/HmsQueryCacheTest.java index 7613a2d284b91c..949b28491d6dba 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/qe/HmsQueryCacheTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/qe/HmsQueryCacheTest.java @@ -367,9 +367,9 @@ public void testHitSqlCache() throws Exception { List scanNodes = Arrays.asList(hiveScanNode1); CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); ca.checkCacheMode(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); - Assert.assertEquals(ca.getCacheMode(), CacheAnalyzer.CacheMode.Sql); + Assert.assertEquals(CacheAnalyzer.CacheMode.Sql, ca.getCacheMode()); SqlCache sqlCache = (SqlCache) ca.getCache(); - Assert.assertEquals(sqlCache.getLatestTime(), NOW); + Assert.assertEquals(NOW, sqlCache.getLatestTime()); } @Test @@ -383,11 +383,11 @@ public void testHitSqlCacheAfterPartitionUpdateTimeChanged() throws Exception { CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); ca.checkCacheMode(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); - Assert.assertEquals(ca.getCacheMode(), CacheAnalyzer.CacheMode.Sql); + Assert.assertEquals(CacheAnalyzer.CacheMode.Sql, ca.getCacheMode()); SqlCache sqlCache1 = (SqlCache) ca.getCache(); // latestTime is equals to the schema update time if not set partition update time - Assert.assertEquals(sqlCache1.getLatestTime(), tbl2.getSchemaUpdateTime()); + Assert.assertEquals(tbl2.getSchemaUpdateTime(), sqlCache1.getLatestTime()); // wait a second and set partition update time try { @@ -401,7 +401,7 @@ public void testHitSqlCacheAfterPartitionUpdateTimeChanged() throws Exception { // check cache mode again ca.checkCacheMode(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); SqlCache sqlCache2 = (SqlCache) ca.getCache(); - Assert.assertEquals(ca.getCacheMode(), CacheAnalyzer.CacheMode.Sql); + Assert.assertEquals(CacheAnalyzer.CacheMode.Sql, ca.getCacheMode()); // the latest time will be changed and is equals to the partition update time Assert.assertEquals(later, sqlCache2.getLatestTime()); @@ -415,9 +415,9 @@ public void testHitSqlCacheByNereids() { List scanNodes = Arrays.asList(hiveScanNode1); CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); ca.checkCacheModeForNereids(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); - Assert.assertEquals(ca.getCacheMode(), CacheAnalyzer.CacheMode.Sql); + Assert.assertEquals(CacheAnalyzer.CacheMode.Sql, ca.getCacheMode()); SqlCache sqlCache = (SqlCache) ca.getCache(); - Assert.assertEquals(sqlCache.getLatestTime(), NOW); + Assert.assertEquals(NOW, sqlCache.getLatestTime()); } @Test @@ -431,11 +431,11 @@ public void testHitSqlCacheByNereidsAfterPartitionUpdateTimeChanged() { CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); ca.checkCacheModeForNereids(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); - Assert.assertEquals(ca.getCacheMode(), CacheAnalyzer.CacheMode.Sql); + Assert.assertEquals(CacheAnalyzer.CacheMode.Sql, ca.getCacheMode()); SqlCache sqlCache1 = (SqlCache) ca.getCache(); // latestTime is equals to the schema update time if not set partition update time - Assert.assertEquals(sqlCache1.getLatestTime(), tbl2.getSchemaUpdateTime()); + Assert.assertEquals(tbl2.getSchemaUpdateTime(), sqlCache1.getLatestTime()); // wait a second and set partition update time try { @@ -449,7 +449,7 @@ public void testHitSqlCacheByNereidsAfterPartitionUpdateTimeChanged() { // check cache mode again ca.checkCacheModeForNereids(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); SqlCache sqlCache2 = (SqlCache) ca.getCache(); - Assert.assertEquals(ca.getCacheMode(), CacheAnalyzer.CacheMode.Sql); + Assert.assertEquals(CacheAnalyzer.CacheMode.Sql, ca.getCacheMode()); // the latest time will be changed and is equals to the partition update time Assert.assertEquals(later, sqlCache2.getLatestTime()); @@ -463,9 +463,9 @@ public void testHitSqlCacheWithHiveView() throws Exception { List scanNodes = Arrays.asList(hiveScanNode2); CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); ca.checkCacheMode(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); - Assert.assertEquals(ca.getCacheMode(), CacheAnalyzer.CacheMode.Sql); + Assert.assertEquals(CacheAnalyzer.CacheMode.Sql, ca.getCacheMode()); SqlCache sqlCache = (SqlCache) ca.getCache(); - Assert.assertEquals(sqlCache.getLatestTime(), NOW); + Assert.assertEquals(NOW, sqlCache.getLatestTime()); } @Test @@ -475,9 +475,9 @@ public void testHitSqlCacheWithHiveViewByNereids() { List scanNodes = Arrays.asList(hiveScanNode2); CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); ca.checkCacheModeForNereids(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); - Assert.assertEquals(ca.getCacheMode(), CacheAnalyzer.CacheMode.Sql); + Assert.assertEquals(CacheAnalyzer.CacheMode.Sql, ca.getCacheMode()); SqlCache sqlCache = (SqlCache) ca.getCache(); - Assert.assertEquals(sqlCache.getLatestTime(), NOW); + Assert.assertEquals(NOW, sqlCache.getLatestTime()); } @Test @@ -487,13 +487,13 @@ public void testHitSqlCacheWithNestedHiveView() throws Exception { List scanNodes = Arrays.asList(hiveScanNode3); CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); ca.checkCacheMode(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); - Assert.assertEquals(ca.getCacheMode(), CacheAnalyzer.CacheMode.Sql); + Assert.assertEquals(CacheAnalyzer.CacheMode.Sql, ca.getCacheMode()); SqlCache sqlCache = (SqlCache) ca.getCache(); String cacheKey = sqlCache.getSqlWithViewStmt(); Assert.assertEquals(cacheKey, "SELECT `hms_ctl`.`hms_db`.`hms_view2`.`k1` AS `k1` " + "FROM `hms_ctl`.`hms_db`.`hms_view2`" + "|SELECT * FROM hms_db.hms_tbl|SELECT * FROM hms_db.hms_view1"); - Assert.assertEquals(sqlCache.getLatestTime(), NOW); + Assert.assertEquals(NOW, sqlCache.getLatestTime()); } @Test @@ -503,12 +503,12 @@ public void testHitSqlCacheWithNestedHiveViewByNereids() { List scanNodes = Arrays.asList(hiveScanNode3); CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); ca.checkCacheModeForNereids(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); - Assert.assertEquals(ca.getCacheMode(), CacheAnalyzer.CacheMode.Sql); + Assert.assertEquals(CacheAnalyzer.CacheMode.Sql, ca.getCacheMode()); SqlCache sqlCache = (SqlCache) ca.getCache(); String cacheKey = sqlCache.getSqlWithViewStmt(); - Assert.assertEquals(cacheKey, "select * from hms_ctl.hms_db.hms_view2" - + "|SELECT * FROM hms_db.hms_tbl|SELECT * FROM hms_db.hms_view1"); - Assert.assertEquals(sqlCache.getLatestTime(), NOW); + Assert.assertEquals("select * from hms_ctl.hms_db.hms_view2" + + "|SELECT * FROM hms_db.hms_tbl|SELECT * FROM hms_db.hms_view1", cacheKey); + Assert.assertEquals(NOW, sqlCache.getLatestTime()); } @Test @@ -518,7 +518,7 @@ public void testNotHitSqlCache() throws Exception { List scanNodes = Arrays.asList(hiveScanNode1); CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); ca.checkCacheMode(0); - Assert.assertEquals(ca.getCacheMode(), CacheAnalyzer.CacheMode.None); + Assert.assertEquals(CacheAnalyzer.CacheMode.None, ca.getCacheMode()); } @Test @@ -528,7 +528,7 @@ public void testNotHitSqlCacheByNereids() { List scanNodes = Arrays.asList(hiveScanNode1); CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); ca.checkCacheModeForNereids(0); - Assert.assertEquals(ca.getCacheMode(), CacheAnalyzer.CacheMode.None); + Assert.assertEquals(CacheAnalyzer.CacheMode.None, ca.getCacheMode()); } @Test @@ -540,7 +540,7 @@ public void testNotHitSqlCacheWithFederatedQuery() throws Exception { List scanNodes = Arrays.asList(hiveScanNode1, olapScanNode); CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); ca.checkCacheMode(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); - Assert.assertEquals(ca.getCacheMode(), CacheAnalyzer.CacheMode.None); + Assert.assertEquals(CacheAnalyzer.CacheMode.None, ca.getCacheMode()); } @Test @@ -552,6 +552,6 @@ public void testNotHitSqlCacheWithFederatedQueryByNereids() { List scanNodes = Arrays.asList(hiveScanNode1, olapScanNode); CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); ca.checkCacheModeForNereids(System.currentTimeMillis() + Config.cache_last_version_interval_second * 1000L * 2); - Assert.assertEquals(ca.getCacheMode(), CacheAnalyzer.CacheMode.None); + Assert.assertEquals(CacheAnalyzer.CacheMode.None, ca.getCacheMode()); } } diff --git a/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy index 7523df43f0cbb8..99bed86beba250 100644 --- a/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy +++ b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +import java.util.stream.Collectors + suite("parse_sql_from_sql_cache") { def assertHasCache = { String sqlStr -> explain { @@ -339,7 +341,7 @@ suite("parse_sql_from_sql_cache") { CREATE ROW POLICY test_cache_row_policy_2 ON ${dbName}.test_use_plan_cache13 AS RESTRICTIVE TO test_cache_user2 - USING (id = 'concat(id, "**")')""" + USING (id = 4)""" sql "set enable_nereids_planner=true" // after row policy changed, the cache is invalidate @@ -374,7 +376,7 @@ suite("parse_sql_from_sql_cache") { CREATE ROW POLICY test_cache_row_policy_3 ON ${dbName}.test_use_plan_cache14 AS RESTRICTIVE TO test_cache_user3 - USING (id = 'concat(id, "**")')""" + USING (id = 4)""" sql "set enable_nereids_planner=true" // after partition changed 10s, the sql cache can be used @@ -545,6 +547,63 @@ suite("parse_sql_from_sql_cache") { assertNoCache "SELECT java_udf_string_test(varchar_col, 2, 3) result FROM test_javaudf_string ORDER BY result;" sql "SELECT java_udf_string_test(varchar_col, 2, 3) result FROM test_javaudf_string ORDER BY result;" assertNoCache "SELECT java_udf_string_test(varchar_col, 2, 3) result FROM test_javaudf_string ORDER BY result;" + }), + extraThread("testMultiFrontends", { + def aliveFrontends = sql_return_maparray("show frontends") + .stream() + .filter { it["Alive"].toString().equalsIgnoreCase("true") } + .collect(Collectors.toList()) + + if (aliveFrontends.size() <= 1) { + return + } + + def fe1 = aliveFrontends[0]["Host"] + ":" + aliveFrontends[0]["QueryPort"] + def fe2 = fe1 + if (aliveFrontends.size() > 1) { + fe2 = aliveFrontends[1]["Host"] + ":" + aliveFrontends[1]["QueryPort"] + } + + log.info("fe1: ${fe1}") + log.info("fe2: ${fe2}") + + def dbName = context.config.getDbNameByFile(context.file) + + log.info("connect to fe: ${fe1}") + connect(user = context.config.jdbcUser, password = context.config.jdbcPassword, url = "jdbc:mysql://${fe1}") { + sql "ADMIN SET FRONTEND CONFIG ('cache_last_version_interval_second' = '10')" + + sql "use ${dbName}" + + createTestTable "test_use_plan_cache18" + + sql "sync" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache18" + sql "select * from test_use_plan_cache18" + assertHasCache "select * from test_use_plan_cache18" + } + + log.info("connect to fe: ${fe2}") + connect(user = context.config.jdbcUser, password = context.config.jdbcPassword, url = "jdbc:mysql://${fe2}") { + sql "ADMIN SET FRONTEND CONFIG ('cache_last_version_interval_second' = '10')" + + sql "use ${dbName}" + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache18" + sql "select * from test_use_plan_cache18" + assertHasCache "select * from test_use_plan_cache18" + } }) ).get() } From 6e63fe920da4633a005e67bb4f4efd598d3e8887 Mon Sep 17 00:00:00 2001 From: 924060929 <924060929@qq.com> Date: Thu, 18 Apr 2024 10:43:22 +0800 Subject: [PATCH 4/6] [fix](Nereids) fix bug of dry run query with sql cache (#33799) 1. dry run query should not use sql cache 2. fix test sql cache in cloud mode 3. enable cache OneRowRelation and EmptyRelation in frontend to skip parse sql (cherry picked from commit dc80ecf7f33da7b8c04832dee88abd09f7db9ffe) --- .../doris/common/NereidsSqlCacheManager.java | 38 ++++++- .../apache/doris/nereids/NereidsPlanner.java | 82 ++++++++++---- .../apache/doris/nereids/SqlCacheContext.java | 10 ++ .../trees/plans/logical/LogicalSqlCache.java | 10 +- .../plans/physical/PhysicalSqlCache.java | 22 +++- .../org/apache/doris/qe/StmtExecutor.java | 12 +-- .../apache/doris/qe/cache/CacheAnalyzer.java | 3 +- .../cache/parse_sql_from_sql_cache.groovy | 100 +++++++++++++++++- 8 files changed, 241 insertions(+), 36 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java b/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java index ad047a1693eb7c..8989375c07f7d2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java @@ -51,6 +51,7 @@ import org.apache.doris.proto.InternalService; import org.apache.doris.proto.Types.PUniqueId; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ResultSet; import org.apache.doris.qe.cache.CacheAnalyzer; import org.apache.doris.qe.cache.SqlCache; @@ -96,8 +97,8 @@ public static synchronized void updateConfig() { } private static Cache buildSqlCaches(int sqlCacheNum, long cacheIntervalSeconds) { - sqlCacheNum = sqlCacheNum <= 0 ? 100 : sqlCacheNum; - cacheIntervalSeconds = cacheIntervalSeconds <= 0 ? 30 : cacheIntervalSeconds; + sqlCacheNum = sqlCacheNum < 0 ? 100 : sqlCacheNum; + cacheIntervalSeconds = cacheIntervalSeconds < 0 ? 30 : cacheIntervalSeconds; return Caffeine.newBuilder() .maximumSize(sqlCacheNum) @@ -107,6 +108,22 @@ private static Cache buildSqlCaches(int sqlCacheNum, lo .build(); } + /** tryAddFeCache */ + public void tryAddFeSqlCache(ConnectContext connectContext, String sql) { + Optional sqlCacheContextOpt = connectContext.getStatementContext().getSqlCacheContext(); + if (!sqlCacheContextOpt.isPresent()) { + return; + } + + SqlCacheContext sqlCacheContext = sqlCacheContextOpt.get(); + UserIdentity currentUserIdentity = connectContext.getCurrentUserIdentity(); + String key = currentUserIdentity.toString() + ":" + sql.trim(); + if ((sqlCaches.getIfPresent(key) == null) && sqlCacheContext.getOrComputeCacheKeyMd5() != null + && sqlCacheContext.getResultSetInFe().isPresent()) { + sqlCaches.put(key, sqlCacheContext); + } + } + /** tryAddCache */ public void tryAddCache( ConnectContext connectContext, String sql, @@ -178,6 +195,19 @@ public Optional tryParseSql(ConnectContext connectContext, Stri } try { + Optional resultSetInFe = sqlCacheContext.getResultSetInFe(); + if (resultSetInFe.isPresent()) { + MetricRepo.COUNTER_CACHE_HIT_SQL.increase(1L); + + String cachedPlan = sqlCacheContext.getPhysicalPlan(); + LogicalSqlCache logicalSqlCache = new LogicalSqlCache( + sqlCacheContext.getQueryId(), sqlCacheContext.getColLabels(), + sqlCacheContext.getResultExprs(), resultSetInFe, ImmutableList.of(), + "none", cachedPlan + ); + return Optional.of(logicalSqlCache); + } + Status status = new Status(); PUniqueId cacheKeyMd5 = sqlCacheContext.getOrComputeCacheKeyMd5(); InternalService.PFetchCacheResult cacheData = @@ -195,7 +225,9 @@ public Optional tryParseSql(ConnectContext connectContext, Stri LogicalSqlCache logicalSqlCache = new LogicalSqlCache( sqlCacheContext.getQueryId(), sqlCacheContext.getColLabels(), - sqlCacheContext.getResultExprs(), cacheValues, backendAddress, cachedPlan); + sqlCacheContext.getResultExprs(), Optional.empty(), + cacheValues, backendAddress, cachedPlan + ); return Optional.of(logicalSqlCache); } return invalidateCache(key); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java index 66e409ae7a96d1..99aa6ed73a1bd8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java @@ -22,6 +22,7 @@ import org.apache.doris.analysis.LiteralExpr; import org.apache.doris.analysis.StatementBase; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.MTMV; import org.apache.doris.common.NereidsException; import org.apache.doris.common.Pair; @@ -53,6 +54,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalSqlCache; import org.apache.doris.nereids.trees.plans.physical.PhysicalCatalogRelation; +import org.apache.doris.nereids.trees.plans.physical.PhysicalEmptyRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalOneRowRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; import org.apache.doris.nereids.trees.plans.physical.PhysicalResultSink; @@ -65,8 +67,10 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ResultSet; import org.apache.doris.qe.ResultSetMetaData; +import org.apache.doris.qe.cache.CacheAnalyzer; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -167,8 +171,9 @@ public Plan plan(LogicalPlan plan, PhysicalProperties requireProperties, LogicalSqlCache logicalSqlCache = (LogicalSqlCache) plan; physicalPlan = new PhysicalSqlCache( logicalSqlCache.getQueryId(), logicalSqlCache.getColumnLabels(), - logicalSqlCache.getResultExprs(), logicalSqlCache.getCacheValues(), - logicalSqlCache.getBackendAddress(), logicalSqlCache.getPlanBody() + logicalSqlCache.getResultExprs(), logicalSqlCache.getResultSetInFe(), + logicalSqlCache.getCacheValues(), logicalSqlCache.getBackendAddress(), + logicalSqlCache.getPlanBody() ); return physicalPlan; } @@ -524,31 +529,66 @@ public Optional handleQueryInFe(StatementBase parsedStmt) { if (!(parsedStmt instanceof LogicalPlanAdapter)) { return Optional.empty(); } - if (!(physicalPlan instanceof PhysicalResultSink)) { - return Optional.empty(); + if (physicalPlan instanceof PhysicalSqlCache + && ((PhysicalSqlCache) physicalPlan).getResultSet().isPresent()) { + return Optional.of(((PhysicalSqlCache) physicalPlan).getResultSet().get()); } - if (!(((PhysicalResultSink) physicalPlan).child() instanceof PhysicalOneRowRelation)) { + if (!(physicalPlan instanceof PhysicalResultSink)) { return Optional.empty(); } - PhysicalOneRowRelation physicalOneRowRelation - = (PhysicalOneRowRelation) ((PhysicalResultSink) physicalPlan).child(); - List columns = Lists.newArrayList(); - List data = Lists.newArrayList(); - for (int i = 0; i < physicalOneRowRelation.getProjects().size(); i++) { - NamedExpression item = physicalOneRowRelation.getProjects().get(i); - NamedExpression output = physicalPlan.getOutput().get(i); - Expression expr = item.child(0); - if (expr instanceof Literal) { - LiteralExpr legacyExpr = ((Literal) expr).toLegacyLiteral(); + + Optional sqlCacheContext = statementContext.getSqlCacheContext(); + boolean enableSqlCache + = CacheAnalyzer.canUseSqlCache(statementContext.getConnectContext().getSessionVariable()); + Plan child = physicalPlan.child(0); + if (child instanceof PhysicalOneRowRelation) { + PhysicalOneRowRelation physicalOneRowRelation = (PhysicalOneRowRelation) physicalPlan.child(0); + List columns = Lists.newArrayList(); + List data = Lists.newArrayList(); + for (int i = 0; i < physicalOneRowRelation.getProjects().size(); i++) { + NamedExpression item = physicalOneRowRelation.getProjects().get(i); + NamedExpression output = physicalPlan.getOutput().get(i); + Expression expr = item.child(0); + if (expr instanceof Literal) { + LiteralExpr legacyExpr = ((Literal) expr).toLegacyLiteral(); + columns.add(new Column(output.getName(), output.getDataType().toCatalogDataType())); + data.add(legacyExpr.getStringValueInFe()); + } else { + return Optional.empty(); + } + } + + ResultSetMetaData metadata = new CommonResultSet.CommonResultSetMetaData(columns); + ResultSet resultSet = new CommonResultSet(metadata, Collections.singletonList(data)); + if (sqlCacheContext.isPresent() && enableSqlCache) { + sqlCacheContext.get().setResultSetInFe(resultSet); + Env.getCurrentEnv().getSqlCacheManager().tryAddFeSqlCache( + statementContext.getConnectContext(), + statementContext.getOriginStatement().originStmt + ); + } + return Optional.of(resultSet); + } else if (child instanceof PhysicalEmptyRelation) { + List columns = Lists.newArrayList(); + PhysicalEmptyRelation physicalEmptyRelation = (PhysicalEmptyRelation) physicalPlan.child(0); + for (int i = 0; i < physicalEmptyRelation.getProjects().size(); i++) { + NamedExpression output = physicalPlan.getOutput().get(i); columns.add(new Column(output.getName(), output.getDataType().toCatalogDataType())); - data.add(legacyExpr.getStringValueInFe()); - } else { - return Optional.empty(); } + + ResultSetMetaData metadata = new CommonResultSet.CommonResultSetMetaData(columns); + ResultSet resultSet = new CommonResultSet(metadata, ImmutableList.of()); + if (sqlCacheContext.isPresent() && enableSqlCache) { + sqlCacheContext.get().setResultSetInFe(resultSet); + Env.getCurrentEnv().getSqlCacheManager().tryAddFeSqlCache( + statementContext.getConnectContext(), + statementContext.getOriginStatement().originStmt + ); + } + return Optional.of(resultSet); + } else { + return Optional.empty(); } - ResultSetMetaData metadata = new CommonResultSet.CommonResultSetMetaData(columns); - ResultSet resultSet = new CommonResultSet(metadata, Collections.singletonList(data)); - return Optional.of(resultSet); } @VisibleForTesting diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java index 6a7dbd2ed9de4f..f3fa61cecaad8b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java @@ -29,6 +29,7 @@ import org.apache.doris.nereids.trees.expressions.Variable; import org.apache.doris.nereids.util.Utils; import org.apache.doris.proto.Types.PUniqueId; +import org.apache.doris.qe.ResultSet; import org.apache.doris.qe.cache.CacheProxy; import org.apache.doris.thrift.TUniqueId; @@ -83,6 +84,7 @@ public class SqlCacheContext { private volatile List colLabels; private volatile PUniqueId cacheKeyMd5; + private volatile ResultSet resultSetInFe; public SqlCacheContext(UserIdentity userIdentity, TUniqueId queryId) { this.userIdentity = Objects.requireNonNull(userIdentity, "userIdentity cannot be null"); @@ -378,6 +380,14 @@ public void setOriginSql(String originSql) { this.originSql = originSql.trim(); } + public Optional getResultSetInFe() { + return Optional.ofNullable(resultSetInFe); + } + + public void setResultSetInFe(ResultSet resultSetInFe) { + this.resultSetInFe = resultSetInFe; + } + /** FullTableName */ @lombok.Data @lombok.AllArgsConstructor diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSqlCache.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSqlCache.java index c0c728d8fc9a86..663044d569fc6e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSqlCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSqlCache.java @@ -32,6 +32,7 @@ import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; import org.apache.doris.proto.InternalService; +import org.apache.doris.qe.ResultSet; import org.apache.doris.thrift.TUniqueId; import com.google.common.collect.ImmutableList; @@ -46,6 +47,7 @@ public class LogicalSqlCache extends LogicalLeaf implements SqlCache, TreeString private final TUniqueId queryId; private final List columnLabels; private final List resultExprs; + private final Optional resultSetInFe; private final List cacheValues; private final String backendAddress; private final String planBody; @@ -53,11 +55,13 @@ public class LogicalSqlCache extends LogicalLeaf implements SqlCache, TreeString /** LogicalSqlCache */ public LogicalSqlCache(TUniqueId queryId, List columnLabels, List resultExprs, - List cacheValues, String backendAddress, String planBody) { + Optional resultSetInFe, List cacheValues, + String backendAddress, String planBody) { super(PlanType.LOGICAL_SQL_CACHE, Optional.empty(), Optional.empty()); this.queryId = Objects.requireNonNull(queryId, "queryId can not be null"); this.columnLabels = Objects.requireNonNull(columnLabels, "columnLabels can not be null"); this.resultExprs = Objects.requireNonNull(resultExprs, "resultExprs can not be null"); + this.resultSetInFe = Objects.requireNonNull(resultSetInFe, "resultSetInFe can not be null"); this.cacheValues = Objects.requireNonNull(cacheValues, "cacheValues can not be null"); this.backendAddress = Objects.requireNonNull(backendAddress, "backendAddress can not be null"); this.planBody = Objects.requireNonNull(planBody, "planBody can not be null"); @@ -67,6 +71,10 @@ public TUniqueId getQueryId() { return queryId; } + public Optional getResultSetInFe() { + return resultSetInFe; + } + public List getCacheValues() { return cacheValues; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSqlCache.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSqlCache.java index 6e223a49cde915..824ca7e8924058 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSqlCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSqlCache.java @@ -32,6 +32,8 @@ import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; import org.apache.doris.proto.InternalService; +import org.apache.doris.proto.InternalService.PCacheValue; +import org.apache.doris.qe.ResultSet; import org.apache.doris.statistics.Statistics; import org.apache.doris.thrift.TUniqueId; @@ -46,6 +48,7 @@ public class PhysicalSqlCache extends PhysicalLeaf implements SqlCache, TreeStri private final TUniqueId queryId; private final List columnLabels; private final List resultExprs; + private final Optional resultSet; private final List cacheValues; private final String backendAddress; private final String planBody; @@ -53,12 +56,14 @@ public class PhysicalSqlCache extends PhysicalLeaf implements SqlCache, TreeStri /** PhysicalSqlCache */ public PhysicalSqlCache(TUniqueId queryId, List columnLabels, List resultExprs, - List cacheValues, String backendAddress, String planBody) { + Optional resultSet, List cacheValues, + String backendAddress, String planBody) { super(PlanType.PHYSICAL_SQL_CACHE, Optional.empty(), new LogicalProperties(() -> ImmutableList.of(), () -> FunctionalDependencies.EMPTY_FUNC_DEPS)); this.queryId = Objects.requireNonNull(queryId, "queryId can not be null"); this.columnLabels = Objects.requireNonNull(columnLabels, "colNames can not be null"); this.resultExprs = Objects.requireNonNull(resultExprs, "resultExprs can not be null"); + this.resultSet = Objects.requireNonNull(resultSet, "resultSet can not be null"); this.cacheValues = Objects.requireNonNull(cacheValues, "cacheValues can not be null"); this.backendAddress = Objects.requireNonNull(backendAddress, "backendAddress can not be null"); this.planBody = Objects.requireNonNull(planBody, "planBody can not be null"); @@ -68,6 +73,10 @@ public TUniqueId getQueryId() { return queryId; } + public Optional getResultSet() { + return resultSet; + } + public List getCacheValues() { return cacheValues; } @@ -90,9 +99,18 @@ public String getPlanBody() { @Override public String toString() { + long rowCount = 0; + if (resultSet.isPresent()) { + rowCount = resultSet.get().getResultRows().size(); + } else { + for (PCacheValue cacheValue : cacheValues) { + rowCount += cacheValue.getRowsCount(); + } + } return Utils.toSqlString("PhysicalSqlCache[" + id.asInt() + "]", "queryId", DebugUtil.printId(queryId), - "backend", backendAddress + "backend", backendAddress, + "rowCount", rowCount ); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java index 04ee42f0198a9f..031062fbd76795 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -1515,8 +1515,7 @@ private boolean sendCachedValues(MysqlChannel channel, List data = Lists.newArrayList(batch.getQueryStatistics() == null ? "0" : batch.getQueryStatistics().getReturnedRows() + ""); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheAnalyzer.java index 161661055a4981..85f37094b02725 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheAnalyzer.java @@ -215,7 +215,8 @@ public static boolean canUseSqlCache(SessionVariable sessionVariable) { } public static boolean commonCacheCondition(SessionVariable sessionVariable) { - return sessionVariable.getSqlSelectLimit() < 0 && sessionVariable.getDefaultOrderByLimit() < 0; + return sessionVariable.getSqlSelectLimit() < 0 && sessionVariable.getDefaultOrderByLimit() < 0 + && !sessionVariable.dryRunQuery; } /** diff --git a/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy index 99bed86beba250..c7759ec5f44919 100644 --- a/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy +++ b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy @@ -68,9 +68,14 @@ suite("parse_sql_from_sql_cache") { sql "select * from test_use_plan_cache2" assertHasCache "select * from test_use_plan_cache2" - // add empty partition can use cache + // NOTE: in cloud mode, add empty partition can not use cache, because the table version already update, + // but in native mode, add empty partition can use cache sql "alter table test_use_plan_cache2 add partition p6 values[('6'),('7'))" - assertHasCache "select * from test_use_plan_cache2" + if (isCloudMode()) { + assertNoCache "select * from test_use_plan_cache2" + } else { + assertHasCache "select * from test_use_plan_cache2" + } // insert data can not use cache sql "insert into test_use_plan_cache2 values(6, 1)" @@ -279,6 +284,13 @@ suite("parse_sql_from_sql_cache") { sql "create user test_cache_user1 identified by 'DORIS@2024'" def dbName = context.config.getDbNameByFile(context.file) sql """GRANT SELECT_PRIV ON *.* TO test_cache_user1""" + //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 test_cache_user1""" + } createTestTable "test_use_plan_cache12" @@ -317,6 +329,13 @@ suite("parse_sql_from_sql_cache") { sql "drop user if exists test_cache_user2" sql "create user test_cache_user2 identified by 'DORIS@2024'" sql """GRANT SELECT_PRIV ON *.* TO test_cache_user2""" + //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 test_cache_user2""" + } createTestTable "test_use_plan_cache13" @@ -368,6 +387,13 @@ suite("parse_sql_from_sql_cache") { sql "drop user if exists test_cache_user3" sql "create user test_cache_user3 identified by 'DORIS@2024'" sql """GRANT SELECT_PRIV ON *.* TO test_cache_user3""" + //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 test_cache_user3""" + } createTestTable "test_use_plan_cache14" @@ -426,6 +452,13 @@ suite("parse_sql_from_sql_cache") { sql "create user test_cache_user4 identified by 'DORIS@2024'" sql "GRANT SELECT_PRIV ON regression_test.* TO test_cache_user4" sql "GRANT SELECT_PRIV ON ${dbName}.test_use_plan_cache15 TO test_cache_user4" + //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 test_cache_user4""" + } extraThread("test_cache_user4_thread", { connect(user = "test_cache_user4", password="DORIS@2024") { @@ -604,6 +637,69 @@ suite("parse_sql_from_sql_cache") { sql "select * from test_use_plan_cache18" assertHasCache "select * from test_use_plan_cache18" } + }), + extraThread("test_dry_run_query", { + createTestTable "test_use_plan_cache19" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + sql "set dry_run_query=true" + assertNoCache "select * from test_use_plan_cache19 order by 1, 2" + def result1 = sql "select * from test_use_plan_cache19 order by 1, 2" + assertTrue(result1.size() == 1) + assertNoCache "select * from test_use_plan_cache19 order by 1, 2" + + sql "set dry_run_query=false" + assertNoCache "select * from test_use_plan_cache19 order by 1, 2" + def result2 = sql "select * from test_use_plan_cache19 order by 1, 2" + assertTrue(result2.size() > 1) + assertHasCache "select * from test_use_plan_cache19 order by 1, 2" + + sql "set dry_run_query=true" + assertNoCache "select * from test_use_plan_cache19 order by 1, 2" + def result3 = sql "select * from test_use_plan_cache19 order by 1, 2" + assertTrue(result3.size() == 1) + assertNoCache "select * from test_use_plan_cache19 order by 1, 2" + }), + extraThread("test_sql_cache_in_fe", { + createTestTable "test_use_plan_cache20" + + sql "alter table test_use_plan_cache20 add partition p6 values[('999'), ('1000'))" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from (select 100 as id)a" + def result1 = sql "select * from (select 100 as id)a" + assertTrue(result1.size() == 1) + + assertHasCache "select * from (select 100 as id)a" + def result2 = sql "select * from (select 100 as id)a" + assertTrue(result2.size() == 1) + + assertNoCache "select * from test_use_plan_cache20 limit 0" + def result3 = sql "select * from test_use_plan_cache20 limit 0" + assertTrue(result3.isEmpty()) + + assertHasCache "select * from test_use_plan_cache20 limit 0" + def result4 = sql "select * from test_use_plan_cache20 limit 0" + assertTrue(result4.isEmpty()) + + assertNoCache "select * from test_use_plan_cache20 where id=999" + def result5 = sql "select * from test_use_plan_cache20 where id=999" + assertTrue(result5.isEmpty()) + assertHasCache "select * from test_use_plan_cache20 where id=999" + def result6 = sql "select * from test_use_plan_cache20 where id=999" + assertTrue(result6.isEmpty()) }) ).get() } From 7691188e604e0fe980c7191732999c4f7f06cfa5 Mon Sep 17 00:00:00 2001 From: 924060929 <924060929@qq.com> Date: Thu, 18 Apr 2024 19:49:31 +0800 Subject: [PATCH 5/6] remove cloud mode --- .../cache/parse_sql_from_sql_cache.groovy | 36 +------------------ 1 file changed, 1 insertion(+), 35 deletions(-) diff --git a/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy index c7759ec5f44919..226ea7a9d6f3d8 100644 --- a/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy +++ b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy @@ -68,14 +68,8 @@ suite("parse_sql_from_sql_cache") { sql "select * from test_use_plan_cache2" assertHasCache "select * from test_use_plan_cache2" - // NOTE: in cloud mode, add empty partition can not use cache, because the table version already update, - // but in native mode, add empty partition can use cache sql "alter table test_use_plan_cache2 add partition p6 values[('6'),('7'))" - if (isCloudMode()) { - assertNoCache "select * from test_use_plan_cache2" - } else { - assertHasCache "select * from test_use_plan_cache2" - } + assertHasCache "select * from test_use_plan_cache2" // insert data can not use cache sql "insert into test_use_plan_cache2 values(6, 1)" @@ -284,13 +278,6 @@ suite("parse_sql_from_sql_cache") { sql "create user test_cache_user1 identified by 'DORIS@2024'" def dbName = context.config.getDbNameByFile(context.file) sql """GRANT SELECT_PRIV ON *.* TO test_cache_user1""" - //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 test_cache_user1""" - } createTestTable "test_use_plan_cache12" @@ -329,13 +316,6 @@ suite("parse_sql_from_sql_cache") { sql "drop user if exists test_cache_user2" sql "create user test_cache_user2 identified by 'DORIS@2024'" sql """GRANT SELECT_PRIV ON *.* TO test_cache_user2""" - //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 test_cache_user2""" - } createTestTable "test_use_plan_cache13" @@ -387,13 +367,6 @@ suite("parse_sql_from_sql_cache") { sql "drop user if exists test_cache_user3" sql "create user test_cache_user3 identified by 'DORIS@2024'" sql """GRANT SELECT_PRIV ON *.* TO test_cache_user3""" - //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 test_cache_user3""" - } createTestTable "test_use_plan_cache14" @@ -452,13 +425,6 @@ suite("parse_sql_from_sql_cache") { sql "create user test_cache_user4 identified by 'DORIS@2024'" sql "GRANT SELECT_PRIV ON regression_test.* TO test_cache_user4" sql "GRANT SELECT_PRIV ON ${dbName}.test_use_plan_cache15 TO test_cache_user4" - //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 test_cache_user4""" - } extraThread("test_cache_user4_thread", { connect(user = "test_cache_user4", password="DORIS@2024") { From 08a5832d18830b359e62b91fde83d01e118c8747 Mon Sep 17 00:00:00 2001 From: 924060929 <924060929@qq.com> Date: Fri, 19 Apr 2024 10:02:59 +0800 Subject: [PATCH 6/6] remove @NotNull --- .../main/groovy/org/apache/doris/regression/suite/Suite.groovy | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy index 2ffb7f8f6d4770..e08658c91cdf4d 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy @@ -41,7 +41,6 @@ import org.apache.doris.regression.util.JdbcUtils import org.apache.doris.regression.util.Hdfs import org.apache.doris.regression.util.SuiteUtils import org.apache.doris.regression.util.DebugPoint -import org.jetbrains.annotations.NotNull import org.junit.jupiter.api.Assertions import org.slf4j.Logger import org.slf4j.LoggerFactory @@ -171,7 +170,7 @@ class Suite implements GroovyInterceptable { String threadName = null, boolean daemon = false, Closure actionSupplier) { def executorService = Executors.newFixedThreadPool(1, new ThreadFactory() { @Override - Thread newThread(@NotNull Runnable r) { + Thread newThread(Runnable r) { def thread = new Thread(r, name) thread.setDaemon(daemon) return thread