You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2021/01/31 16:12:43 UTC

[incubator-doris] branch master updated: [Backup]Support content, exclude and whole database in backup (#5314)

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

morningman pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git


The following commit(s) were added to refs/heads/master by this push:
     new bb7ba00  [Backup]Support content, exclude and whole database in backup (#5314)
bb7ba00 is described below

commit bb7ba00ccf91e2a66b1f8932d134a06b63a73263
Author: EmmyMiao87 <52...@qq.com>
AuthorDate: Mon Feb 1 00:12:35 2021 +0800

    [Backup]Support content, exclude and whole database in backup (#5314)
    
    This PR support following functions:
    1. Support content properties in backup stmt. It means user can backup only metadata or
    meta+data which use content [METADATA_ONLY| ALL]attribute to distinguish.
    2. Support exclude some tables in backup and restore stmt. This means that some
    very large and unimportant tables can be excluded when the entire database is backed up.
    3. Support backup and restore whole database instead of declaring each table name
    in the backup and restore statement.
    
    The backup and restore api has changed as following:
    ```
    BACKUP SNAPSHOT [db_name].{snapshot_name}
    TO 'repo_name'
    [ON|EXCLUDE (
        'table_name' [partition (p1,...)]
    )]
    [properties (
        "content" = "metadata_only|all"
    )]
    
    RESTORE SNAPSHOT [db_name].{snapshot_name}
    TO 'repo_name'
    [EXCLUDE|ON (
        'table_name' [partition (p1,...)]
    )]
    [properties (
    )]
    ```
---
 fe/fe-core/src/main/cup/sql_parser.cup             |  37 ++++++-
 .../apache/doris/analysis/AbstractBackupStmt.java  |  64 +++++-------
 .../analysis/AbstractBackupTableRefClause.java     |  87 +++++++++++++++++
 .../java/org/apache/doris/analysis/BackupStmt.java |  60 ++++++++----
 .../org/apache/doris/analysis/RestoreStmt.java     |  41 ++++----
 .../org/apache/doris/backup/BackupHandler.java     |  62 ++++++++++--
 .../java/org/apache/doris/backup/BackupJob.java    |  61 ++++++++----
 .../org/apache/doris/backup/BackupJobInfo.java     | 108 ++++++++++++++++++++-
 .../java/org/apache/doris/backup/RestoreJob.java   |  25 +++--
 fe/fe-core/src/main/jflex/sql_scanner.flex         |   1 +
 .../org/apache/doris/backup/BackupHandlerTest.java |  29 +++---
 .../org/apache/doris/backup/BackupJobTest.java     |  18 ++--
 12 files changed, 445 insertions(+), 148 deletions(-)

diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup
index 62d363e..1f43b43 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -235,7 +235,8 @@ terminal String KW_ADD, KW_ADMIN, KW_AFTER, KW_AGGREGATE, KW_ALL, KW_ALTER, KW_A
     KW_CONFIG, KW_CONNECTION, KW_CONNECTION_ID, KW_CONSISTENT, KW_CONVERT, KW_COUNT, KW_CREATE, KW_CROSS, KW_CUBE, KW_CURRENT, KW_CURRENT_USER,
     KW_DATA, KW_DATABASE, KW_DATABASES, KW_DATE, KW_DATETIME, KW_DAY, KW_DECIMAL, KW_DECOMMISSION, KW_DEFAULT, KW_DESC, KW_DESCRIBE,
     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_ENABLE, KW_END, KW_ENGINE, KW_ENGINES, KW_ENTER, KW_ERRORS, KW_EVENTS, KW_EXCEPT, KW_EXISTS, KW_EXPORT, KW_EXTERNAL, KW_EXTRACT,
+    KW_ELSE, KW_ENABLE, KW_END, KW_ENGINE, KW_ENGINES, KW_ENTER, KW_ERRORS, KW_EVENTS, KW_EXCEPT, KW_EXCLUDE,
+    KW_EXISTS, KW_EXPORT, KW_EXTERNAL, KW_EXTRACT,
     KW_FALSE, KW_FEATURE, 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,
@@ -326,6 +327,8 @@ nonterminal InsertTarget insert_target;
 nonterminal InsertSource insert_source;
 
 nonterminal BackupStmt backup_stmt;
+nonterminal AbstractBackupTableRefClause opt_backup_table_ref_list;
+nonterminal Boolean backup_exclude_or_not;
 nonterminal RestoreStmt restore_stmt;
 
 nonterminal SelectList select_clause, select_list, select_sublist;
@@ -2943,10 +2946,32 @@ insert_source ::=
 backup_stmt ::=
     KW_BACKUP KW_SNAPSHOT job_label:label
     KW_TO ident:repoName
-    KW_ON LPAREN base_table_ref_list:tbls RPAREN
+    opt_backup_table_ref_list:tblRefClause
     opt_properties:properties
     {:
-        RESULT = new BackupStmt(label, repoName, tbls, properties);
+        RESULT = new BackupStmt(label, repoName, tblRefClause, properties);
+    :}
+    ;
+
+opt_backup_table_ref_list ::=
+    backup_exclude_or_not:isExclude LPAREN base_table_ref_list:tbls RPAREN
+    {:
+        RESULT = new AbstractBackupTableRefClause(isExclude, tbls);
+    :}
+    | /* empty */
+    {:
+        RESULT = null;
+    :}
+    ;
+
+backup_exclude_or_not ::=
+    KW_ON
+    {:
+        RESULT = false;
+    :}
+    | KW_EXCLUDE
+    {:
+        RESULT = true;
     :}
     ;
 
@@ -2954,10 +2979,10 @@ backup_stmt ::=
 restore_stmt ::=
     KW_RESTORE KW_SNAPSHOT job_label:label
     KW_FROM ident:repoName
-    KW_ON LPAREN base_table_ref_list:tbls RPAREN
+    opt_backup_table_ref_list:tblRefClause
     opt_properties:properties
     {:
-        RESULT = new RestoreStmt(label, repoName, tbls, properties);
+        RESULT = new RestoreStmt(label, repoName, tblRefClause, properties);
     :}
     ;
 
@@ -4677,6 +4702,8 @@ keyword ::=
     {: RESULT = id; :}
     | KW_ERRORS:id
     {: RESULT = id; :}
+    | KW_EXCLUDE:id
+    {: RESULT = id; :}
     | KW_EVENTS:id
     {: RESULT = id; :}
     | KW_EXTERNAL:id
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AbstractBackupStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AbstractBackupStmt.java
index 383dc08..d6a54fa 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AbstractBackupStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AbstractBackupStmt.java
@@ -26,15 +26,12 @@ import org.apache.doris.common.UserException;
 import org.apache.doris.mysql.privilege.PrivPredicate;
 import org.apache.doris.qe.ConnectContext;
 
-import com.google.common.base.Joiner;
 import com.google.common.base.Strings;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
-import java.util.List;
 import java.util.Map;
 
 public class AbstractBackupStmt extends DdlStmt {
@@ -45,27 +42,23 @@ public class AbstractBackupStmt extends DdlStmt {
 
     protected LabelName labelName;
     protected String repoName;
-    protected List<TableRef> tblRefs;
+    protected AbstractBackupTableRefClause abstractBackupTableRefClause;
     protected Map<String, String> properties;
 
     protected long timeoutMs;
 
-    public AbstractBackupStmt(LabelName labelName, String repoName, List<TableRef> tableRefs,
-            Map<String, String> properties) {
+    public AbstractBackupStmt(LabelName labelName, String repoName, AbstractBackupTableRefClause abstractBackupTableRefClause,
+                              Map<String, String> properties) {
         this.labelName = labelName;
         this.repoName = repoName;
-        this.tblRefs = tableRefs;
-        if (this.tblRefs == null) {
-            this.tblRefs = Lists.newArrayList();
-        }
-
+        this.abstractBackupTableRefClause = abstractBackupTableRefClause;
         this.properties = properties == null ? Maps.newHashMap() : properties;
     }
 
     @Override
-    public void analyze(Analyzer analyzer) throws AnalysisException, UserException {
+    public void analyze(Analyzer analyzer) throws UserException {
         labelName.analyze(analyzer);
-        
+
         // user need database level privilege(not table level), because when doing restore operation,
         // the restore table may be newly created, so we can not judge its privileges.
         if (!Catalog.getCurrentCatalog().getAuth().checkDbPriv(ConnectContext.get(),
@@ -73,13 +66,21 @@ public class AbstractBackupStmt extends DdlStmt {
             ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "LOAD");
         }
 
-        checkAndNormalizeBackupObjs();
-
+        analyzeTableRefClause();
         analyzeProperties();
     }
 
-    private void checkAndNormalizeBackupObjs() throws AnalysisException {
-        for (TableRef tblRef : tblRefs) {
+    private void analyzeTableRefClause() throws UserException {
+        if (abstractBackupTableRefClause == null) {
+            return;
+        }
+        checkTableRefWithoutDatabase();
+        abstractBackupTableRefClause.analyze(analyzer);
+        customAnalyzeTableRefClause();
+    }
+
+    private void checkTableRefWithoutDatabase() throws AnalysisException {
+        for (TableRef tblRef : abstractBackupTableRefClause.getTableRefList()) {
             if (!Strings.isNullOrEmpty(tblRef.getName().getDb())) {
                 throw new AnalysisException("Cannot specify database name on backup objects: "
                         + tblRef.getName().getTbl() + ". Specify database name before label");
@@ -87,27 +88,9 @@ public class AbstractBackupStmt extends DdlStmt {
             // set db name because we can not persist empty string when writing bdbje log
             tblRef.getName().setDb(labelName.getDbName());
         }
-        
-        // normalize
-        // table name => table ref
-        Map<String, TableRef> tblPartsMap = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
-        for (TableRef tblRef : tblRefs) {
-            String tblName = tblRef.getName().getTbl();
-
-            if (!tblPartsMap.containsKey(tblName)) {
-                tblPartsMap.put(tblName, tblRef);
-            } else {
-                throw new AnalysisException("Duplicated restore table: " + tblName);
-            }
-        }
-        
-        // update table ref
-        tblRefs.clear();
-        for (TableRef tableRef : tblPartsMap.values()) {
-            tblRefs.add(tableRef);
-        }
+    }
 
-        LOG.debug("table refs after normalization: \n{}", Joiner.on("\n").join(tblRefs));
+    protected void customAnalyzeTableRefClause() throws AnalysisException {
     }
 
     protected void analyzeProperties() throws AnalysisException {
@@ -117,8 +100,7 @@ public class AbstractBackupStmt extends DdlStmt {
                 timeoutMs = Long.valueOf(properties.get(PROP_TIMEOUT));
             } catch (NumberFormatException e) {
                 ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR,
-                                                    "Invalid timeout format: "
-                                                            + properties.get(PROP_TIMEOUT));
+                        "Invalid timeout format: " + properties.get(PROP_TIMEOUT));
             }
 
             if (timeoutMs * 1000 < MIN_TIMEOUT_MS) {
@@ -148,8 +130,8 @@ public class AbstractBackupStmt extends DdlStmt {
         return repoName;
     }
 
-    public List<TableRef> getTableRefs() {
-        return tblRefs;
+    public AbstractBackupTableRefClause getAbstractBackupTableRefClause() {
+        return abstractBackupTableRefClause;
     }
 
     public Map<String, String> getProperties() {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AbstractBackupTableRefClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AbstractBackupTableRefClause.java
new file mode 100644
index 0000000..d3e8bac
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AbstractBackupTableRefClause.java
@@ -0,0 +1,87 @@
+// 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.common.AnalysisException;
+import org.apache.doris.common.UserException;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Maps;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.List;
+import java.util.Map;
+
+public class AbstractBackupTableRefClause implements ParseNode {
+    private static final Logger LOG = LogManager.getLogger(AbstractBackupTableRefClause.class);
+
+    private boolean isExclude;
+    private List<TableRef> tableRefList;
+
+    public AbstractBackupTableRefClause(boolean isExclude, List<TableRef> tableRefList) {
+        this.isExclude = isExclude;
+        this.tableRefList = tableRefList;
+    }
+
+    @Override
+    public void analyze(Analyzer analyzer) throws UserException {
+        // normalize
+        // table name => table ref
+        Map<String, TableRef> tblPartsMap = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
+        for (TableRef tblRef : tableRefList) {
+            String tblName = tblRef.getName().getTbl();
+            if (!tblPartsMap.containsKey(tblName)) {
+                tblPartsMap.put(tblName, tblRef);
+            } else {
+                throw new AnalysisException("Duplicated restore table: " + tblName);
+            }
+        }
+
+        // update table ref
+        tableRefList.clear();
+        for (TableRef tableRef : tblPartsMap.values()) {
+            tableRefList.add(tableRef);
+        }
+
+        LOG.debug("table refs after normalization: {}", Joiner.on(",").join(tableRefList));
+    }
+
+    public boolean isExclude() {
+        return isExclude;
+    }
+
+    public List<TableRef> getTableRefList() {
+        return tableRefList;
+    }
+
+    @Override
+    public String toSql() {
+        StringBuilder sb = new StringBuilder();
+        if (isExclude) {
+            sb.append("EXCLUDE ");
+        } else {
+            sb.append("ON ");
+        }
+        sb.append("\n(");
+        sb.append(Joiner.on(",\n").join(tableRefList));
+        sb.append("\n)");
+        return sb.toString();
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/BackupStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/BackupStmt.java
index 21bd9aa..89709b9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BackupStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/BackupStmt.java
@@ -23,23 +23,29 @@ import org.apache.doris.common.ErrorReport;
 import org.apache.doris.common.UserException;
 import org.apache.doris.common.util.PrintableMap;
 
-import com.google.common.base.Joiner;
 import com.google.common.collect.Maps;
 
-import java.util.List;
 import java.util.Map;
 
 public class BackupStmt extends AbstractBackupStmt {
     private final static String PROP_TYPE = "type";
+    public final static String PROP_CONTENT = "content";
 
     public enum BackupType {
         INCREMENTAL, FULL
     }
 
+    public enum BackupContent {
+        METADATA_ONLY, ALL
+    }
+
     private BackupType type = BackupType.FULL;
+    private BackupContent content = BackupContent.ALL;
 
-    public BackupStmt(LabelName labelName, String repoName, List<TableRef> tblRefs, Map<String, String> properties) {
-        super(labelName, repoName, tblRefs, properties);
+
+    public BackupStmt(LabelName labelName, String repoName, AbstractBackupTableRefClause abstractBackupTableRefClause,
+                      Map<String, String> properties) {
+        super(labelName, repoName, abstractBackupTableRefClause, properties);
     }
 
     public long getTimeoutMs() {
@@ -50,12 +56,19 @@ public class BackupStmt extends AbstractBackupStmt {
         return type;
     }
 
+    public BackupContent getContent() {
+        return content;
+    }
+
     @Override
     public void analyze(Analyzer analyzer) throws UserException {
         super.analyze(analyzer);
+    }
 
+    @Override
+    protected void customAnalyzeTableRefClause() throws AnalysisException {
         // tbl refs can not set alias in backup
-        for (TableRef tblRef : tblRefs) {
+        for (TableRef tblRef : abstractBackupTableRefClause.getTableRefList()) {
             if (tblRef.hasExplicitAlias()) {
                 throw new AnalysisException("Can not set alias for table in Backup Stmt: " + tblRef);
             }
@@ -65,23 +78,34 @@ public class BackupStmt extends AbstractBackupStmt {
     @Override
     protected void analyzeProperties() throws AnalysisException {
         super.analyzeProperties();
-        
+
         Map<String, String> copiedProperties = Maps.newHashMap(properties);
         // type
-        if (copiedProperties.containsKey(PROP_TYPE)) {
+        String typeProp = copiedProperties.get(PROP_TYPE);
+        if (typeProp != null) {
             try {
-                type = BackupType.valueOf(copiedProperties.get(PROP_TYPE).toUpperCase());
-            } catch (Exception e) { 
+                type = BackupType.valueOf(typeProp.toUpperCase());
+            } catch (Exception e) {
                 ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR,
-                                                    "Invalid backup job type: "
-                                                            + copiedProperties.get(PROP_TYPE));
+                        "Invalid backup job type: " + typeProp);
             }
             copiedProperties.remove(PROP_TYPE);
         }
+        // content
+        String contentProp = copiedProperties.get(PROP_CONTENT);
+        if (contentProp != null) {
+            try {
+                content = BackupContent.valueOf(contentProp.toUpperCase());
+            } catch (IllegalArgumentException e) {
+                ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR,
+                        "Invalid backup job content:" + contentProp);
+            }
+            copiedProperties.remove(PROP_CONTENT);
+        }
 
         if (!copiedProperties.isEmpty()) {
             ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR,
-                                                "Unknown backup job properties: " + copiedProperties.keySet());
+                    "Unknown backup job properties: " + copiedProperties.keySet());
         }
     }
 
@@ -94,12 +118,12 @@ public class BackupStmt extends AbstractBackupStmt {
     public String toSql() {
         StringBuilder sb = new StringBuilder();
         sb.append("BACKUP SNAPSHOT ").append(labelName.toSql());
-        sb.append("\n").append("TO ").append(repoName).append("\nON\n(");
-
-        sb.append(Joiner.on(",\n").join(tblRefs));
-
-        sb.append("\n)\nPROPERTIES\n(");
-        sb.append(new PrintableMap<String, String>(properties, " = ", true, true));
+        sb.append("\n").append("TO ").append(repoName).append("\n");
+        if (abstractBackupTableRefClause != null) {
+            sb.append(abstractBackupTableRefClause.toSql()).append("\n");
+        }
+        sb.append("PROPERTIES\n(");
+        sb.append(new PrintableMap<>(properties, " = ", true, true));
         sb.append("\n)");
         return sb.toString();
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java
index f77562e..a397d44 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java
@@ -24,11 +24,9 @@ import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.UserException;
 import org.apache.doris.common.util.PrintableMap;
 
-import com.google.common.base.Joiner;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
-import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
@@ -43,14 +41,15 @@ public class RestoreStmt extends AbstractBackupStmt {
     private String backupTimestamp = null;
     private int metaVersion = -1;
 
-    public RestoreStmt(LabelName labelName, String repoName, List<TableRef> tblRefs, Map<String, String> properties) {
-        super(labelName, repoName, tblRefs, properties);
+    public RestoreStmt(LabelName labelName, String repoName, AbstractBackupTableRefClause restoreTableRefClause,
+                       Map<String, String> properties) {
+        super(labelName, repoName, restoreTableRefClause, properties);
     }
-    
+
     public boolean allowLoad() {
         return allowLoad;
     }
-    
+
     public int getReplicationNum() {
         return replicationNum;
     }
@@ -66,14 +65,17 @@ public class RestoreStmt extends AbstractBackupStmt {
     @Override
     public void analyze(Analyzer analyzer) throws UserException {
         super.analyze(analyzer);
+    }
 
+    @Override
+    protected void customAnalyzeTableRefClause() throws AnalysisException {
         // check if alias is duplicated
         Set<String> aliasSet = Sets.newHashSet();
-        for (TableRef tblRef : tblRefs) {
+        for (TableRef tblRef : abstractBackupTableRefClause.getTableRefList()) {
             aliasSet.add(tblRef.getName().getTbl());
         }
 
-        for (TableRef tblRef : tblRefs) {
+        for (TableRef tblRef : abstractBackupTableRefClause.getTableRefList()) {
             if (tblRef.hasExplicitAlias() && !aliasSet.add(tblRef.getExplicitAlias())) {
                 throw new AnalysisException("Duplicated alias name: " + tblRef.getExplicitAlias());
             }
@@ -93,8 +95,7 @@ public class RestoreStmt extends AbstractBackupStmt {
                 allowLoad = false;
             } else {
                 ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR,
-                                                    "Invalid allow load value: "
-                                                            + copiedProperties.get(PROP_ALLOW_LOAD));
+                        "Invalid allow load value: " + copiedProperties.get(PROP_ALLOW_LOAD));
             }
             copiedProperties.remove(PROP_ALLOW_LOAD);
         }
@@ -105,8 +106,7 @@ public class RestoreStmt extends AbstractBackupStmt {
                 replicationNum = Integer.valueOf(copiedProperties.get(PROP_REPLICATION_NUM));
             } catch (NumberFormatException e) {
                 ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR,
-                                                    "Invalid replication num format: "
-                                                            + copiedProperties.get(PROP_REPLICATION_NUM));
+                        "Invalid replication num format: " + copiedProperties.get(PROP_REPLICATION_NUM));
             }
             copiedProperties.remove(PROP_REPLICATION_NUM);
         }
@@ -117,7 +117,7 @@ public class RestoreStmt extends AbstractBackupStmt {
             copiedProperties.remove(PROP_BACKUP_TIMESTAMP);
         } else {
             ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR,
-                                                "Missing " + PROP_BACKUP_TIMESTAMP + " property");
+                    "Missing " + PROP_BACKUP_TIMESTAMP + " property");
         }
 
         // meta version
@@ -126,15 +126,14 @@ public class RestoreStmt extends AbstractBackupStmt {
                 metaVersion = Integer.valueOf(copiedProperties.get(PROP_META_VERSION));
             } catch (NumberFormatException e) {
                 ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR,
-                        "Invalid meta version format: "
-                                + copiedProperties.get(PROP_META_VERSION));
+                        "Invalid meta version format: " + copiedProperties.get(PROP_META_VERSION));
             }
             copiedProperties.remove(PROP_META_VERSION);
         }
 
         if (!copiedProperties.isEmpty()) {
             ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR,
-                                                "Unknown restore job properties: " + copiedProperties.keySet());
+                    "Unknown restore job properties: " + copiedProperties.keySet());
         }
     }
 
@@ -147,11 +146,11 @@ public class RestoreStmt extends AbstractBackupStmt {
     public String toSql() {
         StringBuilder sb = new StringBuilder();
         sb.append("RESTORE SNAPSHOT ").append(labelName.toSql());
-        sb.append("\n").append("FROM ").append(repoName).append("\nON\n(");
-
-        sb.append(Joiner.on(",\n").join(tblRefs));
-
-        sb.append("\n)\nPROPERTIES\n(");
+        sb.append("\n").append("FROM ").append(repoName).append("\n");
+        if (abstractBackupTableRefClause != null) {
+            sb.append(abstractBackupTableRefClause.toSql()).append("\n");
+        }
+        sb.append("PROPERTIES\n(");
         sb.append(new PrintableMap<String, String>(properties, " = ", true, true));
         sb.append("\n)");
         return sb.toString();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java
index d454ce0..77c111b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java
@@ -18,6 +18,7 @@
 package org.apache.doris.backup;
 
 import org.apache.doris.analysis.AbstractBackupStmt;
+import org.apache.doris.analysis.AbstractBackupTableRefClause;
 import org.apache.doris.analysis.BackupStmt;
 import org.apache.doris.analysis.BackupStmt.BackupType;
 import org.apache.doris.analysis.CancelBackupStmt;
@@ -25,6 +26,7 @@ import org.apache.doris.analysis.CreateRepositoryStmt;
 import org.apache.doris.analysis.DropRepositoryStmt;
 import org.apache.doris.analysis.PartitionNames;
 import org.apache.doris.analysis.RestoreStmt;
+import org.apache.doris.analysis.TableName;
 import org.apache.doris.analysis.TableRef;
 import org.apache.doris.backup.AbstractJob.JobType;
 import org.apache.doris.backup.BackupJob.BackupJobState;
@@ -269,10 +271,33 @@ public class BackupHandler extends MasterDaemon implements Writable {
                     + " is read only");
         }
 
+        // Determine the tables to be backed up
+        Set<String> tableNames = Sets.newHashSet();
+        AbstractBackupTableRefClause abstractBackupTableRefClause = stmt.getAbstractBackupTableRefClause();
+        if (abstractBackupTableRefClause == null) {
+            tableNames = db.getTableNamesWithLock();
+        } else if (abstractBackupTableRefClause.isExclude()) {
+            tableNames = db.getTableNamesWithLock();
+            for (TableRef tableRef : abstractBackupTableRefClause.getTableRefList()) {
+                if (!tableNames.remove(tableRef.getName().getTbl())) {
+                    LOG.info("exclude table " + tableRef.getName().getTbl()
+                            + " of backup stmt is not exists in db " + db.getFullName());
+                }
+            }
+        }
+        List<TableRef> tblRefs = Lists.newArrayList();
+        if (abstractBackupTableRefClause != null && !abstractBackupTableRefClause.isExclude()) {
+            tblRefs = abstractBackupTableRefClause.getTableRefList();
+        } else {
+            for (String tableName : tableNames) {
+                TableRef tableRef = new TableRef(new TableName(db.getFullName(), tableName), null);
+                tblRefs.add(tableRef);
+            }
+        }
+
         // Check if backup objects are valid
         // This is just a pre-check to avoid most of invalid backup requests.
         // Also calculate the signature for incremental backup check.
-        List<TableRef> tblRefs = stmt.getTableRefs();
         for (TableRef tblRef : tblRefs) {
             String tblName = tblRef.getName().getTbl();
             Table tbl = db.getTable(tblName);
@@ -331,7 +356,7 @@ public class BackupHandler extends MasterDaemon implements Writable {
         // Create a backup job
         BackupJob backupJob = new BackupJob(stmt.getLabel(), db.getId(),
                 ClusterNamespace.getNameFromFullName(db.getFullName()),
-                tblRefs, stmt.getTimeoutMs(),
+                tblRefs, stmt.getTimeoutMs(), stmt.getContent(),
                 catalog, repository.getId());
         // write log
         catalog.getEditLog().logBackupJob(backupJob);
@@ -356,7 +381,7 @@ public class BackupHandler extends MasterDaemon implements Writable {
         // Also remove all unrelated objs
         Preconditions.checkState(infos.size() == 1);
         BackupJobInfo jobInfo = infos.get(0);
-        checkAndFilterRestoreObjsExistInSnapshot(jobInfo, stmt.getTableRefs());
+        checkAndFilterRestoreObjsExistInSnapshot(jobInfo, stmt.getAbstractBackupTableRefClause());
 
         // Create a restore job
         RestoreJob restoreJob = new RestoreJob(stmt.getLabel(), stmt.getBackupTimestamp(),
@@ -370,12 +395,37 @@ public class BackupHandler extends MasterDaemon implements Writable {
         LOG.info("finished to submit restore job: {}", restoreJob);
     }
 
-    private void checkAndFilterRestoreObjsExistInSnapshot(BackupJobInfo jobInfo, List<TableRef> tblRefs)
+    private void checkAndFilterRestoreObjsExistInSnapshot(BackupJobInfo jobInfo,
+                                                          AbstractBackupTableRefClause backupTableRefClause)
             throws DdlException {
+        // case1: all table in job info
+        if (backupTableRefClause == null) {
+            return;
+        }
+
+        // case2: exclude table ref
+        if (backupTableRefClause.isExclude()) {
+            for (TableRef tblRef : backupTableRefClause.getTableRefList()) {
+                String tblName = tblRef.getName().getTbl();
+                TableType tableType = jobInfo.getTypeByTblName(tblName);
+                if (tableType == null) {
+                    LOG.info("Ignore error : exclude table " + tblName + " does not exist in snapshot "
+                            + jobInfo.name);
+                    continue;
+                }
+                if (tblRef.hasExplicitAlias()) {
+                    ErrorReport.reportDdlException(ErrorCode.ERR_COMMON_ERROR,
+                            "The table alias in exclude clause does not make sense");
+                }
+                jobInfo.removeTable(tblRef, tableType);
+            }
+            return;
+        }
+        // case3: include table ref
         Set<String> olapTableNames = Sets.newHashSet();
         Set<String> viewNames = Sets.newHashSet();
         Set<String> odbcTableNames = Sets.newHashSet();
-        for (TableRef tblRef : tblRefs) {
+        for (TableRef tblRef : backupTableRefClause.getTableRefList()) {
             String tblName = tblRef.getName().getTbl();
             TableType tableType = jobInfo.getTypeByTblName(tblName);
             if (tableType == null) {
@@ -402,8 +452,6 @@ public class BackupHandler extends MasterDaemon implements Writable {
                 jobInfo.setAlias(tblName, tblRef.getExplicitAlias());
             }
         }
-
-        // only retain restore tables
         jobInfo.retainOlapTables(olapTableNames);
         jobInfo.retainView(viewNames);
         jobInfo.retainOdbcTables(odbcTableNames);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java
index dd96fa8..cbb6c8a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java
@@ -17,6 +17,8 @@
 
 package org.apache.doris.backup;
 
+import org.apache.doris.analysis.BackupStmt;
+import org.apache.doris.analysis.BackupStmt.BackupContent;
 import org.apache.doris.analysis.TableRef;
 import org.apache.doris.backup.Status.ErrCode;
 import org.apache.doris.catalog.Catalog;
@@ -91,11 +93,12 @@ public class BackupJob extends AbstractJob {
 
     // all objects which need backup
     private List<TableRef> tableRefs = Lists.newArrayList();
+//    private BackupContent content = BackupContent.ALL;
 
     private BackupJobState state;
 
     private long snapshotFinishedTime = -1;
-    private long snapshopUploadFinishedTime = -1;
+    private long snapshotUploadFinishedTime = -1;
 
     // save task id map to the backend it be executed
     private Map<Long, Long> unfinishedTaskIds = Maps.newConcurrentMap();
@@ -120,10 +123,11 @@ public class BackupJob extends AbstractJob {
     }
 
     public BackupJob(String label, long dbId, String dbName, List<TableRef> tableRefs, long timeoutMs,
-                     Catalog catalog, long repoId) {
+                     BackupContent content, Catalog catalog, long repoId) {
         super(JobType.BACKUP, label, dbId, dbName, timeoutMs, catalog, repoId);
         this.tableRefs = tableRefs;
         this.state = BackupJobState.PENDING;
+        properties.put(BackupStmt.PROP_CONTENT, content.name());
     }
 
     public BackupJobState getState() {
@@ -146,6 +150,13 @@ public class BackupJob extends AbstractJob {
         return localMetaInfoFilePath;
     }
 
+    public BackupContent getContent() {
+        if (properties.containsKey(BackupStmt.PROP_CONTENT)) {
+            return BackupStmt.BackupContent.valueOf(properties.get(BackupStmt.PROP_CONTENT).toUpperCase());
+        }
+        return BackupContent.ALL;
+    }
+
     public synchronized boolean finishTabletSnapshotTask(SnapshotTask task, TFinishTaskRequest request) {
         Preconditions.checkState(task.getJobId() == jobId);
         
@@ -360,7 +371,10 @@ public class BackupJob extends AbstractJob {
             }
             switch (tbl.getType()){
                 case OLAP:
-                    prepareAndSendSnapshotTaskForOlapTable((OlapTable) tbl, tableRef, batchTask);
+                    checkOlapTable((OlapTable) tbl, tableRef);
+                    if (getContent() == BackupContent.ALL) {
+                        prepareSnapshotTaskForOlapTable((OlapTable) tbl, tableRef, batchTask);
+                    }
                     break;
                 case VIEW:
                     break;
@@ -400,18 +414,26 @@ public class BackupJob extends AbstractJob {
         LOG.info("finished to send snapshot tasks to backend. {}", this);
     }
 
-    private void prepareAndSendSnapshotTaskForOlapTable(OlapTable olapTable, TableRef backupTableRef, AgentBatchTask batchTask) {
-        String tblName = backupTableRef.getName().getTbl();
-        if (backupTableRef.getPartitionNames() != null) {
-            for (String partName : backupTableRef.getPartitionNames().getPartitionNames()) {
-                Partition partition = olapTable.getPartition(partName);
-                if (partition == null) {
-                    status = new Status(ErrCode.NOT_FOUND, "partition " + partName
-                            + " does not exist  in table" + tblName);
-                    return;
+    private void checkOlapTable(OlapTable olapTable, TableRef backupTableRef) {
+        olapTable.readLock();
+        try {
+            // check backup table again
+            if (backupTableRef.getPartitionNames() != null) {
+                for (String partName : backupTableRef.getPartitionNames().getPartitionNames()) {
+                    Partition partition = olapTable.getPartition(partName);
+                    if (partition == null) {
+                        status = new Status(ErrCode.NOT_FOUND, "partition " + partName
+                                + " does not exist  in table" + backupTableRef.getName().getTbl());
+                        return;
+                    }
                 }
             }
+        }  finally {
+            olapTable.readUnlock();
         }
+    }
+
+    private void prepareSnapshotTaskForOlapTable(OlapTable olapTable, TableRef backupTableRef, AgentBatchTask batchTask) {
         olapTable.readLock();
         try {
             // check backup table again
@@ -420,7 +442,7 @@ public class BackupJob extends AbstractJob {
                     Partition partition = olapTable.getPartition(partName);
                     if (partition == null) {
                         status = new Status(ErrCode.NOT_FOUND, "partition " + partName
-                                + " does not exist  in table" + tblName);
+                                + " does not exist  in table" + backupTableRef.getName().getTbl());
                         return;
                     }
                 }
@@ -520,7 +542,7 @@ public class BackupJob extends AbstractJob {
                         copiedResources.add(copiedResource);
                     }
                 }
-            }finally {
+            } finally {
                 table.readUnlock();
             }
         }
@@ -605,7 +627,7 @@ public class BackupJob extends AbstractJob {
 
     private void waitingAllUploadingFinished() {
         if (unfinishedTaskIds.isEmpty()) {
-            snapshopUploadFinishedTime = System.currentTimeMillis();
+            snapshotUploadFinishedTime = System.currentTimeMillis();
             state = BackupJobState.SAVE_META;
 
             // log
@@ -648,8 +670,7 @@ public class BackupJob extends AbstractJob {
             localMetaInfoFilePath = metaInfoFile.getAbsolutePath();
 
             // 3. save job info file
-            jobInfo = BackupJobInfo.fromCatalog(createTime, label, dbName, dbId, backupMeta,
-                                                snapshotInfos);
+            jobInfo = BackupJobInfo.fromCatalog(createTime, label, dbName, dbId, getContent(), backupMeta, snapshotInfos);
             LOG.debug("job info: {}. {}", jobInfo, this);
             File jobInfoFile = new File(jobDir, Repository.PREFIX_JOB_INFO + createTimeStr);
             if (!jobInfoFile.createNewFile()) {
@@ -809,7 +830,7 @@ public class BackupJob extends AbstractJob {
         info.add(getBackupObjs());
         info.add(TimeUtils.longToTimeString(createTime));
         info.add(TimeUtils.longToTimeString(snapshotFinishedTime));
-        info.add(TimeUtils.longToTimeString(snapshopUploadFinishedTime));
+        info.add(TimeUtils.longToTimeString(snapshotUploadFinishedTime));
         info.add(TimeUtils.longToTimeString(finishedTime));
         info.add(Joiner.on(", ").join(unfinishedTaskIds.entrySet()));
         info.add(Joiner.on(", ").join(taskProgress.entrySet().stream().map(
@@ -848,7 +869,7 @@ public class BackupJob extends AbstractJob {
 
         // times
         out.writeLong(snapshotFinishedTime);
-        out.writeLong(snapshopUploadFinishedTime);
+        out.writeLong(snapshotUploadFinishedTime);
 
         // snapshot info
         out.writeInt(snapshotInfos.size());
@@ -905,7 +926,7 @@ public class BackupJob extends AbstractJob {
 
         // times
         snapshotFinishedTime = in.readLong();
-        snapshopUploadFinishedTime = in.readLong();
+        snapshotUploadFinishedTime = in.readLong();
 
         // snapshot info
         size = in.readInt();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJobInfo.java b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJobInfo.java
index b2dff76..072f01a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJobInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJobInfo.java
@@ -17,6 +17,9 @@
 
 package org.apache.doris.backup;
 
+import org.apache.doris.analysis.BackupStmt.BackupContent;
+import org.apache.doris.analysis.PartitionNames;
+import org.apache.doris.analysis.TableRef;
 import org.apache.doris.backup.RestoreFileMapping.IdChain;
 import org.apache.doris.catalog.MaterializedIndex;
 import org.apache.doris.catalog.MaterializedIndex.IndexExtState;
@@ -26,6 +29,7 @@ import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
 import org.apache.doris.catalog.Resource;
 import org.apache.doris.catalog.Table;
+import org.apache.doris.catalog.Table.TableType;
 import org.apache.doris.catalog.Tablet;
 import org.apache.doris.catalog.View;
 import org.apache.doris.common.FeConstants;
@@ -75,6 +79,8 @@ public class BackupJobInfo implements Writable {
     public long dbId;
     @SerializedName("backup_time")
     public long backupTime;
+    @SerializedName("content")
+    public BackupContent content;
     // only include olap table
     @SerializedName("backup_objects")
     public Map<String, BackupOlapTableInfo> backupOlapTableObjects = Maps.newHashMap();
@@ -146,6 +152,74 @@ public class BackupJobInfo implements Writable {
         return backupOlapTableObjects.get(tblName);
     }
 
+    public void removeTable(TableRef tableRef, TableType tableType) {
+        switch (tableType) {
+            case OLAP:
+                removeOlapTable(tableRef);
+                break;
+            case VIEW:
+                removeView(tableRef);
+                break;
+            case ODBC:
+                removeOdbcTable(tableRef);
+                break;
+            default:
+                break;
+        }
+    }
+
+    public void removeOlapTable(TableRef tableRef) {
+        String tblName = tableRef.getName().getTbl();
+        BackupOlapTableInfo tblInfo = backupOlapTableObjects.get(tblName);
+        if (tblInfo == null) {
+            LOG.info("Ignore error: exclude table " + tblName + " does not exist in snapshot " + name);
+            return;
+        }
+        PartitionNames partitionNames = tableRef.getPartitionNames();
+        if (partitionNames == null) {
+            backupOlapTableObjects.remove(tblInfo);
+            return;
+        }
+        // check the selected partitions
+        for (String partName : partitionNames.getPartitionNames()) {
+            if (tblInfo.containsPart(partName)) {
+                tblInfo.partitions.remove(partName);
+            } else {
+                LOG.info("Ignore error: exclude partition " + partName + " of table " + tblName
+                        + " does not exist in snapshot");
+            }
+        }
+    }
+
+    public void removeView(TableRef tableRef) {
+        Iterator<BackupViewInfo> iter = newBackupObjects.views.listIterator();
+        while (iter.hasNext()) {
+            if (iter.next().name.equals(tableRef.getName().getTbl())) {
+                iter.remove();
+                return;
+            }
+        }
+    }
+
+    public void removeOdbcTable(TableRef tableRef) {
+        Iterator<BackupOdbcTableInfo> iter = newBackupObjects.odbcTables.listIterator();
+        while (iter.hasNext()) {
+            BackupOdbcTableInfo backupOdbcTableInfo = iter.next();
+            if (backupOdbcTableInfo.dorisTableName.equals(tableRef.getName().getTbl())) {
+                if (backupOdbcTableInfo.resourceName != null) {
+                    Iterator<BackupOdbcResourceInfo> resourceIter = newBackupObjects.odbcResources.listIterator();
+                    while (resourceIter.hasNext()) {
+                        if (resourceIter.next().name.equals(backupOdbcTableInfo.resourceName)) {
+                            resourceIter.remove();
+                        }
+                    }
+                }
+                iter.remove();
+                return;
+            }
+        }
+    }
+
     public void retainOlapTables(Set<String> tblNames) {
         Iterator<Map.Entry<String, BackupOlapTableInfo>> iter = backupOlapTableObjects.entrySet().iterator();
         while (iter.hasNext()) {
@@ -204,6 +278,7 @@ public class BackupJobInfo implements Writable {
         public String database;
         @SerializedName("backup_time")
         public long backupTime;
+        public BackupContent content;
         @SerializedName("olap_table_list")
         public List<BriefBackupOlapTable> olapTableList = Lists.newArrayList();
         @SerializedName("view_list")
@@ -218,6 +293,7 @@ public class BackupJobInfo implements Writable {
             briefBackupJobInfo.name = backupJobInfo.name;
             briefBackupJobInfo.database = backupJobInfo.dbName;
             briefBackupJobInfo.backupTime = backupJobInfo.backupTime;
+            briefBackupJobInfo.content = backupJobInfo.content;
             for (Map.Entry<String, BackupOlapTableInfo> olapTableEntry :
                     backupJobInfo.backupOlapTableObjects.entrySet()) {
                 BriefBackupOlapTable briefBackupOlapTable = new BriefBackupOlapTable();
@@ -336,6 +412,12 @@ public class BackupJobInfo implements Writable {
         public String linkedOdbcTableName;
         @SerializedName("resource_name")
         public String resourceName;
+        public String host;
+        public String port;
+        public String user;
+        public String driver;
+        @SerializedName("odbc_type")
+        public String odbcType;
     }
 
     public static class BackupOdbcResourceInfo {
@@ -390,7 +472,8 @@ public class BackupJobInfo implements Writable {
     }
 
     public static BackupJobInfo fromCatalog(long backupTime, String label, String dbName, long dbId,
-                                            BackupMeta backupMeta, Map<Long, SnapshotInfo> snapshotInfos) {
+                                            BackupContent content, BackupMeta backupMeta,
+                                            Map<Long, SnapshotInfo> snapshotInfos) {
 
         BackupJobInfo jobInfo = new BackupJobInfo();
         jobInfo.backupTime = backupTime;
@@ -398,6 +481,7 @@ public class BackupJobInfo implements Writable {
         jobInfo.dbName = dbName;
         jobInfo.dbId = dbId;
         jobInfo.metaVersion = FeConstants.meta_version;
+        jobInfo.content = content;
 
         Collection<Table> tbls = backupMeta.getTables().values();
         // tbls
@@ -421,9 +505,15 @@ public class BackupJobInfo implements Writable {
                         idxInfo.schemaHash = olapTbl.getSchemaHashByIndexId(index.getId());
                         partitionInfo.indexes.put(olapTbl.getIndexNameById(index.getId()), idxInfo);
                         // tablets
-                        for (Tablet tablet : index.getTablets()) {
-                            idxInfo.tablets.put(tablet.getId(),
-                                    Lists.newArrayList(snapshotInfos.get(tablet.getId()).getFiles()));
+                        if (content == BackupContent.METADATA_ONLY) {
+                            for (Tablet tablet: index.getTablets()) {
+                                idxInfo.tablets.put(tablet.getId(), Lists.newArrayList());
+                            }
+                        } else {
+                            for (Tablet tablet : index.getTablets()) {
+                                idxInfo.tablets.put(tablet.getId(),
+                                        Lists.newArrayList(snapshotInfos.get(tablet.getId()).getFiles()));
+                            }
                         }
                         idxInfo.tabletsOrder.addAll(index.getTabletIdsInOrder());
                     }
@@ -441,7 +531,15 @@ public class BackupJobInfo implements Writable {
                 backupOdbcTableInfo.dorisTableName = odbcTable.getName();
                 backupOdbcTableInfo.linkedOdbcDatabaseName = odbcTable.getOdbcDatabaseName();
                 backupOdbcTableInfo.linkedOdbcTableName = odbcTable.getOdbcTableName();
-                backupOdbcTableInfo.resourceName = odbcTable.getOdbcCatalogResourceName();
+                if (odbcTable.getOdbcCatalogResourceName() != null) {
+                    backupOdbcTableInfo.resourceName = odbcTable.getOdbcCatalogResourceName();
+                } else {
+                    backupOdbcTableInfo.host = odbcTable.getHost();
+                    backupOdbcTableInfo.port = odbcTable.getPort();
+                    backupOdbcTableInfo.user = odbcTable.getUserName();
+                    backupOdbcTableInfo.driver = odbcTable.getOdbcDriver();
+                    backupOdbcTableInfo.odbcType = odbcTable.getOdbcTableTypeName();
+                }
                 jobInfo.newBackupObjects.odbcTables.add(backupOdbcTableInfo);
             }
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java
index 4f3d1c8..f7b42db 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java
@@ -17,6 +17,7 @@
 
 package org.apache.doris.backup;
 
+import org.apache.doris.analysis.BackupStmt.BackupContent;
 import org.apache.doris.backup.BackupJobInfo.BackupIndexInfo;
 import org.apache.doris.backup.BackupJobInfo.BackupOlapTableInfo;
 import org.apache.doris.backup.BackupJobInfo.BackupPartitionInfo;
@@ -805,16 +806,26 @@ public class RestoreJob extends AbstractJob {
                     "Failed to create replicas for restore. unfinished marks: " + idStr);
             return;
         }
+        LOG.info("finished to prepare meta. {}", this);
 
-        LOG.info("finished to prepare meta. begin to make snapshot. {}", this);
+        if (jobInfo.content == null || jobInfo.content == BackupContent.ALL) {
+            prepareAndSendSnapshotTaskForOlapTable(db);
+        }
 
+        metaPreparedTime = System.currentTimeMillis();
+        state = RestoreJobState.SNAPSHOTING;
+        // No log here, PENDING state restore job will redo this method
+    }
+
+    private void prepareAndSendSnapshotTaskForOlapTable(Database db) {
+        LOG.info("begin to make snapshot. {} when restore content is ALL", this);
         // begin to make snapshots for all replicas
         // snapshot is for incremental download
         unfinishedSignatureToId.clear();
         taskProgress.clear();
         taskErrMsg.clear();
         Multimap<Long, Long> bePathsMap = HashMultimap.create();
-        batchTask = new AgentBatchTask();
+        AgentBatchTask batchTask = new AgentBatchTask();
         db.readLock();
         try {
             for (IdChain idChain : fileMapping.getMapping().keySet()) {
@@ -842,7 +853,7 @@ public class RestoreJob extends AbstractJob {
         } finally {
             db.readUnlock();
         }
-        
+
         // check disk capacity
         org.apache.doris.common.Status st = Catalog.getCurrentSystemInfo().checkExceedDiskCapacityLimit(bePathsMap, true);
         if (!st.ok()) {
@@ -855,13 +866,7 @@ public class RestoreJob extends AbstractJob {
             AgentTaskQueue.addTask(task);
         }
         AgentTaskExecutor.submit(batchTask);
-
-        metaPreparedTime = System.currentTimeMillis();
-        state = RestoreJobState.SNAPSHOTING;
-
-        // No log here, PENDING state restore job will redo this method
-        LOG.info("finished to prepare meta and send snapshot tasks, num: {}. {}",
-                batchTask.getTaskNum(), this);
+        LOG.info("finished to send snapshot tasks, num: {}. {}", batchTask.getTaskNum(), this);
     }
 
     private void checkAndRestoreResources() {
diff --git a/fe/fe-core/src/main/jflex/sql_scanner.flex b/fe/fe-core/src/main/jflex/sql_scanner.flex
index e50be25..8863bf1 100644
--- a/fe/fe-core/src/main/jflex/sql_scanner.flex
+++ b/fe/fe-core/src/main/jflex/sql_scanner.flex
@@ -176,6 +176,7 @@ import org.apache.doris.qe.SqlModeHelper;
         keywordMap.put("errors", new Integer(SqlParserSymbols.KW_ERRORS));
         keywordMap.put("events", new Integer(SqlParserSymbols.KW_EVENTS));
         keywordMap.put("except", new Integer(SqlParserSymbols.KW_EXCEPT));
+        keywordMap.put("exclude", new Integer(SqlParserSymbols.KW_EXCLUDE));
         keywordMap.put("exists", new Integer(SqlParserSymbols.KW_EXISTS));
         keywordMap.put("explain", new Integer(SqlParserSymbols.KW_DESCRIBE));
         keywordMap.put("export", new Integer(SqlParserSymbols.KW_EXPORT));
diff --git a/fe/fe-core/src/test/java/org/apache/doris/backup/BackupHandlerTest.java b/fe/fe-core/src/test/java/org/apache/doris/backup/BackupHandlerTest.java
index 01ae668..fae305c 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/backup/BackupHandlerTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/backup/BackupHandlerTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.doris.backup;
 
+import org.apache.doris.analysis.AbstractBackupTableRefClause;
 import org.apache.doris.analysis.BackupStmt;
 import org.apache.doris.analysis.CancelBackupStmt;
 import org.apache.doris.analysis.CreateRepositoryStmt;
@@ -52,11 +53,6 @@ import org.apache.doris.thrift.TStatusCode;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.File;
@@ -76,6 +72,10 @@ import mockit.Expectations;
 import mockit.Mock;
 import mockit.MockUp;
 import mockit.Mocked;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
 
 public class BackupHandlerTest {
 
@@ -222,10 +222,11 @@ public class BackupHandlerTest {
                         }
                     }
                 }
-                
+
                 BackupJobInfo info = BackupJobInfo.fromCatalog(System.currentTimeMillis(),
-                                                               "ss2", CatalogMocker.TEST_DB_NAME, 
-                                                               CatalogMocker.TEST_DB_ID, backupMeta, snapshotInfos);
+                        "ss2", CatalogMocker.TEST_DB_NAME,
+                        CatalogMocker.TEST_DB_ID, BackupStmt.BackupContent.ALL,
+                        backupMeta, snapshotInfos);
                 infos.add(info);
                 return Status.OK;
             }
@@ -253,15 +254,16 @@ public class BackupHandlerTest {
         // process backup
         List<TableRef> tblRefs = Lists.newArrayList();
         tblRefs.add(new TableRef(new TableName(CatalogMocker.TEST_DB_NAME, CatalogMocker.TEST_TBL_NAME), null));
-        BackupStmt backupStmt = new BackupStmt(new LabelName(CatalogMocker.TEST_DB_NAME, "label1"), "repo", tblRefs,
-                null);
+        AbstractBackupTableRefClause tableRefClause = new AbstractBackupTableRefClause(false, tblRefs);
+        BackupStmt backupStmt = new BackupStmt(new LabelName(CatalogMocker.TEST_DB_NAME, "label1"), "repo",
+                tableRefClause, null);
         try {
             handler.process(backupStmt);
         } catch (DdlException e1) {
             e1.printStackTrace();
             Assert.fail();
         }
-        
+
         // handleFinishedSnapshotTask
         BackupJob backupJob = (BackupJob) handler.getJob(CatalogMocker.TEST_DB_ID);
         SnapshotTask snapshotTask = new SnapshotTask(null, 0, 0, backupJob.getJobId(), CatalogMocker.TEST_DB_ID,
@@ -313,8 +315,9 @@ public class BackupHandlerTest {
         tblRefs2.add(new TableRef(new TableName(CatalogMocker.TEST_DB_NAME, CatalogMocker.TEST_TBL_NAME), null));
         Map<String, String> properties = Maps.newHashMap();
         properties.put("backup_timestamp", "2018-08-08-08-08-08");
-        RestoreStmt restoreStmt = new RestoreStmt(new LabelName(CatalogMocker.TEST_DB_NAME, "ss2"), "repo", tblRefs2,
-                properties);
+        AbstractBackupTableRefClause abstractBackupTableRefClause = new AbstractBackupTableRefClause(false, tblRefs2);
+        RestoreStmt restoreStmt = new RestoreStmt(new LabelName(CatalogMocker.TEST_DB_NAME, "ss2"), "repo",
+                abstractBackupTableRefClause, properties);
         try {
             restoreStmt.analyzeProperties();
         } catch (AnalysisException e2) {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/backup/BackupJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/backup/BackupJobTest.java
index 62a1a3b..f0dba66 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/backup/BackupJobTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/backup/BackupJobTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.doris.backup;
 
+import org.apache.doris.analysis.BackupStmt;
 import org.apache.doris.analysis.TableName;
 import org.apache.doris.analysis.TableRef;
 import org.apache.doris.backup.BackupJob.BackupJobState;
@@ -44,12 +45,6 @@ import org.apache.doris.thrift.TTaskType;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
 import java.io.File;
 import java.io.IOException;
 import java.nio.file.FileVisitOption;
@@ -65,6 +60,11 @@ import mockit.Expectations;
 import mockit.Mock;
 import mockit.MockUp;
 import mockit.Mocked;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
 
 public class BackupJobTest {
 
@@ -200,7 +200,8 @@ public class BackupJobTest {
 
         List<TableRef> tableRefs = Lists.newArrayList();
         tableRefs.add(new TableRef(new TableName(UnitTestUtil.DB_NAME, UnitTestUtil.TABLE_NAME), null));
-        job = new BackupJob("label", dbId, UnitTestUtil.DB_NAME, tableRefs, 13600 * 1000, catalog, repo.getId());
+        job = new BackupJob("label", dbId, UnitTestUtil.DB_NAME, tableRefs, 13600 * 1000, BackupStmt.BackupContent.ALL,
+                catalog, repo.getId());
     }
 
     @Test
@@ -334,7 +335,8 @@ public class BackupJobTest {
 
         List<TableRef> tableRefs = Lists.newArrayList();
         tableRefs.add(new TableRef(new TableName(UnitTestUtil.DB_NAME, "unknown_tbl"), null));
-        job = new BackupJob("label", dbId, UnitTestUtil.DB_NAME, tableRefs, 13600 * 1000, catalog, repo.getId());
+        job = new BackupJob("label", dbId, UnitTestUtil.DB_NAME, tableRefs, 13600 * 1000, BackupStmt.BackupContent.ALL,
+                catalog, repo.getId());
         job.run();
         Assert.assertEquals(Status.ErrCode.NOT_FOUND, job.getStatus().getErrCode());
         Assert.assertEquals(BackupJobState.CANCELLED, job.getState());


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