From a3279a260441692cd0494c0eb8a93e5f97aa2945 Mon Sep 17 00:00:00 2001 From: Mryange Date: Thu, 19 Dec 2024 17:04:03 +0800 Subject: [PATCH 01/14] [refine](exchange) Use is_merge from FE for judgment instead of relying on the operator in BE. (#45592) ### What problem does this PR solve? Previously, determining whether the receiver is a merge exchange relied on checking if the specific operator was a sort node. However, this approach is incorrect because there are many types of sort operators: regular sort, partitioned sort, and spill sort. --- be/src/pipeline/exec/exchange_sink_operator.cpp | 10 ++-------- be/src/pipeline/exec/exchange_sink_operator.h | 4 +++- 2 files changed, 5 insertions(+), 9 deletions(-) diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index e7fed76be8fa16..cc789f6e25b20b 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -32,7 +32,6 @@ #include "pipeline/exec/operator.h" #include "pipeline/exec/sort_source_operator.h" #include "pipeline/local_exchange/local_exchange_sink_operator.h" -#include "pipeline/local_exchange/local_exchange_source_operator.h" #include "pipeline/pipeline_fragment_context.h" #include "util/runtime_profile.h" #include "util/uid_util.h" @@ -279,6 +278,7 @@ ExchangeSinkOperatorX::ExchangeSinkOperatorX( _tablet_sink_txn_id(sink.tablet_sink_txn_id), _t_tablet_sink_exprs(&sink.tablet_sink_exprs), _enable_local_merge_sort(state->enable_local_merge_sort()), + _dest_is_merge(sink.__isset.is_merge && sink.is_merge), _fragment_instance_ids(fragment_instance_ids) { DCHECK_GT(destinations.size(), 0); DCHECK(sink.output_partition.type == TPartitionType::UNPARTITIONED || @@ -571,19 +571,13 @@ std::shared_ptr ExchangeSinkOperatorX::_create_buffer( // Therefore, a shared sink buffer is used here to limit the number of concurrent RPCs. // (Note: This does not reduce the total number of RPCs.) // In a merge sort scenario, there are only n RPCs, so a shared sink buffer is not needed. -/// TODO: Modify this to let FE handle the judgment instead of BE. std::shared_ptr ExchangeSinkOperatorX::get_sink_buffer( InstanceLoId sender_ins_id) { - if (!_child) { - throw doris::Exception(ErrorCode::INTERNAL_ERROR, - "ExchangeSinkOperatorX did not correctly set the child."); - } // When the child is SortSourceOperatorX or LocalExchangeSourceOperatorX, // it is an order-by scenario. // In this case, there is only one target instance, and no n * n RPC concurrency will occur. // Therefore, sharing a sink buffer is not necessary. - if (std::dynamic_pointer_cast(_child) || - std::dynamic_pointer_cast(_child)) { + if (_dest_is_merge) { return _create_buffer({sender_ins_id}); } if (_state->enable_shared_exchange_sink_buffer()) { diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h index 85575beb9f7e47..3d6eeb4b39e94f 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.h +++ b/be/src/pipeline/exec/exchange_sink_operator.h @@ -205,7 +205,6 @@ class ExchangeSinkOperatorX final : public DataSinkOperatorX get_sink_buffer(InstanceLoId sender_ins_id); vectorized::VExprContextSPtrs& tablet_sink_expr_ctxs() { return _tablet_sink_expr_ctxs; } @@ -260,6 +259,9 @@ class ExchangeSinkOperatorX final : public DataSinkOperatorX& _fragment_instance_ids; }; From a3de177da0799b93c02a173c659800fde7f5278d Mon Sep 17 00:00:00 2001 From: bobhan1 Date: Thu, 19 Dec 2024 17:17:28 +0800 Subject: [PATCH 02/14] [fix](cloud) Adjust rowset state check in `CloudTablet::create_transient_rowset_writer` (#45496) https://github.com/apache/doris/pull/32257 checks if the current rowset state is `BEGIN_PARTIAL_UPDATE` in `CloudTablet::create_transient_rowset_writer`. But if this is a retry calculate task, the rowset's state may have been changed to `COMMITTED` in the first try. This PR adjust this check to avoid DCHECK fails. --- be/src/cloud/cloud_tablet.cpp | 23 ++-- .../test_cloud_mow_partial_update_retry.out | 16 +++ ...test_cloud_mow_partial_update_retry.groovy | 100 ++++++++++++++++++ 3 files changed, 131 insertions(+), 8 deletions(-) create mode 100644 regression-test/data/fault_injection_p0/cloud/test_cloud_mow_partial_update_retry.out create mode 100644 regression-test/suites/fault_injection_p0/cloud/test_cloud_mow_partial_update_retry.groovy diff --git a/be/src/cloud/cloud_tablet.cpp b/be/src/cloud/cloud_tablet.cpp index 93c7128756738c..c7d3170726b2d5 100644 --- a/be/src/cloud/cloud_tablet.cpp +++ b/be/src/cloud/cloud_tablet.cpp @@ -54,6 +54,7 @@ namespace doris { using namespace ErrorCode; static constexpr int COMPACTION_DELETE_BITMAP_LOCK_ID = -1; +static constexpr int LOAD_INITIATOR_ID = -1; CloudTablet::CloudTablet(CloudStorageEngine& engine, TabletMetaSharedPtr tablet_meta) : BaseTablet(std::move(tablet_meta)), _engine(engine) {} @@ -504,13 +505,19 @@ Result> CloudTablet::create_rowset_writer( Result> CloudTablet::create_transient_rowset_writer( const Rowset& rowset, std::shared_ptr partial_update_info, int64_t txn_expiration) { - if (rowset.rowset_meta()->rowset_state() != RowsetStatePB::BEGIN_PARTIAL_UPDATE) [[unlikely]] { - // May cause the segment files generated by the transient rowset writer unable to be - // recycled, see `CloudRowsetWriter::build` for detail. - LOG(WARNING) << "Wrong rowset state: " << rowset.rowset_meta()->rowset_state(); - DCHECK(false) << rowset.rowset_meta()->rowset_state(); + if (rowset.rowset_meta_state() != RowsetStatePB::BEGIN_PARTIAL_UPDATE && + rowset.rowset_meta_state() != RowsetStatePB::COMMITTED) [[unlikely]] { + auto msg = fmt::format( + "wrong rowset state when create_transient_rowset_writer, rowset state should be " + "BEGIN_PARTIAL_UPDATE or COMMITTED, but found {}, rowset_id={}, tablet_id={}", + RowsetStatePB_Name(rowset.rowset_meta_state()), rowset.rowset_id().to_string(), + tablet_id()); + // see `CloudRowsetWriter::build` for detail. + // if this is in a retry task, the rowset state may have been changed to RowsetStatePB::COMMITTED + // in `RowsetMeta::merge_rowset_meta()` in previous trials. + LOG(WARNING) << msg; + DCHECK(false) << msg; } - RowsetWriterContext context; context.rowset_state = PREPARED; context.segments_overlap = OVERLAPPING; @@ -719,8 +726,8 @@ Status CloudTablet::save_delete_bitmap(const TabletTxnInfo* txn_info, int64_t tx } auto ms_lock_id = lock_id == -1 ? txn_id : lock_id; - RETURN_IF_ERROR(_engine.meta_mgr().update_delete_bitmap( - *this, ms_lock_id, COMPACTION_DELETE_BITMAP_LOCK_ID, new_delete_bitmap.get())); + RETURN_IF_ERROR(_engine.meta_mgr().update_delete_bitmap(*this, ms_lock_id, LOAD_INITIATOR_ID, + new_delete_bitmap.get())); // store the delete bitmap with sentinel marks in txn_delete_bitmap_cache because if the txn is retried for some reason, // it will use the delete bitmap from txn_delete_bitmap_cache when re-calculating the delete bitmap, during which it will do diff --git a/regression-test/data/fault_injection_p0/cloud/test_cloud_mow_partial_update_retry.out b/regression-test/data/fault_injection_p0/cloud/test_cloud_mow_partial_update_retry.out new file mode 100644 index 00000000000000..3b24419bdc6fc1 --- /dev/null +++ b/regression-test/data/fault_injection_p0/cloud/test_cloud_mow_partial_update_retry.out @@ -0,0 +1,16 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +1 1 1 1 +2 2 2 2 +3 3 3 2 + +-- !sql -- +1 1 888 1 +2 2 777 2 +3 3 3 2 + +-- !sql -- +1 999 888 1 +2 666 777 2 +3 3 3 2 + diff --git a/regression-test/suites/fault_injection_p0/cloud/test_cloud_mow_partial_update_retry.groovy b/regression-test/suites/fault_injection_p0/cloud/test_cloud_mow_partial_update_retry.groovy new file mode 100644 index 00000000000000..13abaf1ffcabd3 --- /dev/null +++ b/regression-test/suites/fault_injection_p0/cloud/test_cloud_mow_partial_update_retry.groovy @@ -0,0 +1,100 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_cloud_mow_partial_update_retry", "nonConcurrent") { + if (!isCloudMode()) { + return + } + + GetDebugPoint().clearDebugPointsForAllFEs() + GetDebugPoint().clearDebugPointsForAllBEs() + + def customFeConfig = [ + delete_bitmap_lock_expiration_seconds : 10, + calculate_delete_bitmap_task_timeout_seconds : 15, + ] + + setFeConfigTemporary(customFeConfig) { + + def table1 = "test_cloud_mow_partial_update_retry" + sql "DROP TABLE IF EXISTS ${table1} FORCE;" + sql """ CREATE TABLE IF NOT EXISTS ${table1} ( + `k1` int NOT NULL, + `c1` int, + `c2` int, + `c3` int + )UNIQUE KEY(k1) + DISTRIBUTED BY HASH(k1) BUCKETS 1 + PROPERTIES ( + "enable_unique_key_merge_on_write" = "true", + "disable_auto_compaction" = "true", + "replication_num" = "1"); """ + + sql "insert into ${table1} values(1,1,1,1);" + sql "insert into ${table1} values(2,2,2,2);" + sql "insert into ${table1} values(3,3,3,2);" + sql "sync;" + qt_sql "select * from ${table1} order by k1;" + + try { + // block the first load + GetDebugPoint().enableDebugPointForAllBEs("BaseTablet::update_delete_bitmap.enable_spin_wait", [token: "token1"]) + GetDebugPoint().enableDebugPointForAllBEs("BaseTablet::update_delete_bitmap.block", [wait_token: "token1"]) + + // the first load + t1 = Thread.start { + sql "set enable_unique_key_partial_update=true;" + sql "sync;" + sql "insert into ${table1}(k1,c1) values(1,999),(2,666);" + } + + // wait util the first partial update load's delete bitmap update lock expired + // to ensure that the second load can take the delete bitmap update lock + // Config.delete_bitmap_lock_expiration_seconds = 10s + Thread.sleep(11 * 1000) + + // the second load + GetDebugPoint().enableDebugPointForAllBEs("BaseTablet::update_delete_bitmap.enable_spin_wait", [token: "token2"]) + Thread.sleep(200) + + sql "set enable_unique_key_partial_update=true;" + sql "sync;" + sql "insert into ${table1}(k1,c2) values(1,888),(2,777);" + + qt_sql "select * from ${table1} order by k1;" + + + // keep waiting util the delete bitmap calculation timeout(Config.calculate_delete_bitmap_task_timeout_seconds = 15s) + // and the first load will retry the calculation of delete bitmap + Thread.sleep(15 * 1000) + + // let the first partial update load finish + GetDebugPoint().enableDebugPointForAllBEs("BaseTablet::update_delete_bitmap.block") + t1.join() + + Thread.sleep(1000) + + qt_sql "select * from ${table1} order by k1;" + + } catch(Exception e) { + logger.info(e.getMessage()) + throw e + } finally { + GetDebugPoint().clearDebugPointsForAllBEs() + } + } +} From 0e2de34bdc8e55382526660cdc9d1e8eed9b8033 Mon Sep 17 00:00:00 2001 From: Sridhar R Manikarnike Date: Thu, 19 Dec 2024 15:00:55 +0530 Subject: [PATCH 03/14] [Enhancement] (nereids)implement DropCatalogCommand in nereids (#45372) Issue Number: close #42613 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../apache/doris/datasource/CatalogMgr.java | 22 ++++-- .../nereids/parser/LogicalPlanBuilder.java | 9 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/DropCatalogCommand.java | 77 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../nereids_p0/test_drop_catalog_command.out | 4 + .../test_drop_catalog_command.groovy | 43 +++++++++++ 8 files changed, 155 insertions(+), 8 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropCatalogCommand.java create mode 100644 regression-test/data/nereids_p0/test_drop_catalog_command.out create mode 100644 regression-test/suites/nereids_p0/test_drop_catalog_command.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 0cd32f3820fda4..b2e3eca37e006b 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -220,6 +220,7 @@ supportedDropStatement | DROP SQL_BLOCK_RULE (IF EXISTS)? identifierSeq #dropSqlBlockRule | DROP USER (IF EXISTS)? userIdentify #dropUser | DROP WORKLOAD GROUP (IF EXISTS)? name=identifierOrText #dropWorkloadGroup + | DROP CATALOG (IF EXISTS)? name=identifier #dropCatalog | DROP FILE name=STRING_LITERAL ((FROM | IN) database=identifier)? properties=propertyClause #dropFile | DROP WORKLOAD POLICY (IF EXISTS)? name=identifierOrText #dropWorkloadPolicy @@ -689,7 +690,6 @@ fromRollup unsupportedDropStatement : DROP (DATABASE | SCHEMA) (IF EXISTS)? name=multipartIdentifier FORCE? #dropDatabase - | DROP CATALOG (IF EXISTS)? name=identifier #dropCatalog | DROP (GLOBAL | SESSION | LOCAL)? FUNCTION (IF EXISTS)? functionIdentifier LEFT_PAREN functionArguments? RIGHT_PAREN #dropFunction | DROP TABLE (IF EXISTS)? name=multipartIdentifier FORCE? #dropTable diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java index 0203aa7020b090..f90a2a32fdc3ec 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java @@ -275,23 +275,24 @@ public void createCatalog(CreateCatalogStmt stmt) throws UserException { /** * Remove the catalog instance by name and write the meta log. */ - public void dropCatalog(DropCatalogStmt stmt) throws UserException { + public void dropCatalog(String catalogName, boolean ifExists) throws UserException { writeLock(); try { - if (stmt.isSetIfExists() && !nameToCatalog.containsKey(stmt.getCatalogName())) { - LOG.warn("Non catalog {} is found.", stmt.getCatalogName()); + if (ifExists && !nameToCatalog.containsKey(catalogName)) { + LOG.warn("Non catalog {} is found.", catalogName); return; } - CatalogIf> catalog = nameToCatalog.get(stmt.getCatalogName()); + CatalogIf> catalog = nameToCatalog.get(catalogName); if (catalog == null) { - throw new DdlException("No catalog found with name: " + stmt.getCatalogName()); + throw new DdlException("No catalog found with name: " + catalogName); } - CatalogLog log = CatalogFactory.createCatalogLog(catalog.getId(), stmt); + CatalogLog log = new CatalogLog(); + log.setCatalogId(catalog.getId()); replayDropCatalog(log); Env.getCurrentEnv().getEditLog().logCatalogLog(OperationType.OP_DROP_CATALOG, log); if (ConnectContext.get() != null) { - ConnectContext.get().removeLastDBOfCatalog(stmt.getCatalogName()); + ConnectContext.get().removeLastDBOfCatalog(catalogName); } Env.getCurrentEnv().getQueryStats().clear(catalog.getId()); } finally { @@ -299,6 +300,13 @@ public void dropCatalog(DropCatalogStmt stmt) throws UserException { } } + /** + * Remove the catalog instance by name and write the meta log. + */ + public void dropCatalog(DropCatalogStmt stmt) throws UserException { + dropCatalog(stmt.getCatalogName(), stmt.isSetIfExists()); + } + /** * Modify the catalog name into a new one and write the meta log. */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 68226f156c7f25..3b570fff8e7584 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -120,6 +120,7 @@ import org.apache.doris.nereids.DorisParser.DecimalLiteralContext; import org.apache.doris.nereids.DorisParser.DeleteContext; import org.apache.doris.nereids.DorisParser.DereferenceContext; +import org.apache.doris.nereids.DorisParser.DropCatalogContext; import org.apache.doris.nereids.DorisParser.DropCatalogRecycleBinContext; import org.apache.doris.nereids.DorisParser.DropConstraintContext; import org.apache.doris.nereids.DorisParser.DropEncryptkeyContext; @@ -526,6 +527,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreateWorkloadGroupCommand; import org.apache.doris.nereids.trees.plans.commands.DeleteFromCommand; import org.apache.doris.nereids.trees.plans.commands.DeleteFromUsingCommand; +import org.apache.doris.nereids.trees.plans.commands.DropCatalogCommand; import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand; import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand.IdType; import org.apache.doris.nereids.trees.plans.commands.DropConstraintCommand; @@ -4969,6 +4971,13 @@ public LogicalPlan visitDropRole(DropRoleContext ctx) { return new DropRoleCommand(ctx.name.getText(), ctx.EXISTS() != null); } + @Override + public LogicalPlan visitDropCatalog(DropCatalogContext ctx) { + String catalogName = stripQuotes(ctx.name.getText()); + boolean ifExists = ctx.EXISTS() != null; + return new DropCatalogCommand(catalogName, ifExists); + } + @Override public LogicalPlan visitCreateEncryptkey(CreateEncryptkeyContext ctx) { List nameParts = visitMultipartIdentifier(ctx.multipartIdentifier()); 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 6395f429db29de..f58a6bf139d2fe 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 @@ -149,6 +149,7 @@ public enum PlanType { CREATE_JOB_COMMAND, PAUSE_JOB_COMMAND, CANCEL_JOB_COMMAND, + DROP_CATALOG_COMMAND, DROP_JOB_COMMAND, RESUME_JOB_COMMAND, ALTER_MTMV_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropCatalogCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropCatalogCommand.java new file mode 100644 index 00000000000000..034ecb1053a5f6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropCatalogCommand.java @@ -0,0 +1,77 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.util.Util; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import java.util.Objects; + +/** + * Command for DROP CATALOG. + */ +public class DropCatalogCommand extends DropCommand { + private final String catalogName; + private final boolean ifExists; + + public DropCatalogCommand(String catalogName, boolean ifExists) { + super(PlanType.DROP_CATALOG_COMMAND); + this.catalogName = Objects.requireNonNull(catalogName, "Catalog name cannot be null"); + this.ifExists = ifExists; + } + + @Override + public void doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + // Validate the catalog name + Util.checkCatalogAllRules(catalogName); + + if (catalogName.equals(InternalCatalog.INTERNAL_CATALOG_NAME)) { + throw new AnalysisException("Internal catalog can't be drop."); + } + + if (!Env.getCurrentEnv().getAccessManager().checkCtlPriv( + ConnectContext.get(), catalogName, PrivPredicate.DROP)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_CATALOG_ACCESS_DENIED, + ConnectContext.get().getQualifiedUser(), catalogName); + } + + Env.getCurrentEnv().getCatalogMgr().dropCatalog(catalogName, ifExists); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitDropCatalogCommand(this, context); + } + + public String getCatalogName() { + return catalogName; + } + + public boolean isIfExists() { + return ifExists; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index a9340894c33590..9c2839b3784093 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -56,6 +56,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreateWorkloadGroupCommand; import org.apache.doris.nereids.trees.plans.commands.DeleteFromCommand; import org.apache.doris.nereids.trees.plans.commands.DeleteFromUsingCommand; +import org.apache.doris.nereids.trees.plans.commands.DropCatalogCommand; import org.apache.doris.nereids.trees.plans.commands.DropCatalogRecycleBinCommand; import org.apache.doris.nereids.trees.plans.commands.DropConstraintCommand; import org.apache.doris.nereids.trees.plans.commands.DropEncryptkeyCommand; @@ -345,6 +346,10 @@ default R visitAlterViewCommand(AlterViewCommand alterViewCommand, C context) { return visitCommand(alterViewCommand, context); } + default R visitDropCatalogCommand(DropCatalogCommand dropCatalogCommand, C context) { + return visitCommand(dropCatalogCommand, context); + } + default R visitAlterCatalogCommentCommand(AlterCatalogCommentCommand alterCatalogCommentCommand, C context) { return visitCommand(alterCatalogCommentCommand, context); } diff --git a/regression-test/data/nereids_p0/test_drop_catalog_command.out b/regression-test/data/nereids_p0/test_drop_catalog_command.out new file mode 100644 index 00000000000000..ddb84e0eb356f7 --- /dev/null +++ b/regression-test/data/nereids_p0/test_drop_catalog_command.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !cmd -- +test_drop_catalog \nCREATE CATALOG `test_drop_catalog` PROPERTIES (\n"type" = "es",\n"hosts" = "http://127.0.0.1:9200"\n); + diff --git a/regression-test/suites/nereids_p0/test_drop_catalog_command.groovy b/regression-test/suites/nereids_p0/test_drop_catalog_command.groovy new file mode 100644 index 00000000000000..a936e52208ea44 --- /dev/null +++ b/regression-test/suites/nereids_p0/test_drop_catalog_command.groovy @@ -0,0 +1,43 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_drop_catalog_command", "nereids_p0") { + def catalogName = "test_drop_catalog" + def catalogProperties = "\"type\"=\"es\", \"hosts\"=\"http://127.0.0.1:9200\"" + + try { + // Drop catalog if it already exists + checkNereidsExecute("DROP CATALOG IF EXISTS ${catalogName}") + + // Create a new catalog + sql(""" + CREATE CATALOG ${catalogName} + PROPERTIES (${catalogProperties}) + """) + + // Verify the catalog was created + checkNereidsExecute("""SHOW CREATE CATALOG ${catalogName}""") + qt_cmd("""SHOW CREATE CATALOG ${catalogName}""") + + // Drop the catalog + checkNereidsExecute("DROP CATALOG ${catalogName}") + } finally { + // Ensure cleanup + checkNereidsExecute("DROP CATALOG IF EXISTS ${catalogName}") + } +} + From a9de07b9ab2226b578821190e846ca4212fc7879 Mon Sep 17 00:00:00 2001 From: Sridhar R Manikarnike Date: Thu, 19 Dec 2024 15:01:22 +0530 Subject: [PATCH 04/14] [Enhancement] (nereids)implement showDataTypesCommand in nereids (#44299) Issue Number: close #42743 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 7 ++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/ShowDataTypesCommand.java | 102 ++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../nereids_p0/show/test_show_data_types.out | 31 ++++++ .../show/test_show_data_types.groovy | 29 +++++ 7 files changed, 176 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDataTypesCommand.java create mode 100644 regression-test/data/nereids_p0/show/test_show_data_types.out create mode 100644 regression-test/suites/nereids_p0/show/test_show_data_types.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index b2e3eca37e006b..93bf6050970930 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -262,6 +262,7 @@ supportedShowStatement | SHOW COLLATION wildWhere? #showCollation | SHOW SQL_BLOCK_RULE (FOR ruleName=identifier)? #showSqlBlockRule | SHOW CREATE VIEW name=multipartIdentifier #showCreateView + | SHOW DATA TYPES #showDataTypes | SHOW CREATE MATERIALIZED VIEW mvName=identifier ON tableName=multipartIdentifier #showCreateMaterializedView | SHOW (WARNINGS | ERRORS) limitClause? #showWarningErrors @@ -330,7 +331,6 @@ unsupportedShowStatement LEFT_PAREN functionArguments? RIGHT_PAREN ((FROM | IN) database=multipartIdentifier)? #showCreateFunction | SHOW (DATABASES | SCHEMAS) (FROM catalog=identifier)? wildWhere? #showDatabases - | SHOW DATA TYPES #showDataTypes | SHOW CATALOGS wildWhere? #showCatalogs | SHOW CATALOG name=identifier #showCatalog | SHOW FULL? (COLUMNS | FIELDS) (FROM | IN) tableName=multipartIdentifier diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 3b570fff8e7584..34f760ff4f524e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -257,6 +257,7 @@ import org.apache.doris.nereids.DorisParser.ShowCreateTableContext; import org.apache.doris.nereids.DorisParser.ShowCreateViewContext; import org.apache.doris.nereids.DorisParser.ShowDataSkewContext; +import org.apache.doris.nereids.DorisParser.ShowDataTypesContext; import org.apache.doris.nereids.DorisParser.ShowDatabaseIdContext; import org.apache.doris.nereids.DorisParser.ShowDeleteContext; import org.apache.doris.nereids.DorisParser.ShowDiagnoseTabletContext; @@ -575,6 +576,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowCreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDataSkewCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowDataTypesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDatabaseIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDeleteCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDiagnoseTabletCommand; @@ -4498,6 +4500,11 @@ public LogicalPlan visitShowLoadProfile(ShowLoadProfileContext ctx) { return new ShowLoadProfileCommand(ctx.loadIdPath.getText()); } + @Override + public LogicalPlan visitShowDataTypes(ShowDataTypesContext ctx) { + return new ShowDataTypesCommand(); + } + @Override public LogicalPlan visitShowGrants(ShowGrantsContext ctx) { boolean all = (ctx.ALL() != null) ? true : false; 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 f58a6bf139d2fe..8eeac54a853e0f 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 @@ -224,6 +224,7 @@ public enum PlanType { SHOW_DYNAMIC_PARTITION_COMMAND, SHOW_ENCRYPT_KEYS_COMMAND, SHOW_EVENTS_COMMAND, + SHOW_DATA_TYPES_COMMAND, SHOW_FRONTENDS_COMMAND, SHOW_GRANTS_COMMAND, SHOW_LAST_INSERT_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDataTypesCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDataTypesCommand.java new file mode 100644 index 00000000000000..6ce9b781bd37f1 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDataTypesCommand.java @@ -0,0 +1,102 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.collect.Lists; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; + +/** + * Represents the command for SHOW DATA TYPES. + */ +public class ShowDataTypesCommand extends ShowCommand { + private static final ShowResultSetMetaData META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("TypeName", ScalarType.createVarchar(20))) + .addColumn(new Column("Size", ScalarType.createVarchar(100))) + .build(); + + public ShowDataTypesCommand() { + super(PlanType.SHOW_DATA_TYPES_COMMAND); + } + + /** + * getTypes(). + */ + public static ArrayList getTypes() { + return PrimitiveType.getSupportedTypes(); + } + + /** + * getTypesAvailableInDdl(). + */ + public static List> getTypesAvailableInDdl() { + ArrayList supportedTypes = getTypes(); + List> rows = Lists.newArrayList(); + for (PrimitiveType type : supportedTypes) { + List row = new ArrayList<>(); + if (type.isAvailableInDdl()) { + row.add(type.toString()); + row.add(Integer.toString(type.getSlotSize())); + rows.add(row); + } + } + return rows; + } + + /** + * sortMetaData(). + */ + public void sortMetaData(List> rows) { + Collections.sort(rows, new Comparator>() { + @Override + public int compare(List row1, List row2) { + return row1.get(0).compareTo(row2.get(0)); + } + }); + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + List> rows = getTypesAvailableInDdl(); + sortMetaData(rows); + return new ShowResultSet(getMetaData(), rows); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowDataTypesCommand(this, context); + } + + public ShowResultSetMetaData getMetaData() { + return META_DATA; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 9c2839b3784093..cce1f41e071531 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -102,6 +102,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowCreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.ShowCreateViewCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDataSkewCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowDataTypesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDatabaseIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDeleteCommand; import org.apache.doris.nereids.trees.plans.commands.ShowDiagnoseTabletCommand; @@ -521,6 +522,10 @@ default R visitCleanAllProfileCommand(CleanAllProfileCommand cleanAllProfileComm return visitCommand(cleanAllProfileCommand, context); } + default R visitShowDataTypesCommand(ShowDataTypesCommand showDataTypesCommand, C context) { + return visitCommand(showDataTypesCommand, context); + } + default R visitShowFrontendsCommand(ShowFrontendsCommand showFrontendsCommand, C context) { return visitCommand(showFrontendsCommand, context); } diff --git a/regression-test/data/nereids_p0/show/test_show_data_types.out b/regression-test/data/nereids_p0/show/test_show_data_types.out new file mode 100644 index 00000000000000..de1d757cbf80cd --- /dev/null +++ b/regression-test/data/nereids_p0/show/test_show_data_types.out @@ -0,0 +1,31 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !cmd -- +AGG_STATE 16 +ARRAY 32 +BIGINT 8 +BITMAP 16 +BOOLEAN 1 +CHAR 16 +DATE 16 +DATETIME 16 +DATETIMEV2 8 +DATEV2 4 +DECIMAL128 16 +DECIMAL32 4 +DECIMAL64 8 +DECIMALV2 16 +DOUBLE 8 +FLOAT 4 +HLL 16 +INT 4 +IPV4 4 +IPV6 16 +JSON 16 +LARGEINT 16 +MAP 24 +QUANTILE_STATE 16 +SMALLINT 2 +STRING 16 +TINYINT 1 +VARCHAR 16 + diff --git a/regression-test/suites/nereids_p0/show/test_show_data_types.groovy b/regression-test/suites/nereids_p0/show/test_show_data_types.groovy new file mode 100644 index 00000000000000..4316fd5545f47f --- /dev/null +++ b/regression-test/suites/nereids_p0/show/test_show_data_types.groovy @@ -0,0 +1,29 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_show_data_types_nereids", "query,datatype") { + try { + // Execute the SHOW DATA TYPES command and verify the output + checkNereidsExecute("SHOW DATA TYPES") + qt_cmd("SHOW DATA TYPES") + } catch (Exception e) { + // Log any exceptions that occur during testing + log.error("Failed to execute SHOW DATA TYPES command", e) + throw e + } +} + From ea6958cb40e225dd3704327964436432696497eb Mon Sep 17 00:00:00 2001 From: Sridhar R Manikarnike Date: Thu, 19 Dec 2024 15:01:40 +0530 Subject: [PATCH 05/14] [Enhancement](nereids)implement showStatusCommand in nereids (#45427) Issue Number: close #42730 --- .../org/apache/doris/nereids/DorisParser.g4 | 2 +- .../nereids/parser/LogicalPlanBuilder.java | 16 +++++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../plans/commands/ShowStatusCommand.java | 61 +++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 ++ .../show/test_show_status_command.out | 7 +++ .../show/test_show_status_command.groovy | 31 ++++++++++ 7 files changed, 122 insertions(+), 1 deletion(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowStatusCommand.java create mode 100644 regression-test/data/nereids_p0/show/test_show_status_command.out create mode 100644 regression-test/suites/nereids_p0/show/test_show_status_command.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 93bf6050970930..97876c231fec69 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -274,6 +274,7 @@ supportedShowStatement | SHOW DATABASE databaseId=INTEGER_VALUE #showDatabaseId | SHOW TABLE tableId=INTEGER_VALUE #showTableId | SHOW TRASH (ON backend=STRING_LITERAL)? #showTrash + | SHOW (GLOBAL | SESSION | LOCAL)? STATUS #showStatus | SHOW WHITELIST #showWhitelist | SHOW TABLETS BELONG tabletIds+=INTEGER_VALUE (COMMA tabletIds+=INTEGER_VALUE)* #showTabletsBelong @@ -325,7 +326,6 @@ unsupportedShowStatement | SHOW TABLE STATUS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTableStatus | SHOW FULL? TABLES ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTables | SHOW FULL? VIEWS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showViews - | SHOW (GLOBAL | SESSION | LOCAL)? STATUS wildWhere? #showStatus | SHOW CREATE MATERIALIZED VIEW name=multipartIdentifier #showMaterializedView | SHOW CREATE (GLOBAL | SESSION | LOCAL)? FUNCTION functionIdentifier LEFT_PAREN functionArguments? RIGHT_PAREN diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 34f760ff4f524e..0332123f9ff584 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -281,6 +281,7 @@ import org.apache.doris.nereids.DorisParser.ShowRolesContext; import org.apache.doris.nereids.DorisParser.ShowSmallFilesContext; import org.apache.doris.nereids.DorisParser.ShowSqlBlockRuleContext; +import org.apache.doris.nereids.DorisParser.ShowStatusContext; import org.apache.doris.nereids.DorisParser.ShowStorageEnginesContext; import org.apache.doris.nereids.DorisParser.ShowSyncJobContext; import org.apache.doris.nereids.DorisParser.ShowTableCreationContext; @@ -599,6 +600,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowRolesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSmallFilesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSqlBlockRuleCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowStatusCommand; import org.apache.doris.nereids.trees.plans.commands.ShowStorageEnginesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSyncJobCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableCreationCommand; @@ -5124,6 +5126,20 @@ public LogicalPlan visitAdminCheckTablets(AdminCheckTabletsContext ctx) { return new AdminCheckTabletsCommand(tabletIdLists, properties); } + @Override + public LogicalPlan visitShowStatus(ShowStatusContext ctx) { + String scope = null; + if (ctx.GLOBAL() != null) { + scope = "GLOBAL"; + } else if (ctx.SESSION() != null) { + scope = "SESSION"; + } else if (ctx.LOCAL() != null) { + scope = "LOCAL"; + } + + return new ShowStatusCommand(scope); + } + @Override public LogicalPlan visitShowDataSkew(ShowDataSkewContext ctx) { TableRefInfo tableRefInfo = visitBaseTableRefContext(ctx.baseTableRef()); 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 8eeac54a853e0f..dfc129f10b0fd6 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 @@ -239,6 +239,7 @@ public enum PlanType { SHOW_REPOSITORIES_COMMAND, SHOW_ROLE_COMMAND, SHOW_SMALL_FILES_COMMAND, + SHOW_STATUS_COMMAND, SHOW_STORAGE_ENGINES_COMMAND, SHOW_SYNC_JOB_COMMAND, SHOW_TABLE_ID_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowStatusCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowStatusCommand.java new file mode 100644 index 00000000000000..3ae5643e068c9a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowStatusCommand.java @@ -0,0 +1,61 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * Command for SHOW STATUS. + */ +public class ShowStatusCommand extends ShowCommand { + private static final ShowResultSetMetaData META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("Variable_name", ScalarType.createVarchar(64))) + .addColumn(new Column("Value", ScalarType.createVarchar(64))) + .build(); + + private final String scope; + + public ShowStatusCommand(String scope) { + super(PlanType.SHOW_STATUS_COMMAND); + this.scope = scope; + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + List> rows = Lists.newArrayList(); + return new ShowResultSet(META_DATA, rows); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowStatusCommand(this, context); + } +} + diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index cce1f41e071531..d3749e94d57d0f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -125,6 +125,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowRolesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSmallFilesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSqlBlockRuleCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowStatusCommand; import org.apache.doris.nereids.trees.plans.commands.ShowStorageEnginesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSyncJobCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableCreationCommand; @@ -416,6 +417,10 @@ default R visitShowGrantsCommand(ShowGrantsCommand showGrantsCommand, C context) return visitCommand(showGrantsCommand, context); } + default R visitShowStatusCommand(ShowStatusCommand showStatusCommand, C context) { + return visitCommand(showStatusCommand, context); + } + default R visitShowPartitionIdCommand(ShowPartitionIdCommand showPartitionIdCommand, C context) { return visitCommand(showPartitionIdCommand, context); } diff --git a/regression-test/data/nereids_p0/show/test_show_status_command.out b/regression-test/data/nereids_p0/show/test_show_status_command.out new file mode 100644 index 00000000000000..0fbf8d052eda80 --- /dev/null +++ b/regression-test/data/nereids_p0/show/test_show_status_command.out @@ -0,0 +1,7 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !cmd -- + +-- !cmd -- + +-- !cmd -- + diff --git a/regression-test/suites/nereids_p0/show/test_show_status_command.groovy b/regression-test/suites/nereids_p0/show/test_show_status_command.groovy new file mode 100644 index 00000000000000..1b611268057111 --- /dev/null +++ b/regression-test/suites/nereids_p0/show/test_show_status_command.groovy @@ -0,0 +1,31 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_show_status_command", "nereids_p0") { + // Verify SESSION status + checkNereidsExecute("SHOW SESSION STATUS") + qt_cmd("SHOW SESSION STATUS") + + // Verify GLOBAL status + checkNereidsExecute("SHOW GLOBAL STATUS") + qt_cmd("SHOW GLOBAL STATUS") + + // Verify default STATUS (SESSION) + checkNereidsExecute("SHOW STATUS") + qt_cmd("SHOW STATUS") +} + From 549abf4a56e047bbba8dd9c75fa5cc46e67d9ea0 Mon Sep 17 00:00:00 2001 From: Calvin Kirs Date: Thu, 19 Dec 2024 17:37:33 +0800 Subject: [PATCH 06/14] =?UTF-8?q?[Fix](Job)Fix=20redundant=20job=20schedul?= =?UTF-8?q?ing=20by=20preventing=20same=20state=20transitions=20(e.g.,=20R?= =?UTF-8?q?UNNING=20=E2=86=92=20RUNNING)=20(#45495)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What problem does this PR solve? In the current job scheduling logic, invalid state transitions (e.g., RUNNING to RUNNING) are not filtered, which causes redundant scheduling during resume operations. This PR adds a check to ensure that jobs cannot transition to the same state, preventing duplicate scheduling triggers and improving state consistency. --- .../java/org/apache/doris/job/manager/JobManager.java | 6 +++++- .../suites/job_p0/test_base_insert_job.groovy | 9 ++++++--- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java index 2a957775e113b8..ac9f15b9d67fd3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java @@ -201,9 +201,13 @@ public void alterJobStatus(String jobName, JobStatus jobStatus) throws JobExcept for (T a : jobMap.values()) { if (a.getJobName().equals(jobName)) { try { + if (jobStatus.equals(a.getJobStatus())) { + throw new JobException("Can't change job status to the same status"); + } alterJobStatus(a.getJobId(), jobStatus); } catch (JobException e) { - throw new JobException("unregister job error, jobName:" + jobName); + throw new JobException("Alter job status error, jobName is %s, errorMsg is %s", + jobName, e.getMessage()); } } } diff --git a/regression-test/suites/job_p0/test_base_insert_job.groovy b/regression-test/suites/job_p0/test_base_insert_job.groovy index 33ae28443b290a..1703b355c95019 100644 --- a/regression-test/suites/job_p0/test_base_insert_job.groovy +++ b/regression-test/suites/job_p0/test_base_insert_job.groovy @@ -190,6 +190,11 @@ suite("test_base_insert_job") { // check job status and succeed task count is 1 pressJob.size() == 1 && '1' == onceJob.get(0).get(0) }) + assertThrows(Exception) { + sql """ + RESUME JOB where jobName='press' + """ + } sql """ DROP JOB IF EXISTS where jobname = 'past_start_time' @@ -299,12 +304,10 @@ suite("test_base_insert_job") { assert e.getMessage().contains("Invalid interval time unit: years") } // assert interval time unit is -1 - try { + assertThrows(Exception) { sql """ CREATE JOB test_error_starts ON SCHEDULE every -1 second comment 'test' DO insert into ${tableName} (timestamp, type, user_id) values ('2023-03-18','1','12213'); """ - } catch (Exception e) { - assert e.getMessage().contains("expecting INTEGER_VALUE") } // test keyword as job name From 40c6c61dbbe67b3c11095427b3c5e471dfee0042 Mon Sep 17 00:00:00 2001 From: 924060929 Date: Thu, 19 Dec 2024 17:39:25 +0800 Subject: [PATCH 07/14] [fix](sql cache) fix prepare statement with sql cache throw NullPointerException (#45640) fix prepare statement with sql cache throw NullPointerException: ```shell java.lang.NullPointerException: Cannot read field "originStmt" because the return value of "org.apache.doris.analysis.StatementBase.getOrigStmt()" is null ``` --- .../org/apache/doris/qe/StmtExecutor.java | 3 +- .../cache/prepare_stmt_with_sql_cache.groovy | 57 +++++++++++++++++++ 2 files changed, 59 insertions(+), 1 deletion(-) create mode 100644 regression-test/suites/nereids_p0/cache/prepare_stmt_with_sql_cache.groovy 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 05df53ed6796cb..5c2566225fe50a 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 @@ -1894,7 +1894,8 @@ private void handleQueryStmt() throws Exception { // TODO support arrow flight sql // NOTE: If you want to add another condition about SessionVariable, please consider whether // add to CacheAnalyzer.commonCacheCondition - if (channel != null && !isOutfileQuery && CacheAnalyzer.canUseCache(context.getSessionVariable())) { + if (channel != null && !isOutfileQuery && CacheAnalyzer.canUseCache(context.getSessionVariable()) + && parsedStmt.getOrigStmt() != null && parsedStmt.getOrigStmt().originStmt != null) { if (queryStmt instanceof QueryStmt || queryStmt instanceof LogicalPlanAdapter) { handleCacheStmt(cacheAnalyzer, channel); LOG.info("Query {} finished", DebugUtil.printId(context.queryId)); diff --git a/regression-test/suites/nereids_p0/cache/prepare_stmt_with_sql_cache.groovy b/regression-test/suites/nereids_p0/cache/prepare_stmt_with_sql_cache.groovy new file mode 100644 index 00000000000000..7819a6ca09d719 --- /dev/null +++ b/regression-test/suites/nereids_p0/cache/prepare_stmt_with_sql_cache.groovy @@ -0,0 +1,57 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import com.mysql.cj.ServerPreparedQuery +import com.mysql.cj.jdbc.ConnectionImpl +import com.mysql.cj.jdbc.JdbcStatement +import com.mysql.cj.jdbc.ServerPreparedStatement +import com.mysql.cj.jdbc.StatementImpl +import org.apache.doris.regression.util.JdbcUtils + +import java.lang.reflect.Field +import java.sql.PreparedStatement +import java.sql.ResultSet +import java.util.concurrent.CopyOnWriteArrayList + +suite("prepare_stmt_with_sql_cache") { + + multi_sql """ + drop table if exists test_prepare_stmt_with_sql_cache; + create table test_prepare_stmt_with_sql_cache(id int) + distributed by hash(id) + properties('replication_num'='1'); + + insert into test_prepare_stmt_with_sql_cache select * from numbers('number'='100'); + """ + + def db = (sql "select database()")[0][0].toString() + + def url = getServerPrepareJdbcUrl(context.config.jdbcUrl, db) + + connect(context.config.jdbcUser, context.config.jdbcPassword, url) { + sql "set enable_sql_cache=true" + for (def i in 0..<10) { + try (PreparedStatement pstmt = prepareStatement("select * from test_prepare_stmt_with_sql_cache where id=?")) { + pstmt.setInt(1, i) + try (ResultSet rs = pstmt.executeQuery()) { + def result = JdbcUtils.toList(rs).v1 + logger.info("result: {}", result) + } + } + } + } +} From 24328d1cc2401b62a62f4d89c944a03866e4a252 Mon Sep 17 00:00:00 2001 From: morrySnow Date: Thu, 19 Dec 2024 17:48:37 +0800 Subject: [PATCH 08/14] [opt](Nereids) lock table in ascending order of table IDs (#45045) ### What problem does this PR solve? Problem Summary: Doris's table locks are fair read-write locks. If two threads acquire read locks on tables in different orders and simultaneously a third thread attempts to acquire a write lock on one of these tables, a deadlock can form between the two threads trying to acquire read locks. This PR changes the lock acquisition order for queries to follow the order of table IDs, ensuring that the lock acquisition order for tables is consistent among different threads. ### Release note Execute table locking operations in ascending order of table IDs --- .../java/org/apache/doris/catalog/Column.java | 5 +- .../java/org/apache/doris/catalog/MTMV.java | 8 +- .../org/apache/doris/catalog/OlapTable.java | 30 +-- .../org/apache/doris/catalog/TableIf.java | 134 ++++------ .../doris/common/NereidsSqlCacheManager.java | 32 ++- .../lock/MonitoredReentrantReadWriteLock.java | 13 + .../doris/common/proc/PartitionsProcDir.java | 37 ++- .../doris/common/profile/SummaryProfile.java | 15 +- .../httpv2/rest/StmtExecutionAction.java | 34 +-- .../doris/job/extensions/mtmv/MTMVTask.java | 38 ++- .../org/apache/doris/mtmv/BaseTableInfo.java | 7 + .../java/org/apache/doris/mtmv/MTMVCache.java | 30 ++- .../org/apache/doris/mtmv/MTMVJobManager.java | 11 +- .../apache/doris/mtmv/MTMVPartitionUtil.java | 5 +- .../org/apache/doris/mtmv/MTMVPlanUtil.java | 68 ++--- .../doris/mtmv/MTMVRelationManager.java | 21 +- .../apache/doris/mtmv/MTMVRewriteUtil.java | 4 +- .../apache/doris/nereids/CascadesContext.java | 239 +----------------- .../apache/doris/nereids/NereidsPlanner.java | 126 +++++---- .../apache/doris/nereids/SqlCacheContext.java | 4 + .../doris/nereids/StatementContext.java | 156 ++++++++---- .../UnboundBaseExternalTableSink.java | 6 - .../analyzer/UnboundOneRowRelation.java | 2 +- .../nereids/analyzer/UnboundRelation.java | 6 - .../nereids/analyzer/UnboundResultSink.java | 6 - .../nereids/analyzer/UnboundTVFRelation.java | 6 - .../nereids/analyzer/UnboundTableSink.java | 6 - .../doris/nereids/jobs/executor/Analyzer.java | 35 +-- .../nereids/jobs/executor/TableCollector.java | 71 ++++++ .../doris/nereids/minidump/MinidumpUtils.java | 11 +- .../nereids/parser/LogicalPlanBuilder.java | 3 +- .../apache/doris/nereids/rules/RuleType.java | 9 +- .../nereids/rules/analysis/BindRelation.java | 64 ++--- .../rules/analysis/CollectRelation.java | 228 +++++++++++++++++ .../mv/AsyncMaterializationContext.java | 5 - .../mv/InitMaterializationContextHook.java | 24 +- .../exploration/mv/MaterializedViewUtils.java | 8 +- .../plans/commands/AddConstraintCommand.java | 31 ++- .../trees/plans/commands/CommandUtils.java | 49 ---- .../plans/commands/DropConstraintCommand.java | 28 +- .../commands/ShowConstraintsCommand.java | 15 +- .../plans/commands/info/CreateMTMVInfo.java | 75 +++--- .../info/MTMVPartitionDefinition.java | 20 +- .../insert/InsertIntoTableCommand.java | 131 ++++++---- .../plans/commands/insert/InsertUtils.java | 11 +- .../trees/plans/visitor/TableCollector.java | 122 --------- .../org/apache/doris/qe/SessionVariable.java | 12 + .../org/apache/doris/qe/StmtExecutor.java | 3 + .../tablefunction/MetadataGenerator.java | 37 ++- .../rules/analysis/BindRelationTest.java | 70 +---- .../nereids/trees/plans/PlanVisitorTest.java | 163 ------------ .../doris/nereids/util/PlanChecker.java | 14 +- .../doris/nereids/util/ReadLockTest.java | 11 +- .../apache/doris/qe/OlapQueryCacheTest.java | 2 +- 54 files changed, 1083 insertions(+), 1218 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/TableCollector.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CommandUtils.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/TableCollector.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java index 0ae6a4f8bdb5eb..3ef5f680e94d15 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java @@ -1054,10 +1054,7 @@ public boolean equals(Object obj) { && isKey == other.isKey && isAllowNull == other.isAllowNull && isAutoInc == other.isAutoInc - && getDataType().equals(other.getDataType()) - && getStrLen() == other.getStrLen() - && getPrecision() == other.getPrecision() - && getScale() == other.getScale() + && Objects.equals(type, other.type) && Objects.equals(comment, other.comment) && visible == other.visible && Objects.equals(children, other.children) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java index daf1aac333d653..19058df1eb904b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java @@ -201,7 +201,7 @@ public void addTaskResult(MTMVTask task, MTMVRelation relation, // to connection issues such as S3, so it is directly set to null if (!isReplay) { // shouldn't do this while holding mvWriteLock - mtmvCache = MTMVCache.from(this, MTMVPlanUtil.createMTMVContext(this), true); + mtmvCache = MTMVCache.from(this, MTMVPlanUtil.createMTMVContext(this), true, true); } } catch (Throwable e) { mtmvCache = null; @@ -323,7 +323,7 @@ public MTMVCache getOrGenerateCache(ConnectContext connectionContext) throws Ana MTMVCache mtmvCache; try { // Should new context with ADMIN user - mtmvCache = MTMVCache.from(this, MTMVPlanUtil.createMTMVContext(this), true); + mtmvCache = MTMVCache.from(this, MTMVPlanUtil.createMTMVContext(this), true, false); } finally { connectionContext.setThreadLocalInfo(); } @@ -362,7 +362,7 @@ public MTMVRefreshSnapshot getRefreshSnapshot() { * * @return mvPartitionName ==> mvPartitionKeyDesc */ - public Map generateMvPartitionDescs() throws AnalysisException { + public Map generateMvPartitionDescs() { Map mtmvItems = getAndCopyPartitionItems(); Map result = Maps.newHashMap(); for (Entry entry : mtmvItems.entrySet()) { @@ -392,7 +392,7 @@ public Pair>, Map> calculateDoublyPartit Map baseToMv = Maps.newHashMap(); Map> relatedPartitionDescs = MTMVPartitionUtil .generateRelatedPartitionDescs(mvPartitionInfo, mvProperties); - Map mvPartitionItems = getAndCopyPartitionItemsWithoutLock(); + Map mvPartitionItems = getAndCopyPartitionItems(); for (Entry entry : mvPartitionItems.entrySet()) { Set basePartitionNames = relatedPartitionDescs.getOrDefault(entry.getValue().toPartitionKeyDesc(), Sets.newHashSet()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index 9f1f455ab35495..477f76301120d2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -114,7 +114,6 @@ import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; /** @@ -3325,33 +3324,26 @@ public PartitionType getPartitionType() { } @Override - public Map getAndCopyPartitionItems(Optional snapshot) - throws AnalysisException { + public Map getAndCopyPartitionItems(Optional snapshot) { return getAndCopyPartitionItems(); } - public Map getAndCopyPartitionItems() throws AnalysisException { - if (!tryReadLock(1, TimeUnit.MINUTES)) { - throw new AnalysisException("get table read lock timeout, database=" + getDBName() + ",table=" + getName()); - } + public Map getAndCopyPartitionItems() { + readLock(); try { - return getAndCopyPartitionItemsWithoutLock(); + Map res = Maps.newHashMap(); + for (Entry entry : getPartitionInfo().getIdToItem(false).entrySet()) { + Partition partition = idToPartition.get(entry.getKey()); + if (partition != null) { + res.put(partition.getName(), entry.getValue()); + } + } + return res; } finally { readUnlock(); } } - public Map getAndCopyPartitionItemsWithoutLock() throws AnalysisException { - Map res = Maps.newHashMap(); - for (Entry entry : getPartitionInfo().getIdToItem(false).entrySet()) { - Partition partition = idToPartition.get(entry.getKey()); - if (partition != null) { - res.put(partition.getName(), entry.getValue()); - } - } - return res; - } - @Override public List getPartitionColumns(Optional snapshot) { return getPartitionColumns(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java index 4761ac9d86db2a..a93c0818d5c91f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java @@ -213,7 +213,6 @@ default Map getConstraintsMapUnsafe() { } default Set getForeignKeyConstraints() { - readLock(); try { return getConstraintsMapUnsafe().values().stream() .filter(ForeignKeyConstraint.class::isInstance) @@ -221,24 +220,18 @@ default Set getForeignKeyConstraints() { .collect(ImmutableSet.toImmutableSet()); } catch (Exception ignored) { return ImmutableSet.of(); - } finally { - readUnlock(); } } default Map getConstraintsMap() { - readLock(); try { return ImmutableMap.copyOf(getConstraintsMapUnsafe()); } catch (Exception ignored) { return ImmutableMap.of(); - } finally { - readUnlock(); } } default Set getPrimaryKeyConstraints() { - readLock(); try { return getConstraintsMapUnsafe().values().stream() .filter(PrimaryKeyConstraint.class::isInstance) @@ -246,13 +239,10 @@ default Set getPrimaryKeyConstraints() { .collect(ImmutableSet.toImmutableSet()); } catch (Exception ignored) { return ImmutableSet.of(); - } finally { - readUnlock(); } } default Set getUniqueConstraints() { - readLock(); try { return getConstraintsMapUnsafe().values().stream() .filter(UniqueConstraint.class::isInstance) @@ -260,8 +250,6 @@ default Set getUniqueConstraints() { .collect(ImmutableSet.toImmutableSet()); } catch (Exception ignored) { return ImmutableSet.of(); - } finally { - readUnlock(); } } @@ -280,34 +268,24 @@ default void checkConstraintNotExistenceUnsafe(String name, Constraint primaryKe } default void addUniqueConstraint(String name, ImmutableList columns, boolean replay) { - writeLock(); - try { - Map constraintMap = getConstraintsMapUnsafe(); - UniqueConstraint uniqueConstraint = new UniqueConstraint(name, ImmutableSet.copyOf(columns)); - checkConstraintNotExistenceUnsafe(name, uniqueConstraint, constraintMap); - constraintMap.put(name, uniqueConstraint); - if (!replay) { - Env.getCurrentEnv().getEditLog().logAddConstraint( - new AlterConstraintLog(uniqueConstraint, this)); - } - } finally { - writeUnlock(); + Map constraintMap = getConstraintsMapUnsafe(); + UniqueConstraint uniqueConstraint = new UniqueConstraint(name, ImmutableSet.copyOf(columns)); + checkConstraintNotExistenceUnsafe(name, uniqueConstraint, constraintMap); + constraintMap.put(name, uniqueConstraint); + if (!replay) { + Env.getCurrentEnv().getEditLog().logAddConstraint( + new AlterConstraintLog(uniqueConstraint, this)); } } default void addPrimaryKeyConstraint(String name, ImmutableList columns, boolean replay) { - writeLock(); - try { - Map constraintMap = getConstraintsMapUnsafe(); - PrimaryKeyConstraint primaryKeyConstraint = new PrimaryKeyConstraint(name, ImmutableSet.copyOf(columns)); - checkConstraintNotExistenceUnsafe(name, primaryKeyConstraint, constraintMap); - constraintMap.put(name, primaryKeyConstraint); - if (!replay) { - Env.getCurrentEnv().getEditLog().logAddConstraint( - new AlterConstraintLog(primaryKeyConstraint, this)); - } - } finally { - writeUnlock(); + Map constraintMap = getConstraintsMapUnsafe(); + PrimaryKeyConstraint primaryKeyConstraint = new PrimaryKeyConstraint(name, ImmutableSet.copyOf(columns)); + checkConstraintNotExistenceUnsafe(name, primaryKeyConstraint, constraintMap); + constraintMap.put(name, primaryKeyConstraint); + if (!replay) { + Env.getCurrentEnv().getEditLog().logAddConstraint( + new AlterConstraintLog(primaryKeyConstraint, this)); } } @@ -326,26 +304,19 @@ default PrimaryKeyConstraint tryGetPrimaryKeyForForeignKeyUnsafe( default void addForeignConstraint(String name, ImmutableList columns, TableIf referencedTable, ImmutableList referencedColumns, boolean replay) { - writeLock(); - referencedTable.writeLock(); - try { - Map constraintMap = getConstraintsMapUnsafe(); - ForeignKeyConstraint foreignKeyConstraint = - new ForeignKeyConstraint(name, columns, referencedTable, referencedColumns); - checkConstraintNotExistenceUnsafe(name, foreignKeyConstraint, constraintMap); - PrimaryKeyConstraint requirePrimaryKeyName = new PrimaryKeyConstraint(name, - foreignKeyConstraint.getReferencedColumnNames()); - PrimaryKeyConstraint primaryKeyConstraint = - tryGetPrimaryKeyForForeignKeyUnsafe(requirePrimaryKeyName, referencedTable); - primaryKeyConstraint.addForeignTable(this); - constraintMap.put(name, foreignKeyConstraint); - if (!replay) { - Env.getCurrentEnv().getEditLog().logAddConstraint( - new AlterConstraintLog(foreignKeyConstraint, this)); - } - } finally { - referencedTable.writeUnlock(); - writeUnlock(); + Map constraintMap = getConstraintsMapUnsafe(); + ForeignKeyConstraint foreignKeyConstraint = + new ForeignKeyConstraint(name, columns, referencedTable, referencedColumns); + checkConstraintNotExistenceUnsafe(name, foreignKeyConstraint, constraintMap); + PrimaryKeyConstraint requirePrimaryKeyName = new PrimaryKeyConstraint(name, + foreignKeyConstraint.getReferencedColumnNames()); + PrimaryKeyConstraint primaryKeyConstraint = + tryGetPrimaryKeyForForeignKeyUnsafe(requirePrimaryKeyName, referencedTable); + primaryKeyConstraint.addForeignTable(this); + constraintMap.put(name, foreignKeyConstraint); + if (!replay) { + Env.getCurrentEnv().getEditLog().logAddConstraint( + new AlterConstraintLog(foreignKeyConstraint, this)); } } @@ -381,40 +352,31 @@ default void replayDropConstraint(String name) { } default void dropConstraint(String name, boolean replay) { - writeLock(); - try { - Map constraintMap = getConstraintsMapUnsafe(); - if (!constraintMap.containsKey(name)) { - throw new AnalysisException( - String.format("Unknown constraint %s on table %s.", name, this.getName())); - } - Constraint constraint = constraintMap.get(name); - constraintMap.remove(name); - if (constraint instanceof PrimaryKeyConstraint) { - ((PrimaryKeyConstraint) constraint).getForeignTables() - .forEach(t -> t.dropFKReferringPK(this, (PrimaryKeyConstraint) constraint)); - } - if (!replay) { - Env.getCurrentEnv().getEditLog().logDropConstraint(new AlterConstraintLog(constraint, this)); - } - } finally { - writeUnlock(); + Map constraintMap = getConstraintsMapUnsafe(); + if (!constraintMap.containsKey(name)) { + throw new AnalysisException( + String.format("Unknown constraint %s on table %s.", name, this.getName())); + } + Constraint constraint = constraintMap.get(name); + constraintMap.remove(name); + if (constraint instanceof PrimaryKeyConstraint) { + ((PrimaryKeyConstraint) constraint).getForeignTables() + .forEach(t -> t.dropFKReferringPK(this, (PrimaryKeyConstraint) constraint)); + } + if (!replay) { + Env.getCurrentEnv().getEditLog().logDropConstraint(new AlterConstraintLog(constraint, this)); } } default void dropFKReferringPK(TableIf table, PrimaryKeyConstraint constraint) { - writeLock(); - try { - Map constraintMap = getConstraintsMapUnsafe(); - Set fkName = constraintMap.entrySet().stream() - .filter(e -> e.getValue() instanceof ForeignKeyConstraint - && ((ForeignKeyConstraint) e.getValue()).isReferringPK(table, constraint)) - .map(Entry::getKey) - .collect(Collectors.toSet()); - fkName.forEach(constraintMap::remove); - } finally { - writeUnlock(); - } + Map constraintMap = getConstraintsMapUnsafe(); + Set fkName = constraintMap.entrySet().stream() + .filter(e -> e.getValue() instanceof ForeignKeyConstraint + && ((ForeignKeyConstraint) e.getValue()).isReferringPK(table, constraint)) + .map(Entry::getKey) + .collect(Collectors.toSet()); + fkName.forEach(constraintMap::remove); + } /** 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 cd32b52034a5d4..86a2b875a93d68 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 @@ -225,6 +225,10 @@ private Optional tryParseSqlWithoutCheckVariable( SqlCacheContext sqlCacheContext, UserIdentity currentUserIdentity) { Env env = connectContext.getEnv(); + if (!tryLockTables(connectContext, env, sqlCacheContext)) { + return invalidateCache(key); + } + // check table and view and their columns authority if (privilegeChanged(connectContext, env, sqlCacheContext)) { return invalidateCache(key); @@ -378,16 +382,38 @@ private boolean dataMaskPoliciesChanged( return false; } - private boolean privilegeChanged(ConnectContext connectContext, Env env, SqlCacheContext sqlCacheContext) { + /** + * Execute table locking operations in ascending order of table IDs. + * + * @return true if obtain all tables lock. + */ + private boolean tryLockTables(ConnectContext connectContext, Env env, SqlCacheContext sqlCacheContext) { StatementContext currentStatementContext = connectContext.getStatementContext(); + for (FullTableName fullTableName : sqlCacheContext.getUsedTables()) { + TableIf tableIf = findTableIf(env, fullTableName); + if (tableIf == null) { + return false; + } + currentStatementContext.getTables().put(fullTableName.toList(), tableIf); + } + for (FullTableName fullTableName : sqlCacheContext.getUsedViews().keySet()) { + TableIf tableIf = findTableIf(env, fullTableName); + if (tableIf == null) { + return false; + } + currentStatementContext.getTables().put(fullTableName.toList(), tableIf); + } + currentStatementContext.lock(); + return true; + } + + private boolean privilegeChanged(ConnectContext connectContext, Env env, SqlCacheContext sqlCacheContext) { 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) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/lock/MonitoredReentrantReadWriteLock.java b/fe/fe-core/src/main/java/org/apache/doris/common/lock/MonitoredReentrantReadWriteLock.java index 7a6f0db5938b23..de825fbdb3ac23 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/lock/MonitoredReentrantReadWriteLock.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/lock/MonitoredReentrantReadWriteLock.java @@ -17,6 +17,12 @@ package org.apache.doris.common.lock; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.qe.ConnectContext; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + import java.util.concurrent.locks.ReentrantReadWriteLock; /** @@ -24,6 +30,8 @@ * monitoring capabilities for read and write locks. */ public class MonitoredReentrantReadWriteLock extends ReentrantReadWriteLock { + + private static final Logger LOG = LogManager.getLogger(MonitoredReentrantReadWriteLock.class); // Monitored read and write lock instances private final ReadLock readLock = new ReadLock(this); private final WriteLock writeLock = new WriteLock(this); @@ -97,6 +105,11 @@ protected WriteLock(ReentrantReadWriteLock lock) { public void lock() { super.lock(); monitor.afterLock(); + if (isFair() && getReadHoldCount() > 0) { + LOG.warn(" read lock count is {}, write lock count is {}, stack is {}, query id is {}", + getReadHoldCount(), getWriteHoldCount(), Thread.currentThread().getStackTrace(), + ConnectContext.get() == null ? "" : DebugUtil.printId(ConnectContext.get().queryId())); + } } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java index 3fd945c013cd5d..3c44874cb7deff 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java @@ -35,6 +35,7 @@ import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.PartitionInfo; import org.apache.doris.catalog.PartitionType; +import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.ErrorCode; @@ -44,9 +45,12 @@ import org.apache.doris.common.Pair; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.common.util.ListComparator; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.common.util.OrderByPair; import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.mtmv.MTMVPartitionUtil; +import org.apache.doris.mtmv.MTMVUtil; import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; @@ -59,6 +63,7 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -250,22 +255,38 @@ private List, TRow>> getPartitionInfosInrernal() throws An List, TRow>> partitionInfos = new ArrayList, TRow>>(); Map> partitionsUnSyncTables = null; String mtmvPartitionSyncErrorMsg = null; + + List needLocked = Lists.newArrayList(); + needLocked.add(olapTable); if (olapTable instanceof MTMV) { - try { - partitionsUnSyncTables = MTMVPartitionUtil - .getPartitionsUnSyncTables((MTMV) olapTable); - } catch (AnalysisException e) { - mtmvPartitionSyncErrorMsg = e.getMessage(); + MTMV mtmv = (MTMV) olapTable; + for (BaseTableInfo baseTableInfo : mtmv.getRelation().getBaseTables()) { + try { + TableIf baseTable = MTMVUtil.getTable(baseTableInfo); + needLocked.add(baseTable); + } catch (Exception e) { + // do nothing, ignore not existed table + } } + needLocked.sort(Comparator.comparing(TableIf::getId)); } - olapTable.readLock(); + MetaLockUtils.readLockTables(needLocked); try { + if (olapTable instanceof MTMV) { + try { + partitionsUnSyncTables = MTMVPartitionUtil + .getPartitionsUnSyncTables((MTMV) olapTable); + } catch (AnalysisException e) { + mtmvPartitionSyncErrorMsg = e.getMessage(); + } + } List partitionIds; PartitionInfo tblPartitionInfo = olapTable.getPartitionInfo(); // for range partitions, we return partitions in ascending range order by default. // this is to be consistent with the behaviour before 0.12 - if (tblPartitionInfo.getType() == PartitionType.RANGE || tblPartitionInfo.getType() == PartitionType.LIST) { + if (tblPartitionInfo.getType() == PartitionType.RANGE + || tblPartitionInfo.getType() == PartitionType.LIST) { partitionIds = tblPartitionInfo.getPartitionItemEntryList(isTempPartition, true).stream() .map(Map.Entry::getKey).collect(Collectors.toList()); } else { @@ -402,7 +423,7 @@ private List, TRow>> getPartitionInfosInrernal() throws An partitionInfos.add(Pair.of(partitionInfo, trow)); } } finally { - olapTable.readUnlock(); + MetaLockUtils.readUnlockTables(needLocked); } return partitionInfos; } 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 ecc4c908809161..6a92e043b6eb20 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 @@ -97,6 +97,7 @@ public class SummaryProfile { public static final String GET_TABLE_VERSION_COUNT = "Get Table Version Count"; public static final String PARSE_SQL_TIME = "Parse SQL Time"; + public static final String NEREIDS_LOCK_TABLE_TIME = "Nereids Lock Table Time"; public static final String NEREIDS_ANALYSIS_TIME = "Nereids Analysis Time"; public static final String NEREIDS_REWRITE_TIME = "Nereids Rewrite Time"; public static final String NEREIDS_OPTIMIZE_TIME = "Nereids Optimize Time"; @@ -136,6 +137,7 @@ public class SummaryProfile { // The display order of execution summary items. public static final ImmutableList EXECUTION_SUMMARY_KEYS = ImmutableList.of( PARSE_SQL_TIME, + NEREIDS_LOCK_TABLE_TIME, NEREIDS_ANALYSIS_TIME, NEREIDS_REWRITE_TIME, NEREIDS_OPTIMIZE_TIME, @@ -224,6 +226,8 @@ public class SummaryProfile { private long parseSqlStartTime = -1; @SerializedName(value = "parseSqlFinishTime") private long parseSqlFinishTime = -1; + @SerializedName(value = "nereidsLockTableFinishTime") + private long nereidsLockTableFinishTime = -1; @SerializedName(value = "nereidsAnalysisFinishTime") private long nereidsAnalysisFinishTime = -1; @SerializedName(value = "nereidsRewriteFinishTime") @@ -410,6 +414,7 @@ private void updateSummaryProfile(Map infos) { private void updateExecutionSummaryProfile() { executionSummaryProfile.addInfoString(PARSE_SQL_TIME, getPrettyParseSqlTime()); + executionSummaryProfile.addInfoString(NEREIDS_LOCK_TABLE_TIME, getPrettyNereidsLockTableTime()); executionSummaryProfile.addInfoString(NEREIDS_ANALYSIS_TIME, getPrettyNereidsAnalysisTime()); executionSummaryProfile.addInfoString(NEREIDS_REWRITE_TIME, getPrettyNereidsRewriteTime()); executionSummaryProfile.addInfoString(NEREIDS_OPTIMIZE_TIME, getPrettyNereidsOptimizeTime()); @@ -506,6 +511,10 @@ public void setParseSqlFinishTime(long parseSqlFinishTime) { this.parseSqlFinishTime = parseSqlFinishTime; } + public void setNereidsLockTableFinishTime() { + this.nereidsLockTableFinishTime = TimeUtils.getStartTimeMs(); + } + public void setNereidsAnalysisTime() { this.nereidsAnalysisFinishTime = TimeUtils.getStartTimeMs(); } @@ -766,8 +775,12 @@ public String getPrettyParseSqlTime() { return getPrettyTime(parseSqlFinishTime, parseSqlStartTime, TUnit.TIME_MS); } + public String getPrettyNereidsLockTableTime() { + return getPrettyTime(nereidsLockTableFinishTime, parseSqlStartTime, TUnit.TIME_MS); + } + public String getPrettyNereidsAnalysisTime() { - return getPrettyTime(nereidsAnalysisFinishTime, queryBeginTime, TUnit.TIME_MS); + return getPrettyTime(nereidsAnalysisFinishTime, nereidsLockTableFinishTime, TUnit.TIME_MS); } public String getPrettyNereidsRewriteTime() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StmtExecutionAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StmtExecutionAction.java index a37d3a11f84c9c..524c228467a194 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StmtExecutionAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StmtExecutionAction.java @@ -186,22 +186,26 @@ private ResponseEntity executeQuery(ActionAuthorizationInfo authInfo, boolean is @NotNull private String getSchema(String sql) { LogicalPlan unboundMvPlan = new NereidsParser().parseSingle(sql); - StatementContext statementContext = new StatementContext(ConnectContext.get(), - new OriginStatement(sql, 0)); - NereidsPlanner planner = new NereidsPlanner(statementContext); - if (statementContext.getConnectContext().getStatementContext() == null) { - statementContext.getConnectContext().setStatementContext(statementContext); + try (StatementContext statementContext = new StatementContext(ConnectContext.get(), + new OriginStatement(sql, 0))) { + StatementContext originalContext = ConnectContext.get().getStatementContext(); + try { + ConnectContext.get().setStatementContext(statementContext); + NereidsPlanner planner = new NereidsPlanner(statementContext); + planner.planWithLock(unboundMvPlan, PhysicalProperties.ANY, ExplainCommand.ExplainLevel.ANALYZED_PLAN); + LogicalPlan logicalPlan = (LogicalPlan) planner.getCascadesContext().getRewritePlan(); + + List createStmts = PlanUtils.getLogicalScanFromRootPlan(logicalPlan).stream().map(plan -> { + TableIf tbl = plan.getTable(); + List createTableStmts = Lists.newArrayList(); + Env.getDdlStmt(tbl, createTableStmts, null, null, false, true, -1L); + return createTableStmts.get(0); + }).collect(Collectors.toList()); + return Joiner.on("\n\n").join(createStmts); + } finally { + ConnectContext.get().setStatementContext(originalContext); + } } - planner.planWithLock(unboundMvPlan, PhysicalProperties.ANY, ExplainCommand.ExplainLevel.ANALYZED_PLAN); - LogicalPlan logicalPlan = (LogicalPlan) planner.getCascadesContext().getRewritePlan(); - - List createStmts = PlanUtils.getLogicalScanFromRootPlan(logicalPlan).stream().map(plan -> { - TableIf tbl = plan.getTable(); - List createTableStmts = Lists.newArrayList(); - Env.getDdlStmt(tbl, createTableStmts, null, null, false, true, -1L); - return createTableStmts.get(0); - }).collect(Collectors.toList()); - return Joiner.on("\n\n").join(createStmts); } private static class StmtRequestBody { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java index c1002faf4078b3..31e6c8353e24b3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java @@ -28,6 +28,7 @@ import org.apache.doris.common.Status; import org.apache.doris.common.UserException; import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.mvcc.MvccSnapshot; import org.apache.doris.datasource.mvcc.MvccTable; @@ -72,6 +73,7 @@ import java.math.BigDecimal; import java.math.RoundingMode; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -180,19 +182,31 @@ public void run() throws JobException { } // Every time a task is run, the relation is regenerated because baseTables and baseViews may change, // such as deleting a table and creating a view with the same name - this.relation = MTMVPlanUtil.generateMTMVRelation(mtmv, ctx); + Set tablesInPlan = MTMVPlanUtil.getBaseTableFromQuery(mtmv.getQuerySql(), ctx); + this.relation = MTMVPlanUtil.generateMTMVRelation(tablesInPlan, ctx); beforeMTMVRefresh(); - if (mtmv.getMvPartitionInfo().getPartitionType() != MTMVPartitionType.SELF_MANAGE) { - MTMVRelatedTableIf relatedTable = mtmv.getMvPartitionInfo().getRelatedTable(); - if (!relatedTable.isValidRelatedTable()) { - throw new JobException("MTMV " + mtmv.getName() + "'s related table " + relatedTable.getName() - + " is not a valid related table anymore, stop refreshing." - + " e.g. Table has multiple partition columns or including not supported transform functions."); + List tableIfs = Lists.newArrayList(tablesInPlan); + tableIfs.sort(Comparator.comparing(TableIf::getId)); + + MTMVRefreshContext context; + // lock table order by id to avoid deadlock + MetaLockUtils.readLockTables(tableIfs); + try { + if (mtmv.getMvPartitionInfo().getPartitionType() != MTMVPartitionType.SELF_MANAGE) { + MTMVRelatedTableIf relatedTable = mtmv.getMvPartitionInfo().getRelatedTable(); + if (!relatedTable.isValidRelatedTable()) { + throw new JobException("MTMV " + mtmv.getName() + "'s related table " + relatedTable.getName() + + " is not a valid related table anymore, stop refreshing." + + " e.g. Table has multiple partition columns" + + " or including not supported transform functions."); + } + MTMVPartitionUtil.alignMvPartition(mtmv); } - MTMVPartitionUtil.alignMvPartition(mtmv); + context = MTMVRefreshContext.buildContext(mtmv); + this.needRefreshPartitions = calculateNeedRefreshPartitions(context); + } finally { + MetaLockUtils.readUnlockTables(tableIfs); } - MTMVRefreshContext context = MTMVRefreshContext.buildContext(mtmv); - this.needRefreshPartitions = calculateNeedRefreshPartitions(context); this.refreshMode = generateRefreshMode(needRefreshPartitions); if (refreshMode == MTMVTaskRefreshMode.NOT_REFRESH) { return; @@ -207,7 +221,7 @@ public void run() throws JobException { int start = i * refreshPartitionNum; int end = start + refreshPartitionNum; Set execPartitionNames = Sets.newHashSet(needRefreshPartitions - .subList(start, end > needRefreshPartitions.size() ? needRefreshPartitions.size() : end)); + .subList(start, Math.min(end, needRefreshPartitions.size()))); // need get names before exec Map execPartitionSnapshots = MTMVPartitionUtil .generatePartitionSnapshots(context, relation.getBaseTablesOneLevel(), execPartitionNames); @@ -217,7 +231,7 @@ public void run() throws JobException { } } catch (Throwable e) { if (getStatus() == TaskStatus.RUNNING) { - LOG.warn("run task failed: ", e.getMessage()); + LOG.warn("run task failed: {}", e.getMessage()); throw new JobException(e.getMessage(), e); } else { // if status is not `RUNNING`,maybe the task was canceled, therefore, it is a normal situation diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseTableInfo.java index fcf18d73a269bb..076a4f4e8bb050 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseTableInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseTableInfo.java @@ -26,11 +26,14 @@ import org.apache.doris.datasource.InternalCatalog; import com.google.common.base.Objects; +import com.google.common.collect.Lists; import com.google.gson.annotations.SerializedName; import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.List; + public class BaseTableInfo { private static final Logger LOG = LogManager.getLogger(BaseTableInfo.class); @@ -167,4 +170,8 @@ public void compatible(CatalogMgr catalogMgr) { LOG.warn("MTMV compatible failed, ctlId: {}, dbId: {}, tableId: {}", ctlId, dbId, tableId, e); } } + + public List toList() { + return Lists.newArrayList(getCtlName(), getDbName(), getTableName()); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java index d3d7f1ad6ebbf5..b185000c14897e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java @@ -87,23 +87,31 @@ public StructInfo getStructInfo() { return structInfo; } - public static MTMVCache from(MTMV mtmv, ConnectContext connectContext, boolean needCost) { + public static MTMVCache from(MTMV mtmv, ConnectContext connectContext, boolean needCost, boolean needLock) { StatementContext mvSqlStatementContext = new StatementContext(connectContext, new OriginStatement(mtmv.getQuerySql(), 0)); + if (needLock) { + mvSqlStatementContext.setNeedLockTables(false); + } if (mvSqlStatementContext.getConnectContext().getStatementContext() == null) { mvSqlStatementContext.getConnectContext().setStatementContext(mvSqlStatementContext); } LogicalPlan unboundMvPlan = new NereidsParser().parseSingle(mtmv.getQuerySql()); NereidsPlanner planner = new NereidsPlanner(mvSqlStatementContext); - - // Can not convert to table sink, because use the same column from different table when self join - // the out slot is wrong - if (needCost) { - // Only in mv rewrite, we need plan with eliminated cost which is used for mv chosen - planner.planWithLock(unboundMvPlan, PhysicalProperties.ANY, ExplainLevel.ALL_PLAN); - } else { - // No need cost for performance - planner.planWithLock(unboundMvPlan, PhysicalProperties.ANY, ExplainLevel.REWRITTEN_PLAN); + boolean originalRewriteFlag = connectContext.getSessionVariable().enableMaterializedViewRewrite; + connectContext.getSessionVariable().enableMaterializedViewRewrite = false; + try { + // Can not convert to table sink, because use the same column from different table when self join + // the out slot is wrong + if (needCost) { + // Only in mv rewrite, we need plan with eliminated cost which is used for mv chosen + planner.planWithLock(unboundMvPlan, PhysicalProperties.ANY, ExplainLevel.ALL_PLAN); + } else { + // No need cost for performance + planner.planWithLock(unboundMvPlan, PhysicalProperties.ANY, ExplainLevel.REWRITTEN_PLAN); + } + } finally { + connectContext.getSessionVariable().enableMaterializedViewRewrite = originalRewriteFlag; } Plan originPlan = planner.getCascadesContext().getRewritePlan(); // Eliminate result sink because sink operator is useless in query rewrite by materialized view @@ -128,6 +136,6 @@ public Plan visitLogicalResultSink(LogicalResultSink logicalResu new BitSet()); return new MTMVCache(mvPlan, originPlan, planner.getAnalyzedPlan(), needCost ? planner.getCascadesContext().getMemo().getRoot().getStatistics() : null, - structInfoOptional.orElseGet(() -> null)); + structInfoOptional.orElse(null)); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobManager.java index 2c03ad16176fea..a9dee132f64b12 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobManager.java @@ -104,17 +104,18 @@ private void setManualJobConfig(JobExecutionConfiguration jobExecutionConfigurat private void setScheduleJobConfig(JobExecutionConfiguration jobExecutionConfiguration, MTMV mtmv) { jobExecutionConfiguration.setExecuteType(JobExecuteType.RECURRING); + MTMVRefreshInfo refreshMTMVInfo = mtmv.getRefreshInfo(); TimerDefinition timerDefinition = new TimerDefinition(); timerDefinition - .setInterval(mtmv.getRefreshInfo().getRefreshTriggerInfo().getIntervalTrigger().getInterval()); + .setInterval(refreshMTMVInfo.getRefreshTriggerInfo().getIntervalTrigger().getInterval()); timerDefinition - .setIntervalUnit(mtmv.getRefreshInfo().getRefreshTriggerInfo().getIntervalTrigger().getTimeUnit()); + .setIntervalUnit(refreshMTMVInfo.getRefreshTriggerInfo().getIntervalTrigger().getTimeUnit()); if (!StringUtils - .isEmpty(mtmv.getRefreshInfo().getRefreshTriggerInfo().getIntervalTrigger().getStartTime())) { + .isEmpty(refreshMTMVInfo.getRefreshTriggerInfo().getIntervalTrigger().getStartTime())) { timerDefinition.setStartTimeMs(TimeUtils.timeStringToLong( - mtmv.getRefreshInfo().getRefreshTriggerInfo().getIntervalTrigger().getStartTime())); + refreshMTMVInfo.getRefreshTriggerInfo().getIntervalTrigger().getStartTime())); } - if (mtmv.getRefreshInfo().getBuildMode().equals(BuildMode.IMMEDIATE)) { + if (refreshMTMVInfo.getBuildMode().equals(BuildMode.IMMEDIATE)) { jobExecutionConfiguration.setImmediate(true); } jobExecutionConfiguration.setTimerDefinition(timerDefinition); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java index 9597378c488cfc..8f715b1b0fb3a0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java @@ -536,11 +536,12 @@ private static Map getPartitionVersions(MTMV mtmv) throws Analysis private static Map getTableVersions(MTMV mtmv) { Map res = Maps.newHashMap(); - if (mtmv.getRelation() == null || mtmv.getRelation().getBaseTablesOneLevel() == null) { + MTMVRelation relation = mtmv.getRelation(); + if (relation == null || relation.getBaseTablesOneLevel() == null) { return res; } List olapTables = Lists.newArrayList(); - for (BaseTableInfo baseTableInfo : mtmv.getRelation().getBaseTablesOneLevel()) { + for (BaseTableInfo baseTableInfo : relation.getBaseTablesOneLevel()) { TableIf table = null; try { table = MTMVUtil.getTable(baseTableInfo); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index 35c06e74d3cc80..3264d6627ead5d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -33,11 +33,8 @@ import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.rules.RuleType; -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.visitor.TableCollector; -import org.apache.doris.nereids.trees.plans.visitor.TableCollector.TableCollectorContext; import org.apache.doris.qe.ConnectContext; import com.google.common.collect.ImmutableSet; @@ -99,41 +96,27 @@ private static void setCatalogAndDb(ConnectContext ctx, MTMV mtmv) { ctx.setDatabase(databaseIf.get().getFullName()); } - public static MTMVRelation generateMTMVRelation(MTMV mtmv, ConnectContext ctx) { - // Should not make table without data to empty relation when analyze the related table, - // so add disable rules - Plan plan = getAnalyzePlanBySql(mtmv.getQuerySql(), ctx); - return generateMTMVRelation(plan, ctx); - } - - public static MTMVRelation generateMTMVRelation(Plan plan, ConnectContext connectContext) { - return new MTMVRelation(getBaseTables(plan, true, connectContext), - getBaseTables(plan, false, connectContext), getBaseViews(plan)); - } - - private static Set getBaseTables(Plan plan, boolean expand, ConnectContext connectContext) { - TableCollectorContext collectorContext = - new TableCollector.TableCollectorContext( - com.google.common.collect.Sets - .newHashSet(TableType.values()), expand, connectContext); - plan.accept(TableCollector.INSTANCE, collectorContext); - Set collectedTables = collectorContext.getCollectedTables(); - return transferTableIfToInfo(collectedTables); - } - - private static Set getBaseViews(Plan plan) { - return Sets.newHashSet(); - } - - private static Set transferTableIfToInfo(Set tables) { - Set result = com.google.common.collect.Sets.newHashSet(); - for (TableIf table : tables) { - result.add(new BaseTableInfo(table)); + public static MTMVRelation generateMTMVRelation(Set tablesInPlan, ConnectContext ctx) { + Set oneLevelTables = Sets.newHashSet(); + Set allLevelTables = Sets.newHashSet(); + Set oneLevelViews = Sets.newHashSet(); + for (TableIf table : tablesInPlan) { + BaseTableInfo baseTableInfo = new BaseTableInfo(table); + if (table.getType() == TableType.VIEW) { + // TODO reopen it after we support mv on view + // oneLevelViews.add(baseTableInfo); + } else { + oneLevelTables.add(baseTableInfo); + allLevelTables.add(baseTableInfo); + if (table instanceof MTMV) { + allLevelTables.addAll(((MTMV) table).getRelation().getBaseTables()); + } + } } - return result; + return new MTMVRelation(allLevelTables, oneLevelTables, oneLevelViews); } - private static Plan getAnalyzePlanBySql(String querySql, ConnectContext ctx) { + public static Set getBaseTableFromQuery(String querySql, ConnectContext ctx) { List statements; try { statements = new NereidsParser().parseSQL(querySql); @@ -143,12 +126,15 @@ private static Plan getAnalyzePlanBySql(String querySql, ConnectContext ctx) { StatementBase parsedStmt = statements.get(0); LogicalPlan logicalPlan = ((LogicalPlanAdapter) parsedStmt).getLogicalPlan(); StatementContext original = ctx.getStatementContext(); - ctx.setStatementContext(new StatementContext()); - try { - NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); - return planner.planWithLock(logicalPlan, PhysicalProperties.ANY, ExplainLevel.ANALYZED_PLAN); - } finally { - ctx.setStatementContext(original); + try (StatementContext tempCtx = new StatementContext()) { + ctx.setStatementContext(tempCtx); + try { + NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); + planner.planWithLock(logicalPlan, PhysicalProperties.ANY, ExplainLevel.ANALYZED_PLAN); + return Sets.newHashSet(ctx.getStatementContext().getTables().values()); + } finally { + ctx.setStatementContext(original); + } } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java index 436427526ba08b..f8f92e25d38d65 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java @@ -59,8 +59,8 @@ public class MTMVRelationManager implements MTMVHookService { // create mv2 as select * from mv1; // `tableMTMVs` will have 3 pair: table1 ==> mv1,mv1==>mv2, table1 ==> mv2 // `tableMTMVsOneLevel` will have 2 pair: table1 ==> mv1,mv1==>mv2 - private Map> tableMTMVs = Maps.newConcurrentMap(); - private Map> tableMTMVsOneLevel = Maps.newConcurrentMap(); + private final Map> tableMTMVs = Maps.newConcurrentMap(); + private final Map> tableMTMVsOneLevel = Maps.newConcurrentMap(); public Set getMtmvsByBaseTable(BaseTableInfo table) { return tableMTMVs.getOrDefault(table, ImmutableSet.of()); @@ -98,6 +98,23 @@ public Set getAvailableMTMVs(List tableInfos, ConnectContex return res; } + /** + * get all mtmv related to tableInfos. + */ + public Set getAllMTMVs(List tableInfos) { + Set mtmvs = Sets.newLinkedHashSet(); + Set mvInfos = getMTMVInfos(tableInfos); + for (BaseTableInfo tableInfo : mvInfos) { + try { + mtmvs.add((MTMV) MTMVUtil.getTable(tableInfo)); + } catch (AnalysisException e) { + // not throw exception to client, just ignore it + LOG.warn("getTable failed: {}", tableInfo.toString(), e); + } + } + return mtmvs; + } + @VisibleForTesting public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent) { long currentTimeMillis = System.currentTimeMillis(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRewriteUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRewriteUtil.java index 7b7d743a36bc8c..ff1b3263d3409e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRewriteUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRewriteUtil.java @@ -51,8 +51,8 @@ public static Collection getMTMVCanRewritePartitions(MTMV mtmv, Conne return res; } // check mv is normal - if (mtmv.getStatus().getState() != MTMVState.NORMAL - || mtmv.getStatus().getRefreshState() == MTMVRefreshState.INIT) { + MTMVStatus mtmvStatus = mtmv.getStatus(); + if (mtmvStatus.getState() != MTMVState.NORMAL || mtmvStatus.getRefreshState() == MTMVRefreshState.INIT) { return res; } MTMVRefreshContext refreshContext = null; 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 bb10996a11bf6a..258704763909f1 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 @@ -17,20 +17,13 @@ package org.apache.doris.nereids; -import org.apache.doris.catalog.DatabaseIf; -import org.apache.doris.catalog.Env; -import org.apache.doris.catalog.TableIf; import org.apache.doris.common.Pair; -import org.apache.doris.datasource.CatalogIf; import org.apache.doris.nereids.analyzer.Scope; -import org.apache.doris.nereids.analyzer.UnboundOneRowRelation; -import org.apache.doris.nereids.analyzer.UnboundRelation; -import org.apache.doris.nereids.analyzer.UnboundTableSink; -import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.hint.Hint; import org.apache.doris.nereids.jobs.Job; import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.jobs.executor.Analyzer; +import org.apache.doris.nereids.jobs.executor.TableCollector; import org.apache.doris.nereids.jobs.rewrite.RewriteBottomUpJob; import org.apache.doris.nereids.jobs.rewrite.RewriteTopDownJob; import org.apache.doris.nereids.jobs.rewrite.RootPlanTreeRewriteJob.RootRewriteJobContext; @@ -46,7 +39,6 @@ import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.rules.RuleFactory; import org.apache.doris.nereids.rules.RuleSet; -import org.apache.doris.nereids.rules.analysis.BindRelation.CustomTableResolver; import org.apache.doris.nereids.rules.exploration.mv.MaterializationContext; import org.apache.doris.nereids.trees.expressions.CTEId; import org.apache.doris.nereids.trees.expressions.Expression; @@ -54,13 +46,7 @@ import org.apache.doris.nereids.trees.expressions.SubqueryExpr; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.RelationId; -import org.apache.doris.nereids.trees.plans.logical.LogicalCTE; import org.apache.doris.nereids.trees.plans.logical.LogicalCTEConsumer; -import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; -import org.apache.doris.nereids.trees.plans.logical.LogicalHaving; -import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; -import org.apache.doris.nereids.trees.plans.logical.LogicalProject; -import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; import org.apache.doris.statistics.ColumnStatistic; @@ -70,7 +56,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; -import org.apache.commons.collections.MapUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -84,8 +69,6 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; -import java.util.Stack; -import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; import javax.annotation.Nullable; @@ -103,7 +86,7 @@ public class CascadesContext implements ScheduleContext { private Memo memo; private final StatementContext statementContext; - private final CTEContext cteContext; + private CTEContext cteContext; private final RuleSet ruleSet; private final JobPool jobPool; private final JobScheduler jobScheduler; @@ -113,7 +96,6 @@ public class CascadesContext implements ScheduleContext { private final RuntimeFilterContext runtimeFilterContext; private final TopnFilterContext topnFilterContext = new TopnFilterContext(); private Optional outerScope = Optional.empty(); - private Map, TableIf> tables = null; private boolean isRewriteRoot; private volatile boolean isTimeout = false; @@ -169,9 +151,6 @@ private CascadesContext(Optional parent, Optional curren } else { this.isEnableExprTrace = false; } - if (parent.isPresent()) { - this.tables = parent.get().tables; - } this.isLeadingDisableJoinReorder = isLeadingDisableJoinReorder; } @@ -245,12 +224,12 @@ public void toMemo() { this.memo = new Memo(getConnectContext(), plan); } - public Analyzer newAnalyzer() { - return newAnalyzer(Optional.empty()); + public TableCollector newTableCollector() { + return new TableCollector(this); } - public Analyzer newAnalyzer(Optional customTableResolver) { - return new Analyzer(this, customTableResolver); + public Analyzer newAnalyzer() { + return new Analyzer(this); } @Override @@ -266,10 +245,6 @@ public void releaseMemo() { this.memo = null; } - public void setTables(Map, TableIf> tables) { - this.tables = tables; - } - public final ConnectContext getConnectContext() { return statementContext.getConnectContext(); } @@ -344,6 +319,10 @@ public CTEContext getCteContext() { return cteContext; } + public void setCteContext(CTEContext cteContext) { + this.cteContext = cteContext; + } + public void setIsRewriteRoot(boolean isRewriteRoot) { this.isRewriteRoot = isRewriteRoot; } @@ -408,204 +387,6 @@ private CascadesContext execute(Job job) { return this; } - /** - * Extract tables. - */ - public void extractTables(LogicalPlan logicalPlan) { - Set> tableNames = getTables(logicalPlan); - tables = Maps.newHashMap(); - for (List tableName : tableNames) { - try { - TableIf table = getTable(tableName); - tables.put(table.getFullQualifiers(), table); - } catch (Throwable e) { - // IGNORE - } - } - - } - - public Map, TableIf> getTables() { - if (tables == null) { - return null; - } else { - return tables; - } - } - - private Set> getTables(LogicalPlan logicalPlan) { - final Set> tableNames = new HashSet<>(); - logicalPlan.foreach(p -> { - if (p instanceof LogicalFilter) { - tableNames.addAll(extractTableNamesFromFilter((LogicalFilter) p)); - } else if (p instanceof LogicalCTE) { - tableNames.addAll(extractTableNamesFromCTE((LogicalCTE) p)); - } else if (p instanceof LogicalProject) { - tableNames.addAll(extractTableNamesFromProject((LogicalProject) p)); - } else if (p instanceof LogicalHaving) { - tableNames.addAll(extractTableNamesFromHaving((LogicalHaving) p)); - } else if (p instanceof UnboundOneRowRelation) { - tableNames.addAll(extractTableNamesFromOneRowRelation((UnboundOneRowRelation) p)); - } else { - Set logicalPlans = p.collect( - n -> (n instanceof UnboundRelation || n instanceof UnboundTableSink)); - for (LogicalPlan plan : logicalPlans) { - if (plan instanceof UnboundRelation) { - tableNames.add(((UnboundRelation) plan).getNameParts()); - } else if (plan instanceof UnboundTableSink) { - tableNames.add(((UnboundTableSink) plan).getNameParts()); - } else { - throw new AnalysisException("get tables from plan failed. meet unknown type node " + plan); - } - } - } - }); - return tableNames; - } - - public Map, TableIf> getOrExtractTables(LogicalPlan logicalPlan) { - if (MapUtils.isEmpty(tables)) { - extractTables(logicalPlan); - } - return tables; - } - - private Set> extractTableNamesFromHaving(LogicalHaving having) { - Set subqueryExprs = having.getPredicate() - .collect(SubqueryExpr.class::isInstance); - Set> tableNames = new HashSet<>(); - for (SubqueryExpr expr : subqueryExprs) { - LogicalPlan plan = expr.getQueryPlan(); - tableNames.addAll(getTables(plan)); - } - return tableNames; - } - - private Set> extractTableNamesFromOneRowRelation(UnboundOneRowRelation oneRowRelation) { - Set subqueryExprs = oneRowRelation.getProjects().stream() - .>map(p -> p.collect(SubqueryExpr.class::isInstance)) - .flatMap(Set::stream) - .collect(Collectors.toSet()); - Set> tableNames = new HashSet<>(); - for (SubqueryExpr expr : subqueryExprs) { - LogicalPlan plan = expr.getQueryPlan(); - tableNames.addAll(getTables(plan)); - } - return tableNames; - } - - private Set> extractTableNamesFromProject(LogicalProject project) { - Set subqueryExprs = project.getProjects().stream() - .>map(p -> p.collect(SubqueryExpr.class::isInstance)) - .flatMap(Set::stream) - .collect(Collectors.toSet()); - Set> tableNames = new HashSet<>(); - for (SubqueryExpr expr : subqueryExprs) { - LogicalPlan plan = expr.getQueryPlan(); - tableNames.addAll(getTables(plan)); - } - return tableNames; - } - - private Set> extractTableNamesFromFilter(LogicalFilter filter) { - Set subqueryExprs = filter.getPredicate() - .collect(SubqueryExpr.class::isInstance); - Set> tableNames = new HashSet<>(); - for (SubqueryExpr expr : subqueryExprs) { - LogicalPlan plan = expr.getQueryPlan(); - tableNames.addAll(getTables(plan)); - } - return tableNames; - } - - private Set> extractTableNamesFromCTE(LogicalCTE cte) { - List> subQueryAliases = cte.getAliasQueries(); - Set> tableNames = new HashSet<>(); - for (LogicalSubQueryAlias subQueryAlias : subQueryAliases) { - tableNames.addAll(getTables(subQueryAlias)); - } - return tableNames; - } - - private TableIf getTable(List nameParts) { - switch (nameParts.size()) { - case 1: { // table - String ctlName = getConnectContext().getEnv().getCurrentCatalog().getName(); - String dbName = getConnectContext().getDatabase(); - return getTable(ctlName, dbName, nameParts.get(0), getConnectContext().getEnv()); - } - case 2: { // db.table - String ctlName = getConnectContext().getEnv().getCurrentCatalog().getName(); - String dbName = nameParts.get(0); - return getTable(ctlName, dbName, nameParts.get(1), getConnectContext().getEnv()); - } - case 3: { // catalog.db.table - return getTable(nameParts.get(0), nameParts.get(1), nameParts.get(2), getConnectContext().getEnv()); - } - default: - throw new IllegalStateException("Table name [" + String.join(".", nameParts) + "] is invalid."); - } - } - - /** - * Find table from catalog. - */ - public TableIf getTable(String ctlName, String dbName, String tableName, Env env) { - CatalogIf catalog = env.getCatalogMgr().getCatalog(ctlName); - if (catalog == null) { - throw new RuntimeException("Catalog [" + ctlName + "] does not exist."); - } - DatabaseIf db = catalog.getDbNullable(dbName); - if (db == null) { - throw new RuntimeException("Database [" + dbName + "] does not exist in catalog [" + ctlName + "]."); - } - - TableIf table = db.getTableNullable(tableName); - if (table == null) { - throw new RuntimeException("Table [" + tableName + "] does not exist in database [" + dbName + "]."); - } - return table; - - } - - /** - * Used to lock table - */ - public static class Lock implements AutoCloseable { - - CascadesContext cascadesContext; - private final Stack locked = new Stack<>(); - - /** - * Try to acquire read locks on tables, throw runtime exception once the acquiring for read lock failed. - */ - public Lock(LogicalPlan plan, CascadesContext cascadesContext) { - this.cascadesContext = cascadesContext; - // tables can also be load from dump file - if (cascadesContext.getTables() == null || cascadesContext.getTables().isEmpty()) { - cascadesContext.extractTables(plan); - cascadesContext.getStatementContext().setTables(cascadesContext.getTables()); - } - for (TableIf table : cascadesContext.tables.values()) { - if (!table.needReadLockWhenPlan()) { - continue; - } - if (!table.tryReadLock(1, TimeUnit.MINUTES)) { - close(); - throw new RuntimeException(String.format("Failed to get read lock on table: %s", table.getName())); - } - locked.push(table); - } - } - - @Override - public void close() { - while (!locked.empty()) { - locked.pop().readUnlock(); - } - } - } - public void putCTEIdToConsumer(LogicalCTEConsumer cteConsumer) { Set consumers = this.statementContext.getCteIdToConsumers() .computeIfAbsent(cteConsumer.getCteId(), k -> new HashSet<>()); 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 16fe1353facfb6..4eafa0e2172f96 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 @@ -30,7 +30,6 @@ import org.apache.doris.common.profile.SummaryProfile; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.mysql.FieldInfo; -import org.apache.doris.nereids.CascadesContext.Lock; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.glue.translator.PhysicalPlanTranslator; @@ -156,7 +155,7 @@ public void plan(StatementBase queryStmt, org.apache.doris.thrift.TQueryOptions } @VisibleForTesting - public void planWithLock(StatementBase queryStmt) { + public void plan(StatementBase queryStmt) { try { plan(queryStmt, statementContext.getConnectContext().getSessionVariable().toThrift()); } catch (Exception e) { @@ -164,14 +163,17 @@ public void planWithLock(StatementBase queryStmt) { } } + @VisibleForTesting public PhysicalPlan planWithLock(LogicalPlan plan, PhysicalProperties outputProperties) { return (PhysicalPlan) planWithLock(plan, outputProperties, ExplainLevel.NONE, false); } + // TODO check all caller public Plan planWithLock(LogicalPlan plan, PhysicalProperties requireProperties, ExplainLevel explainLevel) { return planWithLock(plan, requireProperties, explainLevel, false); } + @VisibleForTesting public Plan planWithLock(LogicalPlan plan, PhysicalProperties requireProperties, ExplainLevel explainLevel, boolean showPlanProcess) { Consumer noCallback = p -> {}; @@ -188,9 +190,8 @@ public Plan planWithLock(LogicalPlan plan, PhysicalProperties requireProperties, * @return plan generated by this planner * @throws AnalysisException throw exception if failed in ant stage */ - public Plan planWithLock(LogicalPlan plan, PhysicalProperties requireProperties, - ExplainLevel explainLevel, boolean showPlanProcess, - Consumer lockCallback) { + private Plan planWithLock(LogicalPlan plan, PhysicalProperties requireProperties, + ExplainLevel explainLevel, boolean showPlanProcess, Consumer lockCallback) { try { long beforePlanGcTime = getGarbageCollectionTime(); if (plan instanceof LogicalSqlCache) { @@ -216,39 +217,37 @@ public Plan planWithLock(LogicalPlan plan, PhysicalProperties requireProperties, plan = preprocess(plan); initCascadesContext(plan, requireProperties); - statementContext.loadSnapshots(cascadesContext.getOrExtractTables(plan)); - try (Lock lock = new Lock(plan, cascadesContext)) { - Plan resultPlan = planWithoutLock(plan, explainLevel, showPlanProcess, requireProperties); - lockCallback.accept(resultPlan); - if (statementContext.getConnectContext().getExecutor() != null) { - statementContext.getConnectContext().getExecutor().getSummaryProfile() - .setNereidsGarbageCollectionTime(getGarbageCollectionTime() - beforePlanGcTime); - } - return resultPlan; + // collect table and lock them in the order of table id + collectAndLockTable(showAnalyzeProcess(explainLevel, showPlanProcess)); + // after table collector, we should use a new context. + statementContext.loadSnapshots(); + Plan resultPlan = planWithoutLock(plan, requireProperties, explainLevel, showPlanProcess); + lockCallback.accept(resultPlan); + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile() + .setNereidsGarbageCollectionTime(getGarbageCollectionTime() - beforePlanGcTime); } + return resultPlan; } finally { statementContext.releasePlannerResources(); } } - protected Plan planWithoutLock( - LogicalPlan plan, ExplainLevel explainLevel, - boolean showPlanProcess, PhysicalProperties requireProperties) { - // resolve column, table and function - // analyze this query - analyze(showAnalyzeProcess(explainLevel, showPlanProcess)); + /** + * do plan but not lock any table + */ + private Plan planWithoutLock( + LogicalPlan plan, PhysicalProperties requireProperties, ExplainLevel explainLevel, + boolean showPlanProcess) { // minidump of input must be serialized first, this process ensure minidump string not null try { - MinidumpUtils.serializeInputsToDumpFile(plan, cascadesContext.getTables()); + + MinidumpUtils.serializeInputsToDumpFile(plan, statementContext); } catch (IOException e) { throw new RuntimeException(e); } - - if (statementContext.getConnectContext().getExecutor() != null) { - statementContext.getConnectContext().getExecutor().getSummaryProfile().setQueryAnalysisFinishTime(); - statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsAnalysisTime(); - } - + // analyze this query, resolve column, table and function + analyze(showAnalyzeProcess(explainLevel, showPlanProcess)); if (explainLevel == ExplainLevel.ANALYZED_PLAN || explainLevel == ExplainLevel.ALL_PLAN) { analyzedPlan = cascadesContext.getRewritePlan(); if (explainLevel == ExplainLevel.ANALYZED_PLAN) { @@ -258,10 +257,6 @@ protected Plan planWithoutLock( // 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) { @@ -269,40 +264,20 @@ protected Plan planWithoutLock( } } - // if we cannot get table row count, skip join reorder - // except: - // 1. user set leading hint - // 2. ut test. In ut test, FeConstants.enableInternalSchemaDb is false or FeConstants.runningUnitTest is true - if (FeConstants.enableInternalSchemaDb && !FeConstants.runningUnitTest - && !cascadesContext.isLeadingDisableJoinReorder()) { - List scans = cascadesContext.getRewritePlan() - .collectToList(CatalogRelation.class::isInstance); - Optional disableJoinReorderReason = StatsCalculator - .disableJoinReorderIfStatsInvalid(scans, cascadesContext); - disableJoinReorderReason.ifPresent(statementContext::setDisableJoinReorderReason); - } - - setRuntimeFilterWaitTimeByTableRowCountAndType(); - 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); + LOG.info("{}\n{}", ConnectContext.get().getQueryIdentifier(), memo); } - 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); + LOG.info("{}\n{}", ConnectContext.get().getQueryIdentifier(), tree); } if (explainLevel == ExplainLevel.OPTIMIZED_PLAN || explainLevel == ExplainLevel.ALL_PLAN @@ -361,8 +336,21 @@ private void setRuntimeFilterWaitTimeByTableRowCountAndType() { private void initCascadesContext(LogicalPlan plan, PhysicalProperties requireProperties) { cascadesContext = CascadesContext.initContext(statementContext, plan, requireProperties); - if (statementContext.getTables() != null) { - cascadesContext.setTables(statementContext.getTables()); + } + + protected void collectAndLockTable(boolean showPlanProcess) { + if (LOG.isDebugEnabled()) { + LOG.debug("Start collect and lock table"); + } + keepOrShowPlanProcess(showPlanProcess, () -> cascadesContext.newTableCollector().collect()); + statementContext.lock(); + cascadesContext.setCteContext(new CTEContext()); + NereidsTracer.logImportantTime("EndCollectAndLockTables"); + if (LOG.isDebugEnabled()) { + LOG.debug("End collect and lock table"); + } + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsLockTableFinishTime(); } } @@ -376,6 +364,11 @@ protected void analyze(boolean showPlanProcess) { if (LOG.isDebugEnabled()) { LOG.debug("End analyze plan"); } + + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile().setQueryAnalysisFinishTime(); + statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsAnalysisTime(); + } } /** @@ -390,10 +383,26 @@ protected void rewrite(boolean showPlanProcess) { if (LOG.isDebugEnabled()) { LOG.debug("End rewrite plan"); } + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsRewriteTime(); + } } // DependsRules: EnsureProjectOnTopJoin.class protected void optimize() { + // if we cannot get table row count, skip join reorder + // except: + // 1. user set leading hint + // 2. ut test. In ut test, FeConstants.enableInternalSchemaDb is false or FeConstants.runningUnitTest is true + if (FeConstants.enableInternalSchemaDb && !FeConstants.runningUnitTest + && !cascadesContext.isLeadingDisableJoinReorder()) { + List scans = cascadesContext.getRewritePlan() + .collectToList(CatalogRelation.class::isInstance); + Optional disableJoinReorderReason = StatsCalculator + .disableJoinReorderIfStatsInvalid(scans, cascadesContext); + disableJoinReorderReason.ifPresent(statementContext::setDisableJoinReorderReason); + } + setRuntimeFilterWaitTimeByTableRowCountAndType(); if (LOG.isDebugEnabled()) { LOG.debug("Start optimize plan"); } @@ -402,6 +411,9 @@ protected void optimize() { if (LOG.isDebugEnabled()) { LOG.debug("End optimize plan"); } + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsOptimizeTime(); + } } protected void splitFragments(PhysicalPlan resultPlan) { @@ -685,6 +697,8 @@ public String getExplainString(ExplainOptions explainOptions) { plan = "========== PARSED PLAN " + getTimeMetricString(SummaryProfile::getPrettyParseSqlTime) + " ==========\n" + parsedPlan.treeString() + "\n\n" + + "========== LOCK TABLE " + + getTimeMetricString(SummaryProfile::getPrettyNereidsLockTableTime) + " ==========\n" + "========== ANALYZED PLAN " + getTimeMetricString(SummaryProfile::getPrettyNereidsAnalysisTime) + " ==========\n" + analyzedPlan.treeString() + "\n\n" @@ -864,7 +878,7 @@ private boolean showRewriteProcess(ExplainLevel explainLevel, boolean showPlanPr } private boolean showPlanProcess(ExplainOptions explainOptions) { - return explainOptions == null ? false : explainOptions.showPlanProcess(); + return explainOptions != null && explainOptions.showPlanProcess(); } private void keepOrShowPlanProcess(boolean showPlanProcess, Runnable task) { 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 ea3e80877db329..29be4af41a7675 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 @@ -424,6 +424,10 @@ public static class FullTableName { public String toString() { return catalog + "." + db + "." + table; } + + public List toList() { + return Lists.newArrayList(catalog, db, table); + } } /** FullColumnName */ 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 4d60b06e7c5cc1..7717c1034bb6b4 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 @@ -19,6 +19,7 @@ import org.apache.doris.analysis.StatementBase; import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.View; import org.apache.doris.catalog.constraint.TableIdentifier; import org.apache.doris.common.FormatOptions; import org.apache.doris.common.Id; @@ -42,9 +43,9 @@ import org.apache.doris.nereids.trees.plans.PlaceholderId; import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.TableId; -import org.apache.doris.nereids.trees.plans.algebra.Relation; import org.apache.doris.nereids.trees.plans.logical.LogicalCTEConsumer; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.OriginStatement; import org.apache.doris.qe.SessionVariable; @@ -54,7 +55,6 @@ import org.apache.doris.system.Backend; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; import com.google.common.base.Stopwatch; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; @@ -70,11 +70,13 @@ import java.util.ArrayList; import java.util.BitSet; import java.util.Collection; +import java.util.Comparator; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.PriorityQueue; import java.util.Set; import java.util.Stack; import java.util.TreeMap; @@ -87,6 +89,18 @@ public class StatementContext implements Closeable { private static final Logger LOG = LogManager.getLogger(StatementContext.class); + /** + * indicate where the table come from. + * QUERY: in query sql directly + * INSERT_TARGET: the insert target table + * MTMV: mtmv itself and its related tables witch do not belong to this sql, but maybe used in rewrite by mtmv. + */ + public enum TableFrom { + QUERY, + INSERT_TARGET, + MTMV + } + private ConnectContext connectContext; private final Stopwatch stopwatch = Stopwatch.createUnstarted(); @@ -140,10 +154,6 @@ public class StatementContext implements Closeable { private final List hints = new ArrayList<>(); - // Map slot to its relation, currently used in SlotReference to find its original - // Relation for example LogicalOlapScan - private final Map slotToRelation = Maps.newHashMap(); - // the columns in Plan.getExpressions(), such as columns in join condition or filter condition, group by expression private final Set keySlots = Sets.newHashSet(); private BitSet disableRules; @@ -154,8 +164,17 @@ public class StatementContext implements Closeable { // placeholder params for prepared statement private List placeholders; - // tables used for plan replayer - private Map, TableIf> tables = null; + // all tables in query + private boolean needLockTables = true; + + // tables in this query directly + private final Map, TableIf> tables = Maps.newHashMap(); + // tables maybe used by mtmv rewritten in this query + private final Map, TableIf> mtmvRelatedTables = Maps.newHashMap(); + // insert into target tables + private final Map, TableIf> insertTargetTables = Maps.newHashMap(); + // save view's def and sql mode to avoid them change before lock + private final Map, Pair> viewInfos = Maps.newHashMap(); // for create view support in nereids // key is the start and end position of the sql substring that needs to be replaced, @@ -178,7 +197,7 @@ public class StatementContext implements Closeable { private FormatOptions formatOptions = FormatOptions.getDefault(); - private List plannerHooks = new ArrayList<>(); + private final List plannerHooks = new ArrayList<>(); private String disableJoinReorderReason; @@ -220,28 +239,67 @@ public StatementContext(ConnectContext connectContext, OriginStatement originSta } } + public void setNeedLockTables(boolean needLockTables) { + this.needLockTables = needLockTables; + } + + /** + * cache view info to avoid view's def and sql mode changed before lock it. + * + * @param qualifiedViewName full qualified name of the view + * @param view view need to cache info + * + * @return view info, first is view's def sql, second is view's sql mode + */ + public Pair getAndCacheViewInfo(List qualifiedViewName, View view) { + return viewInfos.computeIfAbsent(qualifiedViewName, k -> { + String viewDef; + long sqlMode; + view.readLock(); + try { + viewDef = view.getInlineViewDef(); + sqlMode = view.getSqlMode(); + } finally { + view.readUnlock(); + } + return Pair.of(viewDef, sqlMode); + }); + } + + public Map, TableIf> getInsertTargetTables() { + return insertTargetTables; + } + + public Map, TableIf> getMtmvRelatedTables() { + return mtmvRelatedTables; + } + public Map, TableIf> getTables() { - if (tables == null) { - tables = Maps.newHashMap(); - } return tables; } public void setTables(Map, TableIf> tables) { - this.tables = tables; + this.tables.clear(); + this.tables.putAll(tables); } /** get table by table name, try to get from information from dumpfile first */ - public TableIf getTableInMinidumpCache(List tableQualifier) { - if (!getConnectContext().getSessionVariable().isPlayNereidsDump()) { - return null; - } - Preconditions.checkState(tables != null, "tables should not be null"); - TableIf table = tables.getOrDefault(tableQualifier, null); - if (getConnectContext().getSessionVariable().isPlayNereidsDump() && table == null) { - throw new AnalysisException("Minidump cache can not find table:" + tableQualifier); + public TableIf getAndCacheTable(List tableQualifier, TableFrom tableFrom) { + Map, TableIf> tables; + switch (tableFrom) { + case QUERY: + tables = this.tables; + break; + case INSERT_TARGET: + tables = this.insertTargetTables; + break; + case MTMV: + tables = this.mtmvRelatedTables; + break; + default: + throw new AnalysisException("Unknown table from " + tableFrom); } - return table; + return tables.computeIfAbsent(tableQualifier, k -> RelationUtil.getTable(k, connectContext.getEnv())); } public void setConnectContext(ConnectContext connectContext) { @@ -303,10 +361,6 @@ public Optional getSqlCacheContext() { return Optional.ofNullable(sqlCacheContext); } - public void addSlotToRelation(Slot slot, Relation relation) { - slotToRelation.put(slot, relation); - } - public boolean isDpHyp() { return isDpHyp; } @@ -475,21 +529,36 @@ public Map getRelationIdToStatisticsMap() { return relationIdToStatisticsMap; } - /** addTableReadLock */ - public synchronized void addTableReadLock(TableIf tableIf) { - if (!tableIf.needReadLockWhenPlan()) { + /** + * lock all table collect by TableCollector + */ + public synchronized void lock() { + if (!needLockTables + || (tables.isEmpty() && mtmvRelatedTables.isEmpty() && insertTargetTables.isEmpty()) + || !plannerResources.isEmpty()) { return; } - if (!tableIf.tryReadLock(1, TimeUnit.MINUTES)) { - close(); - throw new RuntimeException(String.format("Failed to get read lock on table: %s", tableIf.getName())); + PriorityQueue tableIfs = new PriorityQueue<>( + tables.size() + mtmvRelatedTables.size() + insertTargetTables.size(), + Comparator.comparing(TableIf::getId)); + tableIfs.addAll(tables.values()); + tableIfs.addAll(mtmvRelatedTables.values()); + tableIfs.addAll(insertTargetTables.values()); + while (!tableIfs.isEmpty()) { + TableIf tableIf = tableIfs.poll(); + if (!tableIf.needReadLockWhenPlan()) { + continue; + } + if (!tableIf.tryReadLock(1, TimeUnit.MINUTES)) { + close(); + throw new RuntimeException("Failed to get read lock on table:" + tableIf.getName()); + } + String fullTableName = tableIf.getNameWithFullQualifiers(); + String resourceName = "tableReadLock(" + fullTableName + ")"; + plannerResources.push(new CloseableResource( + resourceName, Thread.currentThread().getName(), + originStatement == null ? null : originStatement.originStmt, tableIf::readUnlock)); } - - String fullTableName = tableIf.getNameWithFullQualifiers(); - String resourceName = "tableReadLock(" + fullTableName + ")"; - plannerResources.push(new CloseableResource( - resourceName, Thread.currentThread().getName(), - originStatement == null ? null : originStatement.originStmt, tableIf::readUnlock)); } /** releasePlannerResources */ @@ -505,7 +574,7 @@ public synchronized void releasePlannerResources() { } } if (throwable != null) { - Throwables.propagateIfInstanceOf(throwable, RuntimeException.class); + Throwables.throwIfInstanceOf(throwable, RuntimeException.class); throw new IllegalStateException("Release resource failed", throwable); } } @@ -552,13 +621,8 @@ public void addPlannerHook(PlannerHook plannerHook) { /** * Load snapshot information of mvcc - * - * @param tables Tables used in queries */ - public void loadSnapshots(Map, TableIf> tables) { - if (tables == null) { - return; - } + public void loadSnapshots() { for (TableIf tableIf : tables.values()) { if (tableIf instanceof MvccTable) { MvccTableInfo mvccTableInfo = new MvccTableInfo(tableIf); @@ -616,7 +680,7 @@ public void close() { try { resource.close(); } catch (Throwable t) { - Throwables.propagateIfInstanceOf(t, RuntimeException.class); + Throwables.throwIfInstanceOf(t, RuntimeException.class); throw new IllegalStateException("Close resource failed: " + t.getMessage(), t); } closed = true; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundBaseExternalTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundBaseExternalTableSink.java index cfdefc59872d4e..2c88e2f4a46ff7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundBaseExternalTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundBaseExternalTableSink.java @@ -21,7 +21,6 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.UnboundLogicalProperties; -import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation; @@ -80,11 +79,6 @@ public UnboundBaseExternalTableSink withOutputExprs(List getExpressions() { - throw new UnsupportedOperationException(this.getClass().getSimpleName() + " don't support getExpression()"); - } - @Override public boolean equals(Object o) { if (this == o) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOneRowRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOneRowRelation.java index 9bc368c8ad6b18..bb61bc93574208 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOneRowRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOneRowRelation.java @@ -71,7 +71,7 @@ public List getProjects() { @Override public List getExpressions() { - throw new UnsupportedOperationException(this.getClass().getSimpleName() + " don't support getExpression()"); + return projects; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java index b8d821e1548be7..12d4a7c74be58f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java @@ -25,7 +25,6 @@ import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.UnboundLogicalProperties; import org.apache.doris.nereids.trees.TableSample; -import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation; import org.apache.doris.nereids.trees.plans.Plan; @@ -186,11 +185,6 @@ public R accept(PlanVisitor visitor, C context) { return visitor.visitUnboundRelation(this, context); } - @Override - public List getExpressions() { - throw new UnsupportedOperationException(this.getClass().getSimpleName() + " don't support getExpression()"); - } - public List getPartNames() { return partNames; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java index 5fd5c18a365d04..d57e518824d3aa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java @@ -21,7 +21,6 @@ import org.apache.doris.nereids.exceptions.UnboundException; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; -import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation; @@ -64,11 +63,6 @@ public R accept(PlanVisitor visitor, C context) { return visitor.visitUnboundResultSink(this, context); } - @Override - public List getExpressions() { - throw new UnsupportedOperationException(this.getClass().getSimpleName() + " don't support getExpression()"); - } - @Override public Plan withGroupExpression(Optional groupExpression) { return new UnboundResultSink<>(groupExpression, Optional.of(getLogicalProperties()), child()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFRelation.java index e876825af6569a..3024058edc7a5d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFRelation.java @@ -21,7 +21,6 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.UnboundLogicalProperties; -import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Properties; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.table.TableValuedFunction; @@ -79,11 +78,6 @@ public R accept(PlanVisitor visitor, C context) { return visitor.visitUnboundTVFRelation(this, context); } - @Override - public List getExpressions() { - throw new UnsupportedOperationException(this.getClass().getSimpleName() + " don't support getExpression()"); - } - @Override public List computeOutput() { throw new UnboundException("output"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSink.java index 23c58ba42fb17e..0e528227dc9742 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSink.java @@ -21,7 +21,6 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.UnboundLogicalProperties; -import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation; @@ -135,11 +134,6 @@ public R accept(PlanVisitor visitor, C context) { return visitor.visitUnboundTableSink(this, context); } - @Override - public List getExpressions() { - throw new UnsupportedOperationException(this.getClass().getSimpleName() + " don't support getExpression()"); - } - @Override public boolean equals(Object o) { if (this == o) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java index 03dbb6c7110a7c..e05fead5901509 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java @@ -24,7 +24,6 @@ import org.apache.doris.nereids.rules.analysis.AnalyzeCTE; import org.apache.doris.nereids.rules.analysis.BindExpression; import org.apache.doris.nereids.rules.analysis.BindRelation; -import org.apache.doris.nereids.rules.analysis.BindRelation.CustomTableResolver; import org.apache.doris.nereids.rules.analysis.BindSink; import org.apache.doris.nereids.rules.analysis.CheckAfterBind; import org.apache.doris.nereids.rules.analysis.CheckAnalysis; @@ -58,8 +57,6 @@ import com.google.common.collect.ImmutableSet; import java.util.List; -import java.util.Objects; -import java.util.Optional; /** * Bind symbols according to metadata in the catalog, perform semantic analysis, etc. @@ -67,38 +64,20 @@ */ public class Analyzer extends AbstractBatchJobExecutor { - public static final List ANALYZE_JOBS = buildAnalyzeJobs(Optional.empty()); - - private final List jobs; - - /** - * Execute the analysis job with scope. - * @param cascadesContext planner context for execute job - */ - public Analyzer(CascadesContext cascadesContext) { - this(cascadesContext, Optional.empty()); - } + public static final List ANALYZE_JOBS = buildAnalyzeJobs(); /** * constructor of Analyzer. For view, we only do bind relation since other analyze step will do by outer Analyzer. * * @param cascadesContext current context for analyzer - * @param customTableResolver custom resolver for outer catalog. */ - public Analyzer(CascadesContext cascadesContext, Optional customTableResolver) { + public Analyzer(CascadesContext cascadesContext) { super(cascadesContext); - Objects.requireNonNull(customTableResolver, "customTableResolver cannot be null"); - - if (customTableResolver.isPresent()) { - this.jobs = buildAnalyzeJobs(customTableResolver); - } else { - this.jobs = ANALYZE_JOBS; - } } @Override public List getJobs() { - return jobs; + return ANALYZE_JOBS; } /** @@ -108,20 +87,20 @@ public void analyze() { execute(); } - private static List buildAnalyzeJobs(Optional customTableResolver) { + private static List buildAnalyzeJobs() { return notTraverseChildrenOf( ImmutableSet.of(LogicalView.class, LogicalCTEAnchor.class), - () -> buildAnalyzerJobs(customTableResolver) + Analyzer::buildAnalyzerJobs ); } - private static List buildAnalyzerJobs(Optional customTableResolver) { + private static List buildAnalyzerJobs() { return jobs( // we should eliminate hint before "Subquery unnesting". topDown(new AnalyzeCTE()), topDown(new EliminateLogicalSelectHint()), bottomUp( - new BindRelation(customTableResolver), + new BindRelation(), new CheckPolicy() ), bottomUp(new BindExpression()), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/TableCollector.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/TableCollector.java new file mode 100644 index 00000000000000..0ae433262efeb9 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/TableCollector.java @@ -0,0 +1,71 @@ +// 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.jobs.executor; + +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.jobs.rewrite.RewriteJob; +import org.apache.doris.nereids.rules.analysis.CollectRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalView; + +import com.google.common.collect.ImmutableSet; + +import java.util.List; + +/** + * Bind symbols according to metadata in the catalog, perform semantic analysis, etc. + * TODO: revisit the interface after subquery analysis is supported. + */ +public class TableCollector extends AbstractBatchJobExecutor { + + public static final List COLLECT_JOBS = buildCollectTableJobs(); + + /** + * constructor of Analyzer. For view, we only do bind relation since other analyze step will do by outer Analyzer. + * + * @param cascadesContext current context for analyzer + */ + public TableCollector(CascadesContext cascadesContext) { + super(cascadesContext); + + } + + @Override + public List getJobs() { + return COLLECT_JOBS; + } + + /** + * nereids analyze sql. + */ + public void collect() { + execute(); + } + + private static List buildCollectTableJobs() { + return notTraverseChildrenOf( + ImmutableSet.of(LogicalView.class), + TableCollector::buildCollectorJobs + ); + } + + private static List buildCollectorJobs() { + return jobs( + topDown(new CollectRelation()) + ); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/minidump/MinidumpUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/minidump/MinidumpUtils.java index c0f88b25341cde..a369772f404b2c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/minidump/MinidumpUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/minidump/MinidumpUtils.java @@ -268,7 +268,7 @@ public static JSONObject executeSql(String sql) { } NereidsPlanner nereidsPlanner = new NereidsPlanner( new StatementContext(ConnectContext.get(), new OriginStatement(sql, 0))); - nereidsPlanner.planWithLock(LogicalPlanAdapter.of(parsed)); + nereidsPlanner.plan(LogicalPlanAdapter.of(parsed)); return ((AbstractPlan) nereidsPlanner.getOptimizedPlan()).toJson(); } @@ -554,10 +554,10 @@ private static JSONObject serializeInputs(Plan parsedPlan, Map, Tab /** * This function is used to serialize inputs of one query * @param parsedPlan input plan - * @param tables all tables relative to this query + * @param statementContext context for this query * @throws IOException this will write to disk, so io exception should be dealed with */ - public static void serializeInputsToDumpFile(Plan parsedPlan, Map, TableIf> tables) + public static void serializeInputsToDumpFile(Plan parsedPlan, StatementContext statementContext) throws IOException { ConnectContext connectContext = ConnectContext.get(); // when playing minidump file, we do not save input again. @@ -566,7 +566,10 @@ public static void serializeInputsToDumpFile(Plan parsedPlan, Map, } MinidumpUtils.init(); - connectContext.setMinidump(serializeInputs(parsedPlan, tables)); + Map, TableIf> allTablesUsedInQuery = Maps.newHashMap(); + allTablesUsedInQuery.putAll(statementContext.getTables()); + allTablesUsedInQuery.putAll(statementContext.getMtmvRelatedTables()); + connectContext.setMinidump(serializeInputs(parsedPlan, allTablesUsedInQuery)); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 0332123f9ff584..bb344e1b376deb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -916,7 +916,8 @@ public LogicalPlan visitInsertTable(InsertTableContext ctx) { command = new InsertOverwriteTableCommand(sink, labelName, cte); } else { if (ConnectContext.get() != null && ConnectContext.get().isTxnModel() - && sink.child() instanceof LogicalInlineTable) { + && sink.child() instanceof LogicalInlineTable + && sink.child().getExpressions().stream().allMatch(Expression::isConstant)) { // FIXME: In legacy, the `insert into select 1` is handled as `insert into values`. // In nereids, the original way is throw an AnalysisException and fallback to legacy. // Now handle it as `insert into select`(a separate load job), should fix it as the legacy. diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index d348889818a5dd..4cf3c75b68dc43 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -26,9 +26,14 @@ public enum RuleType { // just for UT TEST_REWRITE(RuleTypeClass.REWRITE), - // binding rules - // **** make sure BINDING_UNBOUND_LOGICAL_PLAN is the lowest priority in the rewrite rules. **** + // collect relation rules + COLLECT_TABLE_FROM_CTE(RuleTypeClass.REWRITE), + COLLECT_TABLE_FROM_RELATION(RuleTypeClass.REWRITE), + COLLECT_TABLE_FROM_SINK(RuleTypeClass.REWRITE), + COLLECT_TABLE_FROM_OTHER(RuleTypeClass.REWRITE), + + // binding rules BINDING_RESULT_SINK(RuleTypeClass.REWRITE), BINDING_INSERT_HIVE_TABLE(RuleTypeClass.REWRITE), BINDING_INSERT_ICEBERG_TABLE(RuleTypeClass.REWRITE), 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 c7d4e9f975e50a..583244f0902896 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 @@ -39,6 +39,7 @@ import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.SqlCacheContext; import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.StatementContext.TableFrom; import org.apache.doris.nereids.analyzer.Unbound; import org.apache.doris.nereids.analyzer.UnboundRelation; import org.apache.doris.nereids.analyzer.UnboundResultSink; @@ -98,25 +99,14 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; -import java.util.function.Function; /** * Rule to bind relations in query plan. */ public class BindRelation extends OneAnalysisRuleFactory { - private final Optional customTableResolver; + public BindRelation() {} - public BindRelation() { - this(Optional.empty()); - } - - public BindRelation(Optional customTableResolver) { - this.customTableResolver = customTableResolver; - } - - // TODO: cte will be copied to a sub-query with different names but the id of the unbound relation in them - // are the same, so we use new relation id when binding relation, and will fix this bug later. @Override public Rule build() { return unboundRelation().thenApply(ctx -> { @@ -168,23 +158,10 @@ private LogicalPlan bindWithCurrentDb(CascadesContext cascadesContext, UnboundRe return consumer; } } - List tableQualifier = RelationUtil.getQualifierName(cascadesContext.getConnectContext(), - unboundRelation.getNameParts()); - TableIf table = null; - table = ConnectContext.get().getStatementContext().getTableInMinidumpCache(tableQualifier); - if (table == null) { - if (customTableResolver.isPresent()) { - table = customTableResolver.get().apply(tableQualifier); - } - } - // In some cases even if we have already called the "cascadesContext.getTableByName", - // it also gets the null. So, we just check it in the catalog again for safety. - if (table == null) { - table = RelationUtil.getTable(tableQualifier, cascadesContext.getConnectContext().getEnv()); - } - ConnectContext.get().getStatementContext().getTables().put(tableQualifier, table); + List tableQualifier = RelationUtil.getQualifierName( + cascadesContext.getConnectContext(), unboundRelation.getNameParts()); + TableIf table = cascadesContext.getStatementContext().getAndCacheTable(tableQualifier, TableFrom.QUERY); - // TODO: should generate different Scan sub class according to table's type LogicalPlan scan = getLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext); if (cascadesContext.isLeadingJoin()) { LeadingHint leading = (LeadingHint) cascadesContext.getHintMap().get("Leading"); @@ -197,17 +174,7 @@ private LogicalPlan bindWithCurrentDb(CascadesContext cascadesContext, UnboundRe private LogicalPlan bind(CascadesContext cascadesContext, UnboundRelation unboundRelation) { List tableQualifier = RelationUtil.getQualifierName(cascadesContext.getConnectContext(), unboundRelation.getNameParts()); - TableIf table = null; - if (customTableResolver.isPresent()) { - table = customTableResolver.get().apply(tableQualifier); - } - table = ConnectContext.get().getStatementContext().getTableInMinidumpCache(tableQualifier); - // In some cases even if we have already called the "cascadesContext.getTableByName", - // it also gets the null. So, we just check it in the catalog again for safety. - if (table == null) { - table = RelationUtil.getTable(tableQualifier, cascadesContext.getConnectContext().getEnv()); - } - ConnectContext.get().getStatementContext().getTables().put(tableQualifier, table); + TableIf table = cascadesContext.getStatementContext().getAndCacheTable(tableQualifier, TableFrom.QUERY); return getLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext); } @@ -415,8 +382,7 @@ private LogicalPlan getLogicalPlan(TableIf table, UnboundRelation unboundRelatio case VIEW: View view = (View) table; isView = true; - String inlineViewDef = view.getInlineViewDef(); - Plan viewBody = parseAndAnalyzeView(view, inlineViewDef, cascadesContext); + Plan viewBody = parseAndAnalyzeDorisView(view, qualifiedTableName, cascadesContext); LogicalView logicalView = new LogicalView<>(view, viewBody); return new LogicalSubQueryAlias<>(qualifiedTableName, logicalView); case HMS_EXTERNAL_TABLE: @@ -496,6 +462,17 @@ private Plan parseAndAnalyzeHiveView( } } + private Plan parseAndAnalyzeDorisView(View view, List tableQualifier, CascadesContext parentContext) { + Pair viewInfo = parentContext.getStatementContext().getAndCacheViewInfo(tableQualifier, view); + long originalSqlMode = parentContext.getConnectContext().getSessionVariable().getSqlMode(); + parentContext.getConnectContext().getSessionVariable().setSqlMode(viewInfo.second); + try { + return parseAndAnalyzeView(view, viewInfo.first, parentContext); + } finally { + parentContext.getConnectContext().getSessionVariable().setSqlMode(originalSqlMode); + } + } + private Plan parseAndAnalyzeView(TableIf view, String ddlSql, CascadesContext parentContext) { parentContext.getStatementContext().addViewDdlSql(ddlSql); Optional sqlCacheContext = parentContext.getStatementContext().getSqlCacheContext(); @@ -510,7 +487,7 @@ private Plan parseAndAnalyzeView(TableIf view, String ddlSql, CascadesContext pa CascadesContext viewContext = CascadesContext.initContext( parentContext.getStatementContext(), parsedViewPlan, PhysicalProperties.ANY); viewContext.keepOrShowPlanProcess(parentContext.showPlanProcess(), () -> { - viewContext.newAnalyzer(customTableResolver).analyze(); + viewContext.newAnalyzer().analyze(); }); parentContext.addPlanProcesses(viewContext.getPlanProcesses()); // we should remove all group expression of the plan which in other memo, so the groupId would not conflict @@ -543,7 +520,4 @@ private List getPartitionIds(TableIf t, UnboundRelation unboundRelation, L return part.getId(); }).collect(ImmutableList.toImmutableList()); } - - /** CustomTableResolver */ - public interface CustomTableResolver extends Function, TableIf> {} } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java new file mode 100644 index 00000000000000..9c6e3adbe74e1b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java @@ -0,0 +1,228 @@ +// 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.rules.analysis; + +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.View; +import org.apache.doris.common.Pair; +import org.apache.doris.mtmv.BaseTableInfo; +import org.apache.doris.nereids.CTEContext; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.StatementContext.TableFrom; +import org.apache.doris.nereids.analyzer.UnboundRelation; +import org.apache.doris.nereids.analyzer.UnboundResultSink; +import org.apache.doris.nereids.analyzer.UnboundTableSink; +import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.pattern.MatchingContext; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.CTEId; +import org.apache.doris.nereids.trees.expressions.SubqueryExpr; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTE; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; +import org.apache.doris.nereids.util.RelationUtil; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Rule to bind relations in query plan. + */ +public class CollectRelation implements AnalysisRuleFactory { + + private static final Logger LOG = LogManager.getLogger(CollectRelation.class); + + public CollectRelation() {} + + @Override + public List buildRules() { + return ImmutableList.of( + // should collect table from cte first to fill collect all cte name to avoid collect wrong table. + logicalCTE() + .thenApply(ctx -> { + ctx.cascadesContext.setCteContext(collectFromCte(ctx.root, ctx.cascadesContext)); + return null; + }) + .toRule(RuleType.COLLECT_TABLE_FROM_CTE), + unboundRelation() + .thenApply(this::collectFromUnboundRelation) + .toRule(RuleType.COLLECT_TABLE_FROM_RELATION), + unboundTableSink() + .thenApply(this::collectFromUnboundTableSink) + .toRule(RuleType.COLLECT_TABLE_FROM_SINK), + any().whenNot(UnboundRelation.class::isInstance) + .whenNot(UnboundTableSink.class::isInstance) + .thenApply(this::collectFromAny) + .toRule(RuleType.COLLECT_TABLE_FROM_OTHER) + ); + } + + /** + * register and store CTEs in CTEContext + */ + private CTEContext collectFromCte( + LogicalCTE logicalCTE, CascadesContext cascadesContext) { + CTEContext outerCteCtx = cascadesContext.getCteContext(); + List> aliasQueries = logicalCTE.getAliasQueries(); + for (LogicalSubQueryAlias aliasQuery : aliasQueries) { + // we should use a chain to ensure visible of cte + LogicalPlan parsedCtePlan = (LogicalPlan) aliasQuery.child(); + CascadesContext innerCascadesCtx = CascadesContext.newContextWithCteContext( + cascadesContext, parsedCtePlan, outerCteCtx); + innerCascadesCtx.newTableCollector().collect(); + LogicalPlan analyzedCtePlan = (LogicalPlan) innerCascadesCtx.getRewritePlan(); + // cteId is not used in CollectTable stage + CTEId cteId = new CTEId(0); + LogicalSubQueryAlias logicalSubQueryAlias = + aliasQuery.withChildren(ImmutableList.of(analyzedCtePlan)); + outerCteCtx = new CTEContext(cteId, logicalSubQueryAlias, outerCteCtx); + outerCteCtx.setAnalyzedPlan(logicalSubQueryAlias); + } + return outerCteCtx; + } + + private Plan collectFromAny(MatchingContext ctx) { + Set subqueryExprs = ctx.root.getExpressions().stream() + .>map(p -> p.collect(SubqueryExpr.class::isInstance)) + .flatMap(Set::stream) + .collect(Collectors.toSet()); + for (SubqueryExpr subqueryExpr : subqueryExprs) { + CascadesContext subqueryContext = CascadesContext.newContextWithCteContext( + ctx.cascadesContext, subqueryExpr.getQueryPlan(), ctx.cteContext); + subqueryContext.keepOrShowPlanProcess(ctx.cascadesContext.showPlanProcess(), + () -> subqueryContext.newTableCollector().collect()); + ctx.cascadesContext.addPlanProcesses(subqueryContext.getPlanProcesses()); + } + return null; + } + + private Plan collectFromUnboundTableSink(MatchingContext> ctx) { + List nameParts = ctx.root.getNameParts(); + switch (nameParts.size()) { + case 1: + // table + // Use current database name from catalog. + case 2: + // db.table + // Use database name from table name parts. + case 3: + // catalog.db.table + // Use catalog and database name from name parts. + collectFromUnboundRelation(ctx.cascadesContext, nameParts, TableFrom.INSERT_TARGET); + return null; + default: + throw new IllegalStateException("Insert target name is invalid."); + } + } + + private Plan collectFromUnboundRelation(MatchingContext ctx) { + List nameParts = ctx.root.getNameParts(); + switch (nameParts.size()) { + case 1: + // table + // Use current database name from catalog. + case 2: + // db.table + // Use database name from table name parts. + case 3: + // catalog.db.table + // Use catalog and database name from name parts. + collectFromUnboundRelation(ctx.cascadesContext, nameParts, TableFrom.QUERY); + return null; + default: + throw new IllegalStateException("Table name [" + ctx.root.getTableName() + "] is invalid."); + } + } + + private void collectFromUnboundRelation(CascadesContext cascadesContext, + List nameParts, TableFrom tableFrom) { + if (nameParts.size() == 1) { + String tableName = nameParts.get(0); + // check if it is a CTE's name + CTEContext cteContext = cascadesContext.getCteContext().findCTEContext(tableName).orElse(null); + if (cteContext != null) { + Optional analyzedCte = cteContext.getAnalyzedCTEPlan(tableName); + if (analyzedCte.isPresent()) { + return; + } + } + } + List tableQualifier = RelationUtil.getQualifierName(cascadesContext.getConnectContext(), nameParts); + TableIf table = cascadesContext.getConnectContext().getStatementContext() + .getAndCacheTable(tableQualifier, tableFrom); + LOG.info("collect table {} from {}", nameParts, tableFrom); + if (tableFrom == TableFrom.QUERY) { + collectMTMVCandidates(table, cascadesContext); + } + if (table instanceof View) { + parseAndCollectFromView(tableQualifier, (View) table, cascadesContext); + } + } + + private void collectMTMVCandidates(TableIf table, CascadesContext cascadesContext) { + if (cascadesContext.getConnectContext().getSessionVariable().enableMaterializedViewRewrite) { + Set mtmvSet = Env.getCurrentEnv().getMtmvService().getRelationManager() + .getAllMTMVs(Lists.newArrayList(new BaseTableInfo(table))); + LOG.info("table {} related mv set is {}", new BaseTableInfo(table), mtmvSet); + for (MTMV mtmv : mtmvSet) { + cascadesContext.getStatementContext().getMtmvRelatedTables().put(mtmv.getFullQualifiers(), mtmv); + mtmv.readMvLock(); + try { + for (BaseTableInfo baseTableInfo : mtmv.getRelation().getBaseTables()) { + LOG.info("mtmv {} related base table include {}", new BaseTableInfo(mtmv), baseTableInfo); + cascadesContext.getStatementContext().getAndCacheTable(baseTableInfo.toList(), TableFrom.MTMV); + } + } finally { + mtmv.readMvUnlock(); + } + } + } + } + + private void parseAndCollectFromView(List tableQualifier, View view, CascadesContext parentContext) { + Pair viewInfo = parentContext.getStatementContext().getAndCacheViewInfo(tableQualifier, view); + long originalSqlMode = parentContext.getConnectContext().getSessionVariable().getSqlMode(); + parentContext.getConnectContext().getSessionVariable().setSqlMode(viewInfo.second); + LogicalPlan parsedViewPlan; + try { + parsedViewPlan = new NereidsParser().parseSingle(viewInfo.first); + } finally { + parentContext.getConnectContext().getSessionVariable().setSqlMode(originalSqlMode); + } + if (parsedViewPlan instanceof UnboundResultSink) { + parsedViewPlan = (LogicalPlan) ((UnboundResultSink) parsedViewPlan).child(); + } + CascadesContext viewContext = CascadesContext.initContext( + parentContext.getStatementContext(), parsedViewPlan, PhysicalProperties.ANY); + viewContext.keepOrShowPlanProcess(parentContext.showPlanProcess(), + () -> viewContext.newTableCollector().collect()); + parentContext.addPlanProcesses(viewContext.getPlanProcesses()); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AsyncMaterializationContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AsyncMaterializationContext.java index 96d37ad546a7b4..593ad986ca797c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AsyncMaterializationContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AsyncMaterializationContext.java @@ -131,11 +131,6 @@ boolean isFinalChosen(Relation relation) { @Override public Plan getScanPlan(StructInfo queryInfo, CascadesContext cascadesContext) { - // If try to get scan plan or rewrite successfully, try to get mv read lock to avoid meta data inconsistent, - // try to get lock which should added before RBO - if (!this.isSuccess()) { - cascadesContext.getStatementContext().addTableReadLock(this.getMtmv()); - } super.getScanPlan(queryInfo, cascadesContext); return scanPlan; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java index 4f8198e0b3c0bd..db270390f9bdd3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java @@ -33,9 +33,6 @@ import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.PlannerHook; import org.apache.doris.nereids.parser.NereidsParser; -import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.visitor.TableCollector; -import org.apache.doris.nereids.trees.plans.visitor.TableCollector.TableCollectorContext; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; @@ -80,24 +77,11 @@ public void initMaterializationContext(CascadesContext cascadesContext) { */ protected void doInitMaterializationContext(CascadesContext cascadesContext) { if (cascadesContext.getConnectContext().getSessionVariable().isInDebugMode()) { - LOG.info(String.format("MaterializationContext init return because is in debug mode, current queryId is %s", - cascadesContext.getConnectContext().getQueryIdentifier())); + LOG.info("MaterializationContext init return because is in debug mode, current queryId is {}", + cascadesContext.getConnectContext().getQueryIdentifier()); return; } - // Only collect the table or mv which query use directly, to avoid useless mv partition in rewrite - // Keep use one connection context when in query, if new connect context, - // the ConnectionContext.get() will change - TableCollectorContext collectorContext = new TableCollectorContext(Sets.newHashSet(), false, - cascadesContext.getConnectContext()); - try { - Plan rewritePlan = cascadesContext.getRewritePlan(); - rewritePlan.accept(TableCollector.INSTANCE, collectorContext); - } catch (Exception e) { - LOG.warn(String.format("MaterializationContext init table collect fail, current queryId is %s", - cascadesContext.getConnectContext().getQueryIdentifier()), e); - return; - } - Set collectedTables = collectorContext.getCollectedTables(); + Set collectedTables = Sets.newHashSet(cascadesContext.getStatementContext().getTables().values()); if (collectedTables.isEmpty()) { return; } @@ -115,7 +99,7 @@ protected void doInitMaterializationContext(CascadesContext cascadesContext) { } // Create async materialization context for (MaterializationContext context : createAsyncMaterializationContext(cascadesContext, - collectorContext.getCollectedTables())) { + collectedTables)) { cascadesContext.addMaterializationContext(context); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java index 4ddb93409379e9..20aad9ecdb25c6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java @@ -128,11 +128,10 @@ public static RelatedTableInfo getRelatedTableInfo(String column, String timeUni materializedViewPlan = new LogicalProject<>(ImmutableList.of(columnExpr), materializedViewPlan); } // Collect table relation map which is used to identify self join - List catalogRelationObjs = - materializedViewPlan.collectToList(CatalogRelation.class::isInstance); + List catalogRelations = materializedViewPlan.collectToList(CatalogRelation.class::isInstance); ImmutableMultimap.Builder tableCatalogRelationMultimapBuilder = ImmutableMultimap.builder(); - for (CatalogRelation catalogRelation : catalogRelationObjs) { + for (CatalogRelation catalogRelation : catalogRelations) { tableCatalogRelationMultimapBuilder.put(new TableIdentifier(catalogRelation.getTable()), catalogRelation); } // Check sql pattern @@ -320,6 +319,7 @@ public static MTMVCache createMTMVCache(String querySql, ConnectContext connectC LogicalPlan unboundMvPlan = new NereidsParser().parseSingle(querySql); StatementContext mvSqlStatementContext = new StatementContext(connectContext, new OriginStatement(querySql, 0)); + mvSqlStatementContext.setNeedLockTables(false); NereidsPlanner planner = new NereidsPlanner(mvSqlStatementContext); if (mvSqlStatementContext.getConnectContext().getStatementContext() == null) { mvSqlStatementContext.getConnectContext().setStatementContext(mvSqlStatementContext); @@ -771,7 +771,7 @@ public static final class RelatedTableInfo { private final String column; private final Set failReasons = new HashSet<>(); // This records the partition expression if exist - private Optional partitionExpression; + private final Optional partitionExpression; public RelatedTableInfo(BaseTableInfo tableInfo, boolean pctPossible, String column, String failReason, Expression partitionExpression) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AddConstraintCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AddConstraintCommand.java index 08954741c806aa..f92a3b6103b345 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AddConstraintCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AddConstraintCommand.java @@ -19,6 +19,7 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.common.Pair; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.properties.PhysicalProperties; @@ -34,9 +35,12 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.Comparator; +import java.util.List; import java.util.Set; /** @@ -61,15 +65,26 @@ public AddConstraintCommand(String name, Constraint constraint) { @Override public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { Pair, TableIf> columnsAndTable = extractColumnsAndTable(ctx, constraint.toProject()); + List tables = Lists.newArrayList(columnsAndTable.second); + Pair, TableIf> referencedColumnsAndTable = null; if (constraint.isForeignKey()) { - Pair, TableIf> referencedColumnsAndTable - = extractColumnsAndTable(ctx, constraint.toReferenceProject()); - columnsAndTable.second.addForeignConstraint(name, columnsAndTable.first, - referencedColumnsAndTable.second, referencedColumnsAndTable.first, false); - } else if (constraint.isPrimaryKey()) { - columnsAndTable.second.addPrimaryKeyConstraint(name, columnsAndTable.first, false); - } else if (constraint.isUnique()) { - columnsAndTable.second.addUniqueConstraint(name, columnsAndTable.first, false); + referencedColumnsAndTable = extractColumnsAndTable(ctx, constraint.toReferenceProject()); + tables.add(referencedColumnsAndTable.second); + } + tables.sort((Comparator.comparing(TableIf::getId))); + MetaLockUtils.writeLockTables(tables); + try { + if (constraint.isForeignKey()) { + Preconditions.checkState(referencedColumnsAndTable != null); + columnsAndTable.second.addForeignConstraint(name, columnsAndTable.first, + referencedColumnsAndTable.second, referencedColumnsAndTable.first, false); + } else if (constraint.isPrimaryKey()) { + columnsAndTable.second.addPrimaryKeyConstraint(name, columnsAndTable.first, false); + } else if (constraint.isUnique()) { + columnsAndTable.second.addUniqueConstraint(name, columnsAndTable.first, false); + } + } finally { + MetaLockUtils.writeUnlockTables(tables); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CommandUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CommandUtils.java deleted file mode 100644 index f9b0c3e18d1b2f..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CommandUtils.java +++ /dev/null @@ -1,49 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.nereids.trees.plans.commands; - -import org.apache.doris.catalog.KeysType; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.TableIf; -import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.util.RelationUtil; -import org.apache.doris.qe.ConnectContext; - -import java.util.List; - -/** - * delete from unique key table. - */ -public class CommandUtils { - - /** - * check delete target table should unique key olap table. If ok, return it. - */ - public static OlapTable checkAndGetDeleteTargetTable(ConnectContext ctx, List nameParts) { - List qualifiedTableName = RelationUtil.getQualifierName(ctx, nameParts); - TableIf table = RelationUtil.getTable(qualifiedTableName, ctx.getEnv()); - if (!(table instanceof OlapTable)) { - throw new AnalysisException("table must be olapTable in delete command"); - } - OlapTable targetTable = ((OlapTable) table); - if (targetTable.getKeysType() != KeysType.UNIQUE_KEYS) { - throw new AnalysisException("Nereids only support delete command on unique key table now"); - } - return targetTable; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropConstraintCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropConstraintCommand.java index fe777ea82972fc..63d432121599e0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropConstraintCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropConstraintCommand.java @@ -18,6 +18,9 @@ package org.apache.doris.nereids.trees.plans.commands; import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.constraint.Constraint; +import org.apache.doris.catalog.constraint.PrimaryKeyConstraint; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.properties.PhysicalProperties; @@ -30,9 +33,12 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.StmtExecutor; +import com.google.common.collect.Lists; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.Comparator; +import java.util.List; import java.util.Set; /** @@ -56,7 +62,27 @@ public DropConstraintCommand(String name, LogicalPlan plan) { @Override public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { TableIf table = extractTable(ctx, plan); - table.dropConstraint(name, false); + List tables = Lists.newArrayList(table); + table.readLock(); + try { + Constraint constraint = table.getConstraintsMapUnsafe().get(name); + if (constraint == null) { + throw new AnalysisException( + String.format("Unknown constraint %s on table %s.", name, table.getName())); + } + if (constraint instanceof PrimaryKeyConstraint) { + tables.addAll(((PrimaryKeyConstraint) constraint).getForeignTables()); + } + } finally { + table.readUnlock(); + } + tables.sort((Comparator.comparing(TableIf::getId))); + MetaLockUtils.writeLockTables(tables); + try { + table.dropConstraint(name, false); + } finally { + MetaLockUtils.writeUnlockTables(tables); + } } private TableIf extractTable(ConnectContext ctx, LogicalPlan plan) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowConstraintsCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowConstraintsCommand.java index 5c3c16137bfe80..0e5c332058d481 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowConstraintsCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowConstraintsCommand.java @@ -52,12 +52,19 @@ public ShowConstraintsCommand(List nameParts) { public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { TableIf tableIf = RelationUtil.getDbAndTable( RelationUtil.getQualifierName(ctx, nameParts), ctx.getEnv()).value(); - List> res = tableIf.getConstraintsMap().entrySet().stream() - .map(e -> Lists.newArrayList(e.getKey(), - e.getValue().getType().getName(), - e.getValue().toString())) + tableIf.readLock(); + List> res; + try { + res = tableIf.getConstraintsMap().entrySet().stream() + .map(e -> Lists.newArrayList(e.getKey(), + e.getValue().getType().getName(), + e.getValue().toString())) .collect(Collectors.toList()); + } finally { + tableIf.readUnlock(); + } executor.handleShowConstraintStmt(res); + } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java index 417ab807cfe4bb..349379285790fa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java @@ -250,42 +250,43 @@ private void analyzeProperties() { /** * analyzeQuery */ - public void analyzeQuery(ConnectContext ctx, Map mvProperties) throws Exception { - // create table as select - StatementContext statementContext = ctx.getStatementContext(); - NereidsPlanner planner = new NereidsPlanner(statementContext); - // this is for expression column name infer when not use alias - LogicalSink logicalSink = new UnboundResultSink<>(logicalQuery); - // Should not make table without data to empty relation when analyze the related table, - // so add disable rules - Set tempDisableRules = ctx.getSessionVariable().getDisableNereidsRuleNames(); - ctx.getSessionVariable().setDisableNereidsRules(CreateMTMVInfo.MTMV_PLANER_DISABLE_RULES); - ctx.getStatementContext().invalidCache(SessionVariable.DISABLE_NEREIDS_RULES); - Plan plan; - try { - // must disable constant folding by be, because be constant folding may return wrong type - ctx.getSessionVariable().setVarOnce(SessionVariable.ENABLE_FOLD_CONSTANT_BY_BE, "false"); - plan = planner.planWithLock(logicalSink, PhysicalProperties.ANY, ExplainLevel.ALL_PLAN); - } finally { - // after operate, roll back the disable rules - ctx.getSessionVariable().setDisableNereidsRules(String.join(",", tempDisableRules)); - ctx.getStatementContext().invalidCache(SessionVariable.DISABLE_NEREIDS_RULES); - } - // can not contain VIEW or MTMV - analyzeBaseTables(planner.getAnalyzedPlan()); - // can not contain Random function - analyzeExpressions(planner.getAnalyzedPlan(), mvProperties); - // can not contain partition or tablets - boolean containTableQueryOperator = MaterializedViewUtils.containTableQueryOperator(planner.getAnalyzedPlan()); - if (containTableQueryOperator) { - throw new AnalysisException("can not contain invalid expression"); - } - getRelation(planner); - this.mvPartitionInfo = mvPartitionDefinition.analyzeAndTransferToMTMVPartitionInfo(planner, ctx); - this.partitionDesc = generatePartitionDesc(ctx); - getColumns(plan, ctx, mvPartitionInfo.getPartitionCol(), distribution); - analyzeKeys(); + public void analyzeQuery(ConnectContext ctx, Map mvProperties) { + try (StatementContext statementContext = ctx.getStatementContext()) { + NereidsPlanner planner = new NereidsPlanner(statementContext); + // this is for expression column name infer when not use alias + LogicalSink logicalSink = new UnboundResultSink<>(logicalQuery); + // Should not make table without data to empty relation when analyze the related table, + // so add disable rules + Set tempDisableRules = ctx.getSessionVariable().getDisableNereidsRuleNames(); + ctx.getSessionVariable().setDisableNereidsRules(CreateMTMVInfo.MTMV_PLANER_DISABLE_RULES); + statementContext.invalidCache(SessionVariable.DISABLE_NEREIDS_RULES); + Plan plan; + try { + // must disable constant folding by be, because be constant folding may return wrong type + ctx.getSessionVariable().setVarOnce(SessionVariable.ENABLE_FOLD_CONSTANT_BY_BE, "false"); + plan = planner.planWithLock(logicalSink, PhysicalProperties.ANY, ExplainLevel.ALL_PLAN); + } finally { + // after operate, roll back the disable rules + ctx.getSessionVariable().setDisableNereidsRules(String.join(",", tempDisableRules)); + statementContext.invalidCache(SessionVariable.DISABLE_NEREIDS_RULES); + } + // can not contain VIEW or MTMV + analyzeBaseTables(planner.getAnalyzedPlan()); + // can not contain Random function + analyzeExpressions(planner.getAnalyzedPlan(), mvProperties); + // can not contain partition or tablets + boolean containTableQueryOperator = MaterializedViewUtils.containTableQueryOperator( + planner.getAnalyzedPlan()); + if (containTableQueryOperator) { + throw new AnalysisException("can not contain invalid expression"); + } + getRelation(Sets.newHashSet(statementContext.getTables().values()), ctx); + this.mvPartitionInfo = mvPartitionDefinition.analyzeAndTransferToMTMVPartitionInfo(planner); + this.partitionDesc = generatePartitionDesc(ctx); + getColumns(plan, ctx, mvPartitionInfo.getPartitionCol(), distribution); + analyzeKeys(); + } } private void analyzeKeys() { @@ -327,8 +328,8 @@ private void analyzeKeys() { } // Should use analyzed plan for collect views and tables - private void getRelation(NereidsPlanner planner) { - this.relation = MTMVPlanUtil.generateMTMVRelation(planner.getAnalyzedPlan(), planner.getConnectContext()); + private void getRelation(Set tables, ConnectContext ctx) { + this.relation = MTMVPlanUtil.generateMTMVRelation(tables, ctx); } private PartitionDesc generatePartitionDesc(ConnectContext ctx) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java index a26a97f7240793..c2e9abd2f0f97c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java @@ -44,7 +44,6 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.scalar.DateTrunc; import org.apache.doris.nereids.trees.expressions.literal.Literal; -import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Sets; @@ -66,10 +65,9 @@ public class MTMVPartitionDefinition { * analyzeAndTransferToMTMVPartitionInfo * * @param planner planner - * @param ctx ctx * @return MTMVPartitionInfo */ - public MTMVPartitionInfo analyzeAndTransferToMTMVPartitionInfo(NereidsPlanner planner, ConnectContext ctx) { + public MTMVPartitionInfo analyzeAndTransferToMTMVPartitionInfo(NereidsPlanner planner) { MTMVPartitionInfo mtmvPartitionInfo = new MTMVPartitionInfo(partitionType); if (this.partitionType == MTMVPartitionType.SELF_MANAGE) { return mtmvPartitionInfo; @@ -77,9 +75,8 @@ public MTMVPartitionInfo analyzeAndTransferToMTMVPartitionInfo(NereidsPlanner pl String partitionColName; String timeUnit; if (this.partitionType == MTMVPartitionType.EXPR) { - String functionName = ((UnboundFunction) functionCallExpression).getName(); - if (functionCallExpression instanceof UnboundFunction - && functionName.equalsIgnoreCase(PARTITION_BY_FUNCTION_NAME)) { + if (functionCallExpression instanceof UnboundFunction && PARTITION_BY_FUNCTION_NAME + .equalsIgnoreCase(((UnboundFunction) functionCallExpression).getName())) { partitionColName = functionCallExpression.getArgument(0) instanceof UnboundSlot ? ((UnboundSlot) functionCallExpression.getArgument(0)).getName() : null; timeUnit = functionCallExpression.getArguments().get(1).isLiteral() @@ -93,7 +90,7 @@ public MTMVPartitionInfo analyzeAndTransferToMTMVPartitionInfo(NereidsPlanner pl timeUnit = null; } mtmvPartitionInfo.setPartitionCol(partitionColName); - RelatedTableInfo relatedTableInfo = getRelatedTableInfo(planner, ctx, partitionColName, timeUnit); + RelatedTableInfo relatedTableInfo = getRelatedTableInfo(planner, partitionColName, timeUnit); mtmvPartitionInfo.setRelatedCol(relatedTableInfo.getColumn()); mtmvPartitionInfo.setRelatedTable(relatedTableInfo.getTableInfo()); if (relatedTableInfo.getPartitionExpression().isPresent()) { @@ -119,8 +116,7 @@ public MTMVPartitionInfo analyzeAndTransferToMTMVPartitionInfo(NereidsPlanner pl } // Should use rewritten plan without view and subQuery to get related partition table - private RelatedTableInfo getRelatedTableInfo(NereidsPlanner planner, ConnectContext ctx, - String partitionColName, String timeUnit) { + private RelatedTableInfo getRelatedTableInfo(NereidsPlanner planner, String partitionColName, String timeUnit) { CascadesContext cascadesContext = planner.getCascadesContext(); RelatedTableInfo relatedTableInfo = MaterializedViewUtils @@ -129,10 +125,10 @@ private RelatedTableInfo getRelatedTableInfo(NereidsPlanner planner, ConnectCont throw new AnalysisException(String.format("Unable to find a suitable base table for partitioning," + " the fail reason is %s", relatedTableInfo.getFailReason())); } - MTMVRelatedTableIf mtmvBaseRealtedTable = MTMVUtil.getRelatedTable(relatedTableInfo.getTableInfo()); + MTMVRelatedTableIf mtmvBaseRelatedTable = MTMVUtil.getRelatedTable(relatedTableInfo.getTableInfo()); Set partitionColumnNames = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); try { - partitionColumnNames.addAll(mtmvBaseRealtedTable.getPartitionColumnNames(Optional.empty())); + partitionColumnNames.addAll(mtmvBaseRelatedTable.getPartitionColumnNames(Optional.empty())); } catch (DdlException e) { throw new AnalysisException(e.getMessage(), e); } @@ -140,7 +136,7 @@ private RelatedTableInfo getRelatedTableInfo(NereidsPlanner planner, ConnectCont if (!partitionColumnNames.contains(relatedTableInfo.getColumn())) { throw new AnalysisException("error related column: " + relatedTableInfo.getColumn()); } - if (!(mtmvBaseRealtedTable instanceof HMSExternalTable) + if (!(mtmvBaseRelatedTable instanceof HMSExternalTable) && partitionColumnNames.size() != 1) { throw new AnalysisException("only hms table support multi column partition."); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java index 0999c4baa79e3b..10f9947974cdb0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java @@ -25,6 +25,7 @@ import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.profile.ProfileManager.ProfileType; +import org.apache.doris.common.util.DebugUtil; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.iceberg.IcebergExternalTable; import org.apache.doris.datasource.jdbc.JdbcExternalTable; @@ -51,6 +52,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalUnion; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.planner.DataSink; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ConnectContext.ConnectType; @@ -83,13 +85,14 @@ public class InsertIntoTableCommand extends Command implements ForwardWithSync, public static final Logger LOG = LogManager.getLogger(InsertIntoTableCommand.class); + private LogicalPlan originalLogicalQuery; private LogicalPlan logicalQuery; private Optional labelName; /** * When source it's from job scheduler,it will be set. */ private long jobId; - private Optional insertCtx; + private final Optional insertCtx; private final Optional cte; /** @@ -98,7 +101,8 @@ public class InsertIntoTableCommand extends Command implements ForwardWithSync, public InsertIntoTableCommand(LogicalPlan logicalQuery, Optional labelName, Optional insertCtx, Optional cte) { super(PlanType.INSERT_INTO_TABLE_COMMAND); - this.logicalQuery = Objects.requireNonNull(logicalQuery, "logicalQuery should not be null"); + this.originalLogicalQuery = Objects.requireNonNull(logicalQuery, "logicalQuery should not be null"); + this.logicalQuery = originalLogicalQuery; this.labelName = Objects.requireNonNull(labelName, "labelName should not be null"); this.insertCtx = insertCtx; this.cte = cte; @@ -145,62 +149,95 @@ public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor executor */ public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor stmtExecutor, boolean needBeginTransaction) throws Exception { - TableIf targetTableIf = InsertUtils.getTargetTable(logicalQuery, ctx); - // check auth - if (!Env.getCurrentEnv().getAccessManager() - .checkTblPriv(ConnectContext.get(), targetTableIf.getDatabase().getCatalog().getName(), - targetTableIf.getDatabase().getFullName(), targetTableIf.getName(), - PrivPredicate.LOAD)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "LOAD", - ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(), - targetTableIf.getDatabase().getFullName() + "." + targetTableIf.getName()); + List qualifiedTargetTableName = InsertUtils.getTargetTableQualified(logicalQuery, ctx); + + AbstractInsertExecutor insertExecutor; + int retryTimes = 0; + while (++retryTimes < Math.max(ctx.getSessionVariable().dmlPlanRetryTimes, 3)) { + TableIf targetTableIf = RelationUtil.getTable(qualifiedTargetTableName, ctx.getEnv()); + // check auth + if (!Env.getCurrentEnv().getAccessManager() + .checkTblPriv(ConnectContext.get(), targetTableIf.getDatabase().getCatalog().getName(), + targetTableIf.getDatabase().getFullName(), targetTableIf.getName(), + PrivPredicate.LOAD)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "LOAD", + ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(), + targetTableIf.getDatabase().getFullName() + "." + targetTableIf.getName()); + } + BuildInsertExecutorResult buildResult; + try { + buildResult = initPlanOnce(ctx, stmtExecutor, targetTableIf); + } catch (Throwable e) { + Throwables.throwIfInstanceOf(e, RuntimeException.class); + throw new IllegalStateException(e.getMessage(), e); + } + insertExecutor = buildResult.executor; + if (!needBeginTransaction) { + return insertExecutor; + } + + // lock after plan and check does table's schema changed to ensure we lock table order by id. + TableIf newestTargetTableIf = RelationUtil.getTable(qualifiedTargetTableName, ctx.getEnv()); + newestTargetTableIf.readLock(); + try { + if (targetTableIf.getId() != newestTargetTableIf.getId()) { + LOG.warn("insert plan failed {} times. query id is {}. table id changed from {} to {}", + retryTimes, DebugUtil.printId(ctx.queryId()), + targetTableIf.getId(), newestTargetTableIf.getId()); + continue; + } + if (!targetTableIf.getFullSchema().equals(newestTargetTableIf.getFullSchema())) { + LOG.warn("insert plan failed {} times. query id is {}. table schema changed from {} to {}", + retryTimes, DebugUtil.printId(ctx.queryId()), + targetTableIf.getFullSchema(), newestTargetTableIf.getFullSchema()); + continue; + } + if (!insertExecutor.isEmptyInsert()) { + insertExecutor.beginTransaction(); + insertExecutor.finalizeSink( + buildResult.planner.getFragments().get(0), buildResult.dataSink, + buildResult.physicalSink + ); + } + newestTargetTableIf.readUnlock(); + } catch (Throwable e) { + newestTargetTableIf.readUnlock(); + // the abortTxn in onFail need to acquire table write lock + if (insertExecutor != null) { + insertExecutor.onFail(e); + } + Throwables.throwIfInstanceOf(e, RuntimeException.class); + throw new IllegalStateException(e.getMessage(), e); + } + stmtExecutor.setProfileType(ProfileType.LOAD); + // We exposed @StmtExecutor#cancel as a unified entry point for statement interruption, + // so we need to set this here + insertExecutor.getCoordinator().setTxnId(insertExecutor.getTxnId()); + stmtExecutor.setCoord(insertExecutor.getCoordinator()); + // for prepare and execute, avoiding normalization for every execute command + this.originalLogicalQuery = this.logicalQuery; + return insertExecutor; } + LOG.warn("insert plan failed {} times. query id is {}.", retryTimes, DebugUtil.printId(ctx.queryId())); + throw new AnalysisException("Insert plan failed. Could not get target table lock."); + } - AbstractInsertExecutor insertExecutor = null; - // should lock target table until we begin transaction. + private BuildInsertExecutorResult initPlanOnce(ConnectContext ctx, + StmtExecutor stmtExecutor, TableIf targetTableIf) throws Throwable { targetTableIf.readLock(); try { - // 1. process inline table (default values, empty values) - this.logicalQuery = (LogicalPlan) InsertUtils.normalizePlan(logicalQuery, targetTableIf, insertCtx); + // process inline table (default values, empty values) + this.logicalQuery = (LogicalPlan) InsertUtils.normalizePlan(originalLogicalQuery, targetTableIf, insertCtx); if (cte.isPresent()) { this.logicalQuery = ((LogicalPlan) cte.get().withChildren(logicalQuery)); } OlapGroupCommitInsertExecutor.analyzeGroupCommit(ctx, targetTableIf, this.logicalQuery, this.insertCtx); - LogicalPlanAdapter logicalPlanAdapter = new LogicalPlanAdapter(logicalQuery, ctx.getStatementContext()); - - BuildInsertExecutorResult buildResult = planInsertExecutor( - ctx, stmtExecutor, logicalPlanAdapter, targetTableIf - ); - - insertExecutor = buildResult.executor; - - if (!needBeginTransaction) { - targetTableIf.readUnlock(); - return insertExecutor; - } - if (!insertExecutor.isEmptyInsert()) { - insertExecutor.beginTransaction(); - insertExecutor.finalizeSink( - buildResult.planner.getFragments().get(0), buildResult.dataSink, buildResult.physicalSink - ); - } - targetTableIf.readUnlock(); - } catch (Throwable e) { + } finally { targetTableIf.readUnlock(); - // the abortTxn in onFail need to acquire table write lock - if (insertExecutor != null) { - insertExecutor.onFail(e); - } - Throwables.propagateIfInstanceOf(e, RuntimeException.class); - throw new IllegalStateException(e.getMessage(), e); } - stmtExecutor.setProfileType(ProfileType.LOAD); - // We exposed @StmtExecutor#cancel as a unified entry point for statement interruption, - // so we need to set this here - insertExecutor.getCoordinator().setTxnId(insertExecutor.getTxnId()); - stmtExecutor.setCoord(insertExecutor.getCoordinator()); - return insertExecutor; + LogicalPlanAdapter logicalPlanAdapter = new LogicalPlanAdapter(logicalQuery, ctx.getStatementContext()); + return planInsertExecutor(ctx, stmtExecutor, logicalPlanAdapter, targetTableIf); } // we should select the factory type first, but we can not initial InsertExecutor at this time, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java index 60e7e5bf805a64..459ffcd04f894a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java @@ -425,6 +425,14 @@ private static Expression castValue(Expression value, DataType targetType) { * get target table from names. */ public static TableIf getTargetTable(Plan plan, ConnectContext ctx) { + List tableQualifier = getTargetTableQualified(plan, ctx); + return RelationUtil.getTable(tableQualifier, ctx.getEnv()); + } + + /** + * get target table from names. + */ + public static List getTargetTableQualified(Plan plan, ConnectContext ctx) { UnboundLogicalSink unboundTableSink; if (plan instanceof UnboundTableSink) { unboundTableSink = (UnboundTableSink) plan; @@ -439,8 +447,7 @@ public static TableIf getTargetTable(Plan plan, ConnectContext ctx) { + " [UnboundTableSink, UnboundHiveTableSink, UnboundIcebergTableSink]," + " but it is " + plan.getType()); } - List tableQualifier = RelationUtil.getQualifierName(ctx, unboundTableSink.getNameParts()); - return RelationUtil.getDbAndTable(tableQualifier, ctx.getEnv()).second; + return RelationUtil.getQualifierName(ctx, unboundTableSink.getNameParts()); } private static NamedExpression generateDefaultExpression(Column column) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/TableCollector.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/TableCollector.java deleted file mode 100644 index 27ff1e4b68c075..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/TableCollector.java +++ /dev/null @@ -1,122 +0,0 @@ -// 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.visitor; - -import org.apache.doris.catalog.MTMV; -import org.apache.doris.catalog.TableIf; -import org.apache.doris.catalog.TableIf.TableType; -import org.apache.doris.common.AnalysisException; -import org.apache.doris.mtmv.MTMVCache; -import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation; -import org.apache.doris.nereids.trees.plans.physical.PhysicalCatalogRelation; -import org.apache.doris.nereids.trees.plans.visitor.TableCollector.TableCollectorContext; -import org.apache.doris.qe.ConnectContext; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.util.HashSet; -import java.util.Set; - -/** - * Collect the table in plan - * Note: will not get table if table is eliminated by EmptyRelation in rewrite. - * View expand is in RBO, if call this method with the plan after RBO, this will get base tables in view, or will not. - * Materialized view is extended or not can be controlled by the field expand - */ -public class TableCollector extends DefaultPlanVisitor { - - public static final TableCollector INSTANCE = new TableCollector(); - private static final Logger LOG = LogManager.getLogger(TableCollector.class); - - @Override - public Plan visitLogicalCatalogRelation(LogicalCatalogRelation catalogRelation, TableCollectorContext context) { - TableIf table = catalogRelation.getTable(); - if (context.getTargetTableTypes().isEmpty() || context.getTargetTableTypes().contains(table.getType())) { - context.getCollectedTables().add(table); - } - if (table instanceof MTMV) { - expandMvAndCollect((MTMV) table, context); - } - return catalogRelation; - } - - @Override - public Plan visitPhysicalCatalogRelation(PhysicalCatalogRelation catalogRelation, TableCollectorContext context) { - TableIf table = catalogRelation.getTable(); - if (context.getTargetTableTypes().isEmpty() || context.getTargetTableTypes().contains(table.getType())) { - context.getCollectedTables().add(table); - } - if (table instanceof MTMV) { - expandMvAndCollect((MTMV) table, context); - } - return catalogRelation; - } - - private void expandMvAndCollect(MTMV mtmv, TableCollectorContext context) { - if (!context.isExpandMaterializedView()) { - return; - } - // Make sure use only one connection context when in query to avoid ConnectionContext.get() wrong - MTMVCache expandedMvCache; - try { - expandedMvCache = mtmv.getOrGenerateCache(context.getConnectContext()); - } catch (AnalysisException exception) { - LOG.warn(String.format("expandMvAndCollect getOrGenerateCache fail, mtmv name is %s", mtmv.getName()), - exception); - expandedMvCache = MTMVCache.from(mtmv, context.getConnectContext(), false); - } - expandedMvCache.getAnalyzedPlan().accept(this, context); - } - - /** - * The context for table collecting, it contains the target collect table types - * and the result of collect. - */ - public static final class TableCollectorContext { - private final Set collectedTables = new HashSet<>(); - private final Set targetTableTypes; - // if expand the mv or not - private final boolean expandMaterializedView; - private final ConnectContext connectContext; - - public TableCollectorContext(Set targetTableTypes, boolean expandMaterializedView, - ConnectContext connectContext) { - this.targetTableTypes = targetTableTypes; - this.expandMaterializedView = expandMaterializedView; - this.connectContext = connectContext; - } - - public Set getCollectedTables() { - return collectedTables; - } - - public Set getTargetTableTypes() { - return targetTableTypes; - } - - public boolean isExpandMaterializedView() { - return expandMaterializedView; - } - - public ConnectContext getConnectContext() { - return connectContext; - } - } -} 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 126ed1135e9dc0..75f21c786b8c37 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 @@ -141,6 +141,7 @@ public class SessionVariable implements Serializable, Writable { public static final String PARALLEL_PIPELINE_TASK_NUM = "parallel_pipeline_task_num"; public static final String PROFILE_LEVEL = "profile_level"; public static final String MAX_INSTANCE_NUM = "max_instance_num"; + public static final String DML_PLAN_RETRY_TIMES = "DML_PLAN_RETRY_TIMES"; public static final String ENABLE_INSERT_STRICT = "enable_insert_strict"; public static final String INSERT_MAX_FILTER_RATIO = "insert_max_filter_ratio"; @@ -1008,6 +1009,17 @@ public enum IgnoreSplitType { @VariableMgr.VarAttr(name = MAX_INSTANCE_NUM) public int maxInstanceNum = 64; + @VariableMgr.VarAttr(name = DML_PLAN_RETRY_TIMES, needForward = true, description = { + "写入规划的最大重试次数。为了避免死锁,写入规划时采用了分阶段加锁。当在两次加锁中间,表结构发生变更时,会尝试重新规划。" + + "此变量限制重新规划的最大尝试次数。", + "Maximum retry attempts for write planning. To avoid deadlocks, " + + "phased locking is adopted during write planning. " + + "When changes occur to the table structure between two locking phases, " + + "re-planning will be attempted. " + + "This variable limits the maximum number of retry attempts for re-planning." + }) + public int dmlPlanRetryTimes = 3; + @VariableMgr.VarAttr(name = ENABLE_INSERT_STRICT, needForward = true) public boolean enableInsertStrict = true; 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 5c2566225fe50a..e757f3153db038 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 @@ -237,6 +237,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -1357,6 +1358,7 @@ public void analyze(TQueryOptions tQueryOptions) throws UserException, Interrupt } // table id in tableList is in ascending order because that table map is a sorted map List tables = Lists.newArrayList(tableMap.values()); + tables.sort((Comparator.comparing(TableIf::getId))); int analyzeTimes = 2; if (Config.isCloudMode()) { // be core and be restarted, need retry more times @@ -2387,6 +2389,7 @@ private void handleInsertStmt() throws Exception { response.getStatus(), i); if (i < maxRetry) { List tables = Lists.newArrayList(insertStmt.getTargetTable()); + tables.sort((Comparator.comparing(TableIf::getId))); MetaLockUtils.readLockTables(tables); try { insertStmt.reset(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java index 01eb92b9be3f40..5f6c12d8eeb477 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java @@ -43,6 +43,7 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.proc.FrontendsProcNode; import org.apache.doris.common.proc.PartitionsProcDir; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.common.util.NetUtils; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.common.util.Util; @@ -62,7 +63,10 @@ import org.apache.doris.job.common.JobType; import org.apache.doris.job.extensions.mtmv.MTMVJob; import org.apache.doris.job.task.AbstractTask; +import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.mtmv.MTMVPartitionUtil; +import org.apache.doris.mtmv.MTMVStatus; +import org.apache.doris.mtmv.MTMVUtil; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.plsql.metastore.PlsqlManager; import org.apache.doris.plsql.metastore.PlsqlProcedureKey; @@ -111,6 +115,7 @@ import java.time.LocalDateTime; import java.util.ArrayList; import java.util.Collection; +import java.util.Comparator; import java.util.Date; import java.util.List; import java.util.Map; @@ -844,22 +849,42 @@ private static TFetchSchemaTableDataResult mtmvMetadataResult(TMetadataTableRequ } MTMV mv = (MTMV) table; if (LOG.isDebugEnabled()) { - LOG.debug("mv: " + mv.toInfoString()); + LOG.debug("mv: {}", mv.toInfoString()); } + List needLocked = Lists.newArrayList(); + needLocked.add(mv); + boolean alwaysNotSync = false; + try { + for (BaseTableInfo baseTableInfo : mv.getRelation().getBaseTables()) { + TableIf baseTable = MTMVUtil.getTable(baseTableInfo); + needLocked.add(baseTable); + } + } catch (Exception e) { + alwaysNotSync = true; + } + needLocked.sort(Comparator.comparing(TableIf::getId)); + MetaLockUtils.readLockTables(needLocked); + boolean isSync; + try { + isSync = !alwaysNotSync && MTMVPartitionUtil.isMTMVSync(mv); + } finally { + MetaLockUtils.readUnlockTables(needLocked); + } + MTMVStatus mtmvStatus = mv.getStatus(); TRow trow = new TRow(); trow.addToColumnValue(new TCell().setLongVal(mv.getId())); trow.addToColumnValue(new TCell().setStringVal(mv.getName())); trow.addToColumnValue(new TCell().setStringVal(mv.getJobInfo().getJobName())); - trow.addToColumnValue(new TCell().setStringVal(mv.getStatus().getState().name())); - trow.addToColumnValue(new TCell().setStringVal(mv.getStatus().getSchemaChangeDetail())); - trow.addToColumnValue(new TCell().setStringVal(mv.getStatus().getRefreshState().name())); + trow.addToColumnValue(new TCell().setStringVal(mtmvStatus.getState().name())); + trow.addToColumnValue(new TCell().setStringVal(mtmvStatus.getSchemaChangeDetail())); + trow.addToColumnValue(new TCell().setStringVal(mtmvStatus.getRefreshState().name())); trow.addToColumnValue(new TCell().setStringVal(mv.getRefreshInfo().toString())); trow.addToColumnValue(new TCell().setStringVal(mv.getQuerySql())); trow.addToColumnValue(new TCell().setStringVal(mv.getMvProperties().toString())); trow.addToColumnValue(new TCell().setStringVal(mv.getMvPartitionInfo().toNameString())); - trow.addToColumnValue(new TCell().setBoolVal(MTMVPartitionUtil.isMTMVSync(mv))); + trow.addToColumnValue(new TCell().setBoolVal(isSync)); if (LOG.isDebugEnabled()) { - LOG.debug("mvend: " + mv.getName()); + LOG.debug("mv end: {}", mv.getName()); } dataBatch.add(trow); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java index 369a57017cba28..eaeaa3b2edda8b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java @@ -17,23 +17,13 @@ package org.apache.doris.nereids.rules.analysis; -import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.DatabaseIf; -import org.apache.doris.catalog.KeysType; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.PartitionInfo; -import org.apache.doris.catalog.RandomDistributionInfo; -import org.apache.doris.catalog.Type; import org.apache.doris.nereids.analyzer.UnboundRelation; import org.apache.doris.nereids.pattern.GeneratedPlanPatterns; import org.apache.doris.nereids.rules.RulePromise; -import org.apache.doris.nereids.rules.analysis.BindRelation.CustomTableResolver; import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; -import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanRewriter; import org.apache.doris.utframe.TestWithFeService; @@ -41,9 +31,6 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import java.util.List; -import java.util.Optional; - class BindRelationTest extends TestWithFeService implements GeneratedPlanPatterns { private static final String DB1 = "db1"; private static final String DB2 = "db2"; @@ -72,7 +59,7 @@ void bindInCurrentDb() { Plan plan = PlanRewriter.bottomUpRewrite(new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("t")), connectContext, new BindRelation()); - Assertions.assertTrue(plan instanceof LogicalOlapScan); + Assertions.assertInstanceOf(LogicalOlapScan.class, plan); Assertions.assertEquals( ImmutableList.of("internal", DEFAULT_CLUSTER_PREFIX + DB1, "t"), ((LogicalOlapScan) plan).qualified()); @@ -84,63 +71,12 @@ void bindByDbQualifier() { Plan plan = PlanRewriter.bottomUpRewrite(new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("db1", "t")), connectContext, new BindRelation()); - Assertions.assertTrue(plan instanceof LogicalOlapScan); + Assertions.assertInstanceOf(LogicalOlapScan.class, plan); Assertions.assertEquals( ImmutableList.of("internal", DEFAULT_CLUSTER_PREFIX + DB1, "t"), ((LogicalOlapScan) plan).qualified()); } - @Test - public void bindExternalRelation() { - connectContext.setDatabase(DEFAULT_CLUSTER_PREFIX + DB1); - String tableName = "external_table"; - - List externalTableColumns = ImmutableList.of( - new Column("id", Type.INT), - new Column("name", Type.VARCHAR) - ); - - Database externalDatabase = new Database(10000, DEFAULT_CLUSTER_PREFIX + DB1); - - OlapTable externalOlapTable = new OlapTable(1, tableName, externalTableColumns, KeysType.DUP_KEYS, - new PartitionInfo(), new RandomDistributionInfo(10)) { - @Override - public List getBaseSchema(boolean full) { - return externalTableColumns; - } - - @Override - public boolean hasDeleteSign() { - return false; - } - - @Override - public DatabaseIf getDatabase() { - return externalDatabase; - } - }; - - CustomTableResolver customTableResolver = qualifiedTable -> { - if (qualifiedTable.get(2).equals(tableName)) { - return externalOlapTable; - } else { - return null; - } - }; - - PlanChecker.from(connectContext) - .parse("select * from " + tableName + " as et join db1.t on et.id = t.a") - .customAnalyzer(Optional.of(customTableResolver)) // analyze internal relation - .matches( - logicalJoin( - logicalSubQueryAlias( - logicalOlapScan().when(r -> r.getTable() == externalOlapTable) - ), - logicalOlapScan().when(r -> r.getTable().getName().equals("t")) - ) - ); - } - @Test void bindRandomAggTable() { connectContext.setDatabase(DEFAULT_CLUSTER_PREFIX + DB1); @@ -148,7 +84,7 @@ void bindRandomAggTable() { Plan plan = PlanRewriter.bottomUpRewrite(new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("tagg")), connectContext, new BindRelation()); - Assertions.assertTrue(plan instanceof LogicalAggregate); + Assertions.assertInstanceOf(LogicalAggregate.class, plan); Assertions.assertEquals( ImmutableList.of("internal", DEFAULT_CLUSTER_PREFIX + DB1, "tagg"), plan.getOutput().get(0).getQualifier()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java index 0c54f8fad5a107..82c8122a18d72f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java @@ -17,34 +17,19 @@ package org.apache.doris.nereids.trees.plans; -import org.apache.doris.catalog.TableIf; -import org.apache.doris.catalog.TableIf.TableType; import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewUtils; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.scalar.CurrentDate; import org.apache.doris.nereids.trees.expressions.functions.scalar.CurrentTime; import org.apache.doris.nereids.trees.expressions.functions.scalar.Now; -import org.apache.doris.nereids.trees.expressions.functions.scalar.Random; import org.apache.doris.nereids.trees.expressions.functions.scalar.UnixTimestamp; -import org.apache.doris.nereids.trees.expressions.functions.scalar.Uuid; -import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; -import org.apache.doris.nereids.trees.plans.visitor.TableCollector; -import org.apache.doris.nereids.trees.plans.visitor.TableCollector.TableCollectorContext; import org.apache.doris.nereids.util.PlanChecker; -import org.apache.doris.qe.SessionVariable; import org.apache.doris.utframe.TestWithFeService; -import com.google.common.collect.Sets; -import mockit.Mock; -import mockit.MockUp; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import java.util.BitSet; -import java.util.HashSet; import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; /** * Tests for plan visitors to make sure the result meets expectation. @@ -115,154 +100,6 @@ protected void runBeforeAll() throws Exception { + "inner join table3 t3 on t1.c1= t3.c2;"); } - @Test - public void test1() { - PlanChecker.from(connectContext) - .checkPlannerResult("SELECT *, random() FROM table1 " - + "LEFT SEMI JOIN table2 ON table1.c1 = table2.c1 " - + "WHERE table1.c1 IN (SELECT c1 FROM table2) OR table1.c1 < 10", - nereidsPlanner -> { - PhysicalPlan physicalPlan = nereidsPlanner.getPhysicalPlan(); - // Check nondeterministic collect - List nondeterministicFunctionSet = - MaterializedViewUtils.extractNondeterministicFunction(physicalPlan); - Assertions.assertEquals(1, nondeterministicFunctionSet.size()); - Assertions.assertTrue(nondeterministicFunctionSet.get(0) instanceof Random); - // Check get tables - TableCollectorContext collectorContext = new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.OLAP), true, connectContext); - physicalPlan.accept(TableCollector.INSTANCE, collectorContext); - Set expectedTables = new HashSet<>(); - expectedTables.add("table1"); - expectedTables.add("table2"); - Assertions.assertEquals( - collectorContext.getCollectedTables().stream() - .map(TableIf::getName) - .collect(Collectors.toSet()), - expectedTables); - }); - } - - @Test - public void test2() { - PlanChecker.from(connectContext) - .checkPlannerResult("SELECT view1.*, uuid() FROM view1 " - + "LEFT SEMI JOIN table2 ON view1.c1 = table2.c1 " - + "WHERE view1.c1 IN (SELECT c1 FROM table2) OR view1.c1 < 10", - nereidsPlanner -> { - PhysicalPlan physicalPlan = nereidsPlanner.getPhysicalPlan(); - // Check nondeterministic collect - List nondeterministicFunctionSet = - MaterializedViewUtils.extractNondeterministicFunction(physicalPlan); - Assertions.assertEquals(2, nondeterministicFunctionSet.size()); - Assertions.assertTrue(nondeterministicFunctionSet.get(0) instanceof Uuid); - Assertions.assertTrue(nondeterministicFunctionSet.get(1) instanceof Random); - // Check get tables - TableCollectorContext collectorContext = new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.OLAP), true, connectContext); - physicalPlan.accept(TableCollector.INSTANCE, collectorContext); - Set expectedTables = new HashSet<>(); - expectedTables.add("table1"); - expectedTables.add("table2"); - Assertions.assertEquals( - collectorContext.getCollectedTables().stream() - .map(TableIf::getName) - .collect(Collectors.toSet()), - expectedTables); - }); - } - - @Test - public void test3() throws Exception { - connectContext.getSessionVariable().setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); - BitSet disableNereidsRules = connectContext.getSessionVariable().getDisableNereidsRules(); - new MockUp() { - @Mock - public BitSet getDisableNereidsRules() { - return disableNereidsRules; - } - }; - PlanChecker.from(connectContext) - .checkPlannerResult("SELECT mv1.*, uuid() FROM mv1 " - + "INNER JOIN view1 on mv1.c1 = view1.c2 " - + "LEFT SEMI JOIN table2 ON mv1.c1 = table2.c1 " - + "WHERE mv1.c1 IN (SELECT c1 FROM table2) OR mv1.c1 < 10", - nereidsPlanner -> { - PhysicalPlan physicalPlan = nereidsPlanner.getPhysicalPlan(); - // Check nondeterministic collect - List nondeterministicFunctionSet = - MaterializedViewUtils.extractNondeterministicFunction(physicalPlan); - Assertions.assertEquals(1, nondeterministicFunctionSet.size()); - Assertions.assertTrue(nondeterministicFunctionSet.get(0) instanceof Uuid); - // Check get tables - TableCollectorContext collectorContext = new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.OLAP), true, connectContext); - physicalPlan.accept(TableCollector.INSTANCE, collectorContext); - Set expectedTables = new HashSet<>(); - expectedTables.add("table1"); - expectedTables.add("table2"); - expectedTables.add("table3"); - Assertions.assertEquals( - collectorContext.getCollectedTables().stream() - .map(TableIf::getName) - .collect(Collectors.toSet()), - expectedTables); - - TableCollectorContext collectorContextWithNoExpand = - new TableCollector.TableCollectorContext(Sets.newHashSet(TableType.OLAP), - false, connectContext); - physicalPlan.accept(TableCollector.INSTANCE, collectorContextWithNoExpand); - Set expectedTablesWithNoExpand = new HashSet<>(); - expectedTablesWithNoExpand.add("table1"); - expectedTablesWithNoExpand.add("table2"); - Assertions.assertEquals( - collectorContextWithNoExpand.getCollectedTables().stream() - .map(TableIf::getName) - .collect(Collectors.toSet()), - expectedTablesWithNoExpand); - - TableCollectorContext mvCollectorContext = new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.MATERIALIZED_VIEW), true, connectContext); - physicalPlan.accept(TableCollector.INSTANCE, mvCollectorContext); - Set expectedMvs = new HashSet<>(); - expectedMvs.add("mv1"); - Assertions.assertEquals( - mvCollectorContext.getCollectedTables().stream() - .map(TableIf::getName) - .collect(Collectors.toSet()), - expectedMvs); - - TableCollectorContext mvCollectorContextWithNoExpand = - new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.MATERIALIZED_VIEW), false, connectContext); - physicalPlan.accept(TableCollector.INSTANCE, mvCollectorContextWithNoExpand); - Set expectedMvsWithNoExpand = new HashSet<>(); - expectedMvsWithNoExpand.add("mv1"); - Assertions.assertEquals( - mvCollectorContextWithNoExpand.getCollectedTables().stream() - .map(TableIf::getName) - .collect(Collectors.toSet()), - expectedMvsWithNoExpand); - - TableCollectorContext allTableTypeWithExpand = - new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.values()), true, connectContext); - physicalPlan.accept(TableCollector.INSTANCE, allTableTypeWithExpand); - // when collect in plan with expand, should collect table which is expended - Set expectedTablesWithExpand = new HashSet<>(); - expectedTablesWithExpand.add("mv1"); - expectedTablesWithExpand.add("table1"); - expectedTablesWithExpand.add("table2"); - expectedTablesWithExpand.add("table3"); - Assertions.assertEquals( - allTableTypeWithExpand.getCollectedTables().stream() - .map(TableIf::getName) - .collect(Collectors.toSet()), - expectedTablesWithExpand); - }); - dropMvByNereids("drop materialized view mv1"); - } - @Test public void testTimeFunction() { PlanChecker.from(connectContext) diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java index f0a45d1e7bc852..77ecbd5dc7c4dd 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java @@ -49,7 +49,6 @@ import org.apache.doris.nereids.rules.RuleFactory; import org.apache.doris.nereids.rules.RuleSet; import org.apache.doris.nereids.rules.RuleType; -import org.apache.doris.nereids.rules.analysis.BindRelation.CustomTableResolver; import org.apache.doris.nereids.rules.exploration.mv.InitMaterializationContextHook; import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory; import org.apache.doris.nereids.trees.plans.GroupPlan; @@ -71,7 +70,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.Optional; import java.util.Set; import java.util.function.Consumer; import java.util.function.Supplier; @@ -147,12 +145,6 @@ public PlanChecker analyze(String sql) { return this; } - public PlanChecker customAnalyzer(Optional customTableResolver) { - this.cascadesContext.newAnalyzer(customTableResolver).analyze(); - this.cascadesContext.toMemo(); - return this; - } - public PlanChecker customRewrite(CustomRewriter customRewriter) { Rewriter.getWholeTreeRewriterWithCustomJobs(cascadesContext, ImmutableList.of(Rewriter.custom(RuleType.TEST_REWRITE, () -> customRewriter))) @@ -277,7 +269,7 @@ public NereidsPlanner plan(String sql) { LogicalPlan parsedPlan = new NereidsParser().parseSingle(sql); LogicalPlanAdapter parsedPlanAdaptor = new LogicalPlanAdapter(parsedPlan, statementContext); statementContext.setParsedStatement(parsedPlanAdaptor); - planner.planWithLock(parsedPlanAdaptor); + planner.plan(parsedPlanAdaptor); return planner; } @@ -576,7 +568,7 @@ public PlanChecker checkExplain(String sql, Consumer consumer) { new StatementContext(connectContext, new OriginStatement(sql, 0))); LogicalPlanAdapter adapter = LogicalPlanAdapter.of(parsed); adapter.setIsExplain(new ExplainOptions(ExplainLevel.ALL_PLAN, false)); - nereidsPlanner.planWithLock(adapter); + nereidsPlanner.plan(adapter); consumer.accept(nereidsPlanner); return this; } @@ -585,7 +577,7 @@ public PlanChecker checkPlannerResult(String sql, Consumer consu LogicalPlan parsed = new NereidsParser().parseSingle(sql); NereidsPlanner nereidsPlanner = new NereidsPlanner( new StatementContext(connectContext, new OriginStatement(sql, 0))); - nereidsPlanner.planWithLock(LogicalPlanAdapter.of(parsed)); + nereidsPlanner.plan(LogicalPlanAdapter.of(parsed)); consumer.accept(nereidsPlanner); return this; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/ReadLockTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/ReadLockTest.java index 1e1535a573610b..6cd85183b3df79 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/ReadLockTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/ReadLockTest.java @@ -118,7 +118,7 @@ public void testScalarSubQuery() { } @Test - public void testInserInto() { + public void testInsertInto() { String sql = "INSERT INTO supplier(s_suppkey, s_name, s_address, s_city, s_nation, s_region, s_phone) " + "SELECT lo_orderkey, '', '', '', '', '', '' FROM lineorder"; StatementContext statementContext = MemoTestUtils.createStatementContext(connectContext, sql); @@ -129,7 +129,6 @@ public void testInserInto() { PhysicalProperties.ANY ); Map, TableIf> f = statementContext.getTables(); - // when table in insert would not be added to statement context, but be lock when insert Assertions.assertEquals(1, f.size()); Set tableNames = new HashSet<>(); for (Map.Entry, TableIf> entry : f.entrySet()) { @@ -137,5 +136,13 @@ public void testInserInto() { tableNames.add(table.getName()); } Assertions.assertTrue(tableNames.contains("lineorder")); + f = statementContext.getInsertTargetTables(); + Assertions.assertEquals(1, f.size()); + tableNames = new HashSet<>(); + for (Map.Entry, TableIf> entry : f.entrySet()) { + TableIf table = entry.getValue(); + tableNames.add(table.getName()); + } + Assertions.assertTrue(tableNames.contains("supplier")); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/OlapQueryCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/OlapQueryCacheTest.java index 3c793cfc72090d..e8b545f3ffe2c1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/qe/OlapQueryCacheTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/qe/OlapQueryCacheTest.java @@ -502,7 +502,7 @@ private StatementBase parseSqlByNereids(String sql) { ctx.setStatementContext(statementContext); NereidsPlanner nereidsPlanner = new NereidsPlanner(statementContext); LogicalPlanAdapter adapter = new LogicalPlanAdapter(plan, statementContext); - nereidsPlanner.planWithLock(adapter); + nereidsPlanner.plan(adapter); statementContext.setParsedStatement(adapter); stmt = adapter; } catch (Throwable throwable) { From b1ccd3696a906c2d295b5af5e21665674d56a96a Mon Sep 17 00:00:00 2001 From: Xinyi Zou Date: Thu, 19 Dec 2024 19:56:19 +0800 Subject: [PATCH 09/14] [fix](memory) Fix adjust cache capacity (#45603) ### What problem does this PR solve? If the cache capacity adjustment is not completed within 500ms (conf::memory_gc_sleep_time_ms), the next adjustment will be skipped. In some scenarios, after Memory GC adjusts the cache capacity to 0, the next adjustment to restore the cache capacity is skipped, the cache capacity will remain at 0 for a long time. --- be/src/common/daemon.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/be/src/common/daemon.cpp b/be/src/common/daemon.cpp index 12bf1749a5694d..2aaa58f4feb597 100644 --- a/be/src/common/daemon.cpp +++ b/be/src/common/daemon.cpp @@ -230,6 +230,11 @@ void refresh_memory_state_after_memory_change() { } void refresh_cache_capacity() { + if (doris::GlobalMemoryArbitrator::cache_adjust_capacity_notify.load( + std::memory_order_relaxed)) { + // the last cache capacity adjustment has not been completed. + return; + } if (refresh_cache_capacity_sleep_time_ms <= 0) { auto cache_capacity_reduce_mem_limit = int64_t( doris::MemInfo::soft_mem_limit() * config::cache_capacity_reduce_mem_limit_frac); @@ -247,6 +252,8 @@ void refresh_cache_capacity() { new_cache_capacity_adjust_weighted; doris::GlobalMemoryArbitrator::notify_cache_adjust_capacity(); refresh_cache_capacity_sleep_time_ms = config::memory_gc_sleep_time_ms; + } else { + refresh_cache_capacity_sleep_time_ms = 0; } } refresh_cache_capacity_sleep_time_ms -= config::memory_maintenance_sleep_time_ms; From 6cce4087f94b5ef91b7ffcba294b4c5da79f1f14 Mon Sep 17 00:00:00 2001 From: Xinyi Zou Date: Thu, 19 Dec 2024 19:56:38 +0800 Subject: [PATCH 10/14] [fix](memory) Process available memory to increase the Jemalloc cache (#45621) ### What problem does this PR solve? Currently, when the Doris BE process exceed memory limit, Jemalloc cache will be manually released. Add the Jemalloc cache to the available memory of the BE process is expected to have little impact on the risk of the process OOM killer. the process memory used has already subtracted the Jemalloc cache. Not merge to 2.1 because 2.1 is stable now --- be/src/runtime/memory/global_memory_arbitrator.h | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/be/src/runtime/memory/global_memory_arbitrator.h b/be/src/runtime/memory/global_memory_arbitrator.h index 075113088fbc5b..a7a85725ab10c9 100644 --- a/be/src/runtime/memory/global_memory_arbitrator.h +++ b/be/src/runtime/memory/global_memory_arbitrator.h @@ -76,7 +76,7 @@ class GlobalMemoryArbitrator { static inline int64_t sys_mem_available() { return MemInfo::_s_sys_mem_available.load(std::memory_order_relaxed) - refresh_interval_memory_growth.load(std::memory_order_relaxed) - - process_reserved_memory(); + process_reserved_memory() + static_cast(MemInfo::allocator_cache_mem()); } static inline std::string sys_mem_available_str() { @@ -91,12 +91,14 @@ class GlobalMemoryArbitrator { static inline std::string sys_mem_available_details_str() { auto msg = fmt::format( "sys available memory {}(= {}[proc/available] - {}[reserved] - " - "{}B[waiting_refresh])", + "{}B[waiting_refresh] + {}[tc/jemalloc_cache])", PrettyPrinter::print(sys_mem_available(), TUnit::BYTES), PrettyPrinter::print(MemInfo::_s_sys_mem_available.load(std::memory_order_relaxed), TUnit::BYTES), PrettyPrinter::print(process_reserved_memory(), TUnit::BYTES), - refresh_interval_memory_growth); + refresh_interval_memory_growth, + PrettyPrinter::print(static_cast(MemInfo::allocator_cache_mem()), + TUnit::BYTES)); #ifdef ADDRESS_SANITIZER msg = "[ASAN]" + msg; #endif From 55c26e03e5f76fe80e6bedd2bc31760e55cd6707 Mon Sep 17 00:00:00 2001 From: linrrarity <142187136+linrrzqqq@users.noreply.github.com> Date: Thu, 19 Dec 2024 20:04:15 +0800 Subject: [PATCH 11/14] [Enhancement](Log) Reduce usage of log fatal(PART I) (#42344) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Proposed changes Issue Number: close #40835 use `throw Exception` to replace them which not in `if constexpr`, and change part of `INTERNAL_ERROR` in this [pr](https://github.com/apache/doris/pull/38144/files)(file `aggregate_function_reader_first_last.h` and `aggregate_function_window.h`) to `FatalError`. for those in `if constexpr else{...}`, use `static_assert` about template argument which used in that judgement to advance them to compile time but there seems to be some bugs with the template parameter instantiation in the files `comparison_predicate.h`, `set_probe_sink_operator.cpp`, `set_sink_operator.cpp`, `comparison_predicate.h`, `in_list_predicate.h` and `set_source_operator.cpp` that I haven't modified yet. --------- Co-authored-by: wyxxxcat <1520358997@qq.com> --- be/src/common/status.h | 11 +++- be/src/gutil/strings/escaping.cc | 5 +- be/src/gutil/strings/numbers.cc | 6 +- be/src/gutil/strings/util.cc | 5 +- .../threading/thread_collision_warner.cc | 8 ++- be/src/io/file_factory.h | 5 +- be/src/olap/block_column_predicate.h | 12 ++-- be/src/olap/data_dir.cpp | 16 +++--- be/src/olap/key_coder.h | 13 ++--- be/src/olap/like_column_predicate.h | 8 +-- be/src/olap/match_predicate.h | 3 +- be/src/olap/null_predicate.h | 4 +- be/src/olap/olap_common.h | 6 +- be/src/olap/page_cache.h | 6 +- be/src/olap/rowset/beta_rowset_writer_v2.h | 3 +- be/src/olap/rowset/rowset_writer.h | 4 +- .../segment_v2/hierarchical_data_reader.cpp | 6 +- be/src/olap/storage_policy.cpp | 6 +- be/src/olap/tablet_reader.cpp | 2 +- be/src/pipeline/dependency.h | 3 +- be/src/pipeline/exec/exchange_sink_buffer.cpp | 3 +- be/src/pipeline/exec/exchange_sink_buffer.h | 5 +- be/src/pipeline/exec/hashjoin_build_sink.cpp | 4 +- be/src/pipeline/exec/operator.cpp | 3 +- be/src/pipeline/exec/operator.h | 6 +- be/src/runtime/exec_env_init.cpp | 10 ++-- be/src/runtime/jsonb_value.h | 33 ++++------- be/src/runtime/memory/cache_manager.h | 3 +- be/src/runtime/memory/cache_policy.h | 6 +- be/src/runtime/memory/lru_cache_policy.h | 3 +- .../runtime/memory/thread_mem_tracker_mgr.h | 4 +- be/src/runtime/snapshot_loader.cpp | 5 +- .../stream_load/stream_load_executor.cpp | 3 +- be/src/runtime/thread_context.h | 7 +-- be/src/util/binary_cast.hpp | 3 +- be/src/util/bit_util.h | 4 +- be/src/util/bitmap_value.h | 3 +- be/src/util/block_compression.cpp | 3 +- be/src/util/easy_json.cc | 5 +- be/src/util/jsonb_utils.h | 4 +- be/src/util/rle_encoding.h | 2 +- be/src/util/threadpool.cpp | 11 ++-- be/src/util/timezone_utils.cpp | 4 +- .../aggregate_function_map.h | 5 +- .../aggregate_function_reader_first_last.h | 17 ++---- .../aggregate_function_window.h | 19 +++---- be/src/vec/columns/column_string.cpp | 10 ++-- be/src/vec/common/assert_cast.h | 19 +++---- .../vec/common/hash_table/string_hash_table.h | 3 +- be/src/vec/common/schema_util.cpp | 5 +- be/src/vec/core/block.cpp | 4 +- be/src/vec/core/decimal_comparison.h | 23 +++----- be/src/vec/core/field.h | 56 ++++++++----------- be/src/vec/core/types.h | 3 +- .../vec/data_types/data_type_number_base.cpp | 3 +- be/src/vec/data_types/data_type_number_base.h | 3 +- be/src/vec/data_types/serde/data_type_serde.h | 5 +- .../exec/format/parquet/bool_rle_decoder.cpp | 9 +-- be/src/vec/exec/format/parquet/decoder.h | 4 +- .../format/parquet/delta_bit_pack_decoder.h | 10 ++-- .../format/parquet/parquet_column_convert.h | 9 +-- .../format/parquet/vparquet_column_reader.h | 4 +- be/src/vec/exec/jni_connector.cpp | 4 +- be/src/vec/exec/scan/split_source_connector.h | 6 +- be/src/vec/exprs/vexpr.h | 12 ++-- .../functions/array/function_array_apply.cpp | 4 +- be/src/vec/functions/function_cast.h | 12 +++- be/src/vec/json/simd_json_parser.h | 4 +- be/src/vec/olap/olap_data_convertor.h | 6 +- be/src/vec/runtime/vdatetime_value.cpp | 6 +- be/test/util/threadpool_test.cpp | 1 + 71 files changed, 250 insertions(+), 282 deletions(-) diff --git a/be/src/common/status.h b/be/src/common/status.h index d059f289402cea..0252ec8564feeb 100644 --- a/be/src/common/status.h +++ b/be/src/common/status.h @@ -293,7 +293,8 @@ namespace ErrorCode { E(ENTRY_NOT_FOUND, -7002, false); \ E(INVALID_TABLET_STATE, -7211, false); \ E(ROWSETS_EXPIRED, -7311, false); \ - E(CGROUP_ERROR, -7411, false); + E(CGROUP_ERROR, -7411, false); \ + E(FATAL_ERROR, -7412, false); // Define constexpr int error_code_name = error_code_value #define M(NAME, ERRORCODE, ENABLESTACKTRACE) constexpr int NAME = ERRORCODE; @@ -446,6 +447,14 @@ class [[nodiscard]] Status { static Status OK() { return {}; } + template + static Status FatalError(std::string_view msg, Args&&... args) { +#ifndef NDEBUG + LOG(FATAL) << fmt::format(msg, std::forward(args)...); +#endif + return Error(msg, std::forward(args)...); + } + // default have stacktrace. could disable manually. #define ERROR_CTOR(name, code) \ template \ diff --git a/be/src/gutil/strings/escaping.cc b/be/src/gutil/strings/escaping.cc index 2ff59104f6d5ce..c6ba8e2f9c375e 100644 --- a/be/src/gutil/strings/escaping.cc +++ b/be/src/gutil/strings/escaping.cc @@ -10,6 +10,8 @@ #include #include +#include "common/exception.h" + using std::numeric_limits; #include @@ -1084,7 +1086,8 @@ int Base64UnescapeInternal(const char* src, int szsrc, char* dest, int szdest, default: // state should have no other values at this point. - LOG(FATAL) << "This can't happen; base64 decoder state = " << state; + throw doris::Exception( + doris::Status::FatalError("This can't happen; base64 decoder state = {}", state)); } // The remainder of the string should be all whitespace, mixed with diff --git a/be/src/gutil/strings/numbers.cc b/be/src/gutil/strings/numbers.cc index f471bf31bd08bb..f044ea08d31551 100644 --- a/be/src/gutil/strings/numbers.cc +++ b/be/src/gutil/strings/numbers.cc @@ -19,6 +19,8 @@ #include #include +#include "common/exception.h" + using std::numeric_limits; #include @@ -772,8 +774,8 @@ uint64 atoi_kmgt(const char* s) { scale = GG_ULONGLONG(1) << 40; break; default: - LOG(FATAL) << "Invalid mnemonic: `" << c << "';" - << " should be one of `K', `M', `G', and `T'."; + throw doris::Exception(doris::Status::FatalError( + "Invalid mnemonic: `{}'; should be one of `K', `M', `G', and `T'.", c)); } } return n * scale; diff --git a/be/src/gutil/strings/util.cc b/be/src/gutil/strings/util.cc index 80d5d463430c77..37c09d63b24fff 100644 --- a/be/src/gutil/strings/util.cc +++ b/be/src/gutil/strings/util.cc @@ -19,6 +19,8 @@ #include #include +#include "common/exception.h" + using std::copy; using std::max; using std::min; @@ -489,8 +491,7 @@ const char* strstr_delimited(const char* haystack, const char* needle, char deli ++haystack; } } - LOG(FATAL) << "Unreachable statement"; - return nullptr; + throw doris::Exception(doris::Status::FatalError("Unreachable statement")); } // ---------------------------------------------------------------------- diff --git a/be/src/gutil/threading/thread_collision_warner.cc b/be/src/gutil/threading/thread_collision_warner.cc index d2f1e47f8e02d9..fd51a9195d629e 100644 --- a/be/src/gutil/threading/thread_collision_warner.cc +++ b/be/src/gutil/threading/thread_collision_warner.cc @@ -4,6 +4,9 @@ #include "gutil/threading/thread_collision_warner.h" +#include "common/exception.h" +#include "common/status.h" + #ifdef __linux__ #include #else @@ -19,8 +22,9 @@ namespace base { void DCheckAsserter::warn(int64_t previous_thread_id, int64_t current_thread_id) { - LOG(FATAL) << "Thread Collision! Previous thread id: " << previous_thread_id - << ", current thread id: " << current_thread_id; + throw doris::Exception(doris::Status::FatalError( + "Thread Collision! Previous thread id: {}, current thread id: {}", previous_thread_id, + current_thread_id)); } static subtle::Atomic64 CurrentThread() { diff --git a/be/src/io/file_factory.h b/be/src/io/file_factory.h index 9d9d714812ffe9..afa54e221664c9 100644 --- a/be/src/io/file_factory.h +++ b/be/src/io/file_factory.h @@ -118,10 +118,9 @@ class FileFactory { case TStorageBackendType::HDFS: return TFileType::FILE_HDFS; default: - LOG(FATAL) << "not match type to convert, from type:" << type; + throw Exception(Status::FatalError("not match type to convert, from type:{}", type)); } - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); + throw Exception(Status::FatalError("__builtin_unreachable")); } }; diff --git a/be/src/olap/block_column_predicate.h b/be/src/olap/block_column_predicate.h index eed5e18329acf7..b6ff115c34c72d 100644 --- a/be/src/olap/block_column_predicate.h +++ b/be/src/olap/block_column_predicate.h @@ -74,25 +74,21 @@ class BlockColumnPredicate { } virtual bool can_do_apply_safely(PrimitiveType input_type, bool is_null) const { - LOG(FATAL) << "should not reach here"; - return true; + throw Exception(Status::FatalError("should not reach here")); } virtual bool support_zonemap() const { return true; } virtual bool evaluate_and(const std::pair& statistic) const { - LOG(FATAL) << "should not reach here"; - return true; + throw Exception(Status::FatalError("should not reach here")); } virtual bool evaluate_and(const segment_v2::BloomFilter* bf) const { - LOG(FATAL) << "should not reach here"; - return true; + throw Exception(Status::FatalError("should not reach here")); } virtual bool evaluate_and(const StringRef* dict_words, const size_t dict_num) const { - LOG(FATAL) << "should not reach here"; - return true; + throw Exception(Status::FatalError("should not reach here")); } virtual bool can_do_bloom_filter(bool ngram) const { return false; } diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp index 4070bd1dd4340e..4aa215e0c2eb16 100644 --- a/be/src/olap/data_dir.cpp +++ b/be/src/olap/data_dir.cpp @@ -316,10 +316,10 @@ Status DataDir::_check_incompatible_old_format_tablet() { std::string_view value) -> bool { // if strict check incompatible old format, then log fatal if (config::storage_strict_check_incompatible_old_format) { - LOG(FATAL) - << "There are incompatible old format metas, current version does not support " - << "and it may lead to data missing!!! " - << "tablet_id = " << tablet_id << " schema_hash = " << schema_hash; + throw Exception(Status::FatalError( + "There are incompatible old format metas, current version does not support and " + "it may lead to data missing!!! tablet_id = {} schema_hash = {}", + tablet_id, schema_hash)); } else { LOG(WARNING) << "There are incompatible old format metas, current version does not support " @@ -451,7 +451,8 @@ Status DataDir::load() { << ", loaded tablet: " << tablet_ids.size() << ", error tablet: " << failed_tablet_ids.size() << ", path: " << _path; if (!config::ignore_load_tablet_failure) { - LOG(FATAL) << "load tablets encounter failure. stop BE process. path: " << _path; + throw Exception(Status::FatalError( + "load tablets encounter failure. stop BE process. path: {}", _path)); } } if (!load_tablet_status) { @@ -495,10 +496,9 @@ Status DataDir::load() { } } if (rowset_partition_id_eq_0_num > config::ignore_invalid_partition_id_rowset_num) { - LOG(FATAL) << fmt::format( + throw Exception(Status::FatalError( "roswet partition id eq 0 is {} bigger than config {}, be exit, plz check be.INFO", - rowset_partition_id_eq_0_num, config::ignore_invalid_partition_id_rowset_num); - exit(-1); + rowset_partition_id_eq_0_num, config::ignore_invalid_partition_id_rowset_num)); } // traverse rowset diff --git a/be/src/olap/key_coder.h b/be/src/olap/key_coder.h index 6885a0d96f251b..549ac53656b647 100644 --- a/be/src/olap/key_coder.h +++ b/be/src/olap/key_coder.h @@ -109,8 +109,8 @@ class KeyCoderTraits< case 16: return BigEndian::FromHost128(val); default: - LOG(FATAL) << "Invalid type to big endian, type=" << int(field_type) - << ", size=" << sizeof(UnsignedCppType); + throw Exception(Status::FatalError("Invalid type to big endian, type={}, size={}", + int(field_type), sizeof(UnsignedCppType))); } } } @@ -300,8 +300,7 @@ class KeyCoderTraits { } static Status decode_ascending(Slice* encoded_key, size_t index_size, uint8_t* cell_ptr) { - LOG(FATAL) << "decode_ascending is not implemented"; - return Status::OK(); + throw Exception(Status::FatalError("decode_ascending is not implemented")); } }; @@ -320,8 +319,7 @@ class KeyCoderTraits { } static Status decode_ascending(Slice* encoded_key, size_t index_size, uint8_t* cell_ptr) { - LOG(FATAL) << "decode_ascending is not implemented"; - return Status::OK(); + throw Exception(Status::FatalError("decode_ascending is not implemented")); } }; @@ -340,8 +338,7 @@ class KeyCoderTraits { } static Status decode_ascending(Slice* encoded_key, size_t index_size, uint8_t* cell_ptr) { - LOG(FATAL) << "decode_ascending is not implemented"; - return Status::OK(); + throw Exception(Status::FatalError("decode_ascending is not implemented")); } }; diff --git a/be/src/olap/like_column_predicate.h b/be/src/olap/like_column_predicate.h index 31763d45f7edc7..e0d185c7bd3e98 100644 --- a/be/src/olap/like_column_predicate.h +++ b/be/src/olap/like_column_predicate.h @@ -128,8 +128,8 @@ class LikeColumnPredicate : public ColumnPredicate { } } } else { - LOG(FATAL) << "vectorized (not) like predicates should be dict column"; - __builtin_unreachable(); + throw Exception(Status::FatalError( + "vectorized (not) like predicates should be dict column")); } } else { if (column.is_column_dictionary()) { @@ -153,8 +153,8 @@ class LikeColumnPredicate : public ColumnPredicate { } } } else { - LOG(FATAL) << "vectorized (not) like predicates should be dict column"; - __builtin_unreachable(); + throw Exception(Status::FatalError( + "vectorized (not) like predicates should be dict column")); } } } diff --git a/be/src/olap/match_predicate.h b/be/src/olap/match_predicate.h index ad202b7b2427cf..3ff1775fd8882a 100644 --- a/be/src/olap/match_predicate.h +++ b/be/src/olap/match_predicate.h @@ -55,8 +55,7 @@ class MatchPredicate : public ColumnPredicate { //evaluate predicate on Bitmap Status evaluate(BitmapIndexIterator* iterator, uint32_t num_rows, roaring::Roaring* roaring) const override { - LOG(FATAL) << "Not Implemented MatchPredicate::evaluate"; - __builtin_unreachable(); + throw Exception(Status::FatalError("Not Implemented MatchPredicate::evaluate")); } //evaluate predicate on inverted diff --git a/be/src/olap/null_predicate.h b/be/src/olap/null_predicate.h index 59480264b46103..8e3fef1ff27695 100644 --- a/be/src/olap/null_predicate.h +++ b/be/src/olap/null_predicate.h @@ -87,8 +87,8 @@ class NullPredicate : public ColumnPredicate { if (_is_null) { return bf->test_bytes(nullptr, 0); } else { - LOG(FATAL) << "Bloom filter is not supported by predicate type: is_null=" << _is_null; - return true; + throw Exception(Status::FatalError( + "Bloom filter is not supported by predicate type: is_null=")); } } diff --git a/be/src/olap/olap_common.h b/be/src/olap/olap_common.h index 11249bafb1e3c0..3b892e5d360e54 100644 --- a/be/src/olap/olap_common.h +++ b/be/src/olap/olap_common.h @@ -36,6 +36,7 @@ #include #include "common/config.h" +#include "common/exception.h" #include "io/io_common.h" #include "olap/olap_define.h" #include "olap/rowset/rowset_fwd.h" @@ -419,7 +420,8 @@ struct RowsetId { LOG(WARNING) << "failed to init rowset id: " << rowset_id_str; high = next_rowset_id().hi; } else { - LOG(FATAL) << "failed to init rowset id: " << rowset_id_str; + throw Exception( + Status::FatalError("failed to init rowset id: {}", rowset_id_str)); } } init(1, high, 0, 0); @@ -440,7 +442,7 @@ struct RowsetId { void init(int64_t id_version, int64_t high, int64_t middle, int64_t low) { version = id_version; if (UNLIKELY(high >= MAX_ROWSET_ID)) { - LOG(FATAL) << "inc rowsetid is too large:" << high; + throw Exception(Status::FatalError("inc rowsetid is too large:{}", high)); } hi = (id_version << 56) + (high & LOW_56_BITS); mi = middle; diff --git a/be/src/olap/page_cache.h b/be/src/olap/page_cache.h index 32b6683e7823b0..db1a6808345525 100644 --- a/be/src/olap/page_cache.h +++ b/be/src/olap/page_cache.h @@ -176,11 +176,9 @@ class StoragePageCache { return _pk_index_page_cache.get(); } default: - LOG(FATAL) << "get error type page cache"; - __builtin_unreachable(); + throw Exception(Status::FatalError("get error type page cache")); } - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); + throw Exception(Status::FatalError("__builtin_unreachable")); } }; diff --git a/be/src/olap/rowset/beta_rowset_writer_v2.h b/be/src/olap/rowset/beta_rowset_writer_v2.h index 78ec4a7dce703c..9040003a68d0d8 100644 --- a/be/src/olap/rowset/beta_rowset_writer_v2.h +++ b/be/src/olap/rowset/beta_rowset_writer_v2.h @@ -99,8 +99,7 @@ class BetaRowsetWriterV2 : public RowsetWriter { }; RowsetSharedPtr manual_build(const RowsetMetaSharedPtr& rowset_meta) override { - LOG(FATAL) << "not implemeted"; - return nullptr; + throw Exception(Status::FatalError("not implemeted")); } PUniqueId load_id() override { return _context.load_id; } diff --git a/be/src/olap/rowset/rowset_writer.h b/be/src/olap/rowset/rowset_writer.h index f84ff964ea3051..0a0d36ea04a661 100644 --- a/be/src/olap/rowset/rowset_writer.h +++ b/be/src/olap/rowset/rowset_writer.h @@ -170,7 +170,9 @@ class RowsetWriter { virtual int32_t allocate_segment_id() = 0; - virtual void set_segment_start_id(int num_segment) { LOG(FATAL) << "not supported!"; } + virtual void set_segment_start_id(int num_segment) { + throw Exception(Status::FatalError("not supported!")); + } virtual int64_t delete_bitmap_ns() { return 0; } diff --git a/be/src/olap/rowset/segment_v2/hierarchical_data_reader.cpp b/be/src/olap/rowset/segment_v2/hierarchical_data_reader.cpp index db6bac6b8b4c09..fe7167e9444a76 100644 --- a/be/src/olap/rowset/segment_v2/hierarchical_data_reader.cpp +++ b/be/src/olap/rowset/segment_v2/hierarchical_data_reader.cpp @@ -80,8 +80,7 @@ Status HierarchicalDataReader::init(const ColumnIteratorOptions& opts) { } Status HierarchicalDataReader::seek_to_first() { - LOG(FATAL) << "Not implemented"; - __builtin_unreachable(); + throw Exception(Status::FatalError("Not implemented")); } Status HierarchicalDataReader::seek_to_ordinal(ordinal_t ord) { @@ -159,8 +158,7 @@ Status ExtractReader::init(const ColumnIteratorOptions& opts) { } Status ExtractReader::seek_to_first() { - LOG(FATAL) << "Not implemented"; - __builtin_unreachable(); + throw Exception(Status::FatalError("Not implemented")); } Status ExtractReader::seek_to_ordinal(ordinal_t ord) { diff --git a/be/src/olap/storage_policy.cpp b/be/src/olap/storage_policy.cpp index 837e9bed178e3a..3b4a1f1a185678 100644 --- a/be/src/olap/storage_policy.cpp +++ b/be/src/olap/storage_policy.cpp @@ -141,8 +141,10 @@ std::vector> get_storage_resource_ids() { namespace { [[noreturn]] void exit_at_unknown_path_version(std::string_view resource_id, int64_t path_version) { - LOG(FATAL) << "unknown path version, please upgrade BE or drop this storage vault. resource_id=" - << resource_id << " path_version=" << path_version; + throw Exception( + Status::FatalError("unknown path version, please upgrade BE or drop this storage " + "vault. resource_id={} path_version={}", + resource_id, path_version)); } } // namespace diff --git a/be/src/olap/tablet_reader.cpp b/be/src/olap/tablet_reader.cpp index a83e0bfdbf4c30..17cab2a3c0c834 100644 --- a/be/src/olap/tablet_reader.cpp +++ b/be/src/olap/tablet_reader.cpp @@ -61,7 +61,7 @@ using namespace ErrorCode; void TabletReader::ReaderParams::check_validation() const { if (UNLIKELY(version.first == -1 && is_segcompaction == false)) { - LOG(FATAL) << "version is not set. tablet=" << tablet->tablet_id(); + throw Exception(Status::FatalError("version is not set. tablet={}", tablet->tablet_id())); } } diff --git a/be/src/pipeline/dependency.h b/be/src/pipeline/dependency.h index f1cfe2b02977e1..ecbd49a5647c2e 100644 --- a/be/src/pipeline/dependency.h +++ b/be/src/pipeline/dependency.h @@ -723,8 +723,7 @@ inline std::string get_exchange_type_name(ExchangeType idx) { case ExchangeType::LOCAL_MERGE_SORT: return "LOCAL_MERGE_SORT"; } - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); + throw Exception(Status::FatalError("__builtin_unreachable")); } struct DataDistribution { diff --git a/be/src/pipeline/exec/exchange_sink_buffer.cpp b/be/src/pipeline/exec/exchange_sink_buffer.cpp index e3f895444d4168..800ef6150738d6 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.cpp +++ b/be/src/pipeline/exec/exchange_sink_buffer.cpp @@ -422,8 +422,7 @@ void ExchangeSinkBuffer::_ended(InstanceLoId id) { } LOG(INFO) << ss.str(); - LOG(FATAL) << "not find the instance id"; - __builtin_unreachable(); + throw Exception(Status::FatalError("not find the instance id")); } else { std::unique_lock lock(*_instance_to_package_queue_mutex[id]); _running_sink_count[id]--; diff --git a/be/src/pipeline/exec/exchange_sink_buffer.h b/be/src/pipeline/exec/exchange_sink_buffer.h index 458c7c3f66e3ee..a381c5aff144f3 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.h +++ b/be/src/pipeline/exec/exchange_sink_buffer.h @@ -155,10 +155,9 @@ class ExchangeSendCallback : public ::doris::DummyBrpcCallback { start_rpc_time); } } catch (const std::exception& exp) { - LOG(FATAL) << "brpc callback error: " << exp.what(); + throw Exception(Status::FatalError("brpc callback error: {}", exp.what())); } catch (...) { - LOG(FATAL) << "brpc callback error."; - __builtin_unreachable(); + throw Exception(Status::FatalError("brpc callback error.")); } } int64_t start_rpc_time; diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index 19e8493e596a7e..47560875b51252 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -303,9 +303,7 @@ Status HashJoinBuildSinkLocalState::process_build_block(RuntimeState* state, [&](std::monostate& arg, auto join_op, auto short_circuit_for_null_in_build_side, auto with_other_conjuncts) -> Status { - LOG(FATAL) << "FATAL: uninited hash table"; - __builtin_unreachable(); - return Status::OK(); + throw Exception(Status::FatalError("FATAL: uninited hash table")); }, [&](auto&& arg, auto&& join_op, auto short_circuit_for_null_in_build_side, auto with_other_conjuncts) -> Status { diff --git a/be/src/pipeline/exec/operator.cpp b/be/src/pipeline/exec/operator.cpp index f6664e147a3dab..bb254aae72b8a7 100644 --- a/be/src/pipeline/exec/operator.cpp +++ b/be/src/pipeline/exec/operator.cpp @@ -414,8 +414,7 @@ std::shared_ptr DataSinkOperatorX::create_shar return nullptr; } else if constexpr (std::is_same_v) { - LOG(FATAL) << "should not reach here!"; - return nullptr; + throw Exception(Status::FatalError("should not reach here!")); } else { auto ss = LocalStateType::SharedStateType::create_shared(); ss->id = operator_id(); diff --git a/be/src/pipeline/exec/operator.h b/be/src/pipeline/exec/operator.h index a2c8e110cedac3..df6e9c913b6b4c 100644 --- a/be/src/pipeline/exec/operator.h +++ b/be/src/pipeline/exec/operator.h @@ -632,12 +632,10 @@ class OperatorXBase : public OperatorBase { _limit(-1) {} virtual Status init(const TPlanNode& tnode, RuntimeState* state); Status init(const TDataSink& tsink) override { - LOG(FATAL) << "should not reach here!"; - return Status::OK(); + throw Exception(Status::FatalError("should not reach here!")); } virtual Status init(ExchangeType type) { - LOG(FATAL) << "should not reach here!"; - return Status::OK(); + throw Exception(Status::FatalError("should not reach here!")); } [[noreturn]] virtual const std::vector& runtime_filter_descs() { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, _op_name); diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index a371cdb947ff56..2d7554e702969f 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -421,9 +421,9 @@ void ExecEnv::init_file_cache_factory(std::vector& cache_paths std::unordered_set cache_path_set; Status rest = doris::parse_conf_cache_paths(doris::config::file_cache_path, cache_paths); if (!rest) { - LOG(FATAL) << "parse config file cache path failed, path=" << doris::config::file_cache_path - << ", reason=" << rest.msg(); - exit(-1); + throw Exception( + Status::FatalError("parse config file cache path failed, path={}, reason={}", + doris::config::file_cache_path, rest.msg())); } doris::Status cache_status; @@ -437,8 +437,8 @@ void ExecEnv::init_file_cache_factory(std::vector& cache_paths cache_path.path, cache_path.init_settings()); if (!cache_status.ok()) { if (!doris::config::ignore_broken_disk) { - LOG(FATAL) << "failed to init file cache, err: " << cache_status; - exit(-1); + throw Exception( + Status::FatalError("failed to init file cache, err: {}", cache_status)); } LOG(WARNING) << "failed to init file cache, err: " << cache_status; } diff --git a/be/src/runtime/jsonb_value.h b/be/src/runtime/jsonb_value.h index 65f4927759c304..5f530db1ac8117 100644 --- a/be/src/runtime/jsonb_value.h +++ b/be/src/runtime/jsonb_value.h @@ -61,58 +61,47 @@ struct JsonBinaryValue { } bool operator==(const JsonBinaryValue& other) const { - LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; - __builtin_unreachable(); + throw Exception(Status::FatalError("comparing between JsonBinaryValue is not supported")); } // != bool ne(const JsonBinaryValue& other) const { - LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; - __builtin_unreachable(); + throw Exception(Status::FatalError("comparing between JsonBinaryValue is not supported")); } // <= bool le(const JsonBinaryValue& other) const { - LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; - __builtin_unreachable(); + throw Exception(Status::FatalError("comparing between JsonBinaryValue is not supported")); } // >= bool ge(const JsonBinaryValue& other) const { - LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; - __builtin_unreachable(); + throw Exception(Status::FatalError("comparing between JsonBinaryValue is not supported")); } // < bool lt(const JsonBinaryValue& other) const { - LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; - __builtin_unreachable(); + throw Exception(Status::FatalError("comparing between JsonBinaryValue is not supported")); } // > bool gt(const JsonBinaryValue& other) const { - LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; - __builtin_unreachable(); + throw Exception(Status::FatalError("comparing between JsonBinaryValue is not supported")); } bool operator!=(const JsonBinaryValue& other) const { - LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; - __builtin_unreachable(); + throw Exception(Status::FatalError("comparing between JsonBinaryValue is not supported")); } bool operator<=(const JsonBinaryValue& other) const { - LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; - __builtin_unreachable(); + throw Exception(Status::FatalError("comparing between JsonBinaryValue is not supported")); } bool operator>=(const JsonBinaryValue& other) const { - LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; - __builtin_unreachable(); + throw Exception(Status::FatalError("comparing between JsonBinaryValue is not supported")); } bool operator<(const JsonBinaryValue& other) const { - LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; - __builtin_unreachable(); + throw Exception(Status::FatalError("comparing between JsonBinaryValue is not supported")); } bool operator>(const JsonBinaryValue& other) const { - LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; - __builtin_unreachable(); + throw Exception(Status::FatalError("comparing between JsonBinaryValue is not supported")); } Status from_json_string(const char* s, size_t len); diff --git a/be/src/runtime/memory/cache_manager.h b/be/src/runtime/memory/cache_manager.h index a2a089b929dbdf..1e89e957ba1ce6 100644 --- a/be/src/runtime/memory/cache_manager.h +++ b/be/src/runtime/memory/cache_manager.h @@ -40,7 +40,8 @@ class CacheManager { #ifdef BE_TEST _caches.erase(it); #else - LOG(FATAL) << "Repeat register cache " << CachePolicy::type_string(cache->type()); + throw Exception(Status::FatalError("Repeat register cache {}", + CachePolicy::type_string(cache->type()))); #endif // BE_TEST } _caches.insert({cache->type(), cache}); diff --git a/be/src/runtime/memory/cache_policy.h b/be/src/runtime/memory/cache_policy.h index 8f077a4eb45bb1..72e61fed2e0013 100644 --- a/be/src/runtime/memory/cache_policy.h +++ b/be/src/runtime/memory/cache_policy.h @@ -99,10 +99,10 @@ class CachePolicy { case CacheType::TABLET_COLUMN_OBJECT_POOL: return "TabletColumnObjectPool"; default: - LOG(FATAL) << "not match type of cache policy :" << static_cast(type); + throw Exception(Status::FatalError("not match type of cache policy :{}", + static_cast(type))); } - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); + throw Exception(Status::FatalError("__builtin_unreachable")); } inline static std::unordered_map StringToType = { diff --git a/be/src/runtime/memory/lru_cache_policy.h b/be/src/runtime/memory/lru_cache_policy.h index 3fdb43facd7715..d4c282dab8274e 100644 --- a/be/src/runtime/memory/lru_cache_policy.h +++ b/be/src/runtime/memory/lru_cache_policy.h @@ -90,7 +90,8 @@ class LRUCachePolicy : public CachePolicy { case LRUCacheType::NUMBER: return "number"; default: - LOG(FATAL) << "not match type of lru cache:" << static_cast(type); + throw Exception( + Status::FatalError("not match type of lru cache:{}", static_cast(type))); } } diff --git a/be/src/runtime/memory/thread_mem_tracker_mgr.h b/be/src/runtime/memory/thread_mem_tracker_mgr.h index db3b32a6298820..9dbf4399492d02 100644 --- a/be/src/runtime/memory/thread_mem_tracker_mgr.h +++ b/be/src/runtime/memory/thread_mem_tracker_mgr.h @@ -246,13 +246,13 @@ inline void ThreadMemTrackerMgr::consume(int64_t size, int skip_large_memory_che } if (doris::config::crash_in_alloc_large_memory_bytes > 0 && size > doris::config::crash_in_alloc_large_memory_bytes) { - LOG(FATAL) << fmt::format( + throw Exception(Status::FatalError( "alloc large memory: {}, {}, crash generate core dumpsto help analyze, " "stacktrace:\n{}", size, is_attach_query() ? "in query or load: " + print_id(_query_id) : "not in query or load", - get_stack_trace()); + get_stack_trace())); } } } diff --git a/be/src/runtime/snapshot_loader.cpp b/be/src/runtime/snapshot_loader.cpp index 784904c78a3fb1..b492a929fca3bf 100644 --- a/be/src/runtime/snapshot_loader.cpp +++ b/be/src/runtime/snapshot_loader.cpp @@ -74,7 +74,7 @@ Status upload_with_checksum(io::RemoteFileSystem& fs, std::string_view local_pat RETURN_IF_ERROR(fs.upload(local_path, full_remote_path)); break; default: - LOG(FATAL) << "unknown fs type: " << static_cast(fs.type()); + throw Exception(Status::FatalError("unknown fs type: {}", static_cast(fs.type()))); } return Status::OK(); } @@ -807,8 +807,7 @@ Status SnapshotLoader::move(const std::string& snapshot_path, TabletSharedPtr ta } } else { - LOG(FATAL) << "only support overwrite now"; - __builtin_unreachable(); + throw Exception(Status::FatalError("only support overwrite now")); } // snapshot loader not need to change tablet uid diff --git a/be/src/runtime/stream_load/stream_load_executor.cpp b/be/src/runtime/stream_load/stream_load_executor.cpp index ad4d22946f1b83..054de96a881425 100644 --- a/be/src/runtime/stream_load/stream_load_executor.cpp +++ b/be/src/runtime/stream_load/stream_load_executor.cpp @@ -390,8 +390,7 @@ bool StreamLoadExecutor::collect_load_stat(StreamLoadContext* ctx, TTxnCommitAtt } switch (ctx->load_type) { case TLoadType::MINI_LOAD: { - LOG(FATAL) << "mini load is not supported any more"; - break; + throw Exception(Status::FatalError("mini load is not supported any more")); } case TLoadType::ROUTINE_LOAD: { attach->loadType = TLoadType::ROUTINE_LOAD; diff --git a/be/src/runtime/thread_context.h b/be/src/runtime/thread_context.h index e0a44af69c1d66..9ba7949ec5afad 100644 --- a/be/src/runtime/thread_context.h +++ b/be/src/runtime/thread_context.h @@ -354,8 +354,7 @@ class ThreadLocalHandle { DCHECK(bthread_context != nullptr); bthread_context->thread_local_handle_count--; } else { - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); + throw Exception(Status::FatalError("__builtin_unreachable")); } } }; @@ -379,8 +378,8 @@ static ThreadContext* thread_context(bool allow_return_null = false) { return nullptr; } // It means that use thread_context() but this thread not attached a query/load using SCOPED_ATTACH_TASK macro. - LOG(FATAL) << "__builtin_unreachable, " << doris::memory_orphan_check_msg; - __builtin_unreachable(); + throw Exception( + Status::FatalError("__builtin_unreachable, {}", doris::memory_orphan_check_msg)); } // belong to one query object member, not be shared by multiple queries. diff --git a/be/src/util/binary_cast.hpp b/be/src/util/binary_cast.hpp index 8a91ab3a579152..e7c62ad45ac091 100644 --- a/be/src/util/binary_cast.hpp +++ b/be/src/util/binary_cast.hpp @@ -137,8 +137,7 @@ To binary_cast(From from) { conv.decimal = from; return conv.i128; } else { - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); + throw Exception(Status::FatalError("__builtin_unreachable")); } } diff --git a/be/src/util/bit_util.h b/be/src/util/bit_util.h index 504b0b27428190..5ec5a8bf8e1aa4 100644 --- a/be/src/util/bit_util.h +++ b/be/src/util/bit_util.h @@ -237,9 +237,7 @@ class BitUtil { } else if constexpr (std::is_same_v) { return value; } else { - __builtin_unreachable(); - LOG(FATAL) << "__builtin_unreachable"; - return value; + throw Exception(Status::FatalError("__builtin_unreachable")); } } diff --git a/be/src/util/bitmap_value.h b/be/src/util/bitmap_value.h index 2d15ac99611274..528dbe40788229 100644 --- a/be/src/util/bitmap_value.h +++ b/be/src/util/bitmap_value.h @@ -2519,8 +2519,7 @@ class BitmapValueIterator { } break; case BitmapValue::BitmapDataType::SET: { - LOG(FATAL) << "BitmapValue with set do not support move"; - break; + throw Exception(Status::FatalError("BitmapValue with set do not support move")); } default: break; diff --git a/be/src/util/block_compression.cpp b/be/src/util/block_compression.cpp index d1788b0948a6f2..7a0aacd4252dec 100644 --- a/be/src/util/block_compression.cpp +++ b/be/src/util/block_compression.cpp @@ -233,7 +233,8 @@ class HadoopLz4BlockCompression : public Lz4BlockCompression { HadoopLz4BlockCompression() { Status st = Decompressor::create_decompressor(CompressType::LZ4BLOCK, &_decompressor); if (!st.ok()) { - LOG(FATAL) << "HadoopLz4BlockCompression construction failed. status = " << st << "\n"; + throw Exception(Status::FatalError( + "HadoopLz4BlockCompression construction failed. status = {}", st)); } } diff --git a/be/src/util/easy_json.cc b/be/src/util/easy_json.cc index 46c3a1867f7b42..fcb8021e3836b2 100644 --- a/be/src/util/easy_json.cc +++ b/be/src/util/easy_json.cc @@ -27,6 +27,8 @@ #include #include #include + +#include "common/exception.h" // IWYU pragma: no_include using rapidjson::SizeType; @@ -200,8 +202,7 @@ EasyJson EasyJson::PushBack(EasyJson::ComplexTypeInitializer val) { } else if (val == kArray) { push_val.SetArray(); } else { - LOG(FATAL) << "Unknown initializer type"; - __builtin_unreachable(); + throw Exception(Status::FatalError("Unknown initializer type")); } value_->PushBack(push_val, alloc_->allocator()); return EasyJson(&(*value_)[value_->Size() - 1], alloc_); diff --git a/be/src/util/jsonb_utils.h b/be/src/util/jsonb_utils.h index 7dba0dca3af1eb..8ec842ef227dd5 100644 --- a/be/src/util/jsonb_utils.h +++ b/be/src/util/jsonb_utils.h @@ -23,6 +23,7 @@ #include +#include "common/exception.h" #include "jsonb_document.h" #include "jsonb_stream.h" #include "jsonb_writer.h" @@ -42,7 +43,8 @@ class JsonbToJson { const std::string to_json_string(const char* data, size_t size) { JsonbDocument* pdoc = doris::JsonbDocument::createDocument(data, size); if (!pdoc) { - LOG(FATAL) << "invalid json binary value: " << std::string_view(data, size); + throw Exception(Status::FatalError("invalid json binary value: {}", + std::string_view(data, size))); } return to_json_string(pdoc->getValue()); } diff --git a/be/src/util/rle_encoding.h b/be/src/util/rle_encoding.h index 206349b472815d..5369ace9eed6ce 100644 --- a/be/src/util/rle_encoding.h +++ b/be/src/util/rle_encoding.h @@ -283,7 +283,7 @@ void RleDecoder::RewindOne() { switch (rewind_state_) { case CANT_REWIND: - LOG(FATAL) << "Can't rewind more than once after each read!"; + throw Exception(Status::FatalError("Can't rewind more than once after each read!")); break; case REWIND_RUN: ++repeat_count_; diff --git a/be/src/util/threadpool.cpp b/be/src/util/threadpool.cpp index f5ea38515def36..e9af13f556e143 100644 --- a/be/src/util/threadpool.cpp +++ b/be/src/util/threadpool.cpp @@ -27,6 +27,7 @@ #include #include +#include "common/exception.h" #include "common/logging.h" #include "gutil/map-util.h" #include "gutil/port.h" @@ -194,7 +195,7 @@ void ThreadPoolToken::transition(State new_state) { CHECK(false); // QUIESCED is a terminal state break; default: - LOG(FATAL) << "Unknown token state: " << _state; + throw Exception(Status::FatalError("Unknown token state: {}", _state)); } #endif @@ -616,10 +617,10 @@ Status ThreadPool::create_thread() { void ThreadPool::check_not_pool_thread_unlocked() { Thread* current = Thread::current_thread(); if (ContainsKey(_threads, current)) { - LOG(FATAL) << strings::Substitute( - "Thread belonging to thread pool '$0' with " - "name '$1' called pool function that would result in deadlock", - _name, current->name()); + throw Exception( + Status::FatalError("Thread belonging to thread pool {} with " + "name {} called pool function that would result in deadlock", + _name, current->name())); } } diff --git a/be/src/util/timezone_utils.cpp b/be/src/util/timezone_utils.cpp index 6bb71ac46471c9..a26ad3703b79b9 100644 --- a/be/src/util/timezone_utils.cpp +++ b/be/src/util/timezone_utils.cpp @@ -35,6 +35,7 @@ #include #include +#include "common/exception.h" #include "common/logging.h" #include "common/status.h" @@ -83,8 +84,7 @@ void TimezoneUtils::load_timezones_to_cache() { const auto root_path = fs::path {base_str}; if (!exists(root_path)) { - LOG(FATAL) << "Cannot find system tzfile. Doris exiting!"; - __builtin_unreachable(); + throw Exception(Status::FatalError("Cannot find system tzfile. Doris exiting!")); } std::set ignore_paths = {"posix", "right"}; // duplications. ignore them. diff --git a/be/src/vec/aggregate_functions/aggregate_function_map.h b/be/src/vec/aggregate_functions/aggregate_function_map.h index 17bc54f7499adb..7273390e7c5342 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_map.h +++ b/be/src/vec/aggregate_functions/aggregate_function_map.h @@ -40,10 +40,7 @@ struct AggregateFunctionMapAggData { using KeyType = std::conditional_t, StringRef, K>; using Map = phmap::flat_hash_map; - AggregateFunctionMapAggData() { - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); - } + AggregateFunctionMapAggData() { throw Exception(Status::FatalError("__builtin_unreachable")); } AggregateFunctionMapAggData(const DataTypes& argument_types) { _key_type = remove_nullable(argument_types[0]); diff --git a/be/src/vec/aggregate_functions/aggregate_function_reader_first_last.h b/be/src/vec/aggregate_functions/aggregate_function_reader_first_last.h index 8efea2dc6fc8e4..6f5d680d3eb0fc 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_reader_first_last.h +++ b/be/src/vec/aggregate_functions/aggregate_function_reader_first_last.h @@ -238,24 +238,17 @@ class ReaderFunctionData final void add_range_single_place(int64_t partition_start, int64_t partition_end, int64_t frame_start, int64_t frame_end, AggregateDataPtr place, const IColumn** columns, Arena*) const override { - throw doris::Exception(ErrorCode::INTERNAL_ERROR, - "ReaderFunctionData do not support add_range_single_place"); - __builtin_unreachable(); + throw doris::Exception( + Status::FatalError("ReaderFunctionData do not support add_range_single_place")); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena*) const override { - throw doris::Exception(ErrorCode::INTERNAL_ERROR, - "ReaderFunctionData do not support merge"); - __builtin_unreachable(); + throw doris::Exception(Status::FatalError("ReaderFunctionData do not support merge")); } void serialize(ConstAggregateDataPtr place, BufferWritable& buf) const override { - throw doris::Exception(ErrorCode::INTERNAL_ERROR, - "ReaderFunctionData do not support serialize"); - __builtin_unreachable(); + throw doris::Exception(Status::FatalError("ReaderFunctionData do not support serialize")); } void deserialize(AggregateDataPtr place, BufferReadable& buf, Arena*) const override { - throw doris::Exception(ErrorCode::INTERNAL_ERROR, - "ReaderFunctionData do not support deserialize"); - __builtin_unreachable(); + throw doris::Exception(Status::FatalError("ReaderFunctionData do not support deserialize")); } private: diff --git a/be/src/vec/aggregate_functions/aggregate_function_window.h b/be/src/vec/aggregate_functions/aggregate_function_window.h index 13fa8e74751df6..0cef4c82d3dbfe 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_window.h +++ b/be/src/vec/aggregate_functions/aggregate_function_window.h @@ -563,24 +563,19 @@ class WindowFunctionData final void add(AggregateDataPtr place, const IColumn** columns, ssize_t row_num, Arena*) const override { - throw doris::Exception(ErrorCode::INTERNAL_ERROR, - "WindowFunctionLeadLagData do not support add"); - __builtin_unreachable(); + throw doris::Exception(Status::FatalError("WindowFunctionLeadLagData do not support add")); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena*) const override { - throw doris::Exception(ErrorCode::INTERNAL_ERROR, - "WindowFunctionLeadLagData do not support merge"); - __builtin_unreachable(); + throw doris::Exception( + Status::FatalError("WindowFunctionLeadLagData do not support merge")); } void serialize(ConstAggregateDataPtr place, BufferWritable& buf) const override { - throw doris::Exception(ErrorCode::INTERNAL_ERROR, - "WindowFunctionLeadLagData do not support serialize"); - __builtin_unreachable(); + throw doris::Exception( + Status::FatalError("WindowFunctionLeadLagData do not support serialize")); } void deserialize(AggregateDataPtr place, BufferReadable& buf, Arena*) const override { - throw doris::Exception(ErrorCode::INTERNAL_ERROR, - "WindowFunctionLeadLagData do not support deserialize"); - __builtin_unreachable(); + throw doris::Exception( + Status::FatalError("WindowFunctionLeadLagData do not support deserialize")); } private: diff --git a/be/src/vec/columns/column_string.cpp b/be/src/vec/columns/column_string.cpp index cb83a29bbada2c..db0088e67c27b6 100644 --- a/be/src/vec/columns/column_string.cpp +++ b/be/src/vec/columns/column_string.cpp @@ -40,16 +40,16 @@ template void ColumnStr::sanity_check() const { auto count = offsets.size(); if (chars.size() != offsets[count - 1]) { - LOG(FATAL) << "row count: " << count << ", chars.size(): " << chars.size() << ", offset[" - << count - 1 << "]: " << offsets[count - 1]; + throw Exception(Status::FatalError("row count: {}, chars.size(): {}, offset[{}]: ", count, + chars.size(), count - 1, offsets[count - 1])); } if (offsets[-1] != 0) { - LOG(FATAL) << "wrong offsets[-1]: " << offsets[-1]; + throw Exception(Status::FatalError("wrong offsets[-1]: {}", offsets[-1])); } for (size_t i = 0; i < count; ++i) { if (offsets[i] < offsets[i - 1]) { - LOG(FATAL) << "row count: " << count << ", offsets[" << i << "]: " << offsets[i] - << ", offsets[" << i - 1 << "]: " << offsets[i - 1]; + throw Exception(Status::FatalError("row count: {}, offsets[{}]: {}, offsets[{}]: {}", + count, i, offsets[i], i - 1, offsets[i - 1])); } } } diff --git a/be/src/vec/common/assert_cast.h b/be/src/vec/common/assert_cast.h index 02dce99e967bdb..1905983a58cc29 100644 --- a/be/src/vec/common/assert_cast.h +++ b/be/src/vec/common/assert_cast.h @@ -23,6 +23,7 @@ #include #include +#include "common/exception.h" #include "common/logging.h" #include "vec/common/demangle.h" @@ -45,35 +46,33 @@ PURE To assert_cast(From&& from) { if (auto ptr = dynamic_cast(from); ptr != nullptr) { return ptr; } - LOG(FATAL) << fmt::format("Bad cast from type:{}* to {}", - demangle(typeid(*from).name()), - demangle(typeid(To).name())); + throw doris::Exception(doris::Status::FatalError("Bad cast from type:{}* to {}", + demangle(typeid(*from).name()), + demangle(typeid(To).name()))); } } else { if (typeid(from) == typeid(To)) { return static_cast(from); } } - LOG(FATAL) << fmt::format("Bad cast from type:{} to {}", demangle(typeid(from).name()), - demangle(typeid(To).name())); - __builtin_unreachable(); + throw doris::Exception(doris::Status::FatalError("Bad cast from type:{} to {}", + demangle(typeid(from).name()), + demangle(typeid(To).name()))); }; #ifndef NDEBUG try { return perform_cast(std::forward(from)); } catch (const std::exception& e) { - LOG(FATAL) << "assert cast err:" << e.what(); + throw doris::Exception(doris::Status::FatalError("assert cast err:{}", e.what())); } - __builtin_unreachable(); #else if constexpr (check == TypeCheckOnRelease::ENABLE) { try { return perform_cast(std::forward(from)); } catch (const std::exception& e) { - LOG(FATAL) << "assert cast err:" << e.what(); + throw doris::Exception(doris::Status::FatalError("assert cast err:{}", e.what())); } - __builtin_unreachable(); } else { return static_cast(from); } diff --git a/be/src/vec/common/hash_table/string_hash_table.h b/be/src/vec/common/hash_table/string_hash_table.h index 74be1e85e1efe8..892598a83263b9 100644 --- a/be/src/vec/common/hash_table/string_hash_table.h +++ b/be/src/vec/common/hash_table/string_hash_table.h @@ -327,8 +327,7 @@ class StringHashTable : private boost::noncopyable { return iterator5 == rhs.iterator5; } } - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); + throw doris::Exception(doris::Status::FatalError("__builtin_unreachable")); } bool operator!=(const iterator_base& rhs) const { return !(*this == rhs); } diff --git a/be/src/vec/common/schema_util.cpp b/be/src/vec/common/schema_util.cpp index fd50af3e1fcd88..2b1c71c643d613 100644 --- a/be/src/vec/common/schema_util.cpp +++ b/be/src/vec/common/schema_util.cpp @@ -133,7 +133,7 @@ size_t get_size_of_interger(TypeIndex type) { case TypeIndex::UInt128: return sizeof(uint128_t); default: - LOG(FATAL) << "Unknown integer type: " << getTypeName(type); + throw Exception(Status::FatalError("Unknown integer type: {}", getTypeName(type))); return 0; } } @@ -231,8 +231,7 @@ void get_column_by_type(const vectorized::DataTypePtr& data_type, const std::str return; } // TODO handle more types like struct/date/datetime/decimal... - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); + throw Exception(Status::FatalError("__builtin_unreachable")); } TabletColumn get_column_by_type(const vectorized::DataTypePtr& data_type, const std::string& name, diff --git a/be/src/vec/core/block.cpp b/be/src/vec/core/block.cpp index 4dc553b1a5790f..951c2661faf172 100644 --- a/be/src/vec/core/block.cpp +++ b/be/src/vec/core/block.cpp @@ -644,10 +644,10 @@ Block Block::clone_with_columns(const Columns& columns) const { size_t num_columns = data.size(); if (num_columns != columns.size()) { - LOG(FATAL) << fmt::format( + throw Exception(Status::FatalError( "Cannot clone block with columns because block has {} columns, but {} columns " "given.", - num_columns, columns.size()); + num_columns, columns.size())); } for (size_t i = 0; i < num_columns; ++i) { diff --git a/be/src/vec/core/decimal_comparison.h b/be/src/vec/core/decimal_comparison.h index 9e9d9ad399ae04..4503a264c28014 100644 --- a/be/src/vec/core/decimal_comparison.h +++ b/be/src/vec/core/decimal_comparison.h @@ -82,8 +82,9 @@ class DecimalComparison { DecimalComparison(Block& block, uint32_t result, const ColumnWithTypeAndName& col_left, const ColumnWithTypeAndName& col_right) { if (!apply(block, result, col_left, col_right)) { - LOG(FATAL) << fmt::format("Wrong decimal comparison with {} and {}", - col_left.type->get_name(), col_right.type->get_name()); + throw Exception(Status::FatalError("Wrong decimal comparison with {} and {}", + col_left.type->get_name(), + col_right.type->get_name())); } } @@ -106,8 +107,7 @@ class DecimalComparison { static bool compare(A a, B b, UInt32 scale_a, UInt32 scale_b) { static const UInt32 max_scale = max_decimal_precision(); if (scale_a > max_scale || scale_b > max_scale) { - LOG(FATAL) << "Bad scale of decimal field"; - __builtin_unreachable(); + throw Exception(Status::FatalError("Bad scale of decimal field")); } Shift shift; @@ -213,8 +213,7 @@ class DecimalComparison { if (const ColVecB* c1_vec = check_and_get_column(c1.get())) constant_vector(a, c1_vec->get_data(), vec_res, scale); else { - LOG(FATAL) << "Wrong column in Decimal comparison"; - __builtin_unreachable(); + throw Exception(Status::FatalError("Wrong column in Decimal comparison")); } } else if (c1_is_const) { const ColumnConst* c1_const = check_and_get_column_const(c1.get()); @@ -222,8 +221,7 @@ class DecimalComparison { if (const ColVecA* c0_vec = check_and_get_column(c0.get())) vector_constant(c0_vec->get_data(), b, vec_res, scale); else { - LOG(FATAL) << "Wrong column in Decimal comparison"; - __builtin_unreachable(); + throw Exception(Status::FatalError("Wrong column in Decimal comparison")); } } else { if (const ColVecA* c0_vec = check_and_get_column(c0.get())) { @@ -231,12 +229,10 @@ class DecimalComparison { vector_vector(c0_vec->get_data(), c1_vec->get_data(), vec_res, scale); else { - LOG(FATAL) << "Wrong column in Decimal comparison"; - __builtin_unreachable(); + throw Exception(Status::FatalError("Wrong column in Decimal comparison")); } } else { - LOG(FATAL) << "Wrong column in Decimal comparison"; - __builtin_unreachable(); + throw Exception(Status::FatalError("Wrong column in Decimal comparison")); } } return c_res; @@ -262,8 +258,7 @@ class DecimalComparison { if constexpr (scale_right) overflow |= common::mul_overflow(y, scale, y); if (overflow) { - LOG(FATAL) << "Can't compare"; - __builtin_unreachable(); + throw Exception(Status::FatalError("Can't compare")); } } else { if constexpr (scale_left) x *= scale; diff --git a/be/src/vec/core/field.h b/be/src/vec/core/field.h index 341f65e075ed11..1176840738a289 100644 --- a/be/src/vec/core/field.h +++ b/be/src/vec/core/field.h @@ -38,6 +38,7 @@ #include #include "common/compiler_util.h" // IWYU pragma: keep +#include "common/exception.h" #include "olap/hll.h" #include "util/bitmap_value.h" #include "util/quantile_state.h" @@ -168,7 +169,7 @@ class JsonbField { JsonbField(const char* ptr, size_t len) : size(len) { data = new char[size]; if (!data) { - LOG(FATAL) << "new data buffer failed, size: " << size; + throw Exception(Status::FatalError("new data buffer failed, size: {}", size)); } memcpy(data, ptr, size); } @@ -176,7 +177,7 @@ class JsonbField { JsonbField(const JsonbField& x) : size(x.size) { data = new char[size]; if (!data) { - LOG(FATAL) << "new data buffer failed, size: " << size; + throw Exception(Status::FatalError("new data buffer failed, size: {}", size)); } memcpy(data, x.data, size); } @@ -189,7 +190,7 @@ class JsonbField { JsonbField& operator=(const JsonbField& x) { data = new char[size]; if (!data) { - LOG(FATAL) << "new data buffer failed, size: " << size; + throw Exception(Status::FatalError("new data buffer failed, size: {}", size)); } memcpy(data, x.data, size); return *this; @@ -216,38 +217,30 @@ class JsonbField { size_t get_size() const { return size; } bool operator<(const JsonbField& r) const { - LOG(FATAL) << "comparing between JsonbField is not supported"; - __builtin_unreachable(); + throw Exception(Status::FatalError("comparing between JsonbField is not supported")); } bool operator<=(const JsonbField& r) const { - LOG(FATAL) << "comparing between JsonbField is not supported"; - __builtin_unreachable(); + throw Exception(Status::FatalError("comparing between JsonbField is not supported")); } bool operator==(const JsonbField& r) const { - LOG(FATAL) << "comparing between JsonbField is not supported"; - __builtin_unreachable(); + throw Exception(Status::FatalError("comparing between JsonbField is not supported")); } bool operator>(const JsonbField& r) const { - LOG(FATAL) << "comparing between JsonbField is not supported"; - __builtin_unreachable(); + throw Exception(Status::FatalError("comparing between JsonbField is not supported")); } bool operator>=(const JsonbField& r) const { - LOG(FATAL) << "comparing between JsonbField is not supported"; - __builtin_unreachable(); + throw Exception(Status::FatalError("comparing between JsonbField is not supported")); } bool operator!=(const JsonbField& r) const { - LOG(FATAL) << "comparing between JsonbField is not supported"; - __builtin_unreachable(); + throw Exception(Status::FatalError("comparing between JsonbField is not supported")); } const JsonbField& operator+=(const JsonbField& r) { - LOG(FATAL) << "Not support plus opration on JsonbField"; - __builtin_unreachable(); + throw Exception(Status::FatalError("Not support plus opration on JsonbField")); } const JsonbField& operator-=(const JsonbField& r) { - LOG(FATAL) << "Not support minus opration on JsonbField"; - __builtin_unreachable(); + throw Exception(Status::FatalError("Not support minus opration on JsonbField")); } private: @@ -305,8 +298,7 @@ class DecimalField { const DecimalField& operator+=(const DecimalField& r) { if (scale != r.get_scale()) { - LOG(FATAL) << "Add different decimal fields"; - __builtin_unreachable(); + throw Exception(Status::FatalError("Add different decimal fields")); } dec += r.get_value(); return *this; @@ -314,8 +306,7 @@ class DecimalField { const DecimalField& operator-=(const DecimalField& r) { if (scale != r.get_scale()) { - LOG(FATAL) << "Sub different decimal fields"; - __builtin_unreachable(); + throw Exception(Status::FatalError("Sub different decimal fields")); } dec -= r.get_value(); return *this; @@ -422,8 +413,8 @@ class Field { case IPv6: return "IPv6"; default: - LOG(FATAL) << "type not supported, type=" << Types::to_string(which); - break; + throw Exception( + Status::FatalError("type not supported, type={}", Types::to_string(which))); } __builtin_unreachable(); } @@ -558,8 +549,9 @@ class Field { return which <=> rhs.which; } if (which != rhs.which) { - LOG(FATAL) << "lhs type not equal with rhs, lhs=" << Types::to_string(which) - << ", rhs=" << Types::to_string(rhs.which); + throw Exception(Status::FatalError("lhs type not equal with rhs, lhs={}, rhs={}", + Types::to_string(which), + Types::to_string(rhs.which))); } switch (which) { @@ -601,9 +593,9 @@ class Field { case Types::Decimal256: return get() <=> rhs.get(); default: - LOG(FATAL) << "lhs type not equal with rhs, lhs=" << Types::to_string(which) - << ", rhs=" << Types::to_string(rhs.which); - break; + throw Exception(Status::FatalError("lhs type not equal with rhs, lhs={}, rhs={}", + Types::to_string(which), + Types::to_string(rhs.which))); } } @@ -675,8 +667,8 @@ class Field { f(field.template get()); return; default: - LOG(FATAL) << "type not supported, type=" << Types::to_string(field.which); - break; + throw Exception(Status::FatalError("type not supported, type={}", + Types::to_string(field.which))); } } diff --git a/be/src/vec/core/types.h b/be/src/vec/core/types.h index c817c6ab273f42..223dc13c8182bd 100644 --- a/be/src/vec/core/types.h +++ b/be/src/vec/core/types.h @@ -942,8 +942,7 @@ inline const char* getTypeName(TypeIndex idx) { return "Time"; } - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); + throw Exception(Status::FatalError("__builtin_unreachable")); } // NOLINTEND(readability-function-size) } // namespace vectorized diff --git a/be/src/vec/data_types/data_type_number_base.cpp b/be/src/vec/data_types/data_type_number_base.cpp index 1afed3d7d1a394..55330bd2797772 100644 --- a/be/src/vec/data_types/data_type_number_base.cpp +++ b/be/src/vec/data_types/data_type_number_base.cpp @@ -158,8 +158,7 @@ Field DataTypeNumberBase::get_field(const TExprNode& node) const { if constexpr (std::is_same_v, TypeId>) { return Float64(node.float_literal.value); } - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); + throw Exception(Status::FatalError("__builtin_unreachable")); } template diff --git a/be/src/vec/data_types/data_type_number_base.h b/be/src/vec/data_types/data_type_number_base.h index a73bd9951891a3..c560fdd01adac3 100644 --- a/be/src/vec/data_types/data_type_number_base.h +++ b/be/src/vec/data_types/data_type_number_base.h @@ -125,8 +125,7 @@ class DataTypeNumberBase : public IDataType { if constexpr (std::is_same_v, TypeId>) { return doris::FieldType::OLAP_FIELD_TYPE_DOUBLE; } - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); + throw Exception(Status::FatalError("__builtin_unreachable")); } Field get_default() const override; diff --git a/be/src/vec/data_types/serde/data_type_serde.h b/be/src/vec/data_types/serde/data_type_serde.h index 1a089bb73fe99c..122a700cf9b20b 100644 --- a/be/src/vec/data_types/serde/data_type_serde.h +++ b/be/src/vec/data_types/serde/data_type_serde.h @@ -395,8 +395,9 @@ inline static NullMap revert_null_map(const NullMap* null_bytemap, size_t start, inline void checkArrowStatus(const arrow::Status& status, const std::string& column, const std::string& format_name) { if (!status.ok()) { - LOG(FATAL) << "arrow serde with arrow: " << format_name << " with column : " << column - << " with error msg: " << status.ToString(); + throw Exception( + Status::FatalError("arrow serde with arrow: {} with column : {} with error msg: {}", + format_name, column, status.ToString())); } } diff --git a/be/src/vec/exec/format/parquet/bool_rle_decoder.cpp b/be/src/vec/exec/format/parquet/bool_rle_decoder.cpp index 17ce68e604e9b8..3f46a9c0073568 100644 --- a/be/src/vec/exec/format/parquet/bool_rle_decoder.cpp +++ b/be/src/vec/exec/format/parquet/bool_rle_decoder.cpp @@ -36,15 +36,16 @@ void BoolRLEDecoder::set_data(Slice* slice) { _offset = 0; _current_value_idx = 0; if (_num_bytes < 4) { - LOG(FATAL) << "Received invalid length : " + std::to_string(_num_bytes) + - " (corrupt data page?)"; + throw Exception(Status::FatalError("Received invalid length : {} (corrupt data page?)", + std::to_string(_num_bytes))); } // Load the first 4 bytes in little-endian, which indicates the length const uint8_t* data = reinterpret_cast(_data->data); uint32_t num_bytes = decode_fixed32_le(data); if (num_bytes > static_cast(_num_bytes - 4)) { - LOG(FATAL) << ("Received invalid number of bytes : " + std::to_string(num_bytes) + - " (corrupt data page?)"); + throw Exception( + Status::FatalError("Received invalid number of bytes : {} (corrupt data page?)", + std::to_string(_num_bytes))); } _num_bytes = num_bytes; auto decoder_data = data + 4; diff --git a/be/src/vec/exec/format/parquet/decoder.h b/be/src/vec/exec/format/parquet/decoder.h index 1654878af80a29..06e131b5b56049 100644 --- a/be/src/vec/exec/format/parquet/decoder.h +++ b/be/src/vec/exec/format/parquet/decoder.h @@ -79,8 +79,8 @@ class Decoder { } virtual MutableColumnPtr convert_dict_column_to_string_column(const ColumnInt32* dict_column) { - LOG(FATAL) << "Method convert_dict_column_to_string_column is not supported"; - __builtin_unreachable(); + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "Method convert_dict_column_to_string_column is not supported"); } protected: diff --git a/be/src/vec/exec/format/parquet/delta_bit_pack_decoder.h b/be/src/vec/exec/format/parquet/delta_bit_pack_decoder.h index 9497aa1cb1cdb5..dbe90acc985a4d 100644 --- a/be/src/vec/exec/format/parquet/delta_bit_pack_decoder.h +++ b/be/src/vec/exec/format/parquet/delta_bit_pack_decoder.h @@ -177,7 +177,8 @@ class DeltaBitPackDecoder final : public DeltaDecoder { _bit_reader.reset(new BitReader((const uint8_t*)slice->data, slice->size)); Status st = _init_header(); if (!st.ok()) { - LOG(FATAL) << "Fail to init delta encoding header for " << st.to_string(); + throw Exception(Status::FatalError("Fail to init delta encoding header for {}", + st.to_string())); } _data = slice; _offset = 0; @@ -189,7 +190,8 @@ class DeltaBitPackDecoder final : public DeltaDecoder { _bit_reader = std::move(bit_reader); Status st = _init_header(); if (!st.ok()) { - LOG(FATAL) << "Fail to init delta encoding header for " << st.to_string(); + throw Exception(Status::FatalError("Fail to init delta encoding header for {}", + st.to_string())); } } @@ -345,7 +347,7 @@ class DeltaByteArrayDecoder : public DeltaDecoder { int ret; Status st = _prefix_len_decoder.decode(_buffered_prefix_length.data(), num_prefix, &ret); if (!st.ok()) { - LOG(FATAL) << "Fail to decode delta prefix, status: " << st; + throw Exception(Status::FatalError("Fail to decode delta prefix, status: {}", st)); } DCHECK_EQ(ret, num_prefix); _prefix_len_offset = 0; @@ -527,7 +529,7 @@ void DeltaLengthByteArrayDecoder::_decode_lengths() { int ret; Status st = _len_decoder.decode(_buffered_length.data(), num_length, &ret); if (!st.ok()) { - LOG(FATAL) << "Fail to decode delta length, status: " << st; + throw Exception(Status::FatalError("Fail to decode delta length, status: {}", st)); } DCHECK_EQ(ret, num_length); _length_idx = 0; diff --git a/be/src/vec/exec/format/parquet/parquet_column_convert.h b/be/src/vec/exec/format/parquet/parquet_column_convert.h index cf6f8aa13fa1d1..d35a69ff59c625 100644 --- a/be/src/vec/exec/format/parquet/parquet_column_convert.h +++ b/be/src/vec/exec/format/parquet/parquet_column_convert.h @@ -423,8 +423,7 @@ class FixedSizeToDecimal : public PhysicalToLogicalConverter { switch (_type_length) { APPLY_FOR_DECIMALS() default: - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); + throw Exception(Status::FatalError("__builtin_unreachable")); } return Status::OK(); #undef APPLY_FOR_DECIMALS @@ -456,8 +455,7 @@ class FixedSizeToDecimal : public PhysicalToLogicalConverter { } else if constexpr (ScaleType == DecimalScaleParams::NO_SCALE) { // do nothing } else { - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); + throw Exception(Status::FatalError("__builtin_unreachable")); } auto& v = reinterpret_cast(data[start_idx + i]); v = (DecimalType)value; @@ -501,8 +499,7 @@ class StringToDecimal : public PhysicalToLogicalConverter { } else if constexpr (ScaleType == DecimalScaleParams::NO_SCALE) { // do nothing } else { - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); + throw Exception(Status::FatalError("__builtin_unreachable")); } } auto& v = reinterpret_cast(data[start_idx + i]); diff --git a/be/src/vec/exec/format/parquet/vparquet_column_reader.h b/be/src/vec/exec/format/parquet/vparquet_column_reader.h index 4c6e5b1eac9f60..a8062d2d9f9b7c 100644 --- a/be/src/vec/exec/format/parquet/vparquet_column_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_column_reader.h @@ -129,8 +129,8 @@ class ParquetColumnReader { } virtual MutableColumnPtr convert_dict_column_to_string_column(const ColumnInt32* dict_column) { - LOG(FATAL) << "Method convert_dict_column_to_string_column is not supported"; - __builtin_unreachable(); + throw Exception( + Status::FatalError("Method convert_dict_column_to_string_column is not supported")); } static Status create(io::FileReaderSPtr file, FieldSchema* field, diff --git a/be/src/vec/exec/jni_connector.cpp b/be/src/vec/exec/jni_connector.cpp index a87ccf987ac7af..11a58e81c98d89 100644 --- a/be/src/vec/exec/jni_connector.cpp +++ b/be/src/vec/exec/jni_connector.cpp @@ -185,8 +185,8 @@ Status JniConnector::close() { jthrowable exc = (env)->ExceptionOccurred(); if (exc != nullptr) { // Ensure successful resource release - LOG(FATAL) << "Failed to release jni resource: " - << JniUtil::GetJniExceptionMsg(env).to_string(); + throw Exception(Status::FatalError("Failed to release jni resource: {}", + JniUtil::GetJniExceptionMsg(env).to_string())); } } return Status::OK(); diff --git a/be/src/vec/exec/scan/split_source_connector.h b/be/src/vec/exec/scan/split_source_connector.h index 8f38cd4f17a18f..abe59562578aaf 100644 --- a/be/src/vec/exec/scan/split_source_connector.h +++ b/be/src/vec/exec/scan/split_source_connector.h @@ -117,7 +117,8 @@ class LocalSplitSourceConnector : public SplitSourceConnector { // for compatibility. return &_scan_ranges[0].scan_range.ext_scan_range.file_scan_range.params; } - LOG(FATAL) << "Unreachable, params is got by file_scan_range_params_map"; + throw Exception( + Status::FatalError("Unreachable, params is got by file_scan_range_params_map")); } }; @@ -160,7 +161,8 @@ class RemoteSplitSourceConnector : public SplitSourceConnector { int num_scan_ranges() override { return _num_splits; } TFileScanRangeParams* get_params() override { - LOG(FATAL) << "Unreachable, params is got by file_scan_range_params_map"; + throw Exception( + Status::FatalError("Unreachable, params is got by file_scan_range_params_map")); } }; diff --git a/be/src/vec/exprs/vexpr.h b/be/src/vec/exprs/vexpr.h index 953fbaa9c38c8d..91786337244013 100644 --- a/be/src/vec/exprs/vexpr.h +++ b/be/src/vec/exprs/vexpr.h @@ -237,18 +237,18 @@ class VExpr { // If this expr is a BloomPredicate, this method will return a BloomFilterFunc virtual std::shared_ptr get_bloom_filter_func() const { - LOG(FATAL) << "Method 'get_bloom_filter_func()' is not supported in expression: " - << this->debug_string(); - return nullptr; + throw Exception(Status::FatalError( + "Method 'get_bloom_filter_func()' is not supported in expression: {}", + this->debug_string())); } virtual std::shared_ptr get_set_func() const { return nullptr; } // If this expr is a BitmapPredicate, this method will return a BitmapFilterFunc virtual std::shared_ptr get_bitmap_filter_func() const { - LOG(FATAL) << "Method 'get_bitmap_filter_func()' is not supported in expression: " - << this->debug_string(); - return nullptr; + throw Exception(Status::FatalError( + "Method 'get_bitmap_filter_func()' is not supported in expression: {}", + this->debug_string())); } // fast_execute can direct copy expr filter result which build by apply index in segment_iterator diff --git a/be/src/vec/functions/array/function_array_apply.cpp b/be/src/vec/functions/array/function_array_apply.cpp index 75425389dd975c..4161441080aac0 100644 --- a/be/src/vec/functions/array/function_array_apply.cpp +++ b/be/src/vec/functions/array/function_array_apply.cpp @@ -24,6 +24,7 @@ #include #include +#include "common/exception.h" #include "common/status.h" #include "runtime/thread_context.h" #include "vec/aggregate_functions/aggregate_function.h" @@ -130,8 +131,7 @@ class FunctionArrayApply : public IFunction { if constexpr (op == ApplyOp::GE) { return data >= comp; } - LOG(FATAL) << "__builtin_unreachable"; - __builtin_unreachable(); + throw Exception(Status::FatalError("__builtin_unreachable")); } // need exception safety diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h index 48619ff85f83c8..af9e9d19267073 100644 --- a/be/src/vec/functions/function_cast.h +++ b/be/src/vec/functions/function_cast.h @@ -665,7 +665,14 @@ struct ConvertImplNumberToJsonb { } else if constexpr (std::is_same_v) { writer.writeDouble(data[i]); } else { - LOG(FATAL) << "unsupported type "; + static_assert(std::is_same_v || + std::is_same_v || + std::is_same_v || + std::is_same_v || + std::is_same_v || + std::is_same_v || + std::is_same_v, + "unsupported type"); __builtin_unreachable(); } column_string->insert_data(writer.getOutput()->getBuffer(), @@ -950,8 +957,7 @@ struct ConvertImplFromJsonb { res[i] = 0; } } else { - LOG(FATAL) << "unsupported type "; - __builtin_unreachable(); + throw Exception(Status::FatalError("unsupported type")); } } diff --git a/be/src/vec/json/simd_json_parser.h b/be/src/vec/json/simd_json_parser.h index 5189e93563cc52..79924a12a3a4ff 100644 --- a/be/src/vec/json/simd_json_parser.h +++ b/be/src/vec/json/simd_json_parser.h @@ -208,8 +208,8 @@ class SimdJSONParser { /// Optional: Allocates memory to parse JSON documents faster. void reserve(size_t max_size) { if (parser.allocate(max_size) != simdjson::error_code::SUCCESS) { - LOG(FATAL) << "Couldn't allocate " + std::to_string(max_size) + - " bytes when parsing JSON"; + throw Exception(Status::FatalError("Couldn't allocate {} bytes when parsing JSON", + std::to_string(max_size))); } } diff --git a/be/src/vec/olap/olap_data_convertor.h b/be/src/vec/olap/olap_data_convertor.h index 3473d9d26b5205..75aff7dfec34cd 100644 --- a/be/src/vec/olap/olap_data_convertor.h +++ b/be/src/vec/olap/olap_data_convertor.h @@ -455,7 +455,8 @@ class OlapBlockDataConvertor { const void* get_data() const override { return _results.data(); }; const void* get_data_at(size_t offset) const override { - LOG(FATAL) << "now not support get_data_at for OlapColumnDataConvertorArray"; + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "now not support get_data_at for OlapColumnDataConvertorArray"); __builtin_unreachable(); }; Status convert_to_olap() override; @@ -484,7 +485,8 @@ class OlapBlockDataConvertor { Status convert_to_olap() override; const void* get_data() const override { return _results.data(); }; const void* get_data_at(size_t offset) const override { - LOG(FATAL) << "now not support get_data_at for OlapColumnDataConvertorMap"; + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "now not support get_data_at for OlapColumnDataConvertorMap"); __builtin_unreachable(); }; diff --git a/be/src/vec/runtime/vdatetime_value.cpp b/be/src/vec/runtime/vdatetime_value.cpp index 86c50f0936f30d..026648319d4be4 100644 --- a/be/src/vec/runtime/vdatetime_value.cpp +++ b/be/src/vec/runtime/vdatetime_value.cpp @@ -3434,8 +3434,7 @@ void DateV2Value::unchecked_set_time(uint8_t hour, uint8_t minute, uint16_t s date_v2_value_.second_ = second; date_v2_value_.microsecond_ = microsecond; } else { - LOG(FATAL) << "Invalid operation 'set_time' for date!"; - __builtin_unreachable(); + throw Exception(Status::FatalError("Invalid operation 'set_time' for date!")); } } @@ -3444,8 +3443,7 @@ void DateV2Value::set_microsecond(uint64_t microsecond) { if constexpr (is_datetime) { date_v2_value_.microsecond_ = microsecond; } else { - LOG(FATAL) << "Invalid operation 'set_microsecond' for date!"; - __builtin_unreachable(); + throw Exception(Status::FatalError("Invalid operation 'set_microsecond' for date!")); } } diff --git a/be/test/util/threadpool_test.cpp b/be/test/util/threadpool_test.cpp index 3859639539dbb7..d331bd0d2ac25d 100644 --- a/be/test/util/threadpool_test.cpp +++ b/be/test/util/threadpool_test.cpp @@ -42,6 +42,7 @@ #include "common/logging.h" #include "common/status.h" +#include "gtest/gtest.h" #include "gtest/gtest_pred_impl.h" #include "gutil/strings/substitute.h" #include "util/barrier.h" From 6c57c3c70dc669ba386782bfa85baa096129852c Mon Sep 17 00:00:00 2001 From: zclllyybb Date: Thu, 19 Dec 2024 22:00:18 +0800 Subject: [PATCH 12/14] [fix](ub) Dont throw in noexcept function (#45672) --- be/src/pipeline/exec/exchange_sink_buffer.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/be/src/pipeline/exec/exchange_sink_buffer.h b/be/src/pipeline/exec/exchange_sink_buffer.h index a381c5aff144f3..458c7c3f66e3ee 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.h +++ b/be/src/pipeline/exec/exchange_sink_buffer.h @@ -155,9 +155,10 @@ class ExchangeSendCallback : public ::doris::DummyBrpcCallback { start_rpc_time); } } catch (const std::exception& exp) { - throw Exception(Status::FatalError("brpc callback error: {}", exp.what())); + LOG(FATAL) << "brpc callback error: " << exp.what(); } catch (...) { - throw Exception(Status::FatalError("brpc callback error.")); + LOG(FATAL) << "brpc callback error."; + __builtin_unreachable(); } } int64_t start_rpc_time; From b5249a9e47149bd5ff33c25933a9ae882c6e6b45 Mon Sep 17 00:00:00 2001 From: zhengyu Date: Fri, 20 Dec 2024 00:08:33 +0800 Subject: [PATCH 13/14] [opt](cloud) reduce cache hotspot table write amplification (#45557) 1. batch insert cloud_cache_hotspot in FE 2. enlarge polling interval in FE 3. shrink bucket num to 1 for cloud_cache_hotspot table 4. ignore stable statistics only catch the dynamic in BE Signed-off-by: zhengyu --- be/src/cloud/cloud_tablet_hotspot.cpp | 94 +++++++++++-------- be/src/cloud/cloud_tablet_hotspot.h | 19 ++++ .../java/org/apache/doris/common/Config.java | 4 +- .../doris/cloud/CacheHotspotManager.java | 2 +- .../doris/cloud/CacheHotspotManagerUtils.java | 3 +- 5 files changed, 80 insertions(+), 42 deletions(-) diff --git a/be/src/cloud/cloud_tablet_hotspot.cpp b/be/src/cloud/cloud_tablet_hotspot.cpp index dd197268646fbc..6391a2dc5c4928 100644 --- a/be/src/cloud/cloud_tablet_hotspot.cpp +++ b/be/src/cloud/cloud_tablet_hotspot.cpp @@ -57,18 +57,55 @@ TabletHotspot::~TabletHotspot() { } } -struct MapKeyHash { - int64_t operator()(const std::pair& key) const { - return std::hash {}(key.first) + std::hash {}(key.second); +void get_return_partitions( + const std::unordered_map, MapKeyHash>& + hot_partition, + const std::unordered_map, MapKeyHash>& + last_hot_partition, + std::vector* hot_tables, int& return_partitions, int N) { + for (const auto& [key, partition_to_value] : hot_partition) { + THotTableMessage msg; + msg.table_id = key.first; + msg.index_id = key.second; + for (const auto& [partition_id, value] : partition_to_value) { + if (return_partitions > N) { + return; + } + auto last_value_iter = last_hot_partition.find(key); + if (last_value_iter != last_hot_partition.end()) { + auto last_partition_iter = last_value_iter->second.find(partition_id); + if (last_partition_iter != last_value_iter->second.end()) { + const auto& last_value = last_partition_iter->second; + if (std::abs(static_cast(value.qpd) - + static_cast(last_value.qpd)) < 5 && + std::abs(static_cast(value.qpw) - + static_cast(last_value.qpw)) < 10 && + std::abs(static_cast(value.last_access_time) - + static_cast(last_value.last_access_time)) < 60) { + LOG(INFO) << "skip partition_id=" << partition_id << " qpd=" << value.qpd + << " qpw=" << value.qpw + << " last_access_time=" << value.last_access_time + << " last_qpd=" << last_value.qpd + << " last_qpw=" << last_value.qpw + << " last_access_time=" << last_value.last_access_time; + continue; + } + } + } + THotPartition hot_partition; + hot_partition.__set_partition_id(partition_id); + hot_partition.__set_query_per_day(value.qpd); + hot_partition.__set_query_per_week(value.qpw); + hot_partition.__set_last_access_time(value.last_access_time); + msg.hot_partitions.push_back(hot_partition); + return_partitions++; + } + msg.__isset.hot_partitions = !msg.hot_partitions.empty(); + hot_tables->push_back(std::move(msg)); } -}; -struct TabletHotspotMapValue { - uint64_t qpd = 0; // query per day - uint64_t qpw = 0; // query per week - int64_t last_access_time; -}; - -using TabletHotspotMapKey = std::pair; +} void TabletHotspot::get_top_n_hot_partition(std::vector* hot_tables) { // map, map> for day @@ -108,33 +145,14 @@ void TabletHotspot::get_top_n_hot_partition(std::vector* hot_t }); constexpr int N = 50; int return_partitions = 0; - auto get_return_partitions = - [=, &return_partitions]( - const std::unordered_map, - MapKeyHash>& hot_partition) { - for (const auto& [key, partition_to_value] : hot_partition) { - THotTableMessage msg; - msg.table_id = key.first; - msg.index_id = key.second; - for (const auto& [partition_id, value] : partition_to_value) { - if (return_partitions > N) { - return; - } - THotPartition hot_partition; - hot_partition.__set_partition_id(partition_id); - hot_partition.__set_query_per_day(value.qpd); - hot_partition.__set_query_per_week(value.qpw); - hot_partition.__set_last_access_time(value.last_access_time); - msg.hot_partitions.push_back(hot_partition); - return_partitions++; - } - msg.__isset.hot_partitions = !msg.hot_partitions.empty(); - hot_tables->push_back(std::move(msg)); - } - }; - get_return_partitions(day_hot_partitions); - get_return_partitions(week_hot_partitions); + + get_return_partitions(day_hot_partitions, _last_day_hot_partitions, hot_tables, + return_partitions, N); + get_return_partitions(week_hot_partitions, _last_week_hot_partitions, hot_tables, + return_partitions, N); + + _last_day_hot_partitions = std::move(day_hot_partitions); + _last_week_hot_partitions = std::move(week_hot_partitions); } void HotspotCounter::make_dot_point() { diff --git a/be/src/cloud/cloud_tablet_hotspot.h b/be/src/cloud/cloud_tablet_hotspot.h index af98f99a558b9b..0be1c085a6c990 100644 --- a/be/src/cloud/cloud_tablet_hotspot.h +++ b/be/src/cloud/cloud_tablet_hotspot.h @@ -49,6 +49,19 @@ struct HotspotCounter { }; using HotspotCounterPtr = std::shared_ptr; +using TabletHotspotMapKey = std::pair; + +struct TabletHotspotMapValue { + uint64_t qpd = 0; // query per day + uint64_t qpw = 0; // query per week + int64_t last_access_time; +}; + +struct MapKeyHash { + int64_t operator()(const std::pair& key) const { + return std::hash {}(key.first) + std::hash {}(key.second); + } +}; class TabletHotspot { public: @@ -71,6 +84,12 @@ class TabletHotspot { bool _closed {false}; std::mutex _mtx; std::condition_variable _cond; + std::unordered_map, + MapKeyHash> + _last_day_hot_partitions; + std::unordered_map, + MapKeyHash> + _last_week_hot_partitions; }; } // namespace doris 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 c601a492162958..935300dee6f2fd 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 @@ -3190,11 +3190,11 @@ public static int metaServiceRpcRetryTimes() { public static boolean enable_fetch_cluster_cache_hotspot = true; @ConfField(mutable = true) - public static long fetch_cluster_cache_hotspot_interval_ms = 600000; + public static long fetch_cluster_cache_hotspot_interval_ms = 3600000; // to control the max num of values inserted into cache hotspot internal table // insert into cache table when the size of batch values reaches this limit @ConfField(mutable = true) - public static long batch_insert_cluster_cache_hotspot_num = 1000; + public static long batch_insert_cluster_cache_hotspot_num = 5000; /** * intervals between be status checks for CloudUpgradeMgr diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/CacheHotspotManager.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/CacheHotspotManager.java index 0b83baa94d6d4a..f4c7392eb75c63 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/CacheHotspotManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/CacheHotspotManager.java @@ -159,9 +159,9 @@ public void runAfterCatalogReady() { } }); } - triggerBatchInsert(); }); }); + triggerBatchInsert(); idToTable.clear(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/CacheHotspotManagerUtils.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/CacheHotspotManagerUtils.java index 20de42f8cdc25a..72710debaefd9a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/CacheHotspotManagerUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/CacheHotspotManagerUtils.java @@ -70,9 +70,10 @@ public class CacheHotspotManagerUtils { + " last_access_time DATETIMEV2)\n" + " UNIQUE KEY(cluster_id, backend_id, table_id, index_id, partition_id, insert_day)\n" + " PARTITION BY RANGE (insert_day) ()\n" - + " DISTRIBUTED BY HASH (cluster_id)\n" + + " DISTRIBUTED BY HASH (cluster_id) BUCKETS 1\n" + " PROPERTIES (\n" + " \"dynamic_partition.enable\" = \"true\",\n" + + " \"dynamic_partition.buckets\" = \"1\",\n" + " \"dynamic_partition.time_unit\" = \"DAY\",\n" + " \"dynamic_partition.start\" = \"-7\",\n" + " \"dynamic_partition.end\" = \"3\",\n" From 1bd8003a1a126e8ea033b9452499da404b2ec0be Mon Sep 17 00:00:00 2001 From: Luwei Date: Fri, 20 Dec 2024 00:28:51 +0800 Subject: [PATCH 14/14] [Enhancement](compaction) enable the compaction producer to generate multiple compaction tasks in a single run (#45411) --- be/src/common/config.cpp | 2 + be/src/common/config.h | 2 + be/src/olap/tablet_manager.cpp | 52 ++++++++++++++++++--- be/test/olap/tablet_mgr_test.cpp | 79 +++++++++++++++++++++++++++++++- 4 files changed, 127 insertions(+), 8 deletions(-) diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 95a3e61fb5517a..083b9f06c9491d 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -1404,6 +1404,8 @@ DEFINE_Bool(enable_table_size_correctness_check, "false"); DEFINE_Bool(force_regenerate_rowsetid_on_start_error, "false"); DEFINE_mBool(enable_sleep_between_delete_cumu_compaction, "false"); +DEFINE_mInt32(compaction_num_per_round, "1"); + // clang-format off #ifdef BE_TEST // test s3 diff --git a/be/src/common/config.h b/be/src/common/config.h index f8a9c3f7480b33..1e3d57ff763417 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1490,6 +1490,8 @@ DECLARE_Bool(enable_table_size_correctness_check); // Enable sleep 5s between delete cumulative compaction. DECLARE_mBool(enable_sleep_between_delete_cumu_compaction); +DECLARE_mInt32(compaction_num_per_round); + #ifdef BE_TEST // test s3 DECLARE_String(test_s3_resource); diff --git a/be/src/olap/tablet_manager.cpp b/be/src/olap/tablet_manager.cpp index 33fee7ca350900..44c26d160eb8bc 100644 --- a/be/src/olap/tablet_manager.cpp +++ b/be/src/olap/tablet_manager.cpp @@ -719,6 +719,11 @@ void TabletManager::get_tablet_stat(TTabletStatResult* result) { result->__set_tablet_stat_list(*local_cache); } +struct TabletScore { + TabletSharedPtr tablet_ptr; + int score; +}; + std::vector TabletManager::find_best_tablets_to_compaction( CompactionType compaction_type, DataDir* data_dir, const std::unordered_set& tablet_submitted_compaction, uint32_t* score, @@ -732,6 +737,9 @@ std::vector TabletManager::find_best_tablets_to_compaction( uint32_t single_compact_highest_score = 0; TabletSharedPtr best_tablet; TabletSharedPtr best_single_compact_tablet; + auto cmp = [](TabletScore left, TabletScore right) { return left.score > right.score; }; + std::priority_queue, decltype(cmp)> top_tablets(cmp); + auto handler = [&](const TabletSharedPtr& tablet_ptr) { if (tablet_ptr->tablet_meta()->tablet_schema()->disable_auto_compaction()) { LOG_EVERY_N(INFO, 500) << "Tablet " << tablet_ptr->tablet_id() @@ -798,13 +806,33 @@ std::vector TabletManager::find_best_tablets_to_compaction( } } - // tablet should do cumu or base compaction - if (current_compaction_score > highest_score && !tablet_ptr->should_fetch_from_peer()) { - bool ret = tablet_ptr->suitable_for_compaction(compaction_type, - cumulative_compaction_policy); - if (ret) { - highest_score = current_compaction_score; - best_tablet = tablet_ptr; + if (config::compaction_num_per_round > 1 && !tablet_ptr->should_fetch_from_peer()) { + TabletScore ts; + ts.score = current_compaction_score; + ts.tablet_ptr = tablet_ptr; + if ((top_tablets.size() >= config::compaction_num_per_round && + current_compaction_score > top_tablets.top().score) || + top_tablets.size() < config::compaction_num_per_round) { + bool ret = tablet_ptr->suitable_for_compaction(compaction_type, + cumulative_compaction_policy); + if (ret) { + top_tablets.push(ts); + if (top_tablets.size() > config::compaction_num_per_round) { + top_tablets.pop(); + } + if (current_compaction_score > highest_score) { + highest_score = current_compaction_score; + } + } + } + } else { + if (current_compaction_score > highest_score && !tablet_ptr->should_fetch_from_peer()) { + bool ret = tablet_ptr->suitable_for_compaction(compaction_type, + cumulative_compaction_policy); + if (ret) { + highest_score = current_compaction_score; + best_tablet = tablet_ptr; + } } } }; @@ -820,6 +848,16 @@ std::vector TabletManager::find_best_tablets_to_compaction( picked_tablet.emplace_back(std::move(best_tablet)); } + std::vector reverse_top_tablets; + while (!top_tablets.empty()) { + reverse_top_tablets.emplace_back(top_tablets.top().tablet_ptr); + top_tablets.pop(); + } + + for (auto it = reverse_top_tablets.rbegin(); it != reverse_top_tablets.rend(); ++it) { + picked_tablet.emplace_back(*it); + } + // pick single compaction tablet needs the highest score if (best_single_compact_tablet != nullptr && single_compact_highest_score >= highest_score) { VLOG_CRITICAL << "Found the best tablet for single compaction. " diff --git a/be/test/olap/tablet_mgr_test.cpp b/be/test/olap/tablet_mgr_test.cpp index 1bcdcdf45c6906..a2551543405328 100644 --- a/be/test/olap/tablet_mgr_test.cpp +++ b/be/test/olap/tablet_mgr_test.cpp @@ -83,6 +83,7 @@ class TabletMgrTest : public testing::Test { SAFE_DELETE(_data_dir); EXPECT_TRUE(io::global_local_filesystem()->delete_directory(_engine_data_path).ok()); _tablet_mgr = nullptr; + config::compaction_num_per_round = 1; } std::unique_ptr k_engine; @@ -463,11 +464,87 @@ TEST_F(TabletMgrTest, FindTabletWithCompact) { ASSERT_EQ(score, 25); // drop all tablets - for (int64_t id = 1; id <= 20; ++id) { + for (int64_t id = 1; id <= 21; ++id) { Status drop_st = _tablet_mgr->drop_tablet(id, id * 10, false); ASSERT_TRUE(drop_st.ok()) << drop_st; } + { + config::compaction_num_per_round = 10; + for (int64_t i = 1; i <= 100; ++i) { + create_tablet(10000 + i, false, i); + } + + compact_tablets = _tablet_mgr->find_best_tablets_to_compaction( + CompactionType::CUMULATIVE_COMPACTION, _data_dir, cumu_set, &score, + cumulative_compaction_policies); + ASSERT_EQ(compact_tablets.size(), 10); + int index = 0; + for (auto t : compact_tablets) { + ASSERT_EQ(t->tablet_id(), 10100 - index); + ASSERT_EQ(t->calc_compaction_score(), 100 - index); + index++; + } + config::compaction_num_per_round = 1; + // drop all tablets + for (int64_t id = 10001; id <= 10100; ++id) { + Status drop_st = _tablet_mgr->drop_tablet(id, id * 10, false); + ASSERT_TRUE(drop_st.ok()) << drop_st; + } + } + + { + config::compaction_num_per_round = 10; + for (int64_t i = 1; i <= 100; ++i) { + create_tablet(20000 + i, false, i); + } + create_tablet(20102, true, 200); + + compact_tablets = _tablet_mgr->find_best_tablets_to_compaction( + CompactionType::CUMULATIVE_COMPACTION, _data_dir, cumu_set, &score, + cumulative_compaction_policies); + ASSERT_EQ(compact_tablets.size(), 11); + for (int i = 0; i < 10; ++i) { + ASSERT_EQ(compact_tablets[i]->tablet_id(), 20100 - i); + ASSERT_EQ(compact_tablets[i]->calc_compaction_score(), 100 - i); + } + ASSERT_EQ(compact_tablets[10]->tablet_id(), 20102); + ASSERT_EQ(compact_tablets[10]->calc_compaction_score(), 200); + + config::compaction_num_per_round = 1; + // drop all tablets + for (int64_t id = 20001; id <= 20100; ++id) { + Status drop_st = _tablet_mgr->drop_tablet(id, id * 10, false); + ASSERT_TRUE(drop_st.ok()) << drop_st; + } + + Status drop_st = _tablet_mgr->drop_tablet(20102, 20102 * 10, false); + ASSERT_TRUE(drop_st.ok()) << drop_st; + } + + { + config::compaction_num_per_round = 10; + for (int64_t i = 1; i <= 5; ++i) { + create_tablet(30000 + i, false, i + 5); + } + + compact_tablets = _tablet_mgr->find_best_tablets_to_compaction( + CompactionType::CUMULATIVE_COMPACTION, _data_dir, cumu_set, &score, + cumulative_compaction_policies); + ASSERT_EQ(compact_tablets.size(), 5); + for (int i = 0; i < 5; ++i) { + ASSERT_EQ(compact_tablets[i]->tablet_id(), 30000 + 5 - i); + ASSERT_EQ(compact_tablets[i]->calc_compaction_score(), 10 - i); + } + + config::compaction_num_per_round = 1; + // drop all tablets + for (int64_t id = 30001; id <= 30005; ++id) { + Status drop_st = _tablet_mgr->drop_tablet(id, id * 10, false); + ASSERT_TRUE(drop_st.ok()) << drop_st; + } + } + Status trash_st = _tablet_mgr->start_trash_sweep(); ASSERT_TRUE(trash_st.ok()) << trash_st; }