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 2023/04/13 15:23:13 UTC

[doris] branch branch-1.2-lts updated (ef99144a72 -> 08f5e46e0c)

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

morningman pushed a change to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git


    from ef99144a72 [Fix](catalog)Fix hudi-catalog get file split error (#18644)
     new b2f45a0e97 [minor](fe) ignore expired alter job when replay, to save memory (#18508)
     new d3731904c7 [FixBug](jdbc Catalog) fix sqlserver column type mapping (#18518)
     new a85f2e3479 [Fix](multi catalog)Refresh table object while refresh external table. (#18592)
     new 2761eb4fa0 [fix](planner)keep all agg functions if there is any virtual slots in group by list (#18630)
     new a3a8cc9505 [refactor](jdbc) refactor jdbc connection num in datasource (#18563)
     new c54dacf0ef [bug](GC)the issue of incorrect disk usage (#18397)
     new 978d7f28b2 [Bugfix](backup) fix show snapshot issue (#17580)
     new 1ed068a4e2 [BugFix](backup) fix show backup with where clause (#17736)
     new 08f5e46e0c [branch1.2] fix test_round ut and remove unused fe ut

The 9 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/olap/rowset/rowset_meta_manager.cpp         |  7 +++++++
 be/src/olap/rowset/rowset_meta_manager.h           |  1 +
 be/src/olap/tablet.cpp                             |  3 ++-
 .../org/apache/doris/analysis/AggregateInfo.java   |  3 ++-
 .../org/apache/doris/analysis/ShowBackupStmt.java  | 22 +++++++++++-----------
 .../java/org/apache/doris/backup/Repository.java   |  9 +++++----
 .../main/java/org/apache/doris/catalog/Env.java    |  4 ++++
 .../doris/catalog/external/EsExternalDatabase.java |  1 +
 .../doris/catalog/external/EsExternalTable.java    |  1 +
 .../doris/catalog/external/ExternalDatabase.java   |  1 +
 .../doris/catalog/external/ExternalTable.java      | 14 +++++++++++++-
 .../catalog/external/HMSExternalDatabase.java      |  1 +
 .../doris/catalog/external/HMSExternalTable.java   | 18 ++++--------------
 .../catalog/external/IcebergExternalDatabase.java  |  1 +
 .../catalog/external/IcebergExternalTable.java     |  1 +
 .../catalog/external/JdbcExternalDatabase.java     |  1 +
 .../doris/catalog/external/JdbcExternalTable.java  |  1 +
 .../org/apache/doris/datasource/CatalogMgr.java    |  4 ++++
 .../apache/doris/datasource/ExternalCatalog.java   |  4 ++++
 .../org/apache/doris/external/jdbc/JdbcClient.java | 11 ++++++++---
 .../java/org/apache/doris/qe/ShowExecutor.java     |  4 +---
 .../java/org/apache/doris/udf/JdbcExecutor.java    | 11 +++++++++--
 .../sql_functions/math_functions/test_round.groovy |  2 ++
 23 files changed, 85 insertions(+), 40 deletions(-)


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


[doris] 05/09: [refactor](jdbc) refactor jdbc connection num in datasource (#18563)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit a3a8cc9505f111c9a9719da2e9c961a135522397
Author: zhangstar333 <87...@users.noreply.github.com>
AuthorDate: Thu Apr 13 22:08:08 2023 +0800

    [refactor](jdbc) refactor jdbc connection num in datasource (#18563)
    
    now maybe jdbc have problem that there are too many connections and they do not release,
    so change the property of datasource: init = 1, min = 1, max = 100, and idle time is 10 minutes.
---
 .../main/java/org/apache/doris/external/jdbc/JdbcClient.java  |  6 ++++--
 .../src/main/java/org/apache/doris/udf/JdbcExecutor.java      | 11 +++++++++--
 2 files changed, 13 insertions(+), 4 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java b/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
index ffeec31e20..38a6f07b04 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
@@ -93,8 +93,10 @@ public class JdbcClient {
             dataSource.setUsername(jdbcUser);
             dataSource.setPassword(password);
             dataSource.setMinIdle(1);
-            dataSource.setInitialSize(2);
-            dataSource.setMaxActive(5);
+            dataSource.setInitialSize(1);
+            dataSource.setMaxActive(100);
+            dataSource.setTimeBetweenEvictionRunsMillis(600000);
+            dataSource.setMinEvictableIdleTimeMillis(300000);
             // set connection timeout to 5s.
             // The default is 30s, which is too long.
             // Because when querying information_schema db, BE will call thrift rpc(default timeout is 30s)
diff --git a/fe/java-udf/src/main/java/org/apache/doris/udf/JdbcExecutor.java b/fe/java-udf/src/main/java/org/apache/doris/udf/JdbcExecutor.java
index b1922ac45e..0a68857eb3 100644
--- a/fe/java-udf/src/main/java/org/apache/doris/udf/JdbcExecutor.java
+++ b/fe/java-udf/src/main/java/org/apache/doris/udf/JdbcExecutor.java
@@ -256,9 +256,16 @@ public class JdbcExecutor {
                 ds.setUsername(jdbcUser);
                 ds.setPassword(jdbcPassword);
                 ds.setMinIdle(1);
-                ds.setInitialSize(2);
-                ds.setMaxActive(5);
+                ds.setInitialSize(1);
+                ds.setMaxActive(100);
+                ds.setMaxWait(5000);
+                ds.setTimeBetweenEvictionRunsMillis(600000);
+                ds.setMinEvictableIdleTimeMillis(300000);
                 druidDataSource = ds;
+                // here is a cache of datasource, which using the string(jdbcUrl + jdbcUser +
+                // jdbcPassword) as key.
+                // and the datasource init = 1, min = 1, max = 100, if one of connection idle
+                // time greater than 10 minutes. then connection will be retrieved.
                 JdbcDataSource.getDataSource().putSource(jdbcUrl, ds);
             }
             conn = druidDataSource.getConnection();


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


[doris] 03/09: [Fix](multi catalog)Refresh table object while refresh external table. (#18592)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit a85f2e347983d84ff44dd50c49681bde7df5eaf9
Author: Jibing-Li <64...@users.noreply.github.com>
AuthorDate: Thu Apr 13 08:49:44 2023 +0800

    [Fix](multi catalog)Refresh table object while refresh external table. (#18592)
    
    Refresh table object while refresh external table. Including:
    Refresh catalog, refresh database and refresh table.
    Before visiting database, need to guarantee catalog has been initialized.
    Before visiting table, need to guarantee catalog and database have been initialized.
---
 .../doris/catalog/external/EsExternalDatabase.java |   1 +
 .../doris/catalog/external/EsExternalTable.java    |   1 +
 .../doris/catalog/external/ExternalDatabase.java   |   1 +
 .../doris/catalog/external/ExternalTable.java      |  14 +-
 .../catalog/external/HMSExternalDatabase.java      |   1 +
 .../doris/catalog/external/HMSExternalTable.java   |  18 +--
 .../catalog/external/IcebergExternalDatabase.java  |   1 +
 .../catalog/external/IcebergExternalTable.java     |   1 +
 .../catalog/external/JdbcExternalDatabase.java     |   1 +
 .../doris/catalog/external/JdbcExternalTable.java  |   1 +
 .../org/apache/doris/datasource/CatalogMgr.java    |   4 +
 .../apache/doris/datasource/ExternalCatalog.java   |   4 +
 .../java/org/apache/doris/qe/ShowExecutor.java     |   2 -
 .../org/apache/doris/catalog/RefreshTableTest.java | 179 +++++++++++++++++++++
 14 files changed, 212 insertions(+), 17 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/EsExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/EsExternalDatabase.java
index bd0e322c5f..b69041952e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/EsExternalDatabase.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/EsExternalDatabase.java
@@ -99,6 +99,7 @@ public class EsExternalDatabase extends ExternalDatabase<EsExternalTable> implem
                     tblId = tableNameToId.get(tableName);
                     tmpTableNameToId.put(tableName, tblId);
                     EsExternalTable table = idToTbl.get(tblId);
+                    table.unsetObjectCreated();
                     tmpIdToTbl.put(tblId, table);
                     initDatabaseLog.addRefreshTable(tblId);
                 } else {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/EsExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/EsExternalTable.java
index 290917e21c..0b2b2cd029 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/EsExternalTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/EsExternalTable.java
@@ -52,6 +52,7 @@ public class EsExternalTable extends ExternalTable {
     }
 
     protected synchronized void makeSureInitialized() {
+        super.makeSureInitialized();
         if (!objectCreated) {
             esTable = toEsTable();
             objectCreated = true;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalDatabase.java
index 4a6c43cf22..2ab0a66ee0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalDatabase.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalDatabase.java
@@ -107,6 +107,7 @@ public class ExternalDatabase<T extends ExternalTable> implements DatabaseIf<T>,
     }
 
     public final synchronized void makeSureInitialized() {
+        extCatalog.makeSureInitialized();
         if (!initialized) {
             if (!Env.getCurrentEnv().isMaster()) {
                 // Forward to master and wait the journal to replay.
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalTable.java
index b12e468425..db01d969bb 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalTable.java
@@ -21,10 +21,12 @@ import org.apache.doris.alter.AlterCancelException;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.TableIf;
+import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
+import org.apache.doris.common.util.Util;
 import org.apache.doris.datasource.ExternalCatalog;
 import org.apache.doris.datasource.ExternalSchemaCache;
 import org.apache.doris.persist.gson.GsonPostProcessable;
@@ -104,7 +106,13 @@ public class ExternalTable implements TableIf, Writable, GsonPostProcessable {
     }
 
     protected void makeSureInitialized() {
-        throw new NotImplementedException();
+        try {
+            // getDbOrAnalysisException will call makeSureInitialized in ExternalCatalog.
+            ExternalDatabase db = catalog.getDbOrAnalysisException(dbName);
+            db.makeSureInitialized();
+        } catch (AnalysisException e) {
+            Util.logAndThrowRuntimeException(LOG, String.format("Exception to get db %s", dbName), e);
+        }
     }
 
     @Override
@@ -315,6 +323,10 @@ public class ExternalTable implements TableIf, Writable, GsonPostProcessable {
         throw new NotImplementedException("implement in sub class");
     }
 
+    public void unsetObjectCreated() {
+        this.objectCreated = false;
+    }
+
     @Override
     public void write(DataOutput out) throws IOException {
         String json = GsonUtils.GSON.toJson(this);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java
index e379dd3c0d..84cf3c4af7 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java
@@ -102,6 +102,7 @@ public class HMSExternalDatabase extends ExternalDatabase<HMSExternalTable> impl
                     tblId = tableNameToId.get(tableName);
                     tmpTableNameToId.put(tableName, tblId);
                     HMSExternalTable table = idToTbl.get(tblId);
+                    table.unsetObjectCreated();
                     tmpIdToTbl.put(tblId, table);
                     initDatabaseLog.addRefreshTable(tblId);
                 } else {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalTable.java
index 79df8bd6bf..1b882f9fe0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalTable.java
@@ -20,7 +20,6 @@ package org.apache.doris.catalog.external;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.HiveMetaStoreClientHelper;
 import org.apache.doris.catalog.Type;
-import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.datasource.HMSExternalCatalog;
 import org.apache.doris.datasource.hive.PooledHiveMetaStoreClient;
 import org.apache.doris.statistics.AnalysisTaskInfo;
@@ -90,12 +89,9 @@ public class HMSExternalTable extends ExternalTable {
     }
 
     protected synchronized void makeSureInitialized() {
+        super.makeSureInitialized();
         if (!objectCreated) {
-            try {
-                getRemoteTable();
-            } catch (MetaNotFoundException e) {
-                // CHECKSTYLE IGNORE THIS LINE
-            }
+            remoteTable = ((HMSExternalCatalog) catalog).getClient().getTable(dbName, name);
             if (remoteTable == null) {
                 dlaType = DLAType.UNKNOWN;
             } else {
@@ -151,14 +147,8 @@ public class HMSExternalTable extends ExternalTable {
     /**
      * Get the related remote hive metastore table.
      */
-    public org.apache.hadoop.hive.metastore.api.Table getRemoteTable() throws MetaNotFoundException {
-        if (remoteTable == null) {
-            synchronized (this) {
-                if (remoteTable == null) {
-                    remoteTable = ((HMSExternalCatalog) catalog).getClient().getTable(dbName, name);
-                }
-            }
-        }
+    public org.apache.hadoop.hive.metastore.api.Table getRemoteTable() {
+        makeSureInitialized();
         return remoteTable;
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalDatabase.java
index 9b110c9dcf..db1d1e13ce 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalDatabase.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalDatabase.java
@@ -92,6 +92,7 @@ public class IcebergExternalDatabase extends ExternalDatabase<IcebergExternalTab
                     tblId = tableNameToId.get(tableName);
                     tmpTableNameToId.put(tableName, tblId);
                     IcebergExternalTable table = idToTbl.get(tblId);
+                    table.unsetObjectCreated();
                     tmpIdToTbl.put(tblId, table);
                     initDatabaseLog.addRefreshTable(tblId);
                 } else {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalTable.java
index 3e95e30e18..21c27bb280 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/IcebergExternalTable.java
@@ -44,6 +44,7 @@ public class IcebergExternalTable extends ExternalTable {
     }
 
     protected synchronized void makeSureInitialized() {
+        super.makeSureInitialized();
         if (!objectCreated) {
             objectCreated = true;
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/JdbcExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/JdbcExternalDatabase.java
index 6eb6340f94..4272d357e0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/JdbcExternalDatabase.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/JdbcExternalDatabase.java
@@ -71,6 +71,7 @@ public class JdbcExternalDatabase extends ExternalDatabase<JdbcExternalTable> im
                     tblId = tableNameToId.get(tableName);
                     tmpTableNameToId.put(tableName, tblId);
                     JdbcExternalTable table = idToTbl.get(tblId);
+                    table.unsetObjectCreated();
                     tmpIdToTbl.put(tblId, table);
                     initDatabaseLog.addRefreshTable(tblId);
                 } else {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/JdbcExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/JdbcExternalTable.java
index 03c145da8c..80e2570fd4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/JdbcExternalTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/JdbcExternalTable.java
@@ -48,6 +48,7 @@ public class JdbcExternalTable extends ExternalTable {
     }
 
     protected synchronized void makeSureInitialized() {
+        super.makeSureInitialized();
         if (!objectCreated) {
             jdbcTable = toJdbcTable();
             objectCreated = true;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java
index 41732927fc..1222f90432 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java
@@ -571,6 +571,9 @@ public class CatalogMgr implements Writable, GsonPostProcessable {
         if (table == null) {
             throw new DdlException("Table " + tableName + " does not exist in db " + dbName);
         }
+        if (table instanceof ExternalTable) {
+            ((ExternalTable) table).unsetObjectCreated();
+        }
         Env.getCurrentEnv().getExtMetaCacheMgr().invalidateTableCache(catalog.getId(), dbName, tableName);
         ExternalObjectLog log = new ExternalObjectLog();
         log.setCatalogId(catalog.getId());
@@ -595,6 +598,7 @@ public class CatalogMgr implements Writable, GsonPostProcessable {
             LOG.warn("No table found with id:[{}], it may have been dropped.", log.getTableId());
             return;
         }
+        table.unsetObjectCreated();
         Env.getCurrentEnv().getExtMetaCacheMgr()
                 .invalidateTableCache(catalog.getId(), db.getFullName(), table.getName());
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
index 3cdf009182..e7f619a742 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
@@ -147,6 +147,10 @@ public abstract class ExternalCatalog implements CatalogIf<ExternalDatabase>, Wr
         }
     }
 
+    public boolean isInitialized() {
+        return this.initialized;
+    }
+
     // init some local objects such as:
     // hms client, read properties from hive-site.xml, es client
     protected abstract void initLocalObjectsImpl();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
index b3969b7b30..e9a8b45e10 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
@@ -906,8 +906,6 @@ public class ShowExecutor {
                         ? new ShowResultSet(showStmt.getMetaData(), rows)
                         : new ShowResultSet(ShowCreateTableStmt.getMaterializedViewMetaData(), rows);
             }
-        } catch (MetaNotFoundException e) {
-            throw new AnalysisException(e.getMessage());
         } finally {
             table.readUnlock();
         }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/RefreshTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/RefreshTableTest.java
new file mode 100644
index 0000000000..1d8965c813
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/RefreshTableTest.java
@@ -0,0 +1,179 @@
+// 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.catalog;
+
+import org.apache.doris.analysis.CreateCatalogStmt;
+import org.apache.doris.analysis.DropCatalogStmt;
+import org.apache.doris.analysis.RefreshCatalogStmt;
+import org.apache.doris.analysis.RefreshDbStmt;
+import org.apache.doris.analysis.RefreshTableStmt;
+import org.apache.doris.analysis.TableName;
+import org.apache.doris.catalog.external.ExternalDatabase;
+import org.apache.doris.catalog.external.TestExternalTable;
+import org.apache.doris.common.FeConstants;
+import org.apache.doris.datasource.CatalogIf;
+import org.apache.doris.datasource.ExternalCatalog;
+import org.apache.doris.datasource.test.TestExternalCatalog;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.DdlExecutor;
+import org.apache.doris.utframe.TestWithFeService;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.Map;
+
+public class RefreshTableTest extends TestWithFeService {
+    private static Env env;
+    private ConnectContext rootCtx;
+
+    @Override
+    protected void runBeforeAll() throws Exception {
+        FeConstants.runningUnitTest = true;
+        rootCtx = createDefaultCtx();
+        env = Env.getCurrentEnv();
+        // 1. create test catalog
+        CreateCatalogStmt testCatalog = (CreateCatalogStmt) parseAndAnalyzeStmt("create catalog test1 properties(\n"
+                + "    \"type\" = \"test\",\n"
+                + "    \"catalog_provider.class\" "
+                + "= \"org.apache.doris.catalog.RefreshTableTest$RefreshTableProvider\"\n"
+                + ");",
+                rootCtx);
+        env.getCatalogMgr().createCatalog(testCatalog);
+    }
+
+    @Override
+    protected void runAfterAll() throws Exception {
+        super.runAfterAll();
+        rootCtx.setThreadLocalInfo();
+        DropCatalogStmt stmt = (DropCatalogStmt) parseAndAnalyzeStmt("drop catalog test1");
+        env.getCatalogMgr().dropCatalog(stmt);
+    }
+
+    @Test
+    public void testRefreshCatalog() throws Exception {
+        CatalogIf test1 = env.getCatalogMgr().getCatalog("test1");
+        TestExternalTable table = (TestExternalTable) test1.getDbNullable("db1").getTable("tbl11").get();
+        Assertions.assertFalse(table.isObjectCreated());
+        table.makeSureInitialized();
+        Assertions.assertTrue(table.isObjectCreated());
+        RefreshCatalogStmt refreshCatalogStmt = new RefreshCatalogStmt("test1", null);
+        try {
+            DdlExecutor.execute(Env.getCurrentEnv(), refreshCatalogStmt);
+        } catch (Exception e) {
+            // Do nothing
+        }
+        Assertions.assertTrue(table.isObjectCreated());
+        test1.getDbNullable("db1").getTables();
+        Assertions.assertFalse(table.isObjectCreated());
+        try {
+            DdlExecutor.execute(Env.getCurrentEnv(), refreshCatalogStmt);
+        } catch (Exception e) {
+            // Do nothing
+        }
+        Assertions.assertFalse(((ExternalCatalog) test1).isInitialized());
+        table.makeSureInitialized();
+        Assertions.assertTrue(((ExternalCatalog) test1).isInitialized());
+        try {
+            DdlExecutor.execute(Env.getCurrentEnv(), refreshCatalogStmt);
+        } catch (Exception e) {
+            // Do nothing
+        }
+    }
+
+    @Test
+    public void testRefreshDatabase() throws Exception {
+        CatalogIf test1 = env.getCatalogMgr().getCatalog("test1");
+        TestExternalTable table = (TestExternalTable) test1.getDbNullable("db1").getTable("tbl11").get();
+        Assertions.assertFalse(table.isObjectCreated());
+        table.makeSureInitialized();
+        Assertions.assertTrue(table.isObjectCreated());
+        RefreshDbStmt refreshDbStmt = new RefreshDbStmt("test1", "db1", null);
+        try {
+            DdlExecutor.execute(Env.getCurrentEnv(), refreshDbStmt);
+        } catch (Exception e) {
+            // Do nothing
+        }
+        Assertions.assertTrue(table.isObjectCreated());
+        test1.getDbNullable("db1").getTables();
+        Assertions.assertFalse(table.isObjectCreated());
+        try {
+            DdlExecutor.execute(Env.getCurrentEnv(), refreshDbStmt);
+        } catch (Exception e) {
+            // Do nothing
+        }
+        Assertions.assertFalse(((ExternalDatabase) test1.getDbNullable("db1")).isInitialized());
+        table.makeSureInitialized();
+        Assertions.assertTrue(((ExternalDatabase) test1.getDbNullable("db1")).isInitialized());
+        try {
+            DdlExecutor.execute(Env.getCurrentEnv(), refreshDbStmt);
+        } catch (Exception e) {
+            // Do nothing
+        }
+    }
+
+    @Test
+    public void testRefreshTable() throws Exception {
+        CatalogIf test1 = env.getCatalogMgr().getCatalog("test1");
+        TestExternalTable table = (TestExternalTable) test1.getDbNullable("db1").getTable("tbl11").get();
+        Assertions.assertFalse(table.isObjectCreated());
+        table.makeSureInitialized();
+        Assertions.assertTrue(table.isObjectCreated());
+        RefreshTableStmt refreshTableStmt = new RefreshTableStmt(new TableName("test1", "db1", "tbl11"));
+        try {
+            DdlExecutor.execute(Env.getCurrentEnv(), refreshTableStmt);
+        } catch (Exception e) {
+            // Do nothing
+        }
+        Assertions.assertFalse(table.isObjectCreated());
+    }
+
+    public static class RefreshTableProvider implements TestExternalCatalog.TestCatalogProvider {
+        public static final Map<String, Map<String, List<Column>>> MOCKED_META;
+
+        static {
+            MOCKED_META = Maps.newHashMap();
+            Map<String, List<Column>> tblSchemaMap1 = Maps.newHashMap();
+            // db1
+            tblSchemaMap1.put("tbl11", Lists.newArrayList(
+                new Column("a11", PrimitiveType.BIGINT),
+                new Column("a12", PrimitiveType.STRING),
+                new Column("a13", PrimitiveType.FLOAT)));
+            tblSchemaMap1.put("tbl12", Lists.newArrayList(
+                new Column("b21", PrimitiveType.BIGINT),
+                new Column("b22", PrimitiveType.STRING),
+                new Column("b23", PrimitiveType.FLOAT)));
+            MOCKED_META.put("db1", tblSchemaMap1);
+            // db2
+            Map<String, List<Column>> tblSchemaMap2 = Maps.newHashMap();
+            tblSchemaMap2.put("tbl21", Lists.newArrayList(
+                new Column("c11", PrimitiveType.BIGINT),
+                new Column("c12", PrimitiveType.STRING),
+                new Column("c13", PrimitiveType.FLOAT)));
+            MOCKED_META.put("db2", tblSchemaMap2);
+        }
+
+        @Override
+        public Map<String, Map<String, List<Column>>> getMetadata() {
+            return MOCKED_META;
+        }
+    }
+}


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


[doris] 06/09: [bug](GC)the issue of incorrect disk usage (#18397)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit c54dacf0efec40b2c9e8faa64e9f10f2180e314d
Author: chenlinzhong <49...@qq.com>
AuthorDate: Sat Apr 8 09:32:36 2023 +0800

    [bug](GC)the issue of incorrect disk usage (#18397)
---
 be/src/olap/rowset/rowset_meta_manager.cpp | 7 +++++++
 be/src/olap/rowset/rowset_meta_manager.h   | 1 +
 be/src/olap/tablet.cpp                     | 3 ++-
 3 files changed, 10 insertions(+), 1 deletion(-)

diff --git a/be/src/olap/rowset/rowset_meta_manager.cpp b/be/src/olap/rowset/rowset_meta_manager.cpp
index 7a58fa9cb6..ac9ae5764e 100644
--- a/be/src/olap/rowset/rowset_meta_manager.cpp
+++ b/be/src/olap/rowset/rowset_meta_manager.cpp
@@ -42,6 +42,13 @@ bool RowsetMetaManager::check_rowset_meta(OlapMeta* meta, TabletUid tablet_uid,
     return meta->key_may_exist(META_COLUMN_FAMILY_INDEX, key, &value);
 }
 
+Status RowsetMetaManager::exists(OlapMeta* meta, TabletUid tablet_uid, const RowsetId& rowset_id) {
+    std::string key = ROWSET_PREFIX + tablet_uid.to_string() + "_" + rowset_id.to_string();
+    std::string value;
+    Status s = meta->get(META_COLUMN_FAMILY_INDEX, key, &value);
+    return s;
+}
+
 Status RowsetMetaManager::get_rowset_meta(OlapMeta* meta, TabletUid tablet_uid,
                                           const RowsetId& rowset_id,
                                           RowsetMetaSharedPtr rowset_meta) {
diff --git a/be/src/olap/rowset/rowset_meta_manager.h b/be/src/olap/rowset/rowset_meta_manager.h
index 8c8f3144e0..cca2304ce3 100644
--- a/be/src/olap/rowset/rowset_meta_manager.h
+++ b/be/src/olap/rowset/rowset_meta_manager.h
@@ -31,6 +31,7 @@ namespace doris {
 class RowsetMetaManager {
 public:
     static bool check_rowset_meta(OlapMeta* meta, TabletUid tablet_uid, const RowsetId& rowset_id);
+    static Status exists(OlapMeta* meta, TabletUid tablet_uid, const RowsetId& rowset_id);
 
     static Status get_rowset_meta(OlapMeta* meta, TabletUid tablet_uid, const RowsetId& rowset_id,
                                   RowsetMetaSharedPtr rowset_meta);
diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp
index 084ee03f24..04e43f719a 100644
--- a/be/src/olap/tablet.cpp
+++ b/be/src/olap/tablet.cpp
@@ -1162,7 +1162,8 @@ bool Tablet::check_rowset_id(const RowsetId& rowset_id) {
             return true;
         }
     }
-    if (RowsetMetaManager::check_rowset_meta(_data_dir->get_meta(), tablet_uid(), rowset_id)) {
+    Status s = RowsetMetaManager::exists(_data_dir->get_meta(), tablet_uid(), rowset_id);
+    if (!s.is<META_KEY_NOT_FOUND>()) {
         return true;
     }
     return false;


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


[doris] 09/09: [branch1.2] fix test_round ut and remove unused fe ut

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 08f5e46e0cc70a8ff2cddd0c312b9630339ecefb
Author: morningman <mo...@163.com>
AuthorDate: Thu Apr 13 23:20:23 2023 +0800

    [branch1.2] fix test_round ut and remove unused fe ut
---
 .../org/apache/doris/catalog/RefreshTableTest.java | 179 ---------------------
 .../sql_functions/math_functions/test_round.groovy |   2 +
 2 files changed, 2 insertions(+), 179 deletions(-)

diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/RefreshTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/RefreshTableTest.java
deleted file mode 100644
index 1d8965c813..0000000000
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/RefreshTableTest.java
+++ /dev/null
@@ -1,179 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-package org.apache.doris.catalog;
-
-import org.apache.doris.analysis.CreateCatalogStmt;
-import org.apache.doris.analysis.DropCatalogStmt;
-import org.apache.doris.analysis.RefreshCatalogStmt;
-import org.apache.doris.analysis.RefreshDbStmt;
-import org.apache.doris.analysis.RefreshTableStmt;
-import org.apache.doris.analysis.TableName;
-import org.apache.doris.catalog.external.ExternalDatabase;
-import org.apache.doris.catalog.external.TestExternalTable;
-import org.apache.doris.common.FeConstants;
-import org.apache.doris.datasource.CatalogIf;
-import org.apache.doris.datasource.ExternalCatalog;
-import org.apache.doris.datasource.test.TestExternalCatalog;
-import org.apache.doris.qe.ConnectContext;
-import org.apache.doris.qe.DdlExecutor;
-import org.apache.doris.utframe.TestWithFeService;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
-
-import java.util.List;
-import java.util.Map;
-
-public class RefreshTableTest extends TestWithFeService {
-    private static Env env;
-    private ConnectContext rootCtx;
-
-    @Override
-    protected void runBeforeAll() throws Exception {
-        FeConstants.runningUnitTest = true;
-        rootCtx = createDefaultCtx();
-        env = Env.getCurrentEnv();
-        // 1. create test catalog
-        CreateCatalogStmt testCatalog = (CreateCatalogStmt) parseAndAnalyzeStmt("create catalog test1 properties(\n"
-                + "    \"type\" = \"test\",\n"
-                + "    \"catalog_provider.class\" "
-                + "= \"org.apache.doris.catalog.RefreshTableTest$RefreshTableProvider\"\n"
-                + ");",
-                rootCtx);
-        env.getCatalogMgr().createCatalog(testCatalog);
-    }
-
-    @Override
-    protected void runAfterAll() throws Exception {
-        super.runAfterAll();
-        rootCtx.setThreadLocalInfo();
-        DropCatalogStmt stmt = (DropCatalogStmt) parseAndAnalyzeStmt("drop catalog test1");
-        env.getCatalogMgr().dropCatalog(stmt);
-    }
-
-    @Test
-    public void testRefreshCatalog() throws Exception {
-        CatalogIf test1 = env.getCatalogMgr().getCatalog("test1");
-        TestExternalTable table = (TestExternalTable) test1.getDbNullable("db1").getTable("tbl11").get();
-        Assertions.assertFalse(table.isObjectCreated());
-        table.makeSureInitialized();
-        Assertions.assertTrue(table.isObjectCreated());
-        RefreshCatalogStmt refreshCatalogStmt = new RefreshCatalogStmt("test1", null);
-        try {
-            DdlExecutor.execute(Env.getCurrentEnv(), refreshCatalogStmt);
-        } catch (Exception e) {
-            // Do nothing
-        }
-        Assertions.assertTrue(table.isObjectCreated());
-        test1.getDbNullable("db1").getTables();
-        Assertions.assertFalse(table.isObjectCreated());
-        try {
-            DdlExecutor.execute(Env.getCurrentEnv(), refreshCatalogStmt);
-        } catch (Exception e) {
-            // Do nothing
-        }
-        Assertions.assertFalse(((ExternalCatalog) test1).isInitialized());
-        table.makeSureInitialized();
-        Assertions.assertTrue(((ExternalCatalog) test1).isInitialized());
-        try {
-            DdlExecutor.execute(Env.getCurrentEnv(), refreshCatalogStmt);
-        } catch (Exception e) {
-            // Do nothing
-        }
-    }
-
-    @Test
-    public void testRefreshDatabase() throws Exception {
-        CatalogIf test1 = env.getCatalogMgr().getCatalog("test1");
-        TestExternalTable table = (TestExternalTable) test1.getDbNullable("db1").getTable("tbl11").get();
-        Assertions.assertFalse(table.isObjectCreated());
-        table.makeSureInitialized();
-        Assertions.assertTrue(table.isObjectCreated());
-        RefreshDbStmt refreshDbStmt = new RefreshDbStmt("test1", "db1", null);
-        try {
-            DdlExecutor.execute(Env.getCurrentEnv(), refreshDbStmt);
-        } catch (Exception e) {
-            // Do nothing
-        }
-        Assertions.assertTrue(table.isObjectCreated());
-        test1.getDbNullable("db1").getTables();
-        Assertions.assertFalse(table.isObjectCreated());
-        try {
-            DdlExecutor.execute(Env.getCurrentEnv(), refreshDbStmt);
-        } catch (Exception e) {
-            // Do nothing
-        }
-        Assertions.assertFalse(((ExternalDatabase) test1.getDbNullable("db1")).isInitialized());
-        table.makeSureInitialized();
-        Assertions.assertTrue(((ExternalDatabase) test1.getDbNullable("db1")).isInitialized());
-        try {
-            DdlExecutor.execute(Env.getCurrentEnv(), refreshDbStmt);
-        } catch (Exception e) {
-            // Do nothing
-        }
-    }
-
-    @Test
-    public void testRefreshTable() throws Exception {
-        CatalogIf test1 = env.getCatalogMgr().getCatalog("test1");
-        TestExternalTable table = (TestExternalTable) test1.getDbNullable("db1").getTable("tbl11").get();
-        Assertions.assertFalse(table.isObjectCreated());
-        table.makeSureInitialized();
-        Assertions.assertTrue(table.isObjectCreated());
-        RefreshTableStmt refreshTableStmt = new RefreshTableStmt(new TableName("test1", "db1", "tbl11"));
-        try {
-            DdlExecutor.execute(Env.getCurrentEnv(), refreshTableStmt);
-        } catch (Exception e) {
-            // Do nothing
-        }
-        Assertions.assertFalse(table.isObjectCreated());
-    }
-
-    public static class RefreshTableProvider implements TestExternalCatalog.TestCatalogProvider {
-        public static final Map<String, Map<String, List<Column>>> MOCKED_META;
-
-        static {
-            MOCKED_META = Maps.newHashMap();
-            Map<String, List<Column>> tblSchemaMap1 = Maps.newHashMap();
-            // db1
-            tblSchemaMap1.put("tbl11", Lists.newArrayList(
-                new Column("a11", PrimitiveType.BIGINT),
-                new Column("a12", PrimitiveType.STRING),
-                new Column("a13", PrimitiveType.FLOAT)));
-            tblSchemaMap1.put("tbl12", Lists.newArrayList(
-                new Column("b21", PrimitiveType.BIGINT),
-                new Column("b22", PrimitiveType.STRING),
-                new Column("b23", PrimitiveType.FLOAT)));
-            MOCKED_META.put("db1", tblSchemaMap1);
-            // db2
-            Map<String, List<Column>> tblSchemaMap2 = Maps.newHashMap();
-            tblSchemaMap2.put("tbl21", Lists.newArrayList(
-                new Column("c11", PrimitiveType.BIGINT),
-                new Column("c12", PrimitiveType.STRING),
-                new Column("c13", PrimitiveType.FLOAT)));
-            MOCKED_META.put("db2", tblSchemaMap2);
-        }
-
-        @Override
-        public Map<String, Map<String, List<Column>>> getMetadata() {
-            return MOCKED_META;
-        }
-    }
-}
diff --git a/regression-test/suites/query_p0/sql_functions/math_functions/test_round.groovy b/regression-test/suites/query_p0/sql_functions/math_functions/test_round.groovy
index 3e2f698c2a..e8db7cc4c6 100644
--- a/regression-test/suites/query_p0/sql_functions/math_functions/test_round.groovy
+++ b/regression-test/suites/query_p0/sql_functions/math_functions/test_round.groovy
@@ -48,6 +48,7 @@ suite("test_round") {
     qt_select """ SELECT truncate(col1, -1), truncate(col2, -1), truncate(col3, -1) FROM `${tableName}`; """
 
     def tableName1 = "test_round1"
+    sql """DROP TABLE IF EXISTS `${tableName1}`"""
     sql """ CREATE TABLE `${tableName1}` (
           `TENANT_ID` varchar(50) NOT NULL,
           `PUBONLN_PRC` decimalv3(18, 4) NULL,
@@ -66,6 +67,7 @@ suite("test_round") {
         ); """
 
     def tableName2 = "test_round2"
+    sql """DROP TABLE IF EXISTS `${tableName2}`"""
     sql """ CREATE TABLE `${tableName2}` (
           `tenant_id` varchar(50) NOT NULL COMMENT '租户ID',
           `prodentp_code` varchar(50) NULL COMMENT '生产企业代码',


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


[doris] 08/09: [BugFix](backup) fix show backup with where clause (#17736)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 1ed068a4e2c9bee378654a6c189b4e15cdeb60e4
Author: GoGoWen <82...@users.noreply.github.com>
AuthorDate: Sun Apr 9 20:11:18 2023 +0800

    [BugFix](backup) fix show backup with where clause (#17736)
    
    1  show backup where SnapshotName="xxx";
    2. show backup where SnapshotName like "%XXX%"
---
 .../org/apache/doris/analysis/ShowBackupStmt.java  | 22 +++++++++++-----------
 .../java/org/apache/doris/qe/ShowExecutor.java     |  2 +-
 2 files changed, 12 insertions(+), 12 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBackupStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBackupStmt.java
index c5adcf45c8..a220517778 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBackupStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBackupStmt.java
@@ -47,7 +47,7 @@ public class ShowBackupStmt extends ShowStmt {
     private String dbName;
     private final Expr where;
     private boolean isAccurateMatch;
-    private String labelValue;
+    private String snapshotName;
 
     public ShowBackupStmt(String dbName, Expr where) {
         this.dbName = dbName;
@@ -81,8 +81,8 @@ public class ShowBackupStmt extends ShowStmt {
         }
         boolean valid = analyzeWhereClause();
         if (!valid) {
-            throw new AnalysisException("Where clause should like: LABEL = \"your_label_name\", "
-                + " or LABEL LIKE \"matcher\"");
+            throw new AnalysisException("Where clause should like: SnapshotName = \"your_snapshot_name\", "
+                + " or SnapshotName LIKE \"matcher\"");
         }
     }
 
@@ -111,7 +111,7 @@ public class ShowBackupStmt extends ShowStmt {
             return false;
         }
         String leftKey = ((SlotRef) where.getChild(0)).getColumnName();
-        if (!"label".equalsIgnoreCase(leftKey)) {
+        if (!"snapshotname".equalsIgnoreCase(leftKey)) {
             return false;
         }
 
@@ -119,8 +119,8 @@ public class ShowBackupStmt extends ShowStmt {
         if (!(where.getChild(1) instanceof StringLiteral)) {
             return false;
         }
-        labelValue = ((StringLiteral) where.getChild(1)).getStringValue();
-        if (Strings.isNullOrEmpty(labelValue)) {
+        snapshotName = ((StringLiteral) where.getChild(1)).getStringValue();
+        if (Strings.isNullOrEmpty(snapshotName)) {
             return false;
         }
 
@@ -165,24 +165,24 @@ public class ShowBackupStmt extends ShowStmt {
         return isAccurateMatch;
     }
 
-    public String getLabelValue() {
-        return labelValue;
+    public String getSnapshotName() {
+        return snapshotName;
     }
 
     public Expr getWhere() {
         return where;
     }
 
-    public Predicate<String> getLabelPredicate() throws AnalysisException {
+    public Predicate<String> getSnapshotPredicate() throws AnalysisException {
         if (null == where) {
             return label -> true;
         }
         if (isAccurateMatch) {
             return CaseSensibility.LABEL.getCaseSensibility()
-                    ? label -> label.equals(labelValue) : label -> label.equalsIgnoreCase(labelValue);
+                    ? label -> label.equals(snapshotName) : label -> label.equalsIgnoreCase(snapshotName);
         } else {
             PatternMatcher patternMatcher = PatternMatcherWrapper.createMysqlPattern(
-                    labelValue, CaseSensibility.LABEL.getCaseSensibility());
+                    snapshotName, CaseSensibility.LABEL.getCaseSensibility());
             return patternMatcher::match;
         }
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
index e9a8b45e10..065c69a68e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
@@ -1764,7 +1764,7 @@ public class ShowExecutor {
         Database db = Env.getCurrentInternalCatalog().getDbOrAnalysisException(showStmt.getDbName());
 
         List<AbstractJob> jobs = Env.getCurrentEnv().getBackupHandler()
-                .getJobs(db.getId(), showStmt.getLabelPredicate());
+                .getJobs(db.getId(), showStmt.getSnapshotPredicate());
 
         List<BackupJob> backupJobs = jobs.stream().filter(job -> job instanceof BackupJob)
                 .map(job -> (BackupJob) job).collect(Collectors.toList());


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


[doris] 07/09: [Bugfix](backup) fix show snapshot issue (#17580)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 978d7f28b22b8d2b7d6c1452b010de76060dceb0
Author: GoGoWen <82...@users.noreply.github.com>
AuthorDate: Sun Apr 9 20:04:39 2023 +0800

    [Bugfix](backup) fix show snapshot issue (#17580)
    
    when there is no snapshot, should no result shows.
---
 fe/fe-core/src/main/java/org/apache/doris/backup/Repository.java | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/Repository.java b/fe/fe-core/src/main/java/org/apache/doris/backup/Repository.java
index 3ce98c9649..61df65ecc5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/backup/Repository.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/backup/Repository.java
@@ -644,8 +644,6 @@ public class Repository implements Writable {
                 info.add(FeConstants.null_string);
                 info.add("ERROR: Failed to get info: " + st.getErrMsg());
             } else {
-                info.add(snapshotName);
-
                 List<String> tmp = Lists.newArrayList();
                 for (RemoteFile file : results) {
                     // __info_2018-04-18-20-11-00.Jdwnd9312sfdn1294343
@@ -657,8 +655,11 @@ public class Repository implements Writable {
                     }
                     tmp.add(disjoinPrefix(PREFIX_JOB_INFO, pureFileName.first));
                 }
-                info.add(Joiner.on("\n").join(tmp));
-                info.add(tmp.isEmpty() ? "ERROR: no snapshot" : "OK");
+                if (!tmp.isEmpty()) {
+                    info.add(snapshotName);
+                    info.add(Joiner.on("\n").join(tmp));
+                    info.add("OK");
+                }
             }
         } else {
             // get specified timestamp


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


[doris] 04/09: [fix](planner)keep all agg functions if there is any virtual slots in group by list (#18630)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 2761eb4fa0c0030ce2fa278449737f66089fd0fb
Author: starocean999 <40...@users.noreply.github.com>
AuthorDate: Thu Apr 13 19:44:46 2023 +0800

    [fix](planner)keep all agg functions if there is any virtual slots in group by list (#18630)
    
    Because of the limitation of ProjectPlanner, we have to keep set agg functions materialized if there is any virtual slots in the group by list, such as 'GROUPING_ID' in the group by list etc.
---
 fe/fe-core/src/main/java/org/apache/doris/analysis/AggregateInfo.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AggregateInfo.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AggregateInfo.java
index 05d5284c08..84e9407693 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AggregateInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AggregateInfo.java
@@ -888,13 +888,14 @@ public final class AggregateInfo extends AggregateInfoBase {
         int aggregateExprsSize = aggregateExprs.size();
         int groupExprsSize = groupingExprs.size();
         boolean isDistinctAgg = isDistinctAgg();
+        boolean hasVirtualSlot = groupingExprs.stream().anyMatch(expr -> expr instanceof VirtualSlotRef);
         for (int i = 0; i < aggregateExprsSize; ++i) {
             FunctionCallExpr functionCallExpr = aggregateExprs.get(i);
             SlotDescriptor slotDesc =
                     outputTupleDesc.getSlots().get(groupExprsSize + i);
             SlotDescriptor intermediateSlotDesc =
                     intermediateTupleDesc.getSlots().get(groupExprsSize + i);
-            if (isDistinctAgg || isUsingSetForDistinct) {
+            if (isDistinctAgg || isUsingSetForDistinct || hasVirtualSlot) {
                 slotDesc.setIsMaterialized(true);
                 intermediateSlotDesc.setIsMaterialized(true);
             }


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


[doris] 02/09: [FixBug](jdbc Catalog) fix sqlserver column type mapping (#18518)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit d3731904c7903d8c2f25551438cf853637c79beb
Author: Tiewei Fang <43...@users.noreply.github.com>
AuthorDate: Wed Apr 12 19:58:30 2023 +0800

    [FixBug](jdbc Catalog) fix sqlserver column type mapping (#18518)
    
    For type int identity of sqlserver, the column type read from JDBC is called int indentity. So we need deal with this case.
---
 .../src/main/java/org/apache/doris/external/jdbc/JdbcClient.java     | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java b/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
index fb5c6dd23c..ffeec31e20 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
@@ -706,7 +706,10 @@ public class JdbcClient {
     }
 
     public Type sqlserverTypeToDoris(JdbcFieldSchema fieldSchema) {
-        String sqlserverType = fieldSchema.getDataTypeName();
+        String originSqlserverType = fieldSchema.getDataTypeName();
+        // For sqlserver IDENTITY type, such as 'INT IDENTITY'
+        // originSqlserverType is "int identity", so we only get "int".
+        String sqlserverType = originSqlserverType.split(" ")[0];
         switch (sqlserverType) {
             case "bit":
                 return Type.BOOLEAN;


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


[doris] 01/09: [minor](fe) ignore expired alter job when replay, to save memory (#18508)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit b2f45a0e9788fa65a53d36b3743bdf26649ed69a
Author: Mingyu Chen <mo...@163.com>
AuthorDate: Mon Apr 10 18:37:27 2023 +0800

    [minor](fe) ignore expired alter job when replay, to save memory (#18508)
    
    Skip expired alter job when replay
---
 fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java | 4 ++++
 1 file changed, 4 insertions(+)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
index 11cbd9aaaa..970d86584d 100755
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
@@ -1841,6 +1841,10 @@ public class Env {
         newChecksum ^= size;
         for (int i = 0; i < size; i++) {
             AlterJobV2 alterJobV2 = AlterJobV2.read(dis);
+            if (alterJobV2.isExpire()) {
+                LOG.info("alter job {} is expired, type: {}, ignore it", alterJobV2.getJobId(), alterJobV2.getType());
+                continue;
+            }
             if (type == JobType.ROLLUP || type == JobType.SCHEMA_CHANGE) {
                 if (type == JobType.ROLLUP) {
                     this.getMaterializedViewHandler().addAlterJobV2(alterJobV2);


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