This is an automated email from the ASF dual-hosted git repository.

yiguolei pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 565de9d7cddb0e6fd84f99e21b121b10162eb7a3
Author: deardeng <565620...@qq.com>
AuthorDate: Sun Apr 14 09:44:25 2024 +0800

    [fix](create table) Fix create table exception without cleaning the e… 
(#33574)
---
 .../apache/doris/datasource/InternalCatalog.java   |  24 +++-
 .../test_create_table_exception.groovy             | 127 +++++++++++++++++++++
 2 files changed, 147 insertions(+), 4 deletions(-)

diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java 
b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
index 9295ab56a18..b94c8b7d076 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
@@ -126,6 +126,7 @@ import org.apache.doris.common.Pair;
 import org.apache.doris.common.UserException;
 import org.apache.doris.common.io.CountingDataOutputStream;
 import org.apache.doris.common.util.DbUtil;
+import org.apache.doris.common.util.DebugPointUtil;
 import org.apache.doris.common.util.DynamicPartitionUtil;
 import org.apache.doris.common.util.IdGeneratorUtil;
 import org.apache.doris.common.util.MetaLockUtils;
@@ -959,15 +960,14 @@ public class InternalCatalog implements 
CatalogIf<Database> {
         return true;
     }
 
-    public void replayDropTable(Database db, long tableId, boolean isForceDrop,
-            Long recycleTime) throws MetaNotFoundException {
+    public void dropTable(Database db, long tableId, boolean isForceDrop,
+                          Long recycleTime) throws MetaNotFoundException {
         Table table = db.getTableOrMetaException(tableId);
         db.writeLock();
         table.writeLock();
         try {
             unprotectDropTable(db, table, isForceDrop, true, recycleTime);
-            
Env.getCurrentEnv().getQueryStats().clear(Env.getCurrentInternalCatalog().getId(),
 db.getId(),
-                    tableId);
+            
Env.getCurrentEnv().getQueryStats().clear(Env.getCurrentInternalCatalog().getId(),
 db.getId(), tableId);
             
Env.getCurrentEnv().getAnalysisManager().removeTableStats(table.getId());
         } finally {
             table.writeUnlock();
@@ -975,6 +975,11 @@ public class InternalCatalog implements 
CatalogIf<Database> {
         }
     }
 
+    public void replayDropTable(Database db, long tableId, boolean isForceDrop,
+            Long recycleTime) throws MetaNotFoundException {
+        dropTable(db, tableId, isForceDrop, recycleTime);
+    }
+
     public void replayEraseTable(long tableId) {
         Env.getCurrentRecycleBin().replayEraseTable(tableId);
     }
@@ -2710,6 +2715,11 @@ public class InternalCatalog implements 
CatalogIf<Database> {
             if (!result.first) {
                 
ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName);
             }
+            if (DebugPointUtil.isEnable("FE.createOlapTable.exception")) {
+                LOG.info("debug point FE.createOlapTable.exception, throw e");
+                // not commit, not log edit
+                throw new DdlException("debug point 
FE.createOlapTable.exception");
+            }
 
             if (result.second) {
                 if (Env.getCurrentColocateIndex().isColocateTable(tableId)) {
@@ -2741,6 +2751,7 @@ public class InternalCatalog implements 
CatalogIf<Database> {
                                 TimeUtils.getCurrentFormatTime());
             }
         } catch (DdlException e) {
+            LOG.warn("create table failed {} - {}", tabletIdSet, 
e.getMessage());
             for (Long tabletId : tabletIdSet) {
                 Env.getCurrentInvertedIndex().deleteTablet(tabletId);
             }
@@ -2748,6 +2759,11 @@ public class InternalCatalog implements 
CatalogIf<Database> {
             if (Env.getCurrentColocateIndex().isColocateTable(tableId)) {
                 Env.getCurrentColocateIndex().removeTable(tableId);
             }
+            try {
+                dropTable(db, tableId, true, 0L);
+            } catch (Exception ex) {
+                LOG.warn("drop table", ex);
+            }
 
             throw e;
         }
diff --git 
a/regression-test/suites/partition_p0/test_create_table_exception.groovy 
b/regression-test/suites/partition_p0/test_create_table_exception.groovy
new file mode 100644
index 00000000000..49cadcd3af4
--- /dev/null
+++ b/regression-test/suites/partition_p0/test_create_table_exception.groovy
@@ -0,0 +1,127 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+import org.apache.doris.regression.suite.ClusterOptions
+import org.apache.doris.regression.util.NodeType
+import org.apache.doris.regression.suite.SuiteCluster
+
+suite("test_create_table_exception") {
+    def options = new ClusterOptions()
+    options.enableDebugPoints()
+    options.setFeNum(3)
+    options.feConfigs.add('max_dynamic_partition_num=2000')
+
+    docker(options) {
+        sleep 2000
+        def table1 = "normal_table"
+        def table2 = "range_table"
+        def table3 = "dynamic_partition_table"
+        try {
+            
GetDebugPoint().enableDebugPointForAllFEs('FE.createOlapTable.exception', null)
+            def createTable = { ->
+                try_sql """
+                    CREATE TABLE $table1 (
+                        `k1` int(11) NULL,
+                        `k2` int(11) NULL
+                    )
+                    DUPLICATE KEY(`k1`, `k2`)
+                    COMMENT 'OLAP'
+                    DISTRIBUTED BY HASH(`k1`) BUCKETS 10
+                    PROPERTIES (
+                    "replication_num"="3"
+                    );
+                """
+
+                try_sql """
+                    CREATE TABLE IF NOT EXISTS $table2 (
+                        lo_orderdate int(11) NOT NULL COMMENT "",
+                        lo_orderkey bigint(20) NOT NULL COMMENT "",
+                        lo_linenumber bigint(20) NOT NULL COMMENT "",
+                        lo_custkey int(11) NOT NULL COMMENT "",
+                        lo_partkey int(11) NOT NULL COMMENT "",
+                        lo_suppkey int(11) NOT NULL COMMENT "",
+                        lo_orderpriority varchar(64) NOT NULL COMMENT "",
+                        lo_shippriority int(11) NOT NULL COMMENT "",
+                        lo_quantity bigint(20) NOT NULL COMMENT "",
+                        lo_extendedprice bigint(20) NOT NULL COMMENT "",
+                        lo_ordtotalprice bigint(20) NOT NULL COMMENT "",
+                        lo_discount bigint(20) NOT NULL COMMENT "",
+                        lo_revenue bigint(20) NOT NULL COMMENT "",
+                        lo_supplycost bigint(20) NOT NULL COMMENT "",
+                        lo_tax bigint(20) NOT NULL COMMENT "",
+                        lo_commitdate bigint(20) NOT NULL COMMENT "",
+                        lo_shipmode varchar(64) NOT NULL COMMENT "" )
+                    ENGINE=OLAP
+                    UNIQUE KEY(lo_orderdate, lo_orderkey, lo_linenumber)
+                    COMMENT "OLAP"
+                    PARTITION BY RANGE(lo_orderdate) (
+                    PARTITION p1992 VALUES [("-2147483648"), ("19930101")),
+                    PARTITION p1993 VALUES [("19930101"), ("19940101")),
+                    PARTITION p1994 VALUES [("19940101"), ("19950101")),
+                    PARTITION p1995 VALUES [("19950101"), ("19960101")),
+                    PARTITION p1996 VALUES [("19960101"), ("19970101")),
+                    PARTITION p1997 VALUES [("19970101"), ("19980101")),
+                    PARTITION p1998 VALUES [("19980101"), ("19990101")))
+                    DISTRIBUTED BY HASH(lo_orderkey) BUCKETS 48;
+                """
+
+                try_sql """
+                    CREATE TABLE $table3 (
+                        time date,
+                        key1 int,
+                        key2 int,
+                        value1 int,
+                        value2 int
+                    ) ENGINE = OLAP UNIQUE KEY(
+                        `time`,
+                        `key1`,
+                        `key2`
+                    ) COMMENT 'OLAP' PARTITION BY RANGE(`time`)()
+                    DISTRIBUTED BY HASH(`key1`) BUCKETS 6 PROPERTIES (
+                    "file_cache_ttl_seconds" = "0",
+                    "bloom_filter_columns" = "time",
+                    "dynamic_partition.enable" = "true",
+                    "dynamic_partition.time_unit" = "DAY",
+                    "dynamic_partition.time_zone" = "Asia/Shanghai",
+                    "dynamic_partition.start" = "-730",
+                    "dynamic_partition.end" = "3",
+                    "dynamic_partition.prefix" = "p",
+                    "dynamic_partition.buckets" = "2",
+                    "dynamic_partition.create_history_partition" = "true",
+                    "dynamic_partition.history_partition_num" = "-1",
+                    "dynamic_partition.hot_partition_num" = "0",
+                    "dynamic_partition.reserved_history_periods" = "NULL",
+                    "enable_unique_key_merge_on_write" = "true",
+                    "light_schema_change" = "true"
+                    );
+                """
+            }
+            createTable()
+            def result = sql """show tables;"""
+            assertEquals(result.size(), 0)
+            
GetDebugPoint().disableDebugPointForAllFEs('FE.createOlapTable.exception')
+            createTable()
+            result = sql """show tables;"""
+            log.info(result.toString())
+            assertEquals(result.size(), 3)
+        } finally {
+            
GetDebugPoint().disableDebugPointForAllFEs('FE.createOlapTable.exception')
+            sql """drop table if exists ${table1}"""
+            sql """drop table if exists ${table2}"""
+            sql """drop table if exists ${table3}"""
+        }
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org
For additional commands, e-mail: commits-h...@doris.apache.org

Reply via email to