diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index 6f40dfe1d58563..5eb3530795a747 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -235,7 +235,7 @@ terminal String KW_ADD, KW_ADMIN, KW_AFTER, KW_AGGREGATE, KW_ALL, KW_ALTER, KW_A KW_DELETE, KW_DISTINCT, KW_DISTINCTPC, KW_DISTINCTPCSA, KW_DISTRIBUTED, KW_DISTRIBUTION, KW_DYNAMIC, KW_BUCKETS, KW_DIV, KW_DOUBLE, KW_DROP, KW_DROPP, KW_DUPLICATE, KW_ELSE, KW_END, KW_ENGINE, KW_ENGINES, KW_ENTER, KW_ERRORS, KW_EVENTS, KW_EXCEPT, KW_EXISTS, KW_EXPORT, KW_EXTERNAL, KW_EXTRACT, - KW_FALSE, KW_FOLLOWER, KW_FOLLOWING, KW_FREE, KW_FROM, KW_FILE, KW_FIRST, KW_FLOAT, KW_FOR, KW_FORMAT, KW_FRONTEND, KW_FRONTENDS, KW_FULL, KW_FUNCTION, KW_FUNCTIONS, + KW_FALSE, KW_FOLLOWER, KW_FOLLOWING, KW_FREE, KW_FROM, KW_FILE, KW_FIRST, KW_FLOAT, KW_FOR, KW_FORCE, KW_FORMAT, KW_FRONTEND, KW_FRONTENDS, KW_FULL, KW_FUNCTION, KW_FUNCTIONS, KW_GLOBAL, KW_GRANT, KW_GRANTS, KW_GROUP, KW_GROUPING, KW_HASH, KW_HAVING, KW_HELP,KW_HLL, KW_HLL_UNION, KW_HOUR, KW_HUB, KW_IDENTIFIED, KW_IF, KW_IN, KW_INDEX, KW_INDEXES, KW_INFILE, KW_INSTALL, @@ -463,6 +463,7 @@ nonterminal String opt_db, procedure_or_function, opt_comment, opt_engine; nonterminal ColumnDef.DefaultValue opt_default_value; nonterminal Boolean opt_if_exists, opt_if_not_exists; nonterminal Boolean opt_external; +nonterminal Boolean opt_force; nonterminal IndexDef.IndexType opt_index_type; nonterminal ShowAlterStmt.AlterType opt_alter_type; @@ -877,9 +878,9 @@ alter_table_clause ::= {: RESULT = new AddPartitionClause(desc, distribution, properties, isTempPartition); :} - | KW_DROP opt_tmp:isTempPartition KW_PARTITION opt_if_exists:ifExists ident:partitionName + | KW_DROP opt_tmp:isTempPartition KW_PARTITION opt_force:force opt_if_exists:ifExists ident:partitionName {: - RESULT = new DropPartitionClause(ifExists, partitionName, isTempPartition); + RESULT = new DropPartitionClause(ifExists, partitionName, isTempPartition, force ? !force : !isTempPartition); :} | KW_MODIFY KW_PARTITION ident:partitionName KW_SET LPAREN key_value_map:properties RPAREN {: @@ -1570,13 +1571,13 @@ revoke_stmt ::= // Drop statement drop_stmt ::= /* Database */ - KW_DROP KW_DATABASE opt_if_exists:ifExists ident:db + KW_DROP KW_DATABASE opt_force:force opt_if_exists:ifExists ident:db {: - RESULT = new DropDbStmt(ifExists, db); + RESULT = new DropDbStmt(ifExists, db, !force); :} - | KW_DROP KW_SCHEMA opt_if_exists:ifExists ident:db + | KW_DROP KW_SCHEMA opt_force:force opt_if_exists:ifExists ident:db {: - RESULT = new DropDbStmt(ifExists, db); + RESULT = new DropDbStmt(ifExists, db, !force); :} /* cluster */ | KW_DROP KW_CLUSTER opt_if_exists:ifExists ident:cluster @@ -1589,9 +1590,9 @@ drop_stmt ::= RESULT = new DropFunctionStmt(functionName, args); :} /* Table */ - | KW_DROP KW_TABLE opt_if_exists:ifExists table_name:name + | KW_DROP KW_TABLE opt_force:force opt_if_exists:ifExists table_name:name {: - RESULT = new DropTableStmt(ifExists, name); + RESULT = new DropTableStmt(ifExists, name, !force); :} /* User */ | KW_DROP KW_USER user_identity:userId @@ -1601,7 +1602,7 @@ drop_stmt ::= /* View */ | KW_DROP KW_VIEW opt_if_exists:ifExists table_name:name {: - RESULT = new DropTableStmt(ifExists, name, true); + RESULT = new DropTableStmt(ifExists, name, true, false); :} | KW_DROP KW_REPOSITORY ident:repoName {: @@ -2020,6 +2021,17 @@ opt_external ::= :} ; +opt_force ::= + /* empty */ + {: + RESULT = false; + :} + | KW_FORCE + {: + RESULT = true; + :} + ; + // Show statement show_stmt ::= KW_SHOW show_param:stmt diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index 2bda6b18be744f..a83f075ec4ed71 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -143,7 +143,6 @@ public void processDropMaterializedView(DropMaterializedViewStmt stmt) throws Dd throw new DdlException("Table[" + table.getName() + "]'s state is not NORMAL. " + "Do not allow doing DROP ops"); } - // drop materialized view ((MaterializedViewHandler)materializedViewHandler).processDropMaterializedView(stmt, db, olapTable); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java index 27ef6d5eff0676..3803baf971b248 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java @@ -701,7 +701,7 @@ public void processBatchDropRollup(List dropRollupClauses, Database public void processDropMaterializedView(DropMaterializedViewStmt dropMaterializedViewStmt, Database db, OlapTable olapTable) throws DdlException, MetaNotFoundException { - db.writeLock(); + Preconditions.checkState(db.isWriteLockHeldByCurrentThread()); try { String mvName = dropMaterializedViewStmt.getMvName(); // Step1: check drop mv index operation @@ -710,7 +710,7 @@ public void processDropMaterializedView(DropMaterializedViewStmt dropMaterialize long mvIndexId = dropMaterializedView(mvName, olapTable); // Step3: log drop mv operation EditLog editLog = Catalog.getCurrentCatalog().getEditLog(); - editLog.logDropRollup(new DropInfo(db.getId(), olapTable.getId(), mvIndexId)); + editLog.logDropRollup(new DropInfo(db.getId(), olapTable.getId(), mvIndexId, false)); LOG.info("finished drop materialized view [{}] in table [{}]", mvName, olapTable.getName()); } catch (MetaNotFoundException e) { if (dropMaterializedViewStmt.isIfExists()) { @@ -718,8 +718,6 @@ public void processDropMaterializedView(DropMaterializedViewStmt dropMaterialize } else { throw e; } - } finally { - db.writeUnlock(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterTableStmt.java index 5e5a7916f310f1..fcc77e3c5539d2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterTableStmt.java @@ -48,7 +48,6 @@ public List getOps() { return ops; } - @Override public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java index ddf540d533679b..d0c2eddbad73e9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropDbStmt.java @@ -33,10 +33,12 @@ public class DropDbStmt extends DdlStmt { private boolean ifExists; private String dbName; + private boolean needCheckCommittedTxns; - public DropDbStmt(boolean ifExists, String dbName) { + public DropDbStmt(boolean ifExists, String dbName, boolean needCheckCommittedTxns) { this.ifExists = ifExists; this.dbName = dbName; + this.needCheckCommittedTxns = needCheckCommittedTxns; } public boolean isSetIfExists() { @@ -47,6 +49,10 @@ public String getDbName() { return this.dbName; } + public boolean isNeedCheckCommittedTxns() { + return this.needCheckCommittedTxns; + } + @Override public void analyze(Analyzer analyzer) throws AnalysisException, UserException { super.analyze(analyzer); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropPartitionClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropPartitionClause.java index 543bceadb4bad3..c5b5d0e035fec9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropPartitionClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropPartitionClause.java @@ -32,13 +32,15 @@ public class DropPartitionClause extends AlterTableClause { private String partitionName; // true if this is to drop a temp partition private boolean isTempPartition; + private boolean needCheckCommittedTxns; - public DropPartitionClause(boolean ifExists, String partitionName, boolean isTempPartition) { + public DropPartitionClause(boolean ifExists, String partitionName, boolean isTempPartition, boolean needCheckCommittedTxns) { super(AlterOpType.DROP_PARTITION); this.ifExists = ifExists; this.partitionName = partitionName; this.isTempPartition = isTempPartition; this.needTableStable = false; + this.needCheckCommittedTxns = needCheckCommittedTxns; } public boolean isSetIfExists() { @@ -53,6 +55,10 @@ public boolean isTempPartition() { return isTempPartition; } + public boolean isNeedCheckCommittedTxns() { + return needCheckCommittedTxns; + } + @Override public void analyze(Analyzer analyzer) throws AnalysisException { if (Strings.isNullOrEmpty(partitionName)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropTableStmt.java index 186ab3cb8e0e58..915592a50cddba 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropTableStmt.java @@ -32,17 +32,20 @@ public class DropTableStmt extends DdlStmt { private boolean ifExists; private final TableName tableName; private final boolean isView; + private boolean needCheckCommittedTxns; - public DropTableStmt(boolean ifExists, TableName tableName) { + public DropTableStmt(boolean ifExists, TableName tableName, boolean needCheckCommittedTxns) { this.ifExists = ifExists; this.tableName = tableName; this.isView = false; + this.needCheckCommittedTxns = needCheckCommittedTxns; } - public DropTableStmt(boolean ifExists, TableName tableName, boolean isView) { + public DropTableStmt(boolean ifExists, TableName tableName, boolean isView, boolean needCheckCommittedTxns) { this.ifExists = ifExists; this.tableName = tableName; this.isView = isView; + this.needCheckCommittedTxns = needCheckCommittedTxns; } public boolean isSetIfExists() { @@ -61,6 +64,10 @@ public boolean isView() { return isView; } + public boolean isNeedCheckCommittedTxns() { + return this.needCheckCommittedTxns; + } + @Override public void analyze(Analyzer analyzer) throws AnalysisException, UserException { if (Strings.isNullOrEmpty(tableName.getDb())) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java index ca7005fcbfc388..ab042b207b88c4 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java @@ -2678,6 +2678,13 @@ public void dropDb(DropDbStmt stmt) throws DdlException { Database db = this.fullNameToDb.get(dbName); db.writeLock(); try { + if (stmt.isNeedCheckCommittedTxns()) { + if (Catalog.getCurrentCatalog().getGlobalTransactionMgr().existCommittedTxns(db.getId(), null, null)) { + throw new DdlException("There are still some transactions in the COMMITTED state waiting to be completed. " + + "The database [" + dbName +"] cannot be dropped. If you want to forcibly drop(cannot be recovered)," + + " please use \"DROP database FORCE\"."); + } + } if (db.getDbState() == DbState.LINK && dbName.equals(db.getAttachDb())) { // We try to drop a hard link. final DropLinkDbAndUpdateDbInfo info = new DropLinkDbAndUpdateDbInfo(); @@ -2713,8 +2720,10 @@ public void dropDb(DropDbStmt stmt) throws DdlException { // save table names for recycling Set tableNames = db.getTableNamesWithLock(); - unprotectDropDb(db); - Catalog.getCurrentRecycleBin().recycleDatabase(db, tableNames); + unprotectDropDb(db, !stmt.isNeedCheckCommittedTxns()); + if (stmt.isNeedCheckCommittedTxns()) { + Catalog.getCurrentRecycleBin().recycleDatabase(db, tableNames); + } } finally { db.writeUnlock(); } @@ -2724,17 +2733,17 @@ public void dropDb(DropDbStmt stmt) throws DdlException { fullNameToDb.remove(db.getFullName()); final Cluster cluster = nameToCluster.get(db.getClusterName()); cluster.removeDb(dbName, db.getId()); - editLog.logDropDb(dbName); + editLog.logDropDb(dbName, !stmt.isNeedCheckCommittedTxns()); } finally { unlock(); } - LOG.info("finish drop database[{}]", dbName); + LOG.info("finish drop database[{}], is force : {}", dbName, !stmt.isNeedCheckCommittedTxns()); } - public void unprotectDropDb(Database db) { + public void unprotectDropDb(Database db, boolean isForeDrop) { for (Table table : db.getTables()) { - unprotectDropTable(db, table.getId()); + unprotectDropTable(db, table.getId(), isForeDrop); } } @@ -2754,15 +2763,17 @@ public void replayDropLinkDb(DropLinkDbAndUpdateDbInfo info) { } } - public void replayDropDb(String dbName) throws DdlException { + public void replayDropDb(String dbName, boolean isForceDrop) throws DdlException { tryLock(true); try { Database db = fullNameToDb.get(dbName); db.writeLock(); try { Set tableNames = db.getTableNamesWithLock(); - unprotectDropDb(db); - Catalog.getCurrentRecycleBin().recycleDatabase(db, tableNames); + unprotectDropDb(db, isForceDrop); + if (!isForceDrop) { + Catalog.getCurrentRecycleBin().recycleDatabase(db, tableNames); + } } finally { db.writeUnlock(); } @@ -3322,14 +3333,24 @@ public void dropPartition(Database db, OlapTable olapTable, DropPartitionClause if (isTempPartition) { olapTable.dropTempPartition(partitionName, true); } else { - olapTable.dropPartition(db.getId(), partitionName); + if (clause.isNeedCheckCommittedTxns()) { + Partition partition = olapTable.getPartition(partitionName); + if (partition != null) { + if (Catalog.getCurrentCatalog().getGlobalTransactionMgr().existCommittedTxns(db.getId(), olapTable.getId(), partition.getId())) { + throw new DdlException("There are still some transactions in the COMMITTED state waiting to be completed." + + " The partition [" + partitionName + "] cannot be dropped. If you want to forcibly drop(cannot be recovered)," + + " please use \"DROP partition FORCE\"."); + } + } + } + olapTable.dropPartition(db.getId(), partitionName, !clause.isNeedCheckCommittedTxns()); } // log - DropPartitionInfo info = new DropPartitionInfo(db.getId(), olapTable.getId(), partitionName, isTempPartition); + DropPartitionInfo info = new DropPartitionInfo(db.getId(), olapTable.getId(), partitionName, isTempPartition, !clause.isNeedCheckCommittedTxns()); editLog.logDropPartition(info); - LOG.info("succeed in droping partition[{}]", partitionName); + LOG.info("succeed in droping partition[{}], is temp : {}, is force : {}", partitionName, isTempPartition, !clause.isNeedCheckCommittedTxns()); } public void replayDropPartition(DropPartitionInfo info) { @@ -3340,7 +3361,7 @@ public void replayDropPartition(DropPartitionInfo info) { if (info.isTempPartition()) { olapTable.dropTempPartition(info.getPartitionName(), true); } else { - olapTable.dropPartition(info.getDbId(), info.getPartitionName()); + olapTable.dropPartition(info.getDbId(), info.getPartitionName(), info.isForceDrop()); } } finally { db.writeUnlock(); @@ -4284,18 +4305,24 @@ public void dropTable(DropTableStmt stmt) throws DdlException { } } - unprotectDropTable(db, table.getId()); - - DropInfo info = new DropInfo(db.getId(), table.getId(), -1L); + if (stmt.isNeedCheckCommittedTxns()) { + if (Catalog.getCurrentCatalog().getGlobalTransactionMgr().existCommittedTxns(db.getId(), table.getId(), null)) { + throw new DdlException("There are still some transactions in the COMMITTED state waiting to be completed. " + + "The table [" + tableName +"] cannot be dropped. If you want to forcibly drop(cannot be recovered)," + + " please use \"DROP table FORCE\"."); + } + } + unprotectDropTable(db, table.getId(), !stmt.isNeedCheckCommittedTxns()); + DropInfo info = new DropInfo(db.getId(), table.getId(), -1L, !stmt.isNeedCheckCommittedTxns()); editLog.logDropTable(info); } finally { db.writeUnlock(); } - LOG.info("finished dropping table: {} from db: {}", tableName, dbName); + LOG.info("finished dropping table: {} from db: {}, is force: {}", tableName, dbName, !stmt.isNeedCheckCommittedTxns()); } - public boolean unprotectDropTable(Database db, long tableId) { + public boolean unprotectDropTable(Database db, long tableId, boolean isForceDrop) { Table table = db.getTable(tableId); // delete from db meta if (table == null) { @@ -4311,17 +4338,18 @@ public boolean unprotectDropTable(Database db, long tableId) { } db.dropTable(table.getName()); - - Catalog.getCurrentRecycleBin().recycleTable(db.getId(), table); + if (!isForceDrop) { + Catalog.getCurrentRecycleBin().recycleTable(db.getId(), table); + } LOG.info("finished dropping table[{}] in db[{}]", table.getName(), db.getFullName()); return true; } - public void replayDropTable(Database db, long tableId) { + public void replayDropTable(Database db, long tableId, boolean isForceDrop) { db.writeLock(); try { - unprotectDropTable(db, tableId); + unprotectDropTable(db, tableId, isForceDrop); } finally { db.writeUnlock(); } 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 10603eb673d30f..b4367eb5f4c792 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 @@ -589,11 +589,7 @@ public void addPartition(Partition partition) { nameToPartition.put(partition.getName(), partition); } - public Partition dropPartition(long dbId, String partitionName) { - return dropPartition(dbId, partitionName, false); - } - - public Partition dropPartition(long dbId, String partitionName, boolean isRestore) { + public Partition dropPartition(long dbId, String partitionName, boolean isForceDrop) { Partition partition = nameToPartition.get(partitionName); if (partition != null) { idToPartition.remove(partition.getId()); @@ -602,7 +598,7 @@ public Partition dropPartition(long dbId, String partitionName, boolean isRestor Preconditions.checkState(partitionInfo.getType() == PartitionType.RANGE); RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) partitionInfo; - if (!isRestore) { + if (!isForceDrop) { // recycle partition Catalog.getCurrentRecycleBin().recyclePartition(dbId, id, partition, rangePartitionInfo.getRange(partition.getId()), diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java index da3d844b9599cb..94110098031036 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java @@ -237,7 +237,7 @@ private ArrayList getDropPartitionClause(Database db, OlapT RangeUtils.checkRangeIntersect(reservePartitionKeyRange, checkDropPartitionKey); if (checkDropPartitionKey.upperEndpoint().compareTo(reservePartitionKeyRange.lowerEndpoint()) <= 0) { String dropPartitionName = olapTable.getPartition(checkDropPartitionId).getName(); - dropPartitionClauses.add(new DropPartitionClause(false, dropPartitionName, false)); + dropPartitionClauses.add(new DropPartitionClause(false, dropPartitionName, false, true)); } } catch (DdlException e) { break; diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/FeMetaVersion.java b/fe/fe-core/src/main/java/org/apache/doris/common/FeMetaVersion.java index ec7205f97c85ec..35f453dcdaddca 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/FeMetaVersion.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/FeMetaVersion.java @@ -187,6 +187,9 @@ public final class FeMetaVersion { public static final int VERSION_87 = 87; // add partition visibleVersionTime public static final int VERSION_88 = 88; + // force drop db, force drop table, force drop partition + // make force drop operation do not recycle meta + public static final int VERSION_89 = 89; // note: when increment meta version, should assign the latest version to VERSION_CURRENT - public static final int VERSION_CURRENT = VERSION_88; + public static final int VERSION_CURRENT = VERSION_89; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java index f76dfedc9a41c3..ccfb541df885c7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java +++ b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java @@ -56,6 +56,7 @@ import org.apache.doris.persist.ConsistencyCheckInfo; import org.apache.doris.persist.CreateTableInfo; import org.apache.doris.persist.DatabaseInfo; +import org.apache.doris.persist.DropDbInfo; import org.apache.doris.persist.DropInfo; import org.apache.doris.persist.DropLinkDbAndUpdateDbInfo; import org.apache.doris.persist.DropPartitionInfo; @@ -149,8 +150,7 @@ public void readFields(DataInput in) throws IOException { break; } case OperationType.OP_DROP_DB: { - data = new Text(); - ((Text) data).readFields(in); + data = DropDbInfo.read(in); isRead = true; break; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/DropDbInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/DropDbInfo.java new file mode 100644 index 00000000000000..6dc01f25ae7257 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/DropDbInfo.java @@ -0,0 +1,90 @@ +// 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.persist; + +import com.google.gson.annotations.SerializedName; +import org.apache.doris.catalog.Catalog; +import org.apache.doris.common.FeMetaVersion; +import org.apache.doris.common.io.Text; +import org.apache.doris.common.io.Writable; +import org.apache.doris.persist.gson.GsonUtils; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +public class DropDbInfo implements Writable { + @SerializedName(value = "dbName") + private String dbName; + @SerializedName(value = "forceDrop") + private boolean forceDrop = false; + + public DropDbInfo() { + this("", false); + } + + public DropDbInfo(String dbName, boolean forceDrop) { + this.dbName = dbName; + this.forceDrop = forceDrop; + } + + public String getDbName() { + return dbName; + } + + public boolean isForceDrop() { + return forceDrop; + } + + private void readFields(DataInput in) throws IOException { + dbName = Text.readString(in); + } + + public static DropDbInfo read(DataInput in) throws IOException { + if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_88) { + DropDbInfo info = new DropDbInfo(); + info.readFields(in); + return info; + } else { + String json = Text.readString(in); + return GsonUtils.GSON.fromJson(json, DropDbInfo.class); + } + } + + @Override + public void write(DataOutput out) throws IOException { + String json = GsonUtils.GSON.toJson(this); + Text.writeString(out, json); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!(obj instanceof DropDbInfo)) { + return false; + } + + DropDbInfo info = (DropDbInfo) obj; + + return (dbName.equals(info.getDbName())) + && (forceDrop == info.isForceDrop()); + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/DropInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/DropInfo.java index 8b72074e7dc160..3606386954655d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/DropInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/DropInfo.java @@ -17,6 +17,8 @@ package org.apache.doris.persist; +import org.apache.doris.catalog.Catalog; +import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.io.Writable; import java.io.DataInput; @@ -28,14 +30,16 @@ public class DropInfo implements Writable { private long tableId; private long indexId; + private boolean forceDrop = false; public DropInfo() { } - public DropInfo(long dbId, long tableId, long indexId) { + public DropInfo(long dbId, long tableId, long indexId, boolean forceDrop) { this.dbId = dbId; this.tableId = tableId; this.indexId = indexId; + this.forceDrop = forceDrop; } public long getDbId() { @@ -49,11 +53,16 @@ public long getTableId() { public long getIndexId() { return this.indexId; } + + public boolean isForceDrop() { + return forceDrop; + } @Override public void write(DataOutput out) throws IOException { out.writeLong(dbId); out.writeLong(tableId); + out.writeBoolean(forceDrop); if (indexId == -1L) { out.writeBoolean(false); } else { @@ -65,7 +74,9 @@ public void write(DataOutput out) throws IOException { public void readFields(DataInput in) throws IOException { dbId = in.readLong(); tableId = in.readLong(); - + if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_89) { + forceDrop = in.readBoolean(); + } boolean hasIndexId = in.readBoolean(); if (hasIndexId) { indexId = in.readLong(); @@ -73,7 +84,7 @@ public void readFields(DataInput in) throws IOException { indexId = -1L; } } - + public static DropInfo read(DataInput in) throws IOException { DropInfo dropInfo = new DropInfo(); dropInfo.readFields(in); @@ -91,6 +102,7 @@ public boolean equals (Object obj) { DropInfo info = (DropInfo) obj; - return dbId == info.dbId && tableId == info.tableId; + return (dbId == info.dbId) && (tableId == info.tableId) && (indexId == info.indexId) + && (forceDrop == info.forceDrop); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/DropPartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/DropPartitionInfo.java index 4587074c5204bc..f10b8d274897d3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/DropPartitionInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/DropPartitionInfo.java @@ -38,15 +38,18 @@ public class DropPartitionInfo implements Writable { private String partitionName; @SerializedName(value = "isTempPartition") private boolean isTempPartition = false; + @SerializedName(value = "forceDrop") + private boolean forceDrop = false; private DropPartitionInfo() { } - public DropPartitionInfo(Long dbId, Long tableId, String partitionName, boolean isTempPartition) { + public DropPartitionInfo(Long dbId, Long tableId, String partitionName, boolean isTempPartition, boolean forceDrop) { this.dbId = dbId; this.tableId = tableId; this.partitionName = partitionName; this.isTempPartition = isTempPartition; + this.forceDrop = forceDrop; } public Long getDbId() { @@ -65,6 +68,10 @@ public boolean isTempPartition() { return isTempPartition; } + public boolean isForceDrop() { + return forceDrop; + } + private void readFields(DataInput in) throws IOException { dbId = in.readLong(); tableId = in.readLong(); @@ -101,6 +108,8 @@ public boolean equals(Object obj) { return (dbId.equals(info.dbId)) && (tableId.equals(info.tableId)) - && (partitionName.equals(info.partitionName)); + && (partitionName.equals(info.partitionName)) + && (isTempPartition == info.isTempPartition) + && (forceDrop == info.forceDrop); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index 728ee0e5f0f29a..a5898e8b765b5f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -145,8 +145,8 @@ public static void loadJournal(Catalog catalog, JournalEntity journal) { break; } case OperationType.OP_DROP_DB: { - String dbName = ((Text) journal.getData()).toString(); - catalog.replayDropDb(dbName); + DropDbInfo dropDbInfo = (DropDbInfo) journal.getData(); + catalog.replayDropDb(dropDbInfo.getDbName(), dropDbInfo.isForceDrop()); break; } case OperationType.OP_ALTER_DB: { @@ -189,7 +189,7 @@ public static void loadJournal(Catalog catalog, JournalEntity journal) { } LOG.info("Begin to unprotect drop table. db = " + db.getFullName() + " table = " + info.getTableId()); - catalog.replayDropTable(db, info.getTableId()); + catalog.replayDropTable(db, info.getTableId(), info.isForceDrop()); break; } case OperationType.OP_ADD_PARTITION: { @@ -297,7 +297,7 @@ public static void loadJournal(Catalog catalog, JournalEntity journal) { BatchDropInfo batchDropInfo = (BatchDropInfo) journal.getData(); for (long indexId : batchDropInfo.getIndexIdSet()) { catalog.getRollupHandler().replayDropRollup( - new DropInfo(batchDropInfo.getDbId(), batchDropInfo.getTableId(), indexId), catalog); + new DropInfo(batchDropInfo.getDbId(), batchDropInfo.getTableId(), indexId, false), catalog); } break; } @@ -889,7 +889,7 @@ public void logCreateDb(Database db) { logEdit(OperationType.OP_CREATE_DB, db); } - public void logDropDb(String dbName) { + public void logDropDb(String dbName, boolean isForceDrop) { logEdit(OperationType.OP_DROP_DB, new Text(dbName)); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java index ce35a81875ec14..f069f77c3d83af 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java @@ -217,14 +217,32 @@ public void abortTransaction(Long dbId, String label, String reason) throws User * get all txns which is ready to publish * a ready-to-publish txn's partition's visible version should be ONE less than txn's commit version. */ - public List getReadyToPublishTransactions() throws UserException { + public List getReadyToPublishTransactions() { List transactionStateList = Lists.newArrayList(); for (DatabaseTransactionMgr dbTransactionMgr : dbIdToDatabaseTransactionMgrs.values()) { transactionStateList.addAll(dbTransactionMgr.getCommittedTxnList()); } return transactionStateList; } - + + public boolean existCommittedTxns(Long dbId, Long tableId, Long partitionId) { + DatabaseTransactionMgr dbTransactionMgr = dbIdToDatabaseTransactionMgrs.get(dbId); + if (tableId == null && partitionId == null) { + return !dbTransactionMgr.getCommittedTxnList().isEmpty(); + } + + for (TransactionState transactionState : dbTransactionMgr.getCommittedTxnList()) { + if (transactionState.getTableIdList().contains(tableId)) { + if (partitionId == null) { + return true; + } else if (transactionState.getTableCommitInfo(tableId).getPartitionCommitInfo(partitionId) != null){ + return true; + } + } + } + return false; + } + /** * if the table is deleted between commit and publish version, then should ignore the partition * diff --git a/fe/fe-core/src/main/jflex/sql_scanner.flex b/fe/fe-core/src/main/jflex/sql_scanner.flex index 973bcf1792f990..f0c0a9d6064798 100644 --- a/fe/fe-core/src/main/jflex/sql_scanner.flex +++ b/fe/fe-core/src/main/jflex/sql_scanner.flex @@ -185,6 +185,7 @@ import org.apache.doris.qe.SqlModeHelper; keywordMap.put("follower", new Integer(SqlParserSymbols.KW_FOLLOWER)); keywordMap.put("following", new Integer(SqlParserSymbols.KW_FOLLOWING)); keywordMap.put("for", new Integer(SqlParserSymbols.KW_FOR)); + keywordMap.put("force", new Integer(SqlParserSymbols.KW_FORCE)); keywordMap.put("format", new Integer(SqlParserSymbols.KW_FORMAT)); keywordMap.put("free", new Integer(SqlParserSymbols.KW_FREE)); keywordMap.put("from", new Integer(SqlParserSymbols.KW_FROM)); diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/AlterTableStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/AlterTableStmtTest.java index 99c800574fcc18..42da34ddbb7740 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/AlterTableStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/AlterTableStmtTest.java @@ -62,7 +62,7 @@ public void setUp() { } @Test - public void testNormal() throws AnalysisException, UserException { + public void testNormal() throws UserException { List ops = Lists.newArrayList(); ops.add(new DropColumnClause("col1", "", null)); ops.add(new DropColumnClause("col2", "", null)); @@ -89,7 +89,7 @@ public void testAddRollup() throws UserException { } @Test(expected = AnalysisException.class) - public void testNoTable() throws AnalysisException, UserException { + public void testNoTable() throws UserException { List ops = Lists.newArrayList(); ops.add(new DropColumnClause("col1", "", null)); AlterTableStmt stmt = new AlterTableStmt(null, ops); @@ -99,7 +99,7 @@ public void testNoTable() throws AnalysisException, UserException { } @Test(expected = AnalysisException.class) - public void testNoClause() throws AnalysisException, UserException { + public void testNoClause() throws UserException { List ops = Lists.newArrayList(); AlterTableStmt stmt = new AlterTableStmt(new TableName("testDb", "testTbl"), ops); stmt.analyze(analyzer); diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java index e2080b4bfa44f9..e281735fe5231a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropDbStmtTest.java @@ -46,7 +46,7 @@ public void setUp() { @Test public void testNormal() throws UserException, AnalysisException { - DropDbStmt stmt = new DropDbStmt(false, "test"); + DropDbStmt stmt = new DropDbStmt(false, "test", true); stmt.analyze(analyzer); Assert.assertEquals("testCluster:test", stmt.getDbName()); @@ -55,7 +55,7 @@ public void testNormal() throws UserException, AnalysisException { @Test(expected = AnalysisException.class) public void testFailed() throws UserException, AnalysisException { - DropDbStmt stmt = new DropDbStmt(false, ""); + DropDbStmt stmt = new DropDbStmt(false, "", true); stmt.analyze(analyzer); Assert.fail("no exception"); @@ -63,7 +63,7 @@ public void testFailed() throws UserException, AnalysisException { @Test(expected = AnalysisException.class) public void testNoPriv() throws UserException, AnalysisException { - DropDbStmt stmt = new DropDbStmt(false, ""); + DropDbStmt stmt = new DropDbStmt(false, "", true); stmt.analyze(AccessTestUtil.fetchBlockAnalyzer()); Assert.fail("no exception"); diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropMaterializedViewStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropMaterializedViewStmtTest.java index 068a03160f95d4..faf39f22554d2d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropMaterializedViewStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropMaterializedViewStmtTest.java @@ -17,7 +17,6 @@ package org.apache.doris.analysis; -import org.apache.doris.common.ErrorCode; import org.apache.doris.common.UserException; import org.apache.doris.mysql.privilege.PaloAuth; import org.apache.doris.mysql.privilege.PrivPredicate; diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropTableStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropTableStmtTest.java index 8c28d0d66561fa..f33fcaab3593be 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropTableStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropTableStmtTest.java @@ -66,7 +66,7 @@ public void setUp() { @Test public void testNormal() throws UserException, AnalysisException { - DropTableStmt stmt = new DropTableStmt(false, tbl); + DropTableStmt stmt = new DropTableStmt(false, tbl, true); stmt.analyze(analyzer); Assert.assertEquals("testCluster:db1", stmt.getDbName()); Assert.assertEquals("table1", stmt.getTableName()); @@ -75,7 +75,7 @@ public void testNormal() throws UserException, AnalysisException { @Test public void testDefaultNormal() throws UserException, AnalysisException { - DropTableStmt stmt = new DropTableStmt(false, noDbTbl); + DropTableStmt stmt = new DropTableStmt(false, noDbTbl, true); stmt.analyze(analyzer); Assert.assertEquals("testCluster:testDb", stmt.getDbName()); Assert.assertEquals("table1", stmt.getTableName()); @@ -84,14 +84,14 @@ public void testDefaultNormal() throws UserException, AnalysisException { @Test(expected = AnalysisException.class) public void testNoDbFail() throws UserException, AnalysisException { - DropTableStmt stmt = new DropTableStmt(false, noDbTbl); + DropTableStmt stmt = new DropTableStmt(false, noDbTbl, true); stmt.analyze(noDbAnalyzer); Assert.fail("No Exception throws."); } @Test(expected = AnalysisException.class) public void testNoTableFail() throws UserException, AnalysisException { - DropTableStmt stmt = new DropTableStmt(false, new TableName("db1", "")); + DropTableStmt stmt = new DropTableStmt(false, new TableName("db1", ""), true); stmt.analyze(noDbAnalyzer); Assert.fail("No Exception throws."); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/persist/DropDbInfoTest.java b/fe/fe-core/src/test/java/org/apache/doris/persist/DropDbInfoTest.java new file mode 100644 index 00000000000000..e66e27f15219fd --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/persist/DropDbInfoTest.java @@ -0,0 +1,75 @@ +// 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.persist; + +import org.apache.doris.catalog.Catalog; +import org.apache.doris.common.FeMetaVersion; +import org.apache.doris.meta.MetaContext; +import org.junit.Assert; +import org.junit.Test; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; + +public class DropDbInfoTest { + @Test + public void testSerialization() throws Exception { + MetaContext metaContext = new MetaContext(); + metaContext.setMetaVersion(FeMetaVersion.VERSION_89); + metaContext.setThreadLocalInfo(); + + // 1. Write objects to file + File file = new File("./dropDbInfo"); + file.createNewFile(); + DataOutputStream dos = new DataOutputStream(new FileOutputStream(file)); + + DropDbInfo info1 = new DropDbInfo(); + info1.write(dos); + + DropDbInfo info2 = new DropDbInfo("test_db", true); + info2.write(dos); + + dos.flush(); + dos.close(); + + // 2. Read objects from file + DataInputStream dis = new DataInputStream(new FileInputStream(file)); + + DropDbInfo rInfo1 = DropDbInfo.read(dis); + Assert.assertTrue(rInfo1.equals(info1)); + + DropDbInfo rInfo2 = DropDbInfo.read(dis); + Assert.assertTrue(rInfo2.equals(info2)); + + Assert.assertEquals("test_db", rInfo2.getDbName()); + Assert.assertTrue(rInfo2.isForceDrop()); + + Assert.assertTrue(rInfo2.equals(rInfo2)); + Assert.assertFalse(rInfo2.equals(this)); + Assert.assertFalse(info2.equals(new DropDbInfo("test_db1", true))); + Assert.assertFalse(info2.equals(new DropDbInfo("test_db", false))); + Assert.assertTrue(info2.equals(new DropDbInfo("test_db", true))); + + // 3. delete files + dis.close(); + file.delete(); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/persist/DropInfoTest.java b/fe/fe-core/src/test/java/org/apache/doris/persist/DropInfoTest.java index a677d5d71b9c53..181a4b4fe3cab7 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/persist/DropInfoTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/persist/DropInfoTest.java @@ -17,6 +17,8 @@ package org.apache.doris.persist; +import org.apache.doris.common.FeMetaVersion; +import org.apache.doris.meta.MetaContext; import org.junit.Assert; import org.junit.Test; @@ -29,6 +31,10 @@ public class DropInfoTest { @Test public void testSerialization() throws Exception { + MetaContext metaContext = new MetaContext(); + metaContext.setMetaVersion(FeMetaVersion.VERSION_89); + metaContext.setThreadLocalInfo(); + // 1. Write objects to file File file = new File("./dropInfo"); file.createNewFile(); @@ -37,7 +43,7 @@ public void testSerialization() throws Exception { DropInfo info1 = new DropInfo(); info1.write(dos); - DropInfo info2 = new DropInfo(1, 2, -1); + DropInfo info2 = new DropInfo(1, 2, -1, true); info2.write(dos); dos.flush(); @@ -54,12 +60,14 @@ public void testSerialization() throws Exception { Assert.assertEquals(1, rInfo2.getDbId()); Assert.assertEquals(2, rInfo2.getTableId()); + Assert.assertTrue(rInfo2.isForceDrop()); Assert.assertTrue(rInfo2.equals(rInfo2)); Assert.assertFalse(rInfo2.equals(this)); - Assert.assertFalse(info2.equals(new DropInfo(0, 2, -1L))); - Assert.assertFalse(info2.equals(new DropInfo(1, 0, -1L))); - Assert.assertTrue(info2.equals(new DropInfo(1, 2, -1L))); + Assert.assertFalse(info2.equals(new DropInfo(0, 2, -1L, true))); + Assert.assertFalse(info2.equals(new DropInfo(1, 0, -1L, true))); + Assert.assertFalse(info2.equals(new DropInfo(1, 2, -1L, false))); + Assert.assertTrue(info2.equals(new DropInfo(1, 2, -1L, true))); // 3. delete files dis.close(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/persist/DropPartitionInfoTest.java b/fe/fe-core/src/test/java/org/apache/doris/persist/DropPartitionInfoTest.java new file mode 100644 index 00000000000000..9bda6deb7beba9 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/persist/DropPartitionInfoTest.java @@ -0,0 +1,73 @@ +// 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.persist; + +import org.apache.doris.common.FeMetaVersion; +import org.apache.doris.meta.MetaContext; +import org.junit.Assert; +import org.junit.Test; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; + +public class DropPartitionInfoTest { + @Test + public void testSerialization() throws Exception { + MetaContext metaContext = new MetaContext(); + metaContext.setMetaVersion(FeMetaVersion.VERSION_89); + metaContext.setThreadLocalInfo(); + + // 1. Write objects to file + File file = new File("./dropPartitionInfo"); + file.createNewFile(); + DataOutputStream dos = new DataOutputStream(new FileOutputStream(file)); + + DropPartitionInfo info1 = new DropPartitionInfo(1L, 2L, "test_partition", false, true); + info1.write(dos); + + dos.flush(); + dos.close(); + + // 2. Read objects from file + DataInputStream dis = new DataInputStream(new FileInputStream(file)); + + DropPartitionInfo rInfo1 = DropPartitionInfo.read(dis); + + Assert.assertEquals(Long.valueOf(1L), rInfo1.getDbId()); + Assert.assertEquals(Long.valueOf(2L), rInfo1.getTableId()); + Assert.assertEquals("test_partition", rInfo1.getPartitionName()); + Assert.assertFalse(rInfo1.isTempPartition()); + Assert.assertTrue(rInfo1.isForceDrop()); + + Assert.assertTrue(rInfo1.equals(info1)); + Assert.assertFalse(rInfo1.equals(this)); + Assert.assertFalse(info1.equals(new DropPartitionInfo(-1L, 2L, "test_partition", false, true))); + Assert.assertFalse(info1.equals(new DropPartitionInfo(1L, -2L, "test_partition", false, true))); + Assert.assertFalse(info1.equals(new DropPartitionInfo(1L, 2L, "test_partition1", false, true))); + Assert.assertFalse(info1.equals(new DropPartitionInfo(1L, 2L, "test_partition", true, true))); + Assert.assertFalse(info1.equals(new DropPartitionInfo(1L, 2L, "test_partition", false, false))); + Assert.assertTrue(info1.equals(new DropPartitionInfo(1L, 2L, "test_partition", false, true))); + + // 3. delete files + dis.close(); + file.delete(); + } +}