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 2022/12/14 05:06:15 UTC

[doris] branch branch-1.2-lts updated (8c507acd71 -> c99d35a49b)

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 8c507acd71 [fix](if) fix coredump of if const (#14858)
     new 15561cfc9c [fix](multi-catalog) check new catalog name is used or not before rename (#14891)
     new fbab211d77 [fix] 'SHOW ROLES' statement does not display resource privilege (#14812) (#14897)
     new 8689265ea7 [Vectorized](jdbc) add check type for jdbc table (#14501)
     new efa4f52539 [fix](multi-catalog) use last used database for catalog when switch back (#14793)
     new e33708ded3 [improvement](config) update high_priority_flush_thread_num_per_store default value to 6 (#14775)
     new c99d35a49b [Enhancement](profile)  enhance column predicates display on profile (#14664)

The 6 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/common/config.h                             |   2 +-
 be/src/olap/bitmap_filter_predicate.h              |   2 +-
 be/src/olap/bloom_filter_predicate.h               |   2 +-
 be/src/olap/column_predicate.h                     |   4 +-
 be/src/olap/comparison_predicate.h                 |   5 +-
 be/src/olap/in_list_predicate.h                    |   5 +-
 be/src/olap/like_column_predicate.h                |   2 +-
 be/src/olap/null_predicate.h                       |   2 +-
 be/src/olap/rowset/segment_v2/segment_iterator.h   |  50 ++++----
 be/src/vec/exec/scan/new_olap_scanner.cpp          |   6 +-
 be/src/vec/exec/scan/new_olap_scanner.h            |   5 -
 be/src/vec/exec/vjdbc_connector.cpp                | 131 +++++++++++++++++++++
 be/src/vec/exec/vjdbc_connector.h                  |   4 +
 .../docs/ecosystem/external-table/jdbc-of-doris.md |  31 +++--
 .../docs/ecosystem/external-table/jdbc-of-doris.md |  30 ++---
 .../main/java/org/apache/doris/catalog/Env.java    |   9 ++
 .../org/apache/doris/datasource/CatalogMgr.java    |  22 ++++
 .../apache/doris/mysql/privilege/RoleManager.java  |  13 +-
 .../apache/doris/datasource/CatalogMgrTest.java    |  12 ++
 .../java/org/apache/doris/udf/JdbcExecutor.java    |  14 +++
 .../suites/jdbc_p0/test_jdbc_query_pg.groovy       |   4 +-
 21 files changed, 268 insertions(+), 87 deletions(-)


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


[doris] 05/06: [improvement](config) update high_priority_flush_thread_num_per_store default value to 6 (#14775)

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 e33708ded318c900b7b7b35f1a7fd3b0324bb6e0
Author: zhannngchen <48...@users.noreply.github.com>
AuthorDate: Wed Dec 7 15:31:00 2022 +0800

    [improvement](config) update high_priority_flush_thread_num_per_store default value to 6 (#14775)
---
 be/src/common/config.h | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/be/src/common/config.h b/be/src/common/config.h
index 4c256f168b..a77a63ecb8 100644
--- a/be/src/common/config.h
+++ b/be/src/common/config.h
@@ -564,7 +564,7 @@ CONF_mInt64(storage_flood_stage_left_capacity_bytes, "1073741824"); // 1GB
 // number of thread for flushing memtable per store
 CONF_Int32(flush_thread_num_per_store, "6");
 // number of thread for flushing memtable per store, for high priority load task
-CONF_Int32(high_priority_flush_thread_num_per_store, "1");
+CONF_Int32(high_priority_flush_thread_num_per_store, "6");
 
 // config for tablet meta checkpoint
 CONF_mInt32(tablet_meta_checkpoint_min_new_rowsets_num, "10");


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


[doris] 01/06: [fix](multi-catalog) check new catalog name is used or not before rename (#14891)

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 15561cfc9c9aee8c46c6b31b7e65e6ff606724bf
Author: Yulei-Yang <yu...@gmail.com>
AuthorDate: Wed Dec 7 21:54:44 2022 +0800

    [fix](multi-catalog) check new catalog name is used or not before rename (#14891)
---
 .../main/java/org/apache/doris/datasource/CatalogMgr.java    |  3 +++
 .../java/org/apache/doris/datasource/CatalogMgrTest.java     | 12 ++++++++++++
 2 files changed, 15 insertions(+)

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 419bf71f24..e1fbb55062 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
@@ -254,6 +254,9 @@ public class CatalogMgr implements Writable, GsonPostProcessable {
             if (catalog == null) {
                 throw new DdlException("No catalog found with name: " + stmt.getCatalogName());
             }
+            if (nameToCatalog.get(stmt.getNewCatalogName()) != null) {
+                throw new DdlException("Catalog with name " + stmt.getNewCatalogName() + " already exist");
+            }
             CatalogLog log = CatalogFactory.constructorCatalogLog(catalog.getId(), stmt);
             replayAlterCatalogName(log);
             Env.getCurrentEnv().getEditLog().logCatalogLog(OperationType.OP_ALTER_CATALOG_NAME, log);
diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/CatalogMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/CatalogMgrTest.java
index 647d5ee690..7340cd8f18 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/datasource/CatalogMgrTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/CatalogMgrTest.java
@@ -36,6 +36,7 @@ import org.apache.doris.catalog.external.HMSExternalDatabase;
 import org.apache.doris.catalog.external.HMSExternalTable;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
+import org.apache.doris.common.DdlException;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.mysql.privilege.PaloAuth;
 import org.apache.doris.qe.ConnectContext;
@@ -167,6 +168,17 @@ public class CatalogMgrTest extends TestWithFeService {
         showResultSet = mgr.showCatalogs(showStmt);
         Assertions.assertEquals(1, showResultSet.getResultRows().size());
 
+        String alterCatalogNameFailSql = "ALTER CATALOG hms_catalog RENAME hive;";
+        AlterCatalogNameStmt alterNameFailStmt = (AlterCatalogNameStmt) parseAndAnalyzeStmt(alterCatalogNameFailSql);
+
+        try {
+            mgr.alterCatalogName(alterNameFailStmt);
+            Assert.fail("Catalog with name hive already exist, rename should be failed");
+        } catch (DdlException e) {
+            Assert.assertEquals(e.getMessage(),
+                    "errCode = 2, detailMessage = Catalog with name hive already exist");
+        }
+
         String alterCatalogNameSql = "ALTER CATALOG hms_catalog RENAME " + MY_CATALOG + ";";
         AlterCatalogNameStmt alterNameStmt = (AlterCatalogNameStmt) parseAndAnalyzeStmt(alterCatalogNameSql);
         mgr.alterCatalogName(alterNameStmt);


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


[doris] 06/06: [Enhancement](profile) enhance column predicates display on profile (#14664)

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 c99d35a49b17794fae07bbe255607cc054b43e8e
Author: Pxl <px...@qq.com>
AuthorDate: Thu Dec 1 13:07:12 2022 +0800

    [Enhancement](profile)  enhance column predicates display on profile (#14664)
---
 be/src/olap/bitmap_filter_predicate.h            |  2 +-
 be/src/olap/bloom_filter_predicate.h             |  2 +-
 be/src/olap/column_predicate.h                   |  4 +-
 be/src/olap/comparison_predicate.h               |  5 +--
 be/src/olap/in_list_predicate.h                  |  5 +--
 be/src/olap/like_column_predicate.h              |  2 +-
 be/src/olap/null_predicate.h                     |  2 +-
 be/src/olap/rowset/segment_v2/segment_iterator.h | 50 +++++++++++-------------
 be/src/vec/exec/scan/new_olap_scanner.cpp        |  6 +--
 be/src/vec/exec/scan/new_olap_scanner.h          |  5 ---
 10 files changed, 34 insertions(+), 49 deletions(-)

diff --git a/be/src/olap/bitmap_filter_predicate.h b/be/src/olap/bitmap_filter_predicate.h
index 053cdb748d..9bf444b889 100644
--- a/be/src/olap/bitmap_filter_predicate.h
+++ b/be/src/olap/bitmap_filter_predicate.h
@@ -100,7 +100,7 @@ private:
         return new_size;
     }
 
-    std::string _debug_string() override {
+    std::string _debug_string() const override {
         return "BitmapFilterColumnPredicate(" + type_to_string(T) + ")";
     }
 
diff --git a/be/src/olap/bloom_filter_predicate.h b/be/src/olap/bloom_filter_predicate.h
index d9d6a964f3..efa00a5930 100644
--- a/be/src/olap/bloom_filter_predicate.h
+++ b/be/src/olap/bloom_filter_predicate.h
@@ -120,7 +120,7 @@ private:
         return new_size;
     }
 
-    std::string _debug_string() override {
+    std::string _debug_string() const override {
         std::string info = "BloomFilterColumnPredicate(" + type_to_string(T) + ")";
         return info;
     }
diff --git a/be/src/olap/column_predicate.h b/be/src/olap/column_predicate.h
index 7e1981cc86..9cdd22860f 100644
--- a/be/src/olap/column_predicate.h
+++ b/be/src/olap/column_predicate.h
@@ -166,13 +166,13 @@ public:
     }
     uint32_t column_id() const { return _column_id; }
 
-    virtual std::string debug_string() {
+    virtual std::string debug_string() const {
         return _debug_string() + ", column_id=" + std::to_string(_column_id) +
                ", opposite=" + (_opposite ? "true" : "false");
     }
 
 protected:
-    virtual std::string _debug_string() = 0;
+    virtual std::string _debug_string() const = 0;
 
     uint32_t _column_id;
     // TODO: the value is only in delete condition, better be template value
diff --git a/be/src/olap/comparison_predicate.h b/be/src/olap/comparison_predicate.h
index 95be483206..909f638813 100644
--- a/be/src/olap/comparison_predicate.h
+++ b/be/src/olap/comparison_predicate.h
@@ -601,7 +601,7 @@ private:
         }
     }
 
-    std::string _debug_string() override {
+    std::string _debug_string() const override {
         std::string info =
                 "ComparisonPredicateBase(" + type_to_string(Type) + ", " + type_to_string(PT) + ")";
         return info;
@@ -611,7 +611,4 @@ private:
     static constexpr PrimitiveType EvalType = (Type == TYPE_CHAR ? TYPE_STRING : Type);
 };
 
-template <PrimitiveType Type, PredicateType PT>
-constexpr PrimitiveType ComparisonPredicateBase<Type, PT>::EvalType;
-
 } //namespace doris
diff --git a/be/src/olap/in_list_predicate.h b/be/src/olap/in_list_predicate.h
index 1a9fac9dcd..503aa340b0 100644
--- a/be/src/olap/in_list_predicate.h
+++ b/be/src/olap/in_list_predicate.h
@@ -609,7 +609,7 @@ private:
         }
     }
 
-    std::string _debug_string() override {
+    std::string _debug_string() const override {
         std::string info =
                 "InListPredicateBase(" + type_to_string(Type) + ", " + type_to_string(PT) + ")";
         return info;
@@ -636,7 +636,4 @@ private:
     std::list<std::string> _temp_datas;
 };
 
-template <PrimitiveType Type, PredicateType PT>
-constexpr PrimitiveType InListPredicateBase<Type, PT>::EvalType;
-
 } //namespace doris
diff --git a/be/src/olap/like_column_predicate.h b/be/src/olap/like_column_predicate.h
index 85f30b5ab4..7a7e8be2cd 100644
--- a/be/src/olap/like_column_predicate.h
+++ b/be/src/olap/like_column_predicate.h
@@ -144,7 +144,7 @@ private:
         }
     }
 
-    std::string _debug_string() override {
+    std::string _debug_string() const override {
         std::string info = "LikeColumnPredicate";
         return info;
     }
diff --git a/be/src/olap/null_predicate.h b/be/src/olap/null_predicate.h
index 7155036bce..3910151f68 100644
--- a/be/src/olap/null_predicate.h
+++ b/be/src/olap/null_predicate.h
@@ -81,7 +81,7 @@ public:
     void evaluate_vec(const vectorized::IColumn& column, uint16_t size, bool* flags) const override;
 
 private:
-    std::string _debug_string() override {
+    std::string _debug_string() const override {
         std::string info = "NullPredicate(" + std::string(_is_null ? "is_null" : "not_null") + ")";
         return info;
     }
diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.h b/be/src/olap/rowset/segment_v2/segment_iterator.h
index 5a7a35773d..09dbc0db9c 100644
--- a/be/src/olap/rowset/segment_v2/segment_iterator.h
+++ b/be/src/olap/rowset/segment_v2/segment_iterator.h
@@ -66,37 +66,34 @@ public:
     uint64_t data_id() const override { return _segment->id(); }
 
     bool update_profile(RuntimeProfile* profile) override {
-        if (_short_cir_eval_predicate.empty() && _pre_eval_block_predicate.empty()) {
-            if (_col_predicates.empty()) {
-                return false;
-            }
-
-            std::string info;
-            for (auto pred : _col_predicates) {
-                info += "\n" + pred->debug_string();
-            }
-            profile->add_info_string("ColumnPredicates", info);
-        } else {
-            if (!_short_cir_eval_predicate.empty()) {
-                std::string info;
-                for (auto pred : _short_cir_eval_predicate) {
-                    info += "\n" + pred->debug_string();
-                }
-                profile->add_info_string("Short Circuit ColumnPredicates", info);
-            }
-            if (!_pre_eval_block_predicate.empty()) {
-                std::string info;
-                for (auto pred : _pre_eval_block_predicate) {
-                    info += "\n" + pred->debug_string();
-                }
-                profile->add_info_string("Pre Evaluate Block ColumnPredicates", info);
-            }
+        bool updated = false;
+        updated |= _update_profile(profile, _short_cir_eval_predicate, "ShortCircuitPredicates");
+        updated |= _update_profile(profile, _pre_eval_block_predicate, "PreEvaluatePredicates");
+
+        if (_opts.delete_condition_predicates != nullptr) {
+            std::set<const ColumnPredicate*> delete_predicate_set;
+            _opts.delete_condition_predicates->get_all_column_predicate(delete_predicate_set);
+            updated |= _update_profile(profile, delete_predicate_set, "DeleteConditionPredicates");
         }
 
-        return true;
+        return updated;
     }
 
 private:
+    template <typename Container>
+    bool _update_profile(RuntimeProfile* profile, const Container& predicates,
+                         const std::string& title) {
+        if (predicates.empty()) {
+            return false;
+        }
+        std::string info;
+        for (auto pred : predicates) {
+            info += "\n" + pred->debug_string();
+        }
+        profile->add_info_string(title, info);
+        return true;
+    }
+
     Status _init(bool is_vec = false);
 
     Status _init_return_column_iterators();
@@ -170,7 +167,6 @@ private:
 
     void _update_max_row(const vectorized::Block* block);
 
-private:
     class BitmapRangeIterator;
     class BackwardBitmapRangeIterator;
 
diff --git a/be/src/vec/exec/scan/new_olap_scanner.cpp b/be/src/vec/exec/scan/new_olap_scanner.cpp
index 224affa4bc..fe1a010df4 100644
--- a/be/src/vec/exec/scan/new_olap_scanner.cpp
+++ b/be/src/vec/exec/scan/new_olap_scanner.cpp
@@ -309,13 +309,13 @@ Status NewOlapScanner::_init_return_columns() {
 }
 
 Status NewOlapScanner::_get_block_impl(RuntimeState* state, Block* block, bool* eof) {
-    if (!_profile_updated) {
-        _profile_updated = _tablet_reader->update_profile(_profile);
-    }
     // Read one block from block reader
     // ATTN: Here we need to let the _get_block_impl method guarantee the semantics of the interface,
     // that is, eof can be set to true only when the returned block is empty.
     RETURN_IF_ERROR(_tablet_reader->next_block_with_aggregation(block, nullptr, nullptr, eof));
+    if (!_profile_updated) {
+        _profile_updated = _tablet_reader->update_profile(_profile);
+    }
     if (block->rows() > 0) {
         *eof = false;
     }
diff --git a/be/src/vec/exec/scan/new_olap_scanner.h b/be/src/vec/exec/scan/new_olap_scanner.h
index e07f38ea2b..5f253ea93f 100644
--- a/be/src/vec/exec/scan/new_olap_scanner.h
+++ b/be/src/vec/exec/scan/new_olap_scanner.h
@@ -18,10 +18,7 @@
 #pragma once
 
 #include "exec/olap_utils.h"
-#include "exprs/bitmapfilter_predicate.h"
-#include "exprs/bloomfilter_predicate.h"
 #include "exprs/function_filter.h"
-#include "exprs/hybrid_set.h"
 #include "olap/reader.h"
 #include "util/runtime_profile.h"
 #include "vec/exec/scan/vscanner.h"
@@ -45,7 +42,6 @@ public:
 
     Status close(RuntimeState* state) override;
 
-public:
     Status prepare(const TPaloScanRange& scan_range, const std::vector<OlapScanRange*>& key_ranges,
                    VExprContext** vconjunct_ctx_ptr, const std::vector<TCondition>& filters,
                    const FilterPredicates& filter_predicates,
@@ -67,7 +63,6 @@ private:
 
     Status _init_return_columns();
 
-private:
     bool _aggregation;
     bool _need_agg_finalize;
 


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


[doris] 02/06: [fix] 'SHOW ROLES' statement does not display resource privilege (#14812) (#14897)

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 fbab211d776a1990c9d63adeb8b2d9efdfad8da2
Author: xu tao <xu...@163.com>
AuthorDate: Thu Dec 8 10:22:09 2022 +0800

    [fix] 'SHOW ROLES' statement does not display resource privilege (#14812) (#14897)
---
 .../java/org/apache/doris/mysql/privilege/RoleManager.java  | 13 +++++++++----
 1 file changed, 9 insertions(+), 4 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/RoleManager.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/RoleManager.java
index 09a7d3aadf..0a31786b84 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/RoleManager.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/RoleManager.java
@@ -145,9 +145,13 @@ public class RoleManager implements Writable {
             info.add(role.getRoleName());
             info.add(Joiner.on(", ").join(role.getUsers()));
 
-            Map<PrivLevel, String> infoMap = role.getTblPatternToPrivs().entrySet().stream()
-                    .collect(Collectors.groupingBy(entry -> entry.getKey().getPrivLevel())).entrySet().stream()
-                    .collect(Collectors.toMap(Entry::getKey, entry -> {
+            Map<PrivLevel, String> infoMap =
+                    Stream.concat(
+                        role.getTblPatternToPrivs().entrySet().stream()
+                            .collect(Collectors.groupingBy(entry -> entry.getKey().getPrivLevel())).entrySet().stream(),
+                        role.getResourcePatternToPrivs().entrySet().stream()
+                            .collect(Collectors.groupingBy(entry -> entry.getKey().getPrivLevel())).entrySet().stream()
+                    ).collect(Collectors.toMap(Entry::getKey, entry -> {
                         if (entry.getKey() == PrivLevel.GLOBAL) {
                             return entry.getValue().stream().findFirst().map(priv -> priv.getValue().toString())
                                     .orElse(FeConstants.null_string);
@@ -156,7 +160,8 @@ public class RoleManager implements Writable {
                                     .map(priv -> priv.getKey() + ": " + priv.getValue())
                                     .collect(Collectors.joining("; "));
                         }
-                    }));
+                    }, (s1, s2) -> s1 + " " + s2
+                ));
             Stream.of(PrivLevel.GLOBAL, PrivLevel.CATALOG, PrivLevel.DATABASE, PrivLevel.TABLE, PrivLevel.RESOURCE)
                     .forEach(level -> {
                         String infoItem = infoMap.get(level);


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


[doris] 04/06: [fix](multi-catalog) use last used database for catalog when switch back (#14793)

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 efa4f525397427ab8cc9ac788dca2b9abe22d830
Author: Yulei-Yang <yu...@gmail.com>
AuthorDate: Thu Dec 8 10:32:30 2022 +0800

    [fix](multi-catalog) use last used database for catalog when switch back (#14793)
    
    remember last used database of every catalog and use it when switch back
---
 .../src/main/java/org/apache/doris/catalog/Env.java   |  9 +++++++++
 .../java/org/apache/doris/datasource/CatalogMgr.java  | 19 +++++++++++++++++++
 2 files changed, 28 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 df08131e42..7c0e261609 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
@@ -4439,7 +4439,16 @@ public class Env {
             throw new DdlException(ErrorCode.ERR_UNKNOWN_CATALOG.formatErrorMsg(catalogName),
                     ErrorCode.ERR_UNKNOWN_CATALOG);
         }
+
+        String currentDB = ctx.getDatabase();
+        if (StringUtils.isNotEmpty(currentDB)) {
+            catalogMgr.addLastDBOfCatalog(ctx.getCurrentCatalog().getName(), currentDB);
+        }
         ctx.changeDefaultCatalog(catalogName);
+        String lastDb = catalogMgr.getLastDB(catalogName);
+        if (StringUtils.isNotEmpty(lastDb)) {
+            ctx.setDatabase(lastDb);
+        }
         if (catalogIf instanceof EsExternalCatalog) {
             ctx.setDatabase(SystemInfoService.DEFAULT_CLUSTER + ClusterNamespace.CLUSTER_DELIMITER
                     + EsExternalCatalog.DEFAULT_DB);
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 e1fbb55062..ecbcdf8790 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
@@ -74,6 +74,8 @@ public class CatalogMgr implements Writable, GsonPostProcessable {
     private final Map<Long, CatalogIf> idToCatalog = Maps.newConcurrentMap();
     // this map will be regenerated from idToCatalog, so not need to persist.
     private final Map<String, CatalogIf> nameToCatalog = Maps.newConcurrentMap();
+    // record last used database of every catalog
+    private final Map<String, String> lastDBOfCatalog = Maps.newConcurrentMap();
 
     // Use a separate instance to facilitate access.
     // internalDataSource still exists in idToDataSource and nameToDataSource
@@ -103,6 +105,7 @@ public class CatalogMgr implements Writable, GsonPostProcessable {
         if (catalog != null) {
             catalog.onClose();
             nameToCatalog.remove(catalog.getName());
+            lastDBOfCatalog.remove(catalog.getName());
             Env.getCurrentEnv().getExtMetaCacheMgr().removeCache(catalog.getName());
         }
         return catalog;
@@ -144,6 +147,14 @@ public class CatalogMgr implements Writable, GsonPostProcessable {
                         ErrorCode.ERR_UNKNOWN_CATALOG));
     }
 
+    public void addLastDBOfCatalog(String catalog, String db) {
+        lastDBOfCatalog.put(catalog, db);
+    }
+
+    public String getLastDB(String catalog) {
+        return lastDBOfCatalog.get(catalog);
+    }
+
     public List<Long> getCatalogIds() {
         return Lists.newArrayList(idToCatalog.keySet());
     }
@@ -239,6 +250,8 @@ public class CatalogMgr implements Writable, GsonPostProcessable {
             CatalogLog log = CatalogFactory.constructorCatalogLog(catalog.getId(), stmt);
             replayDropCatalog(log);
             Env.getCurrentEnv().getEditLog().logCatalogLog(OperationType.OP_DROP_CATALOG, log);
+
+            lastDBOfCatalog.remove(stmt.getCatalogName());
         } finally {
             writeUnlock();
         }
@@ -260,6 +273,12 @@ public class CatalogMgr implements Writable, GsonPostProcessable {
             CatalogLog log = CatalogFactory.constructorCatalogLog(catalog.getId(), stmt);
             replayAlterCatalogName(log);
             Env.getCurrentEnv().getEditLog().logCatalogLog(OperationType.OP_ALTER_CATALOG_NAME, log);
+
+            String db = lastDBOfCatalog.get(stmt.getCatalogName());
+            if (db != null) {
+                lastDBOfCatalog.remove(stmt.getCatalogName());
+                lastDBOfCatalog.put(log.getNewCatalogName(), db);
+            }
         } finally {
             writeUnlock();
         }


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


[doris] 03/06: [Vectorized](jdbc) add check type for jdbc table (#14501)

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 8689265ea7f82db17a719286dde156f2a33ec617
Author: zhangstar333 <87...@users.noreply.github.com>
AuthorDate: Thu Dec 8 10:27:47 2022 +0800

    [Vectorized](jdbc) add check type for jdbc table (#14501)
---
 be/src/vec/exec/vjdbc_connector.cpp                | 131 +++++++++++++++++++++
 be/src/vec/exec/vjdbc_connector.h                  |   4 +
 .../docs/ecosystem/external-table/jdbc-of-doris.md |  31 +++--
 .../docs/ecosystem/external-table/jdbc-of-doris.md |  30 ++---
 .../java/org/apache/doris/udf/JdbcExecutor.java    |  14 +++
 .../suites/jdbc_p0/test_jdbc_query_pg.groovy       |   4 +-
 6 files changed, 181 insertions(+), 33 deletions(-)

diff --git a/be/src/vec/exec/vjdbc_connector.cpp b/be/src/vec/exec/vjdbc_connector.cpp
index f3a7688d38..ce171ad569 100644
--- a/be/src/vec/exec/vjdbc_connector.cpp
+++ b/be/src/vec/exec/vjdbc_connector.cpp
@@ -22,6 +22,7 @@
 #include "gen_cpp/Types_types.h"
 #include "gutil/strings/substitute.h"
 #include "jni.h"
+#include "runtime/define_primitive_type.h"
 #include "runtime/user_function_cache.h"
 #include "util/jni-util.h"
 #include "vec/columns/column_nullable.h"
@@ -34,6 +35,7 @@ const char* JDBC_EXECUTOR_CTOR_SIGNATURE = "([B)V";
 const char* JDBC_EXECUTOR_WRITE_SIGNATURE = "(Ljava/lang/String;)I";
 const char* JDBC_EXECUTOR_HAS_NEXT_SIGNATURE = "()Z";
 const char* JDBC_EXECUTOR_GET_BLOCK_SIGNATURE = "(I)Ljava/util/List;";
+const char* JDBC_EXECUTOR_GET_TYPES_SIGNATURE = "()Ljava/util/List;";
 const char* JDBC_EXECUTOR_CLOSE_SIGNATURE = "()V";
 const char* JDBC_EXECUTOR_CONVERT_DATE_SIGNATURE = "(Ljava/lang/Object;)J";
 const char* JDBC_EXECUTOR_CONVERT_DATETIME_SIGNATURE = "(Ljava/lang/Object;)J";
@@ -172,6 +174,125 @@ Status JdbcConnector::query() {
     if (colunm_count != materialize_num) {
         return Status::InternalError("input and output column num not equal of jdbc query.");
     }
+    LOG(INFO) << "JdbcConnector::query has exec success: " << _sql_str;
+    RETURN_IF_ERROR(_check_column_type());
+    return Status::OK();
+}
+
+Status JdbcConnector::_check_column_type() {
+    JNIEnv* env = nullptr;
+    RETURN_IF_ERROR(JniUtil::GetJNIEnv(&env));
+    jobject type_lists =
+            env->CallNonvirtualObjectMethod(_executor_obj, _executor_clazz, _executor_get_types_id);
+    auto column_size = _tuple_desc->slots().size();
+    for (int column_index = 0, materialized_column_index = 0; column_index < column_size;
+         ++column_index) {
+        auto slot_desc = _tuple_desc->slots()[column_index];
+        if (!slot_desc->is_materialized()) {
+            continue;
+        }
+        jobject column_type =
+                env->CallObjectMethod(type_lists, _executor_get_list_id, materialized_column_index);
+
+        const std::string& type_str = _jobject_to_string(env, column_type);
+        RETURN_IF_ERROR(_check_type(slot_desc, type_str));
+        env->DeleteLocalRef(column_type);
+        materialized_column_index++;
+    }
+    env->DeleteLocalRef(type_lists);
+    return JniUtil::GetJniExceptionMsg(env);
+}
+/* type mapping: https://doris.apache.org/zh-CN/docs/dev/ecosystem/external-table/jdbc-of-doris?_highlight=jdbc
+
+Doris            MYSQL                      PostgreSQL                  Oracle                      SQLServer
+
+BOOLEAN      java.lang.Boolean          java.lang.Boolean                                       java.lang.Boolean
+TINYINT      java.lang.Integer                                                                  java.lang.Short    
+SMALLINT     java.lang.Integer          java.lang.Integer           java.math.BigDecimal        java.lang.Short    
+INT          java.lang.Integer          java.lang.Integer           java.math.BigDecimal        java.lang.Integer
+BIGINT       java.lang.Long             java.lang.Long                                          java.lang.Long
+LARGET       java.math.BigInteger
+DECIMAL      java.math.BigDecimal       java.math.BigDecimal        java.math.BigDecimal        java.math.BigDecimal
+VARCHAR      java.lang.String           java.lang.String            java.lang.String            java.lang.String
+DOUBLE       java.lang.Double           java.lang.Double            java.lang.Double            java.lang.Double
+FLOAT        java.lang.Float            java.lang.Float                                         java.lang.Float
+DATE         java.sql.Date              java.sql.Date                                           java.sql.Date
+DATETIME     java.sql.Timestamp         java.sql.Timestamp          java.sql.Timestamp          java.sql.Timestamp
+
+NOTE: because oracle always use number(p,s) to create all numerical type, so it's java type maybe java.math.BigDecimal
+*/
+
+Status JdbcConnector::_check_type(SlotDescriptor* slot_desc, const std::string& type_str) {
+    const std::string error_msg = fmt::format(
+            "Fail to convert jdbc type of {} to doris type {} on column: {}. You need to "
+            "check this column type between external table and doris table.",
+            type_str, slot_desc->type().debug_string(), slot_desc->col_name());
+    switch (slot_desc->type().type) {
+    case TYPE_BOOLEAN: {
+        if (type_str != "java.lang.Boolean" && type_str != "java.math.BigDecimal") {
+            return Status::InternalError(error_msg);
+        }
+        break;
+    }
+    case TYPE_TINYINT:
+    case TYPE_SMALLINT:
+    case TYPE_INT: {
+        if (type_str != "java.lang.Short" && type_str != "java.lang.Integer" &&
+            type_str != "java.math.BigDecimal") {
+            return Status::InternalError(error_msg);
+        }
+        break;
+    }
+    case TYPE_BIGINT:
+    case TYPE_LARGEINT: {
+        if (type_str != "java.lang.Long" && type_str != "java.math.BigDecimal" &&
+            type_str != "java.math.BigInteger") {
+            return Status::InternalError(error_msg);
+        }
+        break;
+    }
+    case TYPE_FLOAT: {
+        if (type_str != "java.lang.Float" && type_str != "java.math.BigDecimal") {
+            return Status::InternalError(error_msg);
+        }
+        break;
+    }
+    case TYPE_DOUBLE: {
+        if (type_str != "java.lang.Double" && type_str != "java.math.BigDecimal") {
+            return Status::InternalError(error_msg);
+        }
+        break;
+    }
+    case TYPE_CHAR:
+    case TYPE_VARCHAR:
+    case TYPE_STRING: {
+        //now here break directly
+        break;
+    }
+    case TYPE_DATE:
+    case TYPE_DATEV2:
+    case TYPE_TIMEV2:
+    case TYPE_DATETIME:
+    case TYPE_DATETIMEV2: {
+        if (type_str != "java.sql.Timestamp" && type_str != "java.time.LocalDateTime" &&
+            type_str != "java.sql.Date") {
+            return Status::InternalError(error_msg);
+        }
+        break;
+    }
+    case TYPE_DECIMALV2:
+    case TYPE_DECIMAL32:
+    case TYPE_DECIMAL64:
+    case TYPE_DECIMAL128I: {
+        if (type_str != "java.math.BigDecimal") {
+            return Status::InternalError(error_msg);
+        }
+        break;
+    }
+    default: {
+        return Status::InternalError(error_msg);
+    }
+    }
     return Status::OK();
 }
 
@@ -262,6 +383,8 @@ Status JdbcConnector::_register_func_id(JNIEnv* env) {
                                 _executor_finish_trans_id));
     RETURN_IF_ERROR(register_id(_executor_clazz, "rollbackTrans",
                                 JDBC_EXECUTOR_TRANSACTION_SIGNATURE, _executor_abort_trans_id));
+    RETURN_IF_ERROR(register_id(_executor_clazz, "getResultColumnTypeNames",
+                                JDBC_EXECUTOR_GET_TYPES_SIGNATURE, _executor_get_types_id));
     return Status::OK();
 }
 
@@ -314,6 +437,14 @@ Status JdbcConnector::_convert_column_data(JNIEnv* env, jobject jobj,
         reinterpret_cast<vectorized::ColumnVector<vectorized::Int64>*>(col_ptr)->insert_value(num);
         break;
     }
+    case TYPE_LARGEINT: {
+        StringParser::ParseResult parse_result = StringParser::PARSE_SUCCESS;
+        std::string data = _jobject_to_string(env, jobj);
+        __int128 num =
+                StringParser::string_to_int<__int128>(data.data(), data.size(), &parse_result);
+        reinterpret_cast<vectorized::ColumnVector<vectorized::Int128>*>(col_ptr)->insert_value(num);
+        break;
+    }
     case TYPE_DECIMALV2: {
         std::string data = _jobject_to_string(env, jobj);
         DecimalV2Value decimal_slot;
diff --git a/be/src/vec/exec/vjdbc_connector.h b/be/src/vec/exec/vjdbc_connector.h
index f3575dfd52..e6da1a0151 100644
--- a/be/src/vec/exec/vjdbc_connector.h
+++ b/be/src/vec/exec/vjdbc_connector.h
@@ -19,6 +19,7 @@
 
 #include <jni.h>
 
+#include "common/status.h"
 #include "exec/table_connector.h"
 
 namespace doris {
@@ -60,6 +61,8 @@ public:
 
 private:
     Status _register_func_id(JNIEnv* env);
+    Status _check_column_type();
+    Status _check_type(SlotDescriptor*, const std::string& type_str);
     Status _convert_column_data(JNIEnv* env, jobject jobj, const SlotDescriptor* slot_desc,
                                 vectorized::IColumn* column_ptr);
     std::string _jobject_to_string(JNIEnv* env, jobject jobj);
@@ -78,6 +81,7 @@ private:
     jmethodID _executor_read_id;
     jmethodID _executor_has_next_id;
     jmethodID _executor_get_blocks_id;
+    jmethodID _executor_get_types_id;
     jmethodID _executor_close_id;
     jmethodID _executor_get_list_id;
     jmethodID _executor_get_list_size_id;
diff --git a/docs/en/docs/ecosystem/external-table/jdbc-of-doris.md b/docs/en/docs/ecosystem/external-table/jdbc-of-doris.md
index 552cd7e848..8d19f7ffab 100644
--- a/docs/en/docs/ecosystem/external-table/jdbc-of-doris.md
+++ b/docs/en/docs/ecosystem/external-table/jdbc-of-doris.md
@@ -162,16 +162,16 @@ There are different data types among different databases. Here is a list of the
 |  MySQL   |  Doris   |
 | :------: | :------: |
 | BOOLEAN  | BOOLEAN  |
-|   CHAR   |   CHAR   |
-| VARCHAR  | VARCHAR  |
-|   DATE   |   DATE   |
-|  FLOAT   |  FLOAT   |
 | TINYINT  | TINYINT  |
 | SMALLINT | SMALLINT |
 |   INT    |   INT    |
 |  BIGINT  |  BIGINT  |
-|  DOUBLE  |  DOUBLE  |
+|BIGINT UNSIGNED|LARGEINT|
+| VARCHAR  | VARCHAR  |
+|   DATE   |   DATE   |
+|  FLOAT   |  FLOAT   |
 | DATETIME | DATETIME |
+|  DOUBLE  |  DOUBLE  |
 | DECIMAL  | DECIMAL  |
 
 
@@ -180,43 +180,42 @@ There are different data types among different databases. Here is a list of the
 |    PostgreSQL    |  Doris   |
 | :--------------: | :------: |
 |     BOOLEAN      | BOOLEAN  |
-|       CHAR       |   CHAR   |
-|     VARCHAR      | VARCHAR  |
-|       DATE       |   DATE   |
-|       REAL       |  FLOAT   |
 |     SMALLINT     | SMALLINT |
 |       INT        |   INT    |
 |      BIGINT      |  BIGINT  |
-| DOUBLE PRECISION |  DOUBLE  |
+|     VARCHAR      | VARCHAR  |
+|       DATE       |   DATE   |
 |    TIMESTAMP     | DATETIME |
+|       REAL       |  FLOAT   |
+|      FLOAT       |  DOUBLE  |
 |     DECIMAL      | DECIMAL  |
 
 ### Oracle
 
 |  Oracle  |  Doris   |
 | :------: | :------: |
-|   CHAR   |   CHAR   |
 | VARCHAR  | VARCHAR  |
 |   DATE   | DATETIME |
 | SMALLINT | SMALLINT |
 |   INT    |   INT    |
+|   REAL   |   FLOAT  |
+|   FLOAT  |   DOUBLE |
 |  NUMBER  | DECIMAL  |
 
-
 ### SQL server
 
 | SQLServer |  Doris   |
 | :-------: | :------: |
 |    BIT    | BOOLEAN  |
-|   CHAR    |   CHAR   |
-|  VARCHAR  | VARCHAR  |
-|   DATE    |   DATE   |
-|   REAL    |  FLOAT   |
 |  TINYINT  | TINYINT  |
 | SMALLINT  | SMALLINT |
 |    INT    |   INT    |
 |  BIGINT   |  BIGINT  |
+|  VARCHAR  | VARCHAR  |
+|   DATE    |   DATE   |
 | DATETIME  | DATETIME |
+|   REAL    |  FLOAT   |
+|   FLOAT   |  DOUBLE  |
 |  DECIMAL  | DECIMAL  |
 
 ### ClickHouse
diff --git a/docs/zh-CN/docs/ecosystem/external-table/jdbc-of-doris.md b/docs/zh-CN/docs/ecosystem/external-table/jdbc-of-doris.md
index d333e8850f..4ef47e1d66 100644
--- a/docs/zh-CN/docs/ecosystem/external-table/jdbc-of-doris.md
+++ b/docs/zh-CN/docs/ecosystem/external-table/jdbc-of-doris.md
@@ -165,16 +165,16 @@ PROPERTIES (
 |  MySQL   |  Doris   |
 | :------: | :------: |
 | BOOLEAN  | BOOLEAN  |
-|   CHAR   |   CHAR   |
-| VARCHAR  | VARCHAR  |
-|   DATE   |   DATE   |
-|  FLOAT   |  FLOAT   |
 | TINYINT  | TINYINT  |
 | SMALLINT | SMALLINT |
 |   INT    |   INT    |
 |  BIGINT  |  BIGINT  |
-|  DOUBLE  |  DOUBLE  |
+|BIGINT UNSIGNED|LARGEINT|
+| VARCHAR  | VARCHAR  |
+|   DATE   |   DATE   |
+|  FLOAT   |  FLOAT   |
 | DATETIME | DATETIME |
+|  DOUBLE  |  DOUBLE  |
 | DECIMAL  | DECIMAL  |
 
 
@@ -183,26 +183,26 @@ PROPERTIES (
 |    PostgreSQL    |  Doris   |
 | :--------------: | :------: |
 |     BOOLEAN      | BOOLEAN  |
-|       CHAR       |   CHAR   |
-|     VARCHAR      | VARCHAR  |
-|       DATE       |   DATE   |
-|       REAL       |  FLOAT   |
 |     SMALLINT     | SMALLINT |
 |       INT        |   INT    |
 |      BIGINT      |  BIGINT  |
-| DOUBLE PRECISION |  DOUBLE  |
+|     VARCHAR      | VARCHAR  |
+|       DATE       |   DATE   |
 |    TIMESTAMP     | DATETIME |
+|       REAL       |  FLOAT   |
+|      FLOAT       |  DOUBLE  |
 |     DECIMAL      | DECIMAL  |
 
 ### Oracle
 
 |  Oracle  |  Doris   |
 | :------: | :------: |
-|   CHAR   |   CHAR   |
 | VARCHAR  | VARCHAR  |
 |   DATE   | DATETIME |
 | SMALLINT | SMALLINT |
 |   INT    |   INT    |
+|   REAL   |   FLOAT  |
+|   FLOAT  |   DOUBLE |
 |  NUMBER  | DECIMAL  |
 
 
@@ -211,15 +211,15 @@ PROPERTIES (
 | SQLServer |  Doris   |
 | :-------: | :------: |
 |    BIT    | BOOLEAN  |
-|   CHAR    |   CHAR   |
-|  VARCHAR  | VARCHAR  |
-|   DATE    |   DATE   |
-|   REAL    |  FLOAT   |
 |  TINYINT  | TINYINT  |
 | SMALLINT  | SMALLINT |
 |    INT    |   INT    |
 |  BIGINT   |  BIGINT  |
+|  VARCHAR  | VARCHAR  |
+|   DATE    |   DATE   |
 | DATETIME  | DATETIME |
+|   REAL    |  FLOAT   |
+|   FLOAT   |  DOUBLE  |
 |  DECIMAL  | DECIMAL  |
 
 ### ClickHouse
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 0a267bd9a6..9539f65aba 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
@@ -53,6 +53,7 @@ public class JdbcExecutor {
     private ResultSetMetaData resultSetMetaData = null;
     // Use HikariDataSource to help us manage the JDBC connections.
     private HikariDataSource dataSource = null;
+    private List<String> resultColumnTypeNames = null;
 
     public JdbcExecutor(byte[] thriftParams) throws Exception {
         TJdbcExecutorCtorParams request = new TJdbcExecutorCtorParams();
@@ -110,6 +111,19 @@ public class JdbcExecutor {
         }
     }
 
+    public List<String> getResultColumnTypeNames() throws UdfRuntimeException {
+        try {
+            int count = resultSetMetaData.getColumnCount();
+            resultColumnTypeNames = new ArrayList<>(count);
+            for (int i = 0; i < count; ++i) {
+                resultColumnTypeNames.add(resultSetMetaData.getColumnClassName(i + 1));
+            }
+            return resultColumnTypeNames;
+        } catch (SQLException e) {
+            throw new UdfRuntimeException("JDBC executor getResultColumnTypeNames has error: ", e);
+        }
+    }
+
     public void openTrans() throws UdfRuntimeException {
         try {
             if (conn != null) {
diff --git a/regression-test/suites/jdbc_p0/test_jdbc_query_pg.groovy b/regression-test/suites/jdbc_p0/test_jdbc_query_pg.groovy
index b84ee57072..a18156527c 100644
--- a/regression-test/suites/jdbc_p0/test_jdbc_query_pg.groovy
+++ b/regression-test/suites/jdbc_p0/test_jdbc_query_pg.groovy
@@ -55,7 +55,7 @@ suite("test_jdbc_query_pg", "p0") {
                 k2 char(100),
                 k3 varchar(128),
                 k4 date,
-                k5 double,
+                k5 float,
                 k6 smallint,
                 k7 int,
                 k8 bigint,
@@ -209,7 +209,7 @@ suite("test_jdbc_query_pg", "p0") {
                 `m_time` DATETIME NULL,
                 `app_id` BIGINT(20) NULL,
                 `t_id` BIGINT(20) NULL,
-                `deleted` TEXT NULL,
+                `deleted` boolean NULL,
                 `w_t_s` DATETIME NULL,
                 `rf_id` TEXT NULL,
                 `e_info` TEXT NULL,


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