You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2022/10/15 10:17:45 UTC

[GitHub] [doris] morningman commented on a diff in pull request #13067: [feature-wip](recover) new recover ddl and support show dropped

morningman commented on code in PR #13067:
URL: https://github.com/apache/doris/pull/13067#discussion_r996279665


##########
fe/fe-core/src/main/cup/sql_parser.cup:
##########
@@ -2743,17 +2745,17 @@ drop_stmt ::=
 
 // Recover statement
 recover_stmt ::=
-    KW_RECOVER KW_DATABASE ident:dbName
+    KW_RECOVER KW_DATABASE ident:dbName opt_id:dbId opt_table_alias:alias

Review Comment:
   if `id` is specified, no need to specify `name`
   so the syntax can be:
   ```
   recover database ident_or_id opt_alias
   ```



##########
fe/fe-core/src/main/cup/sql_parser.cup:
##########
@@ -3619,6 +3621,10 @@ show_param ::=
     {:
         RESULT = new ShowAnalyzeStmt(tbl, parser.where, orderByClause, limitClause);
     :}
+    | KW_DROPPED opt_wild_where

Review Comment:
   How about:
   ```
   show catalog recycle bin where xxx?
   ```
   `dropped` is too colloquial



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java:
##########
@@ -76,46 +81,56 @@ public synchronized boolean recycleDatabase(Database db, Set<String> tableNames)
         // db should be empty. all tables are recycled before
         Preconditions.checkState(db.getTables().isEmpty());
 
-        // erase db with same name
-        eraseDatabaseWithSameName(db.getFullName());
+        // erase db with same id
+        eraseDatabaseWithSameId(db.getId());
 
         // recycle db
-        RecycleDatabaseInfo databaseInfo = new RecycleDatabaseInfo(db, tableNames);
+        RecycleDatabaseInfo databaseInfo = new RecycleDatabaseInfo(db, tableNames, tableIds);
         idToDatabase.put(db.getId(), databaseInfo);
-        idToRecycleTime.put(db.getId(), System.currentTimeMillis());
+        if (!isReplay || replayRecycleTime == 0) {
+            recycleTime = System.currentTimeMillis();

Review Comment:
   Looks like the `recycleTime` is not written into edit log,
   so it is not persisted?



##########
fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDroppedStmt.java:
##########
@@ -0,0 +1,154 @@
+// 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.analysis;
+
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.ScalarType;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.CaseSensibility;
+import org.apache.doris.common.PatternMatcher;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ShowResultSetMetaData;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+
+import java.util.function.Predicate;
+
+public class ShowDroppedStmt extends ShowStmt {
+    public static final ImmutableList<String> TITLE_NAMES = new ImmutableList.Builder<String>()
+            .add("Type").add("Name").add("DbId").add("TableId").add("PartitionId").add("DropTime")

Review Comment:
   We can know what id it is using `type` column.
   So no need to use 2 columns to distinguish `db/tbl/partition` ids,



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java:
##########
@@ -76,46 +81,56 @@ public synchronized boolean recycleDatabase(Database db, Set<String> tableNames)
         // db should be empty. all tables are recycled before
         Preconditions.checkState(db.getTables().isEmpty());
 
-        // erase db with same name
-        eraseDatabaseWithSameName(db.getFullName());
+        // erase db with same id
+        eraseDatabaseWithSameId(db.getId());

Review Comment:
   Why does the same id appear?



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java:
##########
@@ -76,46 +81,56 @@ public synchronized boolean recycleDatabase(Database db, Set<String> tableNames)
         // db should be empty. all tables are recycled before
         Preconditions.checkState(db.getTables().isEmpty());
 
-        // erase db with same name
-        eraseDatabaseWithSameName(db.getFullName());
+        // erase db with same id
+        eraseDatabaseWithSameId(db.getId());
 
         // recycle db
-        RecycleDatabaseInfo databaseInfo = new RecycleDatabaseInfo(db, tableNames);
+        RecycleDatabaseInfo databaseInfo = new RecycleDatabaseInfo(db, tableNames, tableIds);

Review Comment:
   if `tableIds` is saved, no need to save `tableNames`



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java:
##########
@@ -76,46 +81,56 @@ public synchronized boolean recycleDatabase(Database db, Set<String> tableNames)
         // db should be empty. all tables are recycled before
         Preconditions.checkState(db.getTables().isEmpty());
 
-        // erase db with same name
-        eraseDatabaseWithSameName(db.getFullName());
+        // erase db with same id
+        eraseDatabaseWithSameId(db.getId());
 
         // recycle db
-        RecycleDatabaseInfo databaseInfo = new RecycleDatabaseInfo(db, tableNames);
+        RecycleDatabaseInfo databaseInfo = new RecycleDatabaseInfo(db, tableNames, tableIds);
         idToDatabase.put(db.getId(), databaseInfo);
-        idToRecycleTime.put(db.getId(), System.currentTimeMillis());
+        if (!isReplay || replayRecycleTime == 0) {
+            recycleTime = System.currentTimeMillis();
+        } else {
+            recycleTime = replayRecycleTime;
+        }
+        idToRecycleTime.put(db.getId(), recycleTime);
         LOG.info("recycle db[{}-{}]", db.getId(), db.getFullName());
         return true;
     }
 
-    public synchronized boolean recycleTable(long dbId, Table table, boolean isReplay) {
+    public synchronized boolean recycleTable(long dbId, Table table, boolean isReplay, long replayRecycleTime) {
+        long recycleTime = 0;
         if (idToTable.containsKey(table.getId())) {
             LOG.error("table[{}] already in recycle bin.", table.getId());
             return false;
         }
 
-        // erase table with same name
-        eraseTableWithSameName(dbId, table.getName(), isReplay);
+        // erase table with same id
+        eraseTableWithSameId(dbId, table.getId(), isReplay);

Review Comment:
   Same question as database, why same id appear?



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java:
##########
@@ -395,9 +463,27 @@ public synchronized void replayRecoverTable(Database db, long tableId) {
             Table table = tableInfo.getTable();
             table.writeLock();
             try {
-                db.createTable(tableInfo.getTable());
+                if (!Strings.isNullOrEmpty(newTableName)) {

Review Comment:
   Better to extract this same logic as a function and use it in both `recoverTable` and `replayRecoverTable`



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java:
##########
@@ -76,46 +81,56 @@ public synchronized boolean recycleDatabase(Database db, Set<String> tableNames)
         // db should be empty. all tables are recycled before
         Preconditions.checkState(db.getTables().isEmpty());
 
-        // erase db with same name
-        eraseDatabaseWithSameName(db.getFullName());
+        // erase db with same id
+        eraseDatabaseWithSameId(db.getId());
 
         // recycle db
-        RecycleDatabaseInfo databaseInfo = new RecycleDatabaseInfo(db, tableNames);
+        RecycleDatabaseInfo databaseInfo = new RecycleDatabaseInfo(db, tableNames, tableIds);
         idToDatabase.put(db.getId(), databaseInfo);
-        idToRecycleTime.put(db.getId(), System.currentTimeMillis());
+        if (!isReplay || replayRecycleTime == 0) {
+            recycleTime = System.currentTimeMillis();
+        } else {
+            recycleTime = replayRecycleTime;
+        }
+        idToRecycleTime.put(db.getId(), recycleTime);
         LOG.info("recycle db[{}-{}]", db.getId(), db.getFullName());
         return true;
     }
 
-    public synchronized boolean recycleTable(long dbId, Table table, boolean isReplay) {
+    public synchronized boolean recycleTable(long dbId, Table table, boolean isReplay, long replayRecycleTime) {
+        long recycleTime = 0;
         if (idToTable.containsKey(table.getId())) {
             LOG.error("table[{}] already in recycle bin.", table.getId());
             return false;
         }
 
-        // erase table with same name
-        eraseTableWithSameName(dbId, table.getName(), isReplay);
+        // erase table with same id
+        eraseTableWithSameId(dbId, table.getId(), isReplay);
 
         // recycle table
         RecycleTableInfo tableInfo = new RecycleTableInfo(dbId, table);
-        idToRecycleTime.put(table.getId(), System.currentTimeMillis());
+        if (!isReplay || replayRecycleTime == 0) {
+            recycleTime = System.currentTimeMillis();
+        } else {
+            recycleTime = replayRecycleTime;
+        }
+        idToRecycleTime.put(table.getId(), recycleTime);
         idToTable.put(table.getId(), tableInfo);
         LOG.info("recycle table[{}-{}]", table.getId(), table.getName());
         return true;
     }
 
     public synchronized boolean recyclePartition(long dbId, long tableId, Partition partition,
                                                  Range<PartitionKey> range, PartitionItem listPartitionItem,
-                                                 DataProperty dataProperty,
-                                                 ReplicaAllocation replicaAlloc,
+                                                 DataProperty dataProperty, ReplicaAllocation replicaAlloc,
                                                  boolean isInMemory) {
         if (idToPartition.containsKey(partition.getId())) {
             LOG.error("partition[{}] already in recycle bin.", partition.getId());
             return false;
         }
 
-        // erase partition with same name
-        erasePartitionWithSameName(dbId, tableId, partition.getName());
+        // erase partition with same id
+        erasePartitionWithSameId(dbId, tableId, partition.getId());

Review Comment:
   Same question as database, why same id appear?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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