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/01/10 15:49:44 UTC

[doris] branch branch-1.2-lts updated (b7d2431ac3 -> 7e9222733a)

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 b7d2431ac3 [fix](having) revert 15143 and fix having clause with multi-conditions (#15745)
     new 221149e590 fix(ui): 1. fix component/table can not change pageSize,affect system/query profile/session page etc. (#15533)
     new 4b69d20152 [fix](tvf) use virtual-hosted style when s3('uri'='s3://xxx') (#15617)
     new 3e700f325e [fix](planner) disconjunct in sub-query failed when plan it on hash join (#15653)
     new 7e9222733a [feature-wip](multi-catalog) add iceberg tvf to read snapshots (#15618)

The 4 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/exec/exec_node.cpp                          |  10 +-
 be/src/runtime/plan_fragment_executor.cpp          |   4 +-
 be/src/vec/CMakeLists.txt                          |   2 +
 ...{new_odbc_scan_node.cpp => vmeta_scan_node.cpp} |  48 +++--
 .../{new_file_scan_node.h => vmeta_scan_node.h}    |  26 ++-
 be/src/vec/exec/scan/vmeta_scanner.cpp             | 209 +++++++++++++++++++++
 .../scan/{new_jdbc_scanner.h => vmeta_scanner.h}   |  38 ++--
 .../apache/doris/planner/SingleNodePlanner.java    |   2 +-
 .../doris/planner/external/MetadataScanNode.java   | 114 +++++++++++
 .../apache/doris/service/FrontendServiceImpl.java  |  93 +++++++++
 .../apache/doris/statistics/StatisticalType.java   |   1 +
 .../tablefunction/IcebergTableValuedFunction.java  | 123 ++++++++++++
 .../MetadataTableValuedFunction.java}              |  31 +--
 .../doris/tablefunction/S3TableValuedFunction.java |   4 +-
 .../doris/tablefunction/TableValuedFunctionIf.java |   2 +
 gensrc/thrift/FrontendService.thrift               |   9 +
 gensrc/thrift/PlanNodes.thrift                     |  18 +-
 ui/src/api/api.ts                                  |  64 ++++---
 ui/src/components/table/index.tsx                  |  48 ++---
 ui/src/i18n.tsx                                    |  15 +-
 ui/src/pages/configuration/index.tsx               |  37 ++--
 ui/src/pages/logs/index.tsx                        |  56 +++---
 .../playground/content/components/data-prev.tsx    |  85 +++++----
 ui/src/pages/playground/page-side/index.tsx        |  20 +-
 ui/src/pages/playground/tree/index.tsx             | 164 ++++++++--------
 ui/src/pages/query-profile/index.tsx               |  80 ++++----
 ui/src/pages/session/index.tsx                     |  45 +++--
 ui/src/pages/system/index.tsx                      |  61 +++---
 ui/src/utils/lazy.tsx                              |  17 +-
 ui/src/utils/utils.ts                              |  37 ++--
 ui/tsconfig.json                                   |  94 ++++-----
 31 files changed, 1098 insertions(+), 459 deletions(-)
 copy be/src/vec/exec/scan/{new_odbc_scan_node.cpp => vmeta_scan_node.cpp} (50%)
 copy be/src/vec/exec/scan/{new_file_scan_node.h => vmeta_scan_node.h} (73%)
 create mode 100644 be/src/vec/exec/scan/vmeta_scanner.cpp
 copy be/src/vec/exec/scan/{new_jdbc_scanner.h => vmeta_scanner.h} (60%)
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/planner/external/MetadataScanNode.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/doris/tablefunction/IcebergTableValuedFunction.java
 copy fe/fe-core/src/main/java/org/apache/doris/{common/profile/PlanTreeNode.java => tablefunction/MetadataTableValuedFunction.java} (54%)


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


[doris] 04/04: [feature-wip](multi-catalog) add iceberg tvf to read snapshots (#15618)

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 7e9222733a778cc695f5246db37eac71b49c9995
Author: slothever <18...@users.noreply.github.com>
AuthorDate: Tue Jan 10 22:37:35 2023 +0800

    [feature-wip](multi-catalog) add iceberg tvf to read snapshots (#15618)
    
    Support new table value function `iceberg_meta("table" = "ctl.db.tbl", "query_type" = "snapshots")`
    we can use the sql `select * from iceberg_meta("table" = "ctl.db.tbl", "query_type" = "snapshots")` to get snapshots info  of a table. The other iceberg metadata will be supported later when needed.
    
    One of the usage:
    
    Before we use following sql to time travel:
    `select * from ice_table FOR TIME AS OF "2022-10-10 11:11:11"`;
    `select * from ice_table FOR VERSION AS OF "snapshot_id"`;
    we can use the snapshots metadata to get the `committed time` or `snapshot_id`,
    and then, we can use it as the time or version in time travel clause
---
 be/src/exec/exec_node.cpp                          |  10 +-
 be/src/runtime/plan_fragment_executor.cpp          |   4 +-
 be/src/vec/CMakeLists.txt                          |   2 +
 be/src/vec/exec/scan/vmeta_scan_node.cpp           |  68 +++++++
 be/src/vec/exec/scan/vmeta_scan_node.h             |  45 +++++
 be/src/vec/exec/scan/vmeta_scanner.cpp             | 209 +++++++++++++++++++++
 be/src/vec/exec/scan/vmeta_scanner.h               |  48 +++++
 .../doris/planner/external/MetadataScanNode.java   | 114 +++++++++++
 .../apache/doris/service/FrontendServiceImpl.java  |  93 +++++++++
 .../apache/doris/statistics/StatisticalType.java   |   1 +
 .../tablefunction/IcebergTableValuedFunction.java  | 123 ++++++++++++
 .../tablefunction/MetadataTableValuedFunction.java |  46 +++++
 .../doris/tablefunction/TableValuedFunctionIf.java |   2 +
 gensrc/thrift/FrontendService.thrift               |   9 +
 gensrc/thrift/PlanNodes.thrift                     |  18 +-
 15 files changed, 787 insertions(+), 5 deletions(-)

diff --git a/be/src/exec/exec_node.cpp b/be/src/exec/exec_node.cpp
index e4bf0b3b13..d804533e88 100644
--- a/be/src/exec/exec_node.cpp
+++ b/be/src/exec/exec_node.cpp
@@ -66,6 +66,7 @@
 #include "vec/exec/scan/new_jdbc_scan_node.h"
 #include "vec/exec/scan/new_odbc_scan_node.h"
 #include "vec/exec/scan/new_olap_scan_node.h"
+#include "vec/exec/scan/vmeta_scan_node.h"
 #include "vec/exec/vaggregation_node.h"
 #include "vec/exec/vanalytic_eval_node.h"
 #include "vec/exec/vassert_num_rows_node.h"
@@ -416,6 +417,7 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN
         case TPlanNodeType::DATA_GEN_SCAN_NODE:
         case TPlanNodeType::FILE_SCAN_NODE:
         case TPlanNodeType::JDBC_SCAN_NODE:
+        case TPlanNodeType::META_SCAN_NODE:
             break;
         default: {
             const auto& i = _TPlanNodeType_VALUES_TO_NAMES.find(tnode.node_type);
@@ -481,6 +483,10 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN
         }
         return Status::OK();
 
+    case TPlanNodeType::META_SCAN_NODE:
+        *node = pool->add(new vectorized::VMetaScanNode(pool, tnode, descs));
+        return Status::OK();
+
     case TPlanNodeType::OLAP_SCAN_NODE:
         if (state->enable_vectorized_exec()) {
             *node = pool->add(new vectorized::NewOlapScanNode(pool, tnode, descs));
@@ -705,6 +711,7 @@ void ExecNode::collect_scan_nodes(vector<ExecNode*>* nodes) {
     collect_nodes(TPlanNodeType::ES_HTTP_SCAN_NODE, nodes);
     collect_nodes(TPlanNodeType::DATA_GEN_SCAN_NODE, nodes);
     collect_nodes(TPlanNodeType::FILE_SCAN_NODE, nodes);
+    collect_nodes(TPlanNodeType::META_SCAN_NODE, nodes);
 }
 
 void ExecNode::try_do_aggregate_serde_improve() {
@@ -728,7 +735,8 @@ void ExecNode::try_do_aggregate_serde_improve() {
         typeid(*child0) == typeid(vectorized::NewFileScanNode) ||
         typeid(*child0) == typeid(vectorized::NewOdbcScanNode) ||
         typeid(*child0) == typeid(vectorized::NewEsScanNode) ||
-        typeid(*child0) == typeid(vectorized::NewJdbcScanNode)) {
+        typeid(*child0) == typeid(vectorized::NewJdbcScanNode) ||
+        typeid(*child0) == typeid(vectorized::VMetaScanNode)) {
         vectorized::VScanNode* scan_node =
                 static_cast<vectorized::VScanNode*>(agg_node[0]->_children[0]);
         scan_node->set_no_agg_finalize();
diff --git a/be/src/runtime/plan_fragment_executor.cpp b/be/src/runtime/plan_fragment_executor.cpp
index e901a738e1..b59cba4896 100644
--- a/be/src/runtime/plan_fragment_executor.cpp
+++ b/be/src/runtime/plan_fragment_executor.cpp
@@ -48,6 +48,7 @@
 #include "vec/exec/scan/new_jdbc_scan_node.h"
 #include "vec/exec/scan/new_odbc_scan_node.h"
 #include "vec/exec/scan/new_olap_scan_node.h"
+#include "vec/exec/scan/vmeta_scan_node.h"
 #include "vec/exec/vexchange_node.h"
 #include "vec/runtime/vdata_stream_mgr.h"
 
@@ -173,7 +174,8 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request,
             typeid(*node) == typeid(vectorized::NewFileScanNode) ||
             typeid(*node) == typeid(vectorized::NewOdbcScanNode) ||
             typeid(*node) == typeid(vectorized::NewEsScanNode) ||
-            typeid(*node) == typeid(vectorized::NewJdbcScanNode)) {
+            typeid(*node) == typeid(vectorized::NewJdbcScanNode) ||
+            typeid(*node) == typeid(vectorized::VMetaScanNode)) {
             vectorized::VScanNode* scan_node = static_cast<vectorized::VScanNode*>(scan_nodes[i]);
             const std::vector<TScanRangeParams>& scan_ranges =
                     find_with_default(params.per_node_scan_ranges, scan_node->id(), no_scan_ranges);
diff --git a/be/src/vec/CMakeLists.txt b/be/src/vec/CMakeLists.txt
index 0e83f98167..b0ce39f80f 100644
--- a/be/src/vec/CMakeLists.txt
+++ b/be/src/vec/CMakeLists.txt
@@ -275,6 +275,8 @@ set(VEC_FILES
   exec/scan/new_jdbc_scan_node.cpp
   exec/scan/new_es_scanner.cpp
   exec/scan/new_es_scan_node.cpp
+  exec/scan/vmeta_scan_node.cpp
+  exec/scan/vmeta_scanner.cpp
   exec/format/csv/csv_reader.cpp
   exec/format/orc/vorc_reader.cpp
   exec/format/json/new_json_reader.cpp
diff --git a/be/src/vec/exec/scan/vmeta_scan_node.cpp b/be/src/vec/exec/scan/vmeta_scan_node.cpp
new file mode 100644
index 0000000000..ae7585ecf4
--- /dev/null
+++ b/be/src/vec/exec/scan/vmeta_scan_node.cpp
@@ -0,0 +1,68 @@
+// 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.
+
+#include "vmeta_scan_node.h"
+
+#include "vmeta_scanner.h"
+
+namespace doris::vectorized {
+
+VMetaScanNode::VMetaScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs)
+        : VScanNode(pool, tnode, descs),
+          _tuple_id(tnode.meta_scan_node.tuple_id),
+          _scan_params(tnode.meta_scan_node) {
+    _output_tuple_id = _tuple_id;
+}
+
+Status VMetaScanNode::init(const TPlanNode& tnode, RuntimeState* state) {
+    RETURN_IF_ERROR(VScanNode::init(tnode, state));
+    return Status::OK();
+}
+
+Status VMetaScanNode::prepare(RuntimeState* state) {
+    RETURN_IF_ERROR(VScanNode::prepare(state));
+    return Status::OK();
+}
+
+void VMetaScanNode::set_scan_ranges(const std::vector<TScanRangeParams>& scan_ranges) {
+    _scan_ranges = scan_ranges;
+}
+
+Status VMetaScanNode::_init_profile() {
+    RETURN_IF_ERROR(VScanNode::_init_profile());
+    return Status::OK();
+}
+
+Status VMetaScanNode::_init_scanners(std::list<VScanner*>* scanners) {
+    if (_eos == true) {
+        return Status::OK();
+    }
+    for (auto& scan_range : _scan_ranges) {
+        VMetaScanner* scanner =
+                new VMetaScanner(_state, this, _tuple_id, scan_range, _limit_per_scanner);
+        RETURN_IF_ERROR(scanner->prepare(_state, _vconjunct_ctx_ptr.get()));
+        _scanner_pool.add(scanner);
+        scanners->push_back(static_cast<VScanner*>(scanner));
+    }
+    return Status::OK();
+}
+
+Status VMetaScanNode::_process_conjuncts() {
+    return Status::OK();
+}
+
+} // namespace doris::vectorized
diff --git a/be/src/vec/exec/scan/vmeta_scan_node.h b/be/src/vec/exec/scan/vmeta_scan_node.h
new file mode 100644
index 0000000000..93774f89d1
--- /dev/null
+++ b/be/src/vec/exec/scan/vmeta_scan_node.h
@@ -0,0 +1,45 @@
+// 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.
+
+#pragma once
+
+#include "runtime/runtime_state.h"
+#include "vec/exec/scan/vscan_node.h"
+
+namespace doris::vectorized {
+
+class VMetaScanNode : public VScanNode {
+public:
+    VMetaScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs);
+    ~VMetaScanNode() override = default;
+
+    Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override;
+    Status prepare(RuntimeState* state) override;
+    void set_scan_ranges(const std::vector<TScanRangeParams>& scan_ranges) override;
+    const TMetaScanNode& scan_params() { return _scan_params; };
+
+private:
+    Status _init_profile() override;
+    Status _init_scanners(std::list<VScanner*>* scanners) override;
+    Status _process_conjuncts() override;
+
+    TupleId _tuple_id;
+    TMetaScanNode _scan_params;
+    std::vector<TScanRangeParams> _scan_ranges;
+};
+
+} // namespace doris::vectorized
\ No newline at end of file
diff --git a/be/src/vec/exec/scan/vmeta_scanner.cpp b/be/src/vec/exec/scan/vmeta_scanner.cpp
new file mode 100644
index 0000000000..c36d7877c0
--- /dev/null
+++ b/be/src/vec/exec/scan/vmeta_scanner.cpp
@@ -0,0 +1,209 @@
+// 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.
+
+#include "vmeta_scanner.h"
+
+#include <gen_cpp/FrontendService_types.h>
+#include <gen_cpp/HeartbeatService_types.h>
+
+#include "gen_cpp/FrontendService.h"
+#include "runtime/client_cache.h"
+#include "util/thrift_rpc_helper.h"
+#include "vec/runtime/vdatetime_value.h"
+
+namespace doris::vectorized {
+
+VMetaScanner::VMetaScanner(RuntimeState* state, VMetaScanNode* parent, int64_t tuple_id,
+                           const TScanRangeParams& scan_range, int64_t limit)
+        : VScanner(state, static_cast<VScanNode*>(parent), limit),
+          _parent(parent),
+          _meta_eos(false),
+          _tuple_id(tuple_id),
+          _scan_range(scan_range.scan_range) {}
+
+Status VMetaScanner::open(RuntimeState* state) {
+    VLOG_CRITICAL << "VMetaScanner::open";
+    RETURN_IF_ERROR(VScanner::open(state));
+    return Status::OK();
+}
+
+Status VMetaScanner::prepare(RuntimeState* state, VExprContext** vconjunct_ctx_ptr) {
+    VLOG_CRITICAL << "VMetaScanner::prepare";
+    if (vconjunct_ctx_ptr != nullptr) {
+        // Copy vconjunct_ctx_ptr from scan node to this scanner's _vconjunct_ctx.
+        RETURN_IF_ERROR((*vconjunct_ctx_ptr)->clone(_state, &_vconjunct_ctx));
+    }
+    _tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_id);
+    if (_scan_range.meta_scan_range.__isset.iceberg_params) {
+        RETURN_IF_ERROR(_fetch_iceberg_metadata_batch());
+    } else {
+        _meta_eos = true;
+    }
+    return Status::OK();
+}
+
+Status VMetaScanner::_get_block_impl(RuntimeState* state, Block* block, bool* eof) {
+    VLOG_CRITICAL << "VMetaScanner::_get_block_impl";
+    if (nullptr == state || nullptr == block || nullptr == eof) {
+        return Status::InternalError("input is NULL pointer");
+    }
+    if (_meta_eos == true) {
+        *eof = true;
+        return Status::OK();
+    }
+
+    auto column_size = _tuple_desc->slots().size();
+    std::vector<MutableColumnPtr> columns(column_size);
+    bool mem_reuse = block->mem_reuse();
+    do {
+        RETURN_IF_CANCELLED(state);
+
+        columns.resize(column_size);
+        for (auto i = 0; i < column_size; i++) {
+            if (mem_reuse) {
+                columns[i] = std::move(*block->get_by_position(i).column).mutate();
+            } else {
+                columns[i] = _tuple_desc->slots()[i]->get_empty_mutable_column();
+            }
+        }
+        // fill block
+        _fill_block_with_remote_data(columns);
+        if (_meta_eos == true) {
+            if (block->rows() == 0) {
+                *eof = true;
+            }
+            break;
+        }
+        // Before really use the Block, must clear other ptr of column in block
+        // So here need do std::move and clear in `columns`
+        if (!mem_reuse) {
+            int column_index = 0;
+            for (const auto slot_desc : _tuple_desc->slots()) {
+                block->insert(ColumnWithTypeAndName(std::move(columns[column_index++]),
+                                                    slot_desc->get_data_type_ptr(),
+                                                    slot_desc->col_name()));
+            }
+        } else {
+            columns.clear();
+        }
+        VLOG_ROW << "VMetaScanNode output rows: " << block->rows();
+    } while (block->rows() == 0 && !(*eof));
+    return Status::OK();
+}
+
+Status VMetaScanner::_fill_block_with_remote_data(const std::vector<MutableColumnPtr>& columns) {
+    VLOG_CRITICAL << "VMetaScanner::_fill_block_with_remote_data";
+    for (int col_idx = 0; col_idx < columns.size(); col_idx++) {
+        auto slot_desc = _tuple_desc->slots()[col_idx];
+        // because the fe planner filter the non_materialize column
+        if (!slot_desc->is_materialized()) {
+            continue;
+        }
+
+        for (int _row_idx = 0; _row_idx < _batch_data.size(); _row_idx++) {
+            vectorized::IColumn* col_ptr = columns[col_idx].get();
+            if (slot_desc->is_nullable() == true) {
+                auto* nullable_column = reinterpret_cast<vectorized::ColumnNullable*>(col_ptr);
+                col_ptr = &nullable_column->get_nested_column();
+            }
+            switch (slot_desc->type().type) {
+            case TYPE_INT: {
+                int64_t data = _batch_data[_row_idx].column_value[col_idx].intVal;
+                reinterpret_cast<vectorized::ColumnVector<vectorized::Int32>*>(col_ptr)
+                        ->insert_value(data);
+                break;
+            }
+            case TYPE_BIGINT: {
+                int64_t data = _batch_data[_row_idx].column_value[col_idx].longVal;
+                reinterpret_cast<vectorized::ColumnVector<vectorized::Int64>*>(col_ptr)
+                        ->insert_value(data);
+                break;
+            }
+            case TYPE_DATETIMEV2: {
+                uint64_t data = _batch_data[_row_idx].column_value[col_idx].longVal;
+                reinterpret_cast<vectorized::ColumnVector<vectorized::UInt64>*>(col_ptr)
+                        ->insert_value(data);
+                break;
+            }
+            case TYPE_STRING:
+            case TYPE_CHAR:
+            case TYPE_VARCHAR: {
+                std::string data = _batch_data[_row_idx].column_value[col_idx].stringVal;
+                reinterpret_cast<vectorized::ColumnString*>(col_ptr)->insert_data(data.c_str(),
+                                                                                  data.length());
+                break;
+            }
+            default: {
+                std::string error_msg =
+                        fmt::format("Invalid column type {} on column: {}.",
+                                    slot_desc->type().debug_string(), slot_desc->col_name());
+                return Status::InternalError(std::string(error_msg));
+            }
+            }
+        }
+    }
+    _meta_eos = true;
+    return Status::OK();
+}
+
+Status VMetaScanner::_fetch_iceberg_metadata_batch() {
+    VLOG_CRITICAL << "VMetaScanner::_fetch_iceberg_metadata_batch";
+    TFetchSchemaTableDataRequest request;
+    request.cluster_name = "";
+    request.__isset.cluster_name = true;
+    request.schema_table_name = TSchemaTableName::ICEBERG_TABLE_META;
+    request.__isset.schema_table_name = true;
+    auto scan_params = _parent->scan_params();
+    TMetadataTableRequestParams meta_table_params = TMetadataTableRequestParams();
+    meta_table_params.catalog = scan_params.catalog;
+    meta_table_params.__isset.catalog = true;
+    meta_table_params.database = scan_params.database;
+    meta_table_params.__isset.database = true;
+    meta_table_params.table = scan_params.table;
+    meta_table_params.__isset.table = true;
+
+    meta_table_params.iceberg_metadata_params = _scan_range.meta_scan_range.iceberg_params;
+    meta_table_params.__isset.iceberg_metadata_params = true;
+
+    request.metada_table_params = meta_table_params;
+    request.__isset.metada_table_params = true;
+
+    TNetworkAddress master_addr = ExecEnv::GetInstance()->master_info()->network_address;
+    TFetchSchemaTableDataResult result;
+
+    RETURN_IF_ERROR(ThriftRpcHelper::rpc<FrontendServiceClient>(
+            master_addr.hostname, master_addr.port,
+            [&request, &result](FrontendServiceConnection& client) {
+                client->fetchSchemaTableData(result, request);
+            },
+            config::txn_commit_rpc_timeout_ms));
+
+    Status status(result.status);
+    if (!status.ok()) {
+        LOG(WARNING) << "fetch schema table data from master failed, errmsg=" << status;
+        return status;
+    }
+    _batch_data = std::move(result.data_batch);
+    return Status::OK();
+}
+
+Status VMetaScanner::close(RuntimeState* state) {
+    VLOG_CRITICAL << "VMetaScanner::close";
+    RETURN_IF_ERROR(VScanner::close(state));
+    return Status::OK();
+}
+} // namespace doris::vectorized
diff --git a/be/src/vec/exec/scan/vmeta_scanner.h b/be/src/vec/exec/scan/vmeta_scanner.h
new file mode 100644
index 0000000000..7663ffe1c8
--- /dev/null
+++ b/be/src/vec/exec/scan/vmeta_scanner.h
@@ -0,0 +1,48 @@
+// 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.
+
+#pragma once
+
+#include "runtime/runtime_state.h"
+#include "vec/exec/scan/vscanner.h"
+#include "vmeta_scan_node.h"
+
+namespace doris::vectorized {
+
+class VMetaScanner : public VScanner {
+public:
+    VMetaScanner(RuntimeState* state, VMetaScanNode* parent, int64_t tuple_id,
+                 const TScanRangeParams& scan_range, int64_t limit);
+
+    Status open(RuntimeState* state) override;
+    Status close(RuntimeState* state) override;
+    Status prepare(RuntimeState* state, VExprContext** vconjunct_ctx_ptr);
+
+protected:
+    Status _get_block_impl(RuntimeState* state, Block* block, bool* eos) override;
+    Status _fill_block_with_remote_data(const std::vector<MutableColumnPtr>& columns);
+    Status _fetch_iceberg_metadata_batch();
+
+private:
+    VMetaScanNode* _parent;
+    bool _meta_eos;
+    TupleId _tuple_id;
+    const TupleDescriptor* _tuple_desc;
+    std::vector<TRow> _batch_data;
+    const TScanRange& _scan_range;
+};
+} // namespace doris::vectorized
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/MetadataScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/MetadataScanNode.java
new file mode 100644
index 0000000000..431a456646
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/MetadataScanNode.java
@@ -0,0 +1,114 @@
+// 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.planner.external;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.TupleDescriptor;
+import org.apache.doris.common.UserException;
+import org.apache.doris.planner.PlanNodeId;
+import org.apache.doris.planner.ScanNode;
+import org.apache.doris.statistics.StatisticalType;
+import org.apache.doris.system.Backend;
+import org.apache.doris.tablefunction.IcebergTableValuedFunction;
+import org.apache.doris.tablefunction.MetadataTableValuedFunction;
+import org.apache.doris.thrift.TIcebergMetadataParams;
+import org.apache.doris.thrift.TIcebergMetadataType;
+import org.apache.doris.thrift.TMetaScanNode;
+import org.apache.doris.thrift.TMetaScanRange;
+import org.apache.doris.thrift.TNetworkAddress;
+import org.apache.doris.thrift.TPlanNode;
+import org.apache.doris.thrift.TPlanNodeType;
+import org.apache.doris.thrift.TScanRange;
+import org.apache.doris.thrift.TScanRangeLocation;
+import org.apache.doris.thrift.TScanRangeLocations;
+
+import com.google.common.collect.Lists;
+
+import java.util.List;
+
+public class MetadataScanNode extends ScanNode {
+
+    private MetadataTableValuedFunction tvf;
+
+    private List<TScanRangeLocations> scanRangeLocations = Lists.newArrayList();
+
+    private final BackendPolicy backendPolicy = new BackendPolicy();
+
+    public MetadataScanNode(PlanNodeId id, TupleDescriptor desc, MetadataTableValuedFunction tvf) {
+        super(id, desc, "METADATA_SCAN_NODE", StatisticalType.METADATA_SCAN_NODE);
+        this.tvf = tvf;
+    }
+
+    @Override
+    public void init(Analyzer analyzer) throws UserException {
+        super.init(analyzer);
+        backendPolicy.init();
+    }
+
+    @Override
+    protected void toThrift(TPlanNode planNode) {
+        planNode.setNodeType(TPlanNodeType.META_SCAN_NODE);
+        TMetaScanNode metaScanNode = new TMetaScanNode();
+        metaScanNode.setCatalog(tvf.getMetadataTableName().getCtl());
+        metaScanNode.setDatabase(tvf.getMetadataTableName().getDb());
+        metaScanNode.setTable(tvf.getMetadataTableName().getTbl());
+        metaScanNode.setTupleId(desc.getId().asInt());
+        planNode.setMetaScanNode(metaScanNode);
+    }
+
+    @Override
+    public List<TScanRangeLocations> getScanRangeLocations(long maxScanRangeLength) {
+        return scanRangeLocations;
+    }
+
+    @Override
+    public void finalize(Analyzer analyzer) throws UserException {
+        buildScanRanges();
+    }
+
+    private void buildScanRanges() {
+        if (tvf.getMetaType() == MetadataTableValuedFunction.MetaType.ICEBERG) {
+            IcebergTableValuedFunction icebergTvf = (IcebergTableValuedFunction) tvf;
+            // todo: split
+            TScanRangeLocations locations = createIcebergTvfLocations(icebergTvf);
+            scanRangeLocations.add(locations);
+        }
+    }
+
+    private TScanRangeLocations createIcebergTvfLocations(IcebergTableValuedFunction icebergTvf) {
+        TScanRange scanRange = new TScanRange();
+        TMetaScanRange metaScanRange = new TMetaScanRange();
+        // set iceberg metadata params
+        TIcebergMetadataParams icebergMetadataParams = new TIcebergMetadataParams();
+        int metadataType = icebergTvf.getMetaQueryType().ordinal();
+        icebergMetadataParams.setMetadataType(TIcebergMetadataType.findByValue(metadataType));
+
+        metaScanRange.setIcebergParams(icebergMetadataParams);
+        scanRange.setMetaScanRange(metaScanRange);
+        // set location
+        TScanRangeLocation location = new TScanRangeLocation();
+        Backend backend = backendPolicy.getNextBe();
+        location.setBackendId(backend.getId());
+        location.setServer(new TNetworkAddress(backend.getHost(), backend.getBePort()));
+
+        TScanRangeLocations result = new TScanRangeLocations();
+        result.addToLocations(location);
+        result.setScanRange(scanRange);
+        return result;
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
index 71784eae21..4c42ca5968 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
@@ -24,6 +24,7 @@ import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.DatabaseIf;
 import org.apache.doris.catalog.Env;
+import org.apache.doris.catalog.HMSResource;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.S3Resource;
 import org.apache.doris.catalog.Table;
@@ -47,6 +48,7 @@ import org.apache.doris.common.Version;
 import org.apache.doris.common.util.TimeUtils;
 import org.apache.doris.datasource.CatalogIf;
 import org.apache.doris.datasource.ExternalCatalog;
+import org.apache.doris.datasource.HMSExternalCatalog;
 import org.apache.doris.datasource.InternalCatalog;
 import org.apache.doris.master.MasterImpl;
 import org.apache.doris.mysql.privilege.PrivPredicate;
@@ -85,6 +87,7 @@ import org.apache.doris.thrift.TGetStoragePolicy;
 import org.apache.doris.thrift.TGetStoragePolicyResult;
 import org.apache.doris.thrift.TGetTablesParams;
 import org.apache.doris.thrift.TGetTablesResult;
+import org.apache.doris.thrift.TIcebergMetadataType;
 import org.apache.doris.thrift.TInitExternalCtlMetaRequest;
 import org.apache.doris.thrift.TInitExternalCtlMetaResult;
 import org.apache.doris.thrift.TListPrivilegesResult;
@@ -100,6 +103,7 @@ import org.apache.doris.thrift.TLoadTxnRollbackResult;
 import org.apache.doris.thrift.TMasterOpRequest;
 import org.apache.doris.thrift.TMasterOpResult;
 import org.apache.doris.thrift.TMasterResult;
+import org.apache.doris.thrift.TMetadataTableRequestParams;
 import org.apache.doris.thrift.TNetworkAddress;
 import org.apache.doris.thrift.TPrivilegeStatus;
 import org.apache.doris.thrift.TReportExecStatusParams;
@@ -131,11 +135,19 @@ import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.gson.Gson;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.catalog.TableIdentifier;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 import org.apache.thrift.TException;
+import org.jetbrains.annotations.NotNull;
 
+import java.time.Instant;
+import java.time.LocalDateTime;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
@@ -997,6 +1009,8 @@ public class FrontendServiceImpl implements FrontendService.Iface {
         switch (request.getSchemaTableName()) {
             case BACKENDS:
                 return getBackendsSchemaTable(request);
+            case ICEBERG_TABLE_META:
+                return getIcebergMetadataTable(request);
             default:
                 break;
         }
@@ -1005,6 +1019,85 @@ public class FrontendServiceImpl implements FrontendService.Iface {
         return result;
     }
 
+    private TFetchSchemaTableDataResult getIcebergMetadataTable(TFetchSchemaTableDataRequest request) {
+        if (!request.isSetMetadaTableParams()) {
+            return errorResult("Metadata table params is not set. ");
+        }
+        TMetadataTableRequestParams params = request.getMetadaTableParams();
+        if (!params.isSetIcebergMetadataParams()) {
+            return errorResult("Iceberg metadata params is not set. ");
+        }
+
+        HMSExternalCatalog catalog = (HMSExternalCatalog) Env.getCurrentEnv().getCatalogMgr()
+                .getCatalog(params.getCatalog());
+        org.apache.iceberg.Table table;
+        try {
+            table = getIcebergTable(catalog, params.getDatabase(), params.getTable());
+        } catch (MetaNotFoundException e) {
+            return errorResult(e.getMessage());
+        }
+        TFetchSchemaTableDataResult result = new TFetchSchemaTableDataResult();
+        List<TRow> dataBatch = Lists.newArrayList();
+        TIcebergMetadataType metadataType = params.getIcebergMetadataParams().getMetadataType();
+        switch (metadataType) {
+            case SNAPSHOTS:
+                for (Snapshot snapshot : table.snapshots()) {
+                    TRow trow = new TRow();
+                    LocalDateTime committedAt = LocalDateTime.ofInstant(Instant.ofEpochMilli(
+                            snapshot.timestampMillis()), TimeUtils.getTimeZone().toZoneId());
+                    long encodedDatetime = convertToDateTimeV2(committedAt.getYear(), committedAt.getMonthValue(),
+                            committedAt.getDayOfMonth(), committedAt.getHour(),
+                            committedAt.getMinute(), committedAt.getSecond());
+
+                    trow.addToColumnValue(new TCell().setLongVal(encodedDatetime));
+                    trow.addToColumnValue(new TCell().setLongVal(snapshot.snapshotId()));
+                    if (snapshot.parentId() == null) {
+                        trow.addToColumnValue(new TCell().setLongVal(-1L));
+                    } else {
+                        trow.addToColumnValue(new TCell().setLongVal(snapshot.parentId()));
+                    }
+                    trow.addToColumnValue(new TCell().setStringVal(snapshot.operation()));
+                    trow.addToColumnValue(new TCell().setStringVal(snapshot.manifestListLocation()));
+                    dataBatch.add(trow);
+                }
+                break;
+            default:
+                return errorResult("Unsupported metadata inspect type: " + metadataType);
+        }
+        result.setDataBatch(dataBatch);
+        result.setStatus(new TStatus(TStatusCode.OK));
+        return result;
+    }
+
+    public static long convertToDateTimeV2(int year, int month, int day, int hour, int minute, int second) {
+        return (long) second << 20 | (long) minute << 26 | (long) hour << 32
+            | (long) day << 37 | (long) month << 42 | (long) year << 46;
+    }
+
+    @NotNull
+    private TFetchSchemaTableDataResult errorResult(String msg) {
+        TFetchSchemaTableDataResult result = new TFetchSchemaTableDataResult();
+        result.setStatus(new TStatus(TStatusCode.INTERNAL_ERROR));
+        result.status.addToErrorMsgs(msg);
+        return result;
+    }
+
+    private org.apache.iceberg.Table getIcebergTable(HMSExternalCatalog catalog, String db, String tbl)
+                throws MetaNotFoundException {
+        org.apache.iceberg.hive.HiveCatalog hiveCatalog = new org.apache.iceberg.hive.HiveCatalog();
+        Configuration conf = new HdfsConfiguration();
+        Map<String, String> properties = catalog.getCatalogProperty().getHadoopProperties();
+        for (Map.Entry<String, String> entry : properties.entrySet()) {
+            conf.set(entry.getKey(), entry.getValue());
+        }
+        hiveCatalog.setConf(conf);
+        Map<String, String> catalogProperties = new HashMap<>();
+        catalogProperties.put(HMSResource.HIVE_METASTORE_URIS, catalog.getHiveMetastoreUris());
+        catalogProperties.put("uri", catalog.getHiveMetastoreUris());
+        hiveCatalog.initialize("hive", catalogProperties);
+        return hiveCatalog.loadTable(TableIdentifier.of(db, tbl));
+    }
+
     private TFetchSchemaTableDataResult getBackendsSchemaTable(TFetchSchemaTableDataRequest request) {
         final SystemInfoService clusterInfoService = Env.getCurrentSystemInfo();
         TFetchSchemaTableDataResult result = new TFetchSchemaTableDataResult();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticalType.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticalType.java
index ebdfd0471e..47ad4a6161 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticalType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticalType.java
@@ -46,5 +46,6 @@ public enum StatisticalType {
     UNION_NODE,
     TABLE_VALUED_FUNCTION_NODE,
     FILE_SCAN_NODE,
+    METADATA_SCAN_NODE,
     JDBC_SCAN_NODE,
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/IcebergTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/IcebergTableValuedFunction.java
new file mode 100644
index 0000000000..4e5aa7dff0
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/IcebergTableValuedFunction.java
@@ -0,0 +1,123 @@
+// 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.tablefunction;
+
+import org.apache.doris.analysis.TableName;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.Env;
+import org.apache.doris.catalog.PrimitiveType;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.ErrorCode;
+import org.apache.doris.common.ErrorReport;
+import org.apache.doris.mysql.privilege.PrivPredicate;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Maps;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * The Implement of table valued function
+ * iceberg_meta("table" = "ctl.db.tbl", "query_type" = "snapshots").
+ */
+public class IcebergTableValuedFunction extends MetadataTableValuedFunction {
+
+    public enum MetadataType { SNAPSHOTS }
+
+    public static final String NAME = "iceberg_meta";
+    private static final String TABLE = "table";
+    private static final String QUERY_TYPE = "query_type";
+
+    private static final ImmutableSet<String> PROPERTIES_SET = new ImmutableSet.Builder<String>()
+            .add(TABLE)
+            .add(QUERY_TYPE)
+            .build();
+
+    private final MetadataType queryType;
+    private final TableName tableName;
+
+    public IcebergTableValuedFunction(Map<String, String> params) throws AnalysisException {
+        super(MetaType.ICEBERG);
+        Map<String, String> validParams = Maps.newHashMap();
+        for (String key : params.keySet()) {
+            if (!PROPERTIES_SET.contains(key.toLowerCase())) {
+                throw new AnalysisException("'" + key + "' is invalid property");
+            }
+            // check ctl db tbl
+            validParams.put(key.toLowerCase(), params.get(key));
+        }
+        String tableName = validParams.get(TABLE);
+        String queryType = validParams.get(QUERY_TYPE);
+        if (tableName == null || queryType == null) {
+            throw new AnalysisException("Invalid iceberg metadata query");
+        }
+        String[] names = tableName.split("\\.");
+        if (names.length != 3) {
+            throw new AnalysisException("The iceberg table name contains the catalogName, databaseName, and tableName");
+        }
+        this.tableName = new TableName(names[0], names[1], names[2]);
+        // check auth
+        if (!Env.getCurrentEnv().getAuth().checkTblPriv(ConnectContext.get(), this.tableName, PrivPredicate.SELECT)) {
+            ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "SELECT",
+                    ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(),
+                    this.tableName.getDb() + ": " + this.tableName.getTbl());
+        }
+        try {
+            this.queryType = MetadataType.valueOf(queryType.toUpperCase());
+        } catch (IllegalArgumentException e) {
+            throw new AnalysisException("Unsupported iceberg metadata query type: " + queryType);
+        }
+    }
+
+    @Override
+    public String getTableName() {
+        return "IcebergMetadataTableValuedFunction";
+    }
+
+    public TableName getMetadataTableName() {
+        return tableName;
+    }
+
+    public MetadataType getMetaQueryType() {
+        return queryType;
+    }
+
+    /**
+     * The tvf can register columns of metadata table
+     * The data is provided by getIcebergMetadataTable in FrontendService
+     * @see org.apache.doris.service.FrontendServiceImpl
+     * @return metadata columns
+     */
+    @Override
+    public List<Column> getTableColumns() throws AnalysisException {
+        List<Column> resColumns = new ArrayList<>();
+        if (queryType == MetadataType.SNAPSHOTS) {
+            resColumns.add(new Column("committed_at", PrimitiveType.DATETIMEV2, false));
+            resColumns.add(new Column("snapshot_id", PrimitiveType.BIGINT, false));
+            resColumns.add(new Column("parent_id", PrimitiveType.BIGINT, false));
+            resColumns.add(new Column("operation", PrimitiveType.STRING, false));
+            // todo: compress manifest_list string
+            resColumns.add(new Column("manifest_list", PrimitiveType.STRING, false));
+            // resColumns.add(new Column("summary", PrimitiveType.MAP, false));
+        }
+        return resColumns;
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataTableValuedFunction.java
new file mode 100644
index 0000000000..fd83c59957
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataTableValuedFunction.java
@@ -0,0 +1,46 @@
+// 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.tablefunction;
+
+import org.apache.doris.analysis.TableName;
+import org.apache.doris.analysis.TupleDescriptor;
+import org.apache.doris.planner.PlanNodeId;
+import org.apache.doris.planner.ScanNode;
+import org.apache.doris.planner.external.MetadataScanNode;
+
+public abstract class MetadataTableValuedFunction extends TableValuedFunctionIf {
+
+    public enum MetaType { ICEBERG }
+
+    private final MetaType metaType;
+
+    public MetadataTableValuedFunction(MetaType metaType) {
+        this.metaType = metaType;
+    }
+
+    public MetaType getMetaType() {
+        return metaType;
+    }
+
+    public abstract TableName getMetadataTableName();
+
+    @Override
+    public ScanNode getScanNode(PlanNodeId id, TupleDescriptor desc) {
+        return new MetadataScanNode(id, desc, this);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java
index 862b986e97..fdd9e106fe 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java
@@ -49,6 +49,8 @@ public abstract class TableValuedFunctionIf {
                 return new S3TableValuedFunction(params);
             case HdfsTableValuedFunction.NAME:
                 return new HdfsTableValuedFunction(params);
+            case IcebergTableValuedFunction.NAME:
+                return new IcebergTableValuedFunction(params);
             default:
                 throw new AnalysisException("Could not find table function " + funcName);
         }
diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift
index 4c7d062727..32c73a5663 100644
--- a/gensrc/thrift/FrontendService.thrift
+++ b/gensrc/thrift/FrontendService.thrift
@@ -694,11 +694,20 @@ struct TInitExternalCtlMetaResult {
 
 enum TSchemaTableName{
   BACKENDS = 0,
+  ICEBERG_TABLE_META = 1,
+}
+
+struct TMetadataTableRequestParams {
+  1: optional PlanNodes.TIcebergMetadataParams iceberg_metadata_params
+  2: optional string catalog
+  3: optional string database
+  4: optional string table
 }
 
 struct TFetchSchemaTableDataRequest {
   1: optional string cluster_name
   2: optional TSchemaTableName schema_table_name
+  3: optional TMetadataTableRequestParams metada_table_params
 }
 
 struct TFetchSchemaTableDataResult {
diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift
index ff97361c6e..35ebab6373 100644
--- a/gensrc/thrift/PlanNodes.thrift
+++ b/gensrc/thrift/PlanNodes.thrift
@@ -370,6 +370,18 @@ struct TDataGenScanRange {
   1: optional TTVFNumbersScanRange numbers_params
 }
 
+enum TIcebergMetadataType {
+  SNAPSHOTS = 0,
+}
+
+struct TIcebergMetadataParams {
+  1: optional TIcebergMetadataType metadata_type
+}
+
+struct TMetaScanRange {
+  1: optional TIcebergMetadataParams iceberg_params
+}
+
 // Specification of an individual data range which is held in its entirety
 // by a storage server
 struct TScanRange {
@@ -380,6 +392,7 @@ struct TScanRange {
   7: optional TEsScanRange es_scan_range
   8: optional TExternalScanRange ext_scan_range
   9: optional TDataGenScanRange data_gen_scan_range
+  10: optional TMetaScanRange meta_scan_range
 }
 
 struct TMySQLScanNode {
@@ -513,10 +526,9 @@ struct TSchemaScanNode {
 
 struct TMetaScanNode {
   1: required Types.TTupleId tuple_id
-  2: required string table_name
-  3: optional string db
+  2: optional string catalog
+  3: optional string database
   4: optional string table
-  5: optional string user
 }
 
 struct TSortInfo {


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


[doris] 01/04: fix(ui): 1. fix component/table can not change pageSize,affect system/query profile/session page etc. (#15533)

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 221149e5900683767ab668d3a7b404b564754b8a
Author: htyoung <ht...@hotmail.com>
AuthorDate: Mon Jan 9 08:46:18 2023 +0800

    fix(ui): 1. fix component/table can not change pageSize,affect system/query profile/session page etc. (#15533)
    
    2. add antd Table Component missing rowKey property to fit react specification
    2. fix system/query profile/session/configuration page maybe lead memory leak when switch these pages fast
    3.other grammar fix to fit typescript and react specification
    
    Co-authored-by: tongyang.hty <ha...@douyu.tv>
---
 ui/src/api/api.ts                                  |  64 +++++---
 ui/src/components/table/index.tsx                  |  48 +++---
 ui/src/i18n.tsx                                    |  15 +-
 ui/src/pages/configuration/index.tsx               |  37 +++--
 ui/src/pages/logs/index.tsx                        |  56 ++++---
 .../playground/content/components/data-prev.tsx    |  85 ++++++-----
 ui/src/pages/playground/page-side/index.tsx        |  20 +--
 ui/src/pages/playground/tree/index.tsx             | 164 +++++++++++----------
 ui/src/pages/query-profile/index.tsx               |  80 +++++-----
 ui/src/pages/session/index.tsx                     |  45 +++---
 ui/src/pages/system/index.tsx                      |  61 ++++----
 ui/src/utils/lazy.tsx                              |  17 ++-
 ui/src/utils/utils.ts                              |  37 ++---
 ui/tsconfig.json                                   |  94 ++++++------
 14 files changed, 442 insertions(+), 381 deletions(-)

diff --git a/ui/src/api/api.ts b/ui/src/api/api.ts
index 935958b3d3..083e44d790 100644
--- a/ui/src/api/api.ts
+++ b/ui/src/api/api.ts
@@ -18,35 +18,41 @@
 import {API_BASE} from 'Constants';
 import request from 'Utils/request';
 import {Result} from '@src/interfaces/http.interface';
+
 //login
 export function login<T>(data: any): Promise<Result<T>> {
     return request('/rest/v1/login', {
         method: 'POST',
-        headers:{Authorization: data.password?`Basic ${btoa(data.username+':'+data.password)}`:`Basic ${btoa(data.username+':')}`},
+        headers: {Authorization: data.password ? `Basic ${btoa(data.username + ':' + data.password)}` : `Basic ${btoa(data.username + ':')}`},
     });
 }
+
 //logout
 export function logOut<T>(): Promise<Result<T>> {
-    return request(`/rest/v1/logout`,{
+    return request(`/rest/v1/logout`, {
         method: 'POST',
     });
 }
+
 //home
 export function getHardwareInfo<T>(): Promise<Result<T>> {
-    return request(`/rest/v1/hardware_info/fe/`,{
+    return request(`/rest/v1/hardware_info/fe/`, {
         method: 'GET',
     });
 }
+
 //system
 export function getSystem<T>(data: any): Promise<Result<T>> {
-    return request(`/rest/v1/system${data.path}`,{
+    return request(`/rest/v1/system${data.path}`, {
         method: 'GET',
+        ...data
     });
 }
+
 //log
 export function getLog<T>(data: any): Promise<Result<T>> {
     let localUrl = '/rest/v1/log';
-    if(data.add_verbose){
+    if (data.add_verbose) {
         localUrl = `/rest/v1/log?add_verbose=${data.add_verbose}`;
     }
     if (data.del_verbose) {
@@ -55,71 +61,81 @@ export function getLog<T>(data: any): Promise<Result<T>> {
     // if (data.add_verbose && data.del_verbose) {
     //     localUrl += `/rest/v1/log?add_verbose=${data.add_verbose}&&del_verbose=${data.del_verbose}`;
     // }
-    return request(localUrl,{
-        method: (data.add_verbose || data.del_verbose)?'POST':'GET',
+    return request(localUrl, {
+        method: (data.add_verbose || data.del_verbose) ? 'POST' : 'GET',
+        ...data
     });
 }
+
 //query_profile
 export function queryProfile<T>(data: any): Promise<Result<T>> {
     let LocalUrl = '/rest/v1/query_profile/';
-    if(data.path){
+    if (data.path) {
         LocalUrl = `/rest/v1/query_profile/${data.path}`;
     }
-    return request(LocalUrl);
+    return request(LocalUrl, data);
 }
+
 //session
 export function getSession<T>(data: any): Promise<Result<T>> {
-    return request('/rest/v1/session');
+    return request('/rest/v1/session', data);
 }
+
 //config
 export function getConfig<T>(data: any): Promise<Result<T>> {
-    return request('/rest/v1/config/fe/');
+    return request('/rest/v1/config/fe/', data);
 }
+
 //query begin
-export function getDatabaseList<T>(data: any): Promise<Result<T>> {
+export function getDatabaseList<T>(data?: any): Promise<Result<T>> {
     let reURL = `${API_BASE}default_cluster/databases`;
-    if(data){
-        if(data.db_name){
+    if (data) {
+        if (data.db_name) {
             reURL += `/${data.db_name}/tables`;
         }
-        if(data.db_name&&data.tbl_name){
+        if (data.db_name && data.tbl_name) {
             reURL += `/${data.tbl_name}/schema`;
         }
     }
-    return request(reURL);
+    return request(reURL, data);
 }
+
 export function doQuery<T>(data: any): Promise<Result<T>> {
     return request(`/api/query/default_cluster/${data.db_name}`, {
-        method: 'POST',...data,
+        method: 'POST', ...data,
     });
 }
+
 export function doUp<T>(data: any): Promise<Result<T>> {
     let localHeader = {
         label: data.label,
         columns: data.columns,
-        column_separator: data. column_separator
+        column_separator: data.column_separator
     }
-    if(!localHeader.columns){
+    if (!localHeader.columns) {
         delete localHeader.columns
     }
     return request(`/api/default_cluster/${data.db_name}/${data.tbl_name}/upload?file_id=${data.file_id}&file_uuid=${data.file_uuid}`, {
-        method: 'PUT',headers:localHeader,
+        method: 'PUT', headers: localHeader,
     });
 }
+
 export function getUploadData<T>(data: any): Promise<Result<T>> {
     let localUrl = `/api/default_cluster/${data.db_name}/${data.tbl_name}/upload`
-    if(data.preview){
+    if (data.preview) {
         localUrl = `/api/default_cluster/${data.db_name}/${data.tbl_name}/upload?file_id=${data.file_id}&file_uuid=${data.file_uuid}&preview=true`
     }
-    return request(localUrl,{
+    return request(localUrl, {
         method: 'GET',
     });
 }
+
 export function deleteUploadData<T>(data: any): Promise<Result<T>> {
-    return request(`/api/default_cluster/${data.db_name}/${data.tbl_name}/upload?file_id=${data.file_id}&file_uuid=${data.file_uuid}`,{
+    return request(`/api/default_cluster/${data.db_name}/${data.tbl_name}/upload?file_id=${data.file_id}&file_uuid=${data.file_uuid}`, {
         method: 'DELETE',
     });
 }
+
 //query end
 export const AdHocAPI = {
     getDatabaseList,
@@ -131,4 +147,4 @@ export const AdHocAPI = {
     getHardwareInfo,
     getUploadData,
     deleteUploadData,
-};
\ No newline at end of file
+};
diff --git a/ui/src/components/table/index.tsx b/ui/src/components/table/index.tsx
index 94afff63c3..bd095c8631 100644
--- a/ui/src/components/table/index.tsx
+++ b/ui/src/components/table/index.tsx
@@ -6,9 +6,9 @@
  * 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
@@ -16,54 +16,54 @@
  * specific language governing permissions and limitations
  * under the License.
  */
- 
-import React, {useState,useEffect} from 'react';
+
+import React, {useState, useEffect} from 'react';
 import {Table, Popover, Input} from 'antd';
 import {FilterFilled} from '@ant-design/icons';
-import {getColumns, filterTableData} from './table.utils.tsx';
+import {getColumns, filterTableData} from './table.utils';
 import './index.less';
 
 export default function SortFilterTable(props: any) {
-    const {isFilter=false, isSort=false, allTableData, isInner, isSystem=false, path=''} = props;
-    const [tableData, setTableData] = useState([]);
-    const [localColumns, setColumns] = useState([]);
+    const {isFilter = false, isSort = false, allTableData, isInner, isSystem = false, path = '', rowKey} = props;
+    const [tableData, setTableData] = useState<any[]>([]);
+    const [localColumns, setColumns] = useState<any[]>([]);
     // function onChange(pagination, filters, sorter, extra) {
     //     console.log('params', pagination, filters, sorter, extra);
     // }
-    function changeTableData(e){
-        const localData = filterTableData(allTableData.rows,e.target.value);
+    function changeTableData(e) {
+        const localData = filterTableData(allTableData.rows, e.target.value);
         setTableData(localData);
     }
+
     const content = (
-        <Input placeholder="Filter data" onChange={e=>changeTableData(e)}/>
+        <Input placeholder="Filter data" onChange={e => changeTableData(e)}/>
     );
     useEffect(() => {
-        if(allTableData.rows&&allTableData.column_names){
-            setColumns(getColumns(allTableData.column_names, isSort, isInner, allTableData.href_columns||allTableData.href_column, path));
+        if (allTableData.rows && allTableData.column_names) {
+            setColumns(getColumns(allTableData.column_names, isSort, isInner, allTableData.href_columns || allTableData.href_column, path));
             setTableData(allTableData.rows);
         }
     }, [allTableData]);
 
-    return(
-        <span className='systemTable' >
-            {isFilter?<Popover className={isSystem?'searchSystem':'search'} content={content} trigger="click">
+    return (
+        <span className='systemTable'>
+            {isFilter ? <Popover className={isSystem ? 'searchSystem' : 'search'} content={content} trigger="click">
                 <FilterFilled/>
-            </Popover>:''}
+            </Popover> : ''}
             <Table
                 columns={localColumns}
+                rowKey={(record) => typeof rowKey === 'function' ? rowKey(record) : (typeof rowKey === 'string' ? rowKey : undefined)}
                 dataSource={tableData}
                 scroll={{ x: 'max-content' }}
                 size='small'
                 bordered
                 // onChange={onChange}
                 pagination={{
-                    size:'small',
-                    showTotal:(total, range) => `${range[0]}-${range[1]} of ${total} items`,
-                    showSizeChanger:true,
-                    showQuickJumper:true,
-                    hideOnSinglePage:true,
-                    pageSize:30,
-                    defaultPageSize:30,
+                    size: 'small',
+                    showTotal: (total, range) => `${range[0]}-${range[1]} of ${total} items`,
+                    showSizeChanger: true,
+                    showQuickJumper: true,
+                    defaultPageSize: 30,
                 }}
             />
         </span>
diff --git a/ui/src/i18n.tsx b/ui/src/i18n.tsx
index 331c465a0b..1a0c8676a5 100644
--- a/ui/src/i18n.tsx
+++ b/ui/src/i18n.tsx
@@ -6,9 +6,9 @@
  * 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
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
- 
+
 import LanguageDetector from 'i18next-browser-languagedetector';
 import i18n from 'i18next';
 import enUsTrans from '../public/locales/en-us.json';
@@ -24,8 +24,9 @@ import zhCnTrans from '../public/locales/zh-cn.json';
 import {
     initReactI18next
 } from 'react-i18next';
-i18n.use(LanguageDetector) 
-    .use(initReactI18next) 
+
+i18n.use(LanguageDetector)
+    .use(initReactI18next)
     .init({
         lng: localStorage.getItem('I18N_LANGUAGE') || "en",
         resources: {
@@ -39,9 +40,9 @@ i18n.use(LanguageDetector)
         fallbackLng: 'en',
         debug: false,
         interpolation: {
-            escapeValue: false 
+            escapeValue: false
         }
     });
 
 export default i18n;
- 
+
diff --git a/ui/src/pages/configuration/index.tsx b/ui/src/pages/configuration/index.tsx
index 0152d84039..cafd9f48b5 100644
--- a/ui/src/pages/configuration/index.tsx
+++ b/ui/src/pages/configuration/index.tsx
@@ -6,9 +6,9 @@
  * 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
@@ -16,39 +16,38 @@
  * specific language governing permissions and limitations
  * under the License.
  */
- 
-import React, {useState, useEffect} from 'react';
-import {Typography, Button, Row, Col} from 'antd';
-const {Title} = Typography;
+
+import React, {useEffect, useState} from 'react';
+import {Typography} from 'antd';
 import {getConfig} from 'Src/api/api';
 import Table from 'Src/components/table';
+
+const {Title} = Typography;
 export default function Configuration(params: any) {
-    const [allTableData, setAllTableData] = useState({});
-    const getConfigData = function(){
-        getConfig({}).then(res=>{
+    const [allTableData, setAllTableData] = useState<any>({column_names: [], rows: []});
+    const getConfigData = function (ac?: AbortController) {
+        getConfig({signal: ac?.signal}).then(res => {
             if (res && res.msg === 'success') {
                 setAllTableData(res.data);
             }
-        })
-            .catch(err=>{
-                setAllTableData({
-                    column_names:[],
-                    rows:[],
-                });
-            });
+        }).catch(err => {
+        });
     };
     useEffect(() => {
-        getConfigData();
+        const ac = new AbortController();
+        getConfigData(ac);
+        return () => ac.abort();
     }, []);
 
-    return(
-        <Typography style={{padding:'30px'}}>
+    return (
+        <Typography style={{padding: '30px'}}>
             <Title level={2}>Configure Info</Title>
             <Table
                 isSort={true}
                 isFilter={true}
                 // isInner={true}
                 allTableData={allTableData}
+                rowKey={record => record.Name}
             />
         </Typography>
     );
diff --git a/ui/src/pages/logs/index.tsx b/ui/src/pages/logs/index.tsx
index 9ca37f2860..065b55d510 100644
--- a/ui/src/pages/logs/index.tsx
+++ b/ui/src/pages/logs/index.tsx
@@ -6,9 +6,9 @@
  * 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
@@ -16,36 +16,44 @@
  * specific language governing permissions and limitations
  * under the License.
  */
- 
-import React,{useState, useEffect, useRef} from 'react';
-import {Typography, Divider, Row, Col, Input, BackTop} from 'antd';
-const {Title, Paragraph, Text} = Typography;
+
+import React, {useEffect, useRef, useState} from 'react';
+import {BackTop, Col, Divider, Input, Row, Typography} from 'antd';
 import {getLog} from 'Src/api/api';
+import {Result} from "@src/interfaces/http.interface";
+
+const {Title, Paragraph} = Typography;
 const {Search} = Input;
-import {Result} from '@src/interfaces/http.interface';
 export default function Logs(params: any) {
-    const container = useRef();
-    const [LogConfiguration, setLogConfiguration] = useState({});
-    const [LogContents, setLogContents] = useState({});
-    function getLogData(data){
-        getLog(data).then(res=>{
-            if(res.data && res.msg === 'success'){
-                if(res.data.LogConfiguration){
+    const container = useRef<HTMLDivElement>(null);
+    const [LogConfiguration, setLogConfiguration] = useState<any>({});
+    const [LogContents, setLogContents] = useState<any>({});
+
+    function getLogData(data) {
+        getLog(data).then((res: Result<any>) => {
+            if (res.data && res.msg === 'success') {
+                if (res.data.LogConfiguration) {
                     setLogConfiguration(res.data.LogConfiguration);
                 }
-                if(res.data.LogContents){
-                    container.current.innerHTML=res.data.LogContents.log;
+                if (res.data.LogContents) {
+                    if (container.current !== null) {
+                        container.current.innerHTML = res.data.LogContents.log;
+                    }
                     setLogContents(res.data.LogContents);
                 }
             }
+        }).catch(err => {
         });
     }
+
     useEffect(() => {
-        getLogData({});
+        const ac = new AbortController();
+        getLogData({signal: ac.signal});
+        return () => ac.abort();
     }, []);
-    return(
-        <Typography style={{padding:'30px'}}>
-            <Title >Log Configuration</Title>
+    return (
+        <Typography style={{padding: '30px'}}>
+            <Title>Log Configuration</Title>
             <Paragraph>
                 <p>Level: {LogConfiguration.VerboseNames}</p>
                 <p>Verbose Names:{LogConfiguration.VerboseNames}</p>
@@ -56,24 +64,24 @@ export default function Logs(params: any) {
                     <Search
                         placeholder="new verbose name"
                         enterButton="Add"
-                        onSearch={value => getLogData({add_verbose:value})}
+                        onSearch={value => getLogData({add_verbose: value})}
                     />
                 </Col>
                 <Col span={4} offset={1}>
                     <Search
                         placeholder="del verbose name"
                         enterButton="Delete"
-                        onSearch={value => getLogData({del_verbose:value})}
+                        onSearch={value => getLogData({del_verbose: value})}
                     />
                 </Col>
             </Row>
             <Divider/>
-            <Title style={{marginTop:'0px'}}>Log Contents</Title>
+            <Title style={{marginTop: '0px'}}>Log Contents</Title>
             <Paragraph>
                 <p>Log path is: {LogContents.logPath}</p>
                 <p>{LogContents.showingLast}</p>
             </Paragraph>
-            <div ref={container} style={{background: '#f9f9f9',padding: '20px'}}>
+            <div ref={container} style={{background: '#f9f9f9', padding: '20px'}}>
                 {/* {LogContents.log} */}
             </div>
             <BackTop></BackTop>
diff --git a/ui/src/pages/playground/content/components/data-prev.tsx b/ui/src/pages/playground/content/components/data-prev.tsx
index d3856848ce..8d027c34e2 100644
--- a/ui/src/pages/playground/content/components/data-prev.tsx
+++ b/ui/src/pages/playground/content/components/data-prev.tsx
@@ -6,9 +6,9 @@
  * 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
@@ -16,65 +16,73 @@
  * specific language governing permissions and limitations
  * under the License.
  */
- 
-import React,{useState,useEffect} from 'react';
+
+import React, {useEffect, useState} from 'react';
 import {AdHocAPI} from 'Src/api/api';
 import {getDbName} from 'Utils/utils';
-import {Row, Empty, notification, Table} from 'antd';
+import {notification, Row, Table} from 'antd';
 import {FlatBtn} from 'Components/flatbtn';
 import {useTranslation} from 'react-i18next';
+
 export function DataPrev(props: any) {
-    let { t } = useTranslation();
-    const {db_name,tbl_name} = getDbName();
-    const [tableData,setTableData] = useState([]);
-    const [columns,setColumns] = useState([]);
+    let {t} = useTranslation();
+    const {db_name, tbl_name} = getDbName();
+    const [tableData, setTableData] = useState<any[]>([]);
+    const [columns, setColumns] = useState<any[]>([]);
+
     function toQuery(): void {
-        if (!tbl_name){
-            notification.error({message: t('selectWarning')});
+        if (!tbl_name) {
+            notification.error({message: t<string>('selectWarning')});
             return;
         }
         AdHocAPI.doQuery({
             db_name,
-            body:{stmt:`SELECT * FROM ${db_name}.${tbl_name} LIMIT 10`},
-        }).then(res=>{
+            body: {stmt: `SELECT * FROM ${db_name}.${tbl_name} LIMIT 10`},
+        }).then((res: any) => {
             if (res && res.msg === 'success') {
-                console.log(getColumns(res.data?.meta),2222)
+                console.log(getColumns(res.data?.meta), 2222)
                 setColumns(getColumns(res.data?.meta))
-                setTableData(getTabledata(res.data));
+                setTableData(getTableData(res.data));
             }
-        })
-        .catch(()=>{
+        }).catch(() => {
             setTableData([]);
         });
     }
-    function getColumns(params: string[]) {
-        console.log(params,2222)
-        if(!params||params.length === 0){return [];}
-        
-        let arr = params.map(item=> {
+
+    function getColumns(params: any[]) {
+        if (!params || params.length === 0) {
+            return [];
+        }
+
+        let arr: any[] = params.map(item => {
             return {
                 title: item.name,
                 dataIndex: item.name,
                 key: item.name,
                 width: 150,
-                render:(text, record, index)=>{return text === '\\N' ? '-' : text}
+                render: (text, record, index) => {
+                    return text === '\\N' ? '-' : text
+                }
             };
         });
         return arr;
     }
-    function getTabledata(data){
-        let meta  = data.meta;
+
+    function getTableData(data): any[] {
+        let meta = data.meta;
         let source = data.data;
-        let res = [];
-        if(!source||source.length === 0){return [];}
-        let metaArr = meta.map(item=>item.name)
-        for (let i=0;i<source.length;i++) {
+        let res: any[] = [];
+        if (!source || source.length === 0) {
+            return [];
+        }
+        let metaArr = meta.map(item => item.name)
+        for (let i = 0; i < source.length; i++) {
             let node = source[i];
-            if(node.length !== meta.length){
-                return {}
+            if (node.length !== meta.length) {
+                return []
             }
             let obj = {}
-            metaArr.map((item,idx)=>{
+            metaArr.map((item, idx) => {
                 obj[item] = node[idx]
             })
             obj['key'] = i
@@ -82,13 +90,14 @@ export function DataPrev(props: any) {
         }
         return res;
     }
-    useEffect(()=>{
+
+    useEffect(() => {
         toQuery();
-    },[location.pathname]);
+    }, [location.pathname]);
     return (
         <div>
             <Row justify="space-between" style={{marginBottom: 10}}>
-                <span style={{paddingBottom:'15px'}}>{t('dataPreview')}({t('display10')})</span>
+                <span style={{paddingBottom: '15px'}}>{t('dataPreview') + "(" + t('display10') + ")"}</span>
                 <span>
                     {db_name}.{tbl_name}
                 </span>
@@ -103,13 +112,13 @@ export function DataPrev(props: any) {
             <Table
                 bordered
                 columns={columns}
-                style={{maxWidth:' calc(100vw - 350px)'}}
+                style={{maxWidth: ' calc(100vw - 350px)'}}
                 // scroll={{ x:'500', y: '36vh'}}
-                scroll={{ x: 1500, y: 300 }}
+                scroll={{x: 1500, y: 300}}
                 dataSource={tableData}
                 size="small"
             />
-           </div>
+        </div>
     );
 }
 
diff --git a/ui/src/pages/playground/page-side/index.tsx b/ui/src/pages/playground/page-side/index.tsx
index 10f3b4a9c4..3347621176 100644
--- a/ui/src/pages/playground/page-side/index.tsx
+++ b/ui/src/pages/playground/page-side/index.tsx
@@ -6,9 +6,9 @@
  * 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
@@ -16,26 +16,27 @@
  * specific language governing permissions and limitations
  * under the License.
  */
- 
-import React, {useState, useEffect, SyntheticEvent} from 'react';
+
+import React, {SyntheticEvent, useState} from 'react';
 import {ResizableBox} from 'react-resizable';
-require('react-resizable/css/styles.css');
 import styles from './index.less';
 
+require('react-resizable/css/styles.css');
+
 export function PageSide(props: any) {
     const {children} = props;
-    const [sideBoxWidth,setSideBoxWidth] = useState(300);
+    const [sideBoxWidth, setSideBoxWidth] = useState(300);
 
-    const onResize=function(e: SyntheticEvent, data: any) {
+    const onResize = function (e: SyntheticEvent, data: any) {
         const width = data.size.width || 300;
         setSideBoxWidth(width);
     };
     return (
-        <div className={styles['side']} >
+        <div className={styles['side']}>
             <ResizableBox
                 width={sideBoxWidth}
                 height={Infinity}
-                style={{'minHeight':'calc(100vh - 64px)','overflow':'hidden'}}
+                style={{'minHeight': 'calc(100vh - 64px)', 'overflow': 'hidden'}}
                 resizeHandles={['e']}
                 onResizeStart={onResize}
                 minConstraints={[300, 300]}
@@ -45,7 +46,6 @@ export function PageSide(props: any) {
                 {children}
             </ResizableBox>
         </div>
-
     );
 }
  
diff --git a/ui/src/pages/playground/tree/index.tsx b/ui/src/pages/playground/tree/index.tsx
index 0f0d440893..e3f56f98e7 100644
--- a/ui/src/pages/playground/tree/index.tsx
+++ b/ui/src/pages/playground/tree/index.tsx
@@ -6,9 +6,9 @@
  * 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
@@ -16,23 +16,25 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-import React, {useState,useEffect} from 'react';
-import {Tree, Spin, Input} from 'antd';
-const { Search } = Input;
-import {TableOutlined, HddOutlined, ReloadOutlined} from '@ant-design/icons';
+import React, {useEffect, useState} from 'react';
+import {Input, Spin, Tree} from 'antd';
+import {HddOutlined, ReloadOutlined, TableOutlined} from '@ant-design/icons';
 import {AdHocAPI} from 'Src/api/api';
 import {useTranslation} from 'react-i18next';
-import {
-    AdhocContentRouteKeyEnum,
-} from '../adhoc.data';
+import {AdhocContentRouteKeyEnum,} from '../adhoc.data';
+import './index.css';
+
+const {Search} = Input;
+
 interface DataNode {
-  title: string;
-  key: string;
-  isLeaf?: boolean;
-  children?: DataNode[];
+    title: string;
+    key: string;
+    isLeaf?: boolean;
+    children?: DataNode[];
 }
-import './index.css';
+
 const initTreeDate: DataNode[] = [];
+
 function updateTreeData(list: DataNode[], key, children) {
     return list.map(node => {
         if (node.key === key) {
@@ -40,26 +42,30 @@ function updateTreeData(list: DataNode[], key, children) {
                 ...node,
                 children,
             };
-        } 
+        }
         return node;
     });
 }
+
 export function AdHocTree(props: any) {
-    let { t } = useTranslation();
+    let {t} = useTranslation();
     const [treeData, setTreeData] = useState(initTreeDate);
     const [realTree, setRealTree] = useState(initTreeDate);
     const [loading, setLoading] = useState(true);
-    const [expandedKeys, setExpandedKeys] = useState([]);
-    const [searchValue, setSearchValue] = useState('');
+    const [expandedKeys, setExpandedKeys] = useState<any[]>([]);
     const [autoExpandParent, setAutoExpandParent] = useState(true);
+
     useEffect(() => {
-        initTreeData()
+        const ac = new AbortController();
+        initTreeData(ac);
+        return () => ac.abort();
     }, []);
-    function initTreeData(){
-        AdHocAPI.getDatabaseList().then(res=>{
+
+    function initTreeData(ac?: AbortController) {
+        AdHocAPI.getDatabaseList({signal: ac?.signal}).then(res => {
             if (res.msg === 'success' && Array.isArray(res.data)) {
                 const num = Math.random()
-                const treeData = res.data.map((item,index)=>{
+                const treeData = res.data.map((item, index) => {
                     return {
                         title: item,
                         key: `${num}-1-${index}-${item}`,
@@ -70,34 +76,35 @@ export function AdHocTree(props: any) {
                 getRealTree(treeData);
             }
             setLoading(false);
+        }).catch(err => {
         });
     }
+
     function onLoadData({key, children}) {
-        const [random, storey, index, db_name] = key.split('-');
+        const [, storey, , db_name] = key.split('-');
         const param = {
             db_name,
             // tbl_name,
         };
-        return AdHocAPI.getDatabaseList(param).then(res=>{
-            if (res.msg=='success' && Array.isArray(res.data)) {
-                const children = res.data.map((item,index)=>{
-                    if (storey === '1'){
+        return AdHocAPI.getDatabaseList(param).then(res => {
+            if (res.msg == 'success' && Array.isArray(res.data)) {
+                const children = res.data.map((item, index) => {
+                    if (storey === '1') {
                         return {
                             title: item,
                             key: `2-${index}-${param.db_name}-${item}`,
-                            icon: <TableOutlined />,
+                            icon: <TableOutlined/>,
                             isLeaf: true,
                         };
                     }
-
                 });
                 const trData = updateTreeData(treeData, key, children);
                 setTreeData(trData);
                 getRealTree(trData);
             }
         });
-
     }
+
     function handleTreeSelect(
         keys: React.ReactText[],
         info: any,
@@ -107,73 +114,79 @@ export function AdHocTree(props: any) {
             props.history.push(`/Playground/${path}/${keys[0].split(':')[1]}`);
         }
     }
-    function onSearch(e){
-        const { value } = e.target;
-        const expandedKeys = treeData
-          .map((item, index) => {
-              if (getParentKey(value, treeData[index].children, index)) {
-                return item.key
-              } else {
-                  return null;
-              }
-          })
+
+    function onSearch(e) {
+        const {value} = e.target;
+        const expandedKeys: any[] = treeData
+            .map((item, index) => {
+                if (getParentKey(value, treeData[index].children, index)) {
+                    return item.key
+                } else {
+                    return null;
+                }
+            })
         setExpandedKeys(expandedKeys);
-        setSearchValue(value);
         setAutoExpandParent(true);
         getRealTree(treeData, value);
-    };
+    }
+
     function onExpand(expandedKeys) {
         setExpandedKeys(expandedKeys);
         setAutoExpandParent(false);
-    };
+    }
+
     const getParentKey = (key, tree, idx) => {
         if (!tree) {
             return false;
         }
         for (let i = 0; i < tree.length; i++) {
-          const node = tree[i];
-          if (node.title.includes(key)) {
-            return true
-          } else {
-            treeData[idx].children ? treeData[idx].children[i].title = node.title : ''
-          }
+            const node = tree[i];
+            if (node.title.includes(key)) {
+                return true
+            } else {
+                treeData[idx].children ? treeData[idx].children[i].title = node.title : ''
+            }
         }
         return false;
     };
-    function getRealTree(treeData, value){
-        const realTree  = inner(treeData);
-        function inner(treeData){
+
+    function getRealTree(treeData, value?) {
+        const realTree = inner(treeData);
+
+        function inner(treeData) {
             return treeData.map(item => {
                 const search = value || '';
                 const index = item.title.indexOf(search);
                 const beforeStr = item.title.substr(0, index);
                 const afterStr = item.title.substr(index + search.length);
                 const title =
-                  index > -1 ? (
-                    <span>
+                    index > -1 ? (
+                        <span>
                       {beforeStr}
-                      <span className="site-tree-search-value">{search}</span>
-                      {afterStr}
+                            <span className="site-tree-search-value">{search}</span>
+                            {afterStr}
                     </span>
-                  ) : (
-                    item.title
-                  );
+                    ) : (
+                        item.title
+                    );
                 if (item.children) {
-                  return {...item, title, children: inner(item.children)};
+                    return {...item, title, children: inner(item.children)};
                 }
                 return {
-                  ...item,
-                  title
+                    ...item,
+                    title
                 };
             });
         }
-        debounce(setRealTree(realTree),300);
+
+        debounce(setRealTree(realTree), 300);
     }
+
     function debounce(fn, wait) {
-        var timer = null;
+        let timer = null;
         return function () {
-            var context = this
-            var args = arguments
+            let context = this
+            let args = arguments
             if (timer) {
                 clearTimeout(timer);
                 timer = null;
@@ -183,19 +196,20 @@ export function AdHocTree(props: any) {
             }, wait)
         }
     }
+
     return (
         <>
             <Spin spinning={loading} size="small"/>
             <div>
-                <Search 
-                    size="small" 
-                    style={{ padding: 5, position: 'fixed', zIndex: '99', width: '300px'}} 
-                    placeholder={t('search')} 
-                    enterButton={<ReloadOutlined />} 
+                <Search
+                    size="small"
+                    style={{padding: 5, position: 'fixed', zIndex: '99', width: '300px'}}
+                    placeholder={t('search')}
+                    enterButton={<ReloadOutlined/>}
                     onSearch={initTreeData}
-                    onChange={onSearch} />
+                    onChange={onSearch}/>
             </div>
-            
+
             <Tree
                 showIcon={true}
                 loadData={onLoadData}
@@ -203,12 +217,12 @@ export function AdHocTree(props: any) {
                 onExpand={onExpand}
                 expandedKeys={expandedKeys}
                 autoExpandParent={autoExpandParent}
-                style={{'width':'100%', height:'86vh' ,paddingTop:'35px',overflowY:'scroll'}}
+                style={{'width': '100%', height: '86vh', paddingTop: '35px', overflowY: 'scroll'}}
                 onSelect={(selectedKeys, info) =>
                     handleTreeSelect(
                         selectedKeys,
                         info,
-                        AdhocContentRouteKeyEnum.Structure                                                        )
+                        AdhocContentRouteKeyEnum.Structure)
                 }
             />
         </>
diff --git a/ui/src/pages/query-profile/index.tsx b/ui/src/pages/query-profile/index.tsx
index 2b296cd233..ccdf64ce26 100644
--- a/ui/src/pages/query-profile/index.tsx
+++ b/ui/src/pages/query-profile/index.tsx
@@ -6,9 +6,9 @@
  * 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
@@ -16,74 +16,82 @@
  * specific language governing permissions and limitations
  * under the License.
  */
- 
-import React, {useState, useEffect, useRef} from 'react';
-import {Typography, Button, Row, Col} from 'antd';
-const {Text, Title, Paragraph} = Typography;
+
+import React, {useEffect, useRef, useState} from 'react';
+import {Button, Col, Row, Typography} from 'antd';
 import {queryProfile} from 'Src/api/api';
 import Table from 'Src/components/table';
 import {useHistory} from 'react-router-dom';
+import {Result} from '@src/interfaces/http.interface';
+
+const {Text, Title} = Typography;
 export default function QueryProfile(params: any) {
     // const [parentUrl, setParentUrl] = useState('');
-    const container = useRef();
-    const [allTableData, setAllTableData] = useState({});
-    const [profile, setProfile] = useState();
+    const container = useRef<HTMLDivElement>(null);
+    const [allTableData, setAllTableData] = useState({column_names: [], rows: []});
+    const [profile, setProfile] = useState<any>();
     const history = useHistory();
-    const doQueryProfile = function(){
+    const doQueryProfile = function (ac?: AbortController) {
         const param = {
             path: getLastPath(),
+            signal: ac?.signal
         };
-        queryProfile(param).then(res=>{
+        queryProfile(param).then((res: Result<any>) => {
             if (res && res.msg === 'success') {
-                if(!res.data.column_names){
+                if (!res.data.column_names) {
                     setProfile(res.data);
-                    container.current.innerHTML=res.data;
-                }else{
+                    if (container.current !== null) {
+                        container.current.innerHTML = res.data;
+                    }
+                } else {
                     setProfile('');
                     setAllTableData(res.data);
                 }
             } else {
                 setAllTableData({
-                    column_names:[],
-                    rows:[],
+                    column_names: [],
+                    rows: [],
                 });
             }
-        })
-            .catch(err=>{
-                setAllTableData({
-                    column_names:[],
-                    rows:[],
-                });
-            });
+        }).catch(err => {
+        });
     };
     useEffect(() => {
-        doQueryProfile();
+        const ac = new AbortController();
+        doQueryProfile(ac);
+        return () => ac.abort();
     }, [location.pathname]);
-    function getLastPath(){
+
+    function getLastPath() {
         let arr = location.pathname.split('/');
         let str = arr.pop();
         return str === 'QueryProfile' ? '' : str;
     }
-    function goPrev(){
-        if (location.pathname === '/QueryProfile/') {return;}
+
+    function goPrev() {
+        if (location.pathname === '/QueryProfile/') {
+            return;
+        }
         history.push('/QueryProfile/');
     }
-    return(
-        <Typography style={{padding:'30px'}}>
-            <Title >Finished Queries</Title>
 
-            <Row style={{paddingBottom:'15px'}}>
-                <Col span={12} ><Text type="strong">This table lists the latest 100 queries</Text></Col>
-                <Col span={12} style={{textAlign:'right'}}>
-                    {profile?<Button  type="primary" onClick={goPrev}>back</Button>:''}
+    return (
+        <Typography style={{padding: '30px'}}>
+            <Title>Finished Queries</Title>
+
+            <Row style={{paddingBottom: '15px'}}>
+                <Col span={12}><Text strong={true}>This table lists the latest 100 queries</Text></Col>
+                <Col span={12} style={{textAlign: 'right'}}>
+                    {profile ? <Button type="primary" onClick={goPrev}>back</Button> : ''}
                 </Col>
             </Row>
             {
                 profile
-                    ?<div ref={container} style={{background: '#f9f9f9',padding: '20px'}}>
+                    ? <div ref={container} style={{background: '#f9f9f9', padding: '20px'}}>
                         {/* {profile} */}
                     </div>
-                    :<Table
+                    : <Table
+                        rowKey={(record) => record['Query ID']}
                         isSort={true}
                         isFilter={true}
                         isInner={'/QueryProfile'}
diff --git a/ui/src/pages/session/index.tsx b/ui/src/pages/session/index.tsx
index 393cf324d7..4c97c095be 100644
--- a/ui/src/pages/session/index.tsx
+++ b/ui/src/pages/session/index.tsx
@@ -6,9 +6,9 @@
  * 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
@@ -16,47 +16,46 @@
  * specific language governing permissions and limitations
  * under the License.
  */
- 
-import React, {useState, useEffect} from 'react';
-import {Typography, Button, Row, Col} from 'antd';
-const {Text, Title, Paragraph} = Typography;
+
+import React, {useEffect, useState} from 'react';
+import {Row, Typography} from 'antd';
 import {getSession} from 'Src/api/api';
 import Table from 'Src/components/table';
+
+const {Text, Title} = Typography;
 // import {useHistory} from 'react-router-dom';
 export default function Session(params: any) {
     // const [parentUrl, setParentUrl] = useState('');
-    const [allTableData, setAllTableData] = useState({});
+    const [allTableData, setAllTableData] = useState<any>({column_names: [], rows: []});
     // const history = useHistory();
-    const getSessionData = function(){
-        getSession({}).then(res=>{
+    const getSessionData = function (ac: AbortController) {
+        getSession({signal: ac.signal}).then(res => {
             if (res && res.msg === 'success') {
                 setAllTableData(res.data);
             }
-        })
-            .catch(err=>{
-                setAllTableData({
-                    column_names:[],
-                    rows:[],
-                });
-            });
+        }).catch(err => {
+        });
     };
     useEffect(() => {
-        getSessionData();
+        const ac = new AbortController();
+        getSessionData(ac);
+        return () => ac.abort();
     }, []);
-    return(
-        <Typography style={{padding:'30px'}}>
-            <Title >Session Info</Title>
+    return (
+        <Typography style={{padding: '30px'}}>
+            <Title>Session Info</Title>
 
-            <Row style={{paddingBottom:'15px'}}>
-                <Text type="strong">This page lists the session info, there are {allTableData?.rows?.length} active sessions.</Text>
+            <Row style={{paddingBottom: '15px'}}>
+                <Text strong={true}>This page lists the session info, there are {allTableData?.rows?.length} active
+                    sessions.</Text>
             </Row>
             <Table
                 isSort={true}
                 isFilter={true}
                 // isInner={true}
                 allTableData={allTableData}
+                rowKey={(record) => record.Id}
             />
-
         </Typography>
     );
 }
diff --git a/ui/src/pages/system/index.tsx b/ui/src/pages/system/index.tsx
index ce96744a5b..f03c366264 100644
--- a/ui/src/pages/system/index.tsx
+++ b/ui/src/pages/system/index.tsx
@@ -6,9 +6,9 @@
  * 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
@@ -16,45 +16,48 @@
  * specific language governing permissions and limitations
  * under the License.
  */
- 
 
-import React, {useState, useEffect} from 'react';
-import {Typography, Button, Row, Col} from 'antd';
-const {Text, Title, Paragraph} = Typography;
+
+import React, {useEffect, useState} from 'react';
+import {Button, Col, Row, Typography} from 'antd';
 import {getSystem} from 'Src/api/api';
 import Table from 'Src/components/table';
 import {useHistory} from 'react-router-dom';
+import {Result} from '@src/interfaces/http.interface';
+
+const {Text, Title, Paragraph} = Typography;
+
 export default function System(params: any) {
-    const [parentUrl, setParentUrl] = useState('');
-    const [allTableData, setAllTableData] = useState({});
+    const [parentUrl, setParentUrl] = useState<string>('');
+    const [allTableData, setAllTableData] = useState<any>({column_names: [], rows: []});
 
     const history = useHistory();
-    const getSystemData = function(){
+    const getSystemData = function (ac?: any) {
         const param = {
-            path:location.search,
+            path: location.search,
+            signal: ac?.signal,
         };
-        getSystem(param).then(res=>{
+        getSystem(param).then((res: Result<any>) => {
             if (res && res.msg === 'success') {
                 setAllTableData(res.data);
                 setParentUrl(res.data.parent_url);
             } else {
                 setAllTableData({
-                    column_names:[],
-                    rows:[],
+                    column_names: [],
+                    rows: [],
                 });
             }
-        })
-            .catch(err=>{
-                setAllTableData({
-                    column_names:[],
-                    rows:[],
-                });
-            });
+        }).catch(err => {
+        });
     };
+
     useEffect(() => {
-        getSystemData();
+        const ac = new AbortController();
+        getSystemData(ac);
+        return () => ac.abort();
     }, [location.search]);
-    function goPrev(){
+
+    function goPrev() {
         if (parentUrl === '/rest/v1/system') {
             history.push('/System?path=/');
             return;
@@ -63,16 +66,17 @@ export default function System(params: any) {
             history.push(parentUrl.split('v1/')[1]);
         }
     }
-    return(
-        <Typography style={{padding:'30px'}}>
+
+    return (
+        <Typography style={{padding: '30px'}}>
             <Title level={2}>System Info</Title>
-            <Text type="strong">This page lists the system info, like /proc in Linux.</Text>
+            <Text strong={true}>This page lists the system info, like /proc in Linux.</Text>
             <Paragraph>
 
             </Paragraph>
-            <Row style={{paddingBottom:'15px'}}>
-                <Col span={12} style={{color:'#02a0f9'}}>Current path: {location.search.split('=')[1]}</Col>
-                <Col span={12} style={{textAlign:'right'}}>
+            <Row style={{paddingBottom: '15px'}}>
+                <Col span={12} style={{color: '#02a0f9'}}>Current path: {location.search.split('=')[1]}</Col>
+                <Col span={12} style={{textAlign: 'right'}}>
                     <Button size='small' type="primary" onClick={goPrev}>Parent Dir</Button>
                 </Col>
             </Row>
@@ -84,6 +88,7 @@ export default function System(params: any) {
                 path = 'System'
                 isSystem = {true}
                 allTableData={allTableData}
+                rowKey={(record) => record.name}
             />
         </Typography>
     );
diff --git a/ui/src/utils/lazy.tsx b/ui/src/utils/lazy.tsx
index df9d483c60..f031556eea 100644
--- a/ui/src/utils/lazy.tsx
+++ b/ui/src/utils/lazy.tsx
@@ -6,9 +6,9 @@
  * 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
@@ -16,23 +16,24 @@
  * specific language governing permissions and limitations
  * under the License.
  */
- 
+
 import React, {Component} from 'react';
 
 const asyncComponent = importComponent => {
-    return class extends Component {
+    return class extends Component<{}, { component: any }> {
         constructor(props) {
             super(props);
             this.state = {
                 component: null
             };
         }
+
         componentDidMount() {
-            importComponent()
-                .then(cmp => {
-                    this.setState({component: cmp.default});
-                });
+            importComponent().then(cmp => {
+                this.setState({component: cmp.default});
+            });
         }
+
         render() {
             const C = this.state.component;
             return C ? <C {...this.props}/> : null;
diff --git a/ui/src/utils/utils.ts b/ui/src/utils/utils.ts
index 584c288edc..eeb5b657f9 100644
--- a/ui/src/utils/utils.ts
+++ b/ui/src/utils/utils.ts
@@ -27,16 +27,14 @@ function isSuccess(response) {
 
     let {code, msg} = response;
 
-    if (code === 0 && msg === 'success') {
-        return true
-    }
-    return false;
+    return code === 0 && msg === 'success';
 }
-function getDbName(params) {
+
+function getDbName(params?: any) {
     const infoArr = location.pathname.split('/');
-    const str = infoArr[infoArr.length-1];
-    const res = {};
-    if(str && str !=='Playground'){
+    const str = infoArr[infoArr.length - 1];
+    const res: any = {};
+    if (str && str !== 'Playground') {
         const db_name = str.split('-')[0];
         const tbl_name = str.split('-')[1];
         res.db_name = db_name;
@@ -44,10 +42,11 @@ function getDbName(params) {
     }
     return res;
 }
+
 function getTimeNow() {
     let dateNow = new Date();
     let fmt = 'yyyy-MM-dd hh:mm:ss';
-    var o = {
+    let o = {
         'M+': dateNow.getMonth() + 1,
         'd+': dateNow.getDate(),
         'h+': dateNow.getHours(),
@@ -58,29 +57,31 @@ function getTimeNow() {
     };
     if (/(y+)/.test(fmt)) {
         fmt = fmt.replace(
-          RegExp.$1,
-          (dateNow.getFullYear() + '').substr(4 - RegExp.$1.length)
+            RegExp.$1,
+            (dateNow.getFullYear() + '').substr(4 - RegExp.$1.length)
         );
     }
-    for (var k in o) {
+    for (let k in o) {
         if (new RegExp('(' + k + ')').test(fmt)) {
             fmt = fmt.replace(
                 RegExp.$1,
                 RegExp.$1.length === 1
-                ? o[k]
-                : ('00' + o[k]).substr(('' + o[k]).length)
+                    ? o[k]
+                    : ('00' + o[k]).substr(('' + o[k]).length)
             );
         }
     }
     return fmt;
 }
-function getBasePath(){
+
+function getBasePath() {
     let arr = location.pathname.split('/');
     let res = '';
-    if(arr.length>5){
-        arr = arr.slice(0,5);
+    if (arr.length > 5) {
+        arr = arr.slice(0, 5);
         res = arr.join('/');
     }
     return res;
 }
-module.exports = {isSuccess, getDbName, getTimeNow, getBasePath};
\ No newline at end of file
+
+export {isSuccess, getDbName, getTimeNow, getBasePath};
diff --git a/ui/tsconfig.json b/ui/tsconfig.json
index 0f07e067e8..19d16e4e61 100644
--- a/ui/tsconfig.json
+++ b/ui/tsconfig.json
@@ -1,29 +1,30 @@
 {
-    "compilerOptions": {
-      "outDir": "dist",
-      "module": "commonjs",
-      "target": "es5",
-      "lib": ["es7", "dom", "es2015.iterable", "es2019"],
-      "sourceMap": true,
-      "allowJs": true,
-      "jsx": "react",
-      "moduleResolution": "node",
-      "forceConsistentCasingInFileNames": false,
-      "noImplicitReturns": true,
-      "noImplicitThis": true,
-      "noImplicitAny": false,
-      "strictNullChecks": true,
-      "esModuleInterop": true,
-      "suppressImplicitAnyIndexErrors": true,
-      "noUnusedLocals": true,
-      "allowSyntheticDefaultImports": true,
-      "experimentalDecorators": true,
-      "emitDecoratorMetadata": true,
-      "noImplicitUseStrict": true,
-      "alwaysStrict": false,
-      "downlevelIteration": true,
-      "baseUrl": "./",
-      "paths": {
+  "compilerOptions": {
+    "outDir": "dist",
+    "module": "commonjs",
+    "target": "es5",
+    "lib": ["es7", "dom", "es2015.iterable", "es2019"],
+    "sourceMap": true,
+    "allowJs": true,
+    "jsx": "react",
+    "moduleResolution": "node",
+    "forceConsistentCasingInFileNames": false,
+    "noImplicitReturns": true,
+    "noImplicitThis": true,
+    "noImplicitAny": false,
+    "strictNullChecks": true,
+    "resolveJsonModule": true,
+    "esModuleInterop": true,
+    "suppressImplicitAnyIndexErrors": true,
+    "noUnusedLocals": true,
+    "allowSyntheticDefaultImports": true,
+    "experimentalDecorators": true,
+    "emitDecoratorMetadata": true,
+    "noImplicitUseStrict": true,
+    "alwaysStrict": false,
+    "downlevelIteration": true,
+    "baseUrl": "./",
+    "paths": {
         "Components/*": ["src/components/*"],
         "Src": ["src"],
         "Utils/*": ["src/utils/*"],
@@ -32,28 +33,27 @@
         "Constants": ["src/constants"],
         "@hooks/*": ["src/hooks/*"],
         "@src/*": ["src/*"]
-      },
-      "typeRoots": [
-        "./node_modules/@types",
-        "custom_typings",
-        "./typings/**/*.d.ts"
-      ]
     },
-    "exclude": [
-      "build",
-      "scripts",
-      "webpack",
-      "jest",
-      "bin",
-      "runtime",
-      "view",
-      "public_gen",
-      "public",
-      "node_modules",
-      "coverage",
-      ".vscode"
-    ],
+    "typeRoots": [
+      "./node_modules/@types",
+      "custom_typings",
+      "./typings/**/*.d.ts"
+    ]
+  },
+  "exclude": [
+    "build",
+    "scripts",
+    "webpack",
+    "jest",
+    "bin",
+    "runtime",
+    "view",
+    "public_gen",
+    "public",
+    "node_modules",
+    "coverage",
+    ".vscode"
+  ],
     "includes": ["src/**/*.ts", "src/**/*.tsx", "global.d.ts"],
     "types": ["typePatches"]
-  }
-  
\ No newline at end of file
+}


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


[doris] 03/04: [fix](planner) disconjunct in sub-query failed when plan it on hash join (#15653)

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 3e700f325ef7850475428ca3613df9eb09a0fe47
Author: morrySnow <10...@users.noreply.github.com>
AuthorDate: Tue Jan 10 11:10:12 2023 +0800

    [fix](planner) disconjunct in sub-query failed when plan it on hash join (#15653)
    
    all conjuncts should be added before HashJoinNode init. Otherwise, some slots on conjuncts linked to the tuple not in intermediate tuple on HashJoinNode
---
 .../src/main/java/org/apache/doris/planner/SingleNodePlanner.java       | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java
index 86044c13b8..8b27b1e1a3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java
@@ -2085,8 +2085,8 @@ public class SingleNodePlanner {
 
         HashJoinNode result = new HashJoinNode(ctx.getNextNodeId(), outer, inner,
                 innerRef, eqJoinConjuncts, ojConjuncts);
-        result.init(analyzer);
         result.addConjuncts(analyzer.getMarkConjuncts(innerRef));
+        result.init(analyzer);
         return result;
     }
 


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


[doris] 02/04: [fix](tvf) use virtual-hosted style when s3('uri'='s3://xxx') (#15617)

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 4b69d201526b25394733f6663abdd27b56df1f08
Author: xueweizhang <zx...@163.com>
AuthorDate: Mon Jan 9 14:09:40 2023 +0800

    [fix](tvf) use virtual-hosted style when s3('uri'='s3://xxx') (#15617)
    
    Signed-off-by: nextdreamblue <zx...@163.com>
    
    Signed-off-by: nextdreamblue <zx...@163.com>
---
 .../java/org/apache/doris/tablefunction/S3TableValuedFunction.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/S3TableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/S3TableValuedFunction.java
index ccf5c3eac5..15cf2264e5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/S3TableValuedFunction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/S3TableValuedFunction.java
@@ -75,8 +75,8 @@ public class S3TableValuedFunction extends ExternalFileTableValuedFunction {
 
         String originUri = validParams.getOrDefault(S3_URI, "");
         if (originUri.toLowerCase().startsWith("s3")) {
-            // s3 protocol
-            forceVirtualHosted = false;
+            // s3 protocol, default virtual-hosted style
+            forceVirtualHosted = true;
         } else {
             // not s3 protocol, forceVirtualHosted is determined by USE_PATH_STYLE.
             forceVirtualHosted = !Boolean.valueOf(validParams.get(USE_PATH_STYLE)).booleanValue();


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