This is an automated email from the ASF dual-hosted git repository.
morningman pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/master by this push:
new a4956d6d378 [feature](metacache) add system table
catalog_meta_cache_statistics (#40155)
a4956d6d378 is described below
commit a4956d6d37892103852099df6cbdeb8a0360d90d
Author: Mingyu Chen <[email protected]>
AuthorDate: Tue Sep 3 14:00:03 2024 +0800
[feature](metacache) add system table catalog_meta_cache_statistics (#40155)
Add new system table `catalog_meta_cache_statistics`
```
mysql> select * from information_schema.catalog_meta_cache_statistics;
+--------------+-----------------------------+----------------------+---------------------+
| CATALOG_NAME | CACHE_NAME | METRIC_NAME |
METRIC_VALUE |
+--------------+-----------------------------+----------------------+---------------------+
| hms | hive_file_cache | eviction_count | 0
|
| hms | hive_file_cache | hit_ratio | 0.6
|
| hms | hive_file_cache | average_load_penalty |
7.60805755E7 |
| hms | hive_file_cache | estimated_size | 2
|
| hms | hive_file_cache | hit_count | 3
|
| hms | hive_file_cache | read_count | 5
|
| hms | hive_partition_cache | eviction_count | 0
|
| hms | hive_partition_cache | hit_ratio | 1.0
|
| hms | hive_partition_cache | average_load_penalty | 0.0
|
| hms | hive_partition_cache | estimated_size | 0
|
| hms | hive_partition_cache | hit_count | 0
|
| hms | hive_partition_cache | read_count | 0
|
| hms | hive_partition_values_cache | eviction_count | 0
|
| hms | hive_partition_values_cache | hit_ratio | 1.0
|
| hms | hive_partition_values_cache | average_load_penalty | 0.0
|
| hms | hive_partition_values_cache | estimated_size | 0
|
| hms | hive_partition_values_cache | hit_count | 0
|
| hms | hive_partition_values_cache | read_count | 0
|
| hms | hudi_partition_cache | eviction_count | 0
|
| hms | hudi_partition_cache | hit_ratio | 1.0
|
| hms | hudi_partition_cache | average_load_penalty | 0.0
|
| hms | hudi_partition_cache | estimated_size | 0
|
| hms | hudi_partition_cache | hit_count | 0
|
| hms | hudi_partition_cache | read_count | 0
|
| iceberg | iceberg_table_cache | eviction_count | 0
|
| iceberg | iceberg_table_cache | hit_ratio |
0.07407407407407407 |
| iceberg | iceberg_table_cache | average_load_penalty |
1.7263263424E8 |
| iceberg | iceberg_table_cache | estimated_size | 25
|
| iceberg | iceberg_table_cache | hit_count | 2
|
| iceberg | iceberg_table_cache | read_count | 27
|
| iceberg | iceberg_snapshot_cache | eviction_count | 0
|
| iceberg | iceberg_snapshot_cache | hit_ratio | 1.0
|
| iceberg | iceberg_snapshot_cache | average_load_penalty | 0.0
|
| iceberg | iceberg_snapshot_cache | estimated_size | 0
|
| iceberg | iceberg_snapshot_cache | hit_count | 0
|
| iceberg | iceberg_snapshot_cache | read_count | 0
|
+--------------+-----------------------------+----------------------+---------------------+
```
---
be/src/exec/schema_scanner.cpp | 3 +
.../schema_catalog_meta_cache_stats_scanner.cpp | 135 +++++++++++++++++++++
.../schema_catalog_meta_cache_stats_scanner.h | 54 +++++++++
be/src/vec/exec/scan/vmeta_scanner.h | 1 -
.../main/java/org/apache/doris/common/Config.java | 2 +-
.../org/apache/doris/analysis/SchemaTableType.java | 4 +-
.../java/org/apache/doris/catalog/SchemaTable.java | 8 ++
.../doris/datasource/ExternalMetaCacheMgr.java | 12 ++
.../doris/datasource/hive/HiveMetaStoreCache.java | 22 +++-
.../hudi/source/HudiCachedPartitionProcessor.java | 12 +-
.../datasource/iceberg/IcebergMetadataCache.java | 15 ++-
.../doris/tablefunction/MetadataGenerator.java | 63 ++++++++++
gensrc/thrift/Descriptors.thrift | 3 +-
gensrc/thrift/FrontendService.thrift | 2 +
gensrc/thrift/PlanNodes.thrift | 4 +
gensrc/thrift/Types.thrift | 2 +-
.../jdbc/test_mariadb_jdbc_catalog.out | 35 ------
.../jdbc/test_mysql_jdbc_catalog.out | 35 ------
.../jdbc/test_mysql_jdbc_catalog_nereids.out | 35 ------
.../jdbc/test_mysql_jdbc_driver5_catalog.out | 35 ------
.../hive/test_external_catalog_hive.groovy | 4 +
.../iceberg/test_iceberg_table_stats.groovy | 4 +
.../jdbc/test_mariadb_jdbc_catalog.groovy | 2 +-
.../jdbc/test_mysql_jdbc_catalog.groovy | 2 +-
.../jdbc/test_mysql_jdbc_catalog_nereids.groovy | 2 +-
.../jdbc/test_mysql_jdbc_driver5_catalog.groovy | 2 +-
26 files changed, 343 insertions(+), 155 deletions(-)
diff --git a/be/src/exec/schema_scanner.cpp b/be/src/exec/schema_scanner.cpp
index b5cd484b650..cce15067035 100644
--- a/be/src/exec/schema_scanner.cpp
+++ b/be/src/exec/schema_scanner.cpp
@@ -28,6 +28,7 @@
#include "exec/schema_scanner/schema_active_queries_scanner.h"
#include "exec/schema_scanner/schema_backend_active_tasks.h"
+#include "exec/schema_scanner/schema_catalog_meta_cache_stats_scanner.h"
#include "exec/schema_scanner/schema_charsets_scanner.h"
#include "exec/schema_scanner/schema_collations_scanner.h"
#include "exec/schema_scanner/schema_columns_scanner.h"
@@ -240,6 +241,8 @@ std::unique_ptr<SchemaScanner>
SchemaScanner::create(TSchemaTableType::type type
return SchemaBackendWorkloadGroupResourceUsage::create_unique();
case TSchemaTableType::SCH_TABLE_PROPERTIES:
return SchemaTablePropertiesScanner::create_unique();
+ case TSchemaTableType::SCH_CATALOG_META_CACHE_STATISTICS:
+ return SchemaCatalogMetaCacheStatsScanner::create_unique();
default:
return SchemaDummyScanner::create_unique();
break;
diff --git
a/be/src/exec/schema_scanner/schema_catalog_meta_cache_stats_scanner.cpp
b/be/src/exec/schema_scanner/schema_catalog_meta_cache_stats_scanner.cpp
new file mode 100644
index 00000000000..4c067057729
--- /dev/null
+++ b/be/src/exec/schema_scanner/schema_catalog_meta_cache_stats_scanner.cpp
@@ -0,0 +1,135 @@
+// 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 "exec/schema_scanner/schema_catalog_meta_cache_stats_scanner.h"
+
+#include "runtime/client_cache.h"
+#include "runtime/exec_env.h"
+#include "runtime/query_context.h"
+#include "runtime/runtime_state.h"
+#include "util/thrift_rpc_helper.h"
+#include "vec/common/string_ref.h"
+#include "vec/core/block.h"
+#include "vec/data_types/data_type_factory.hpp"
+
+namespace doris {
+std::vector<SchemaScanner::ColumnDesc>
SchemaCatalogMetaCacheStatsScanner::_s_tbls_columns = {
+ {"CATALOG_NAME", TYPE_STRING, sizeof(StringRef), true},
+ {"CACHE_NAME", TYPE_STRING, sizeof(StringRef), true},
+ {"METRIC_NAME", TYPE_STRING, sizeof(StringRef), true},
+ {"METRIC_VALUE", TYPE_STRING, sizeof(StringRef), true},
+};
+
+SchemaCatalogMetaCacheStatsScanner::SchemaCatalogMetaCacheStatsScanner()
+ : SchemaScanner(_s_tbls_columns,
TSchemaTableType::SCH_CATALOG_META_CACHE_STATISTICS) {}
+
+SchemaCatalogMetaCacheStatsScanner::~SchemaCatalogMetaCacheStatsScanner() {}
+
+Status SchemaCatalogMetaCacheStatsScanner::start(RuntimeState* state) {
+ _block_rows_limit = state->batch_size();
+ _rpc_timeout = state->execution_timeout() * 1000;
+ _fe_addr = state->get_query_ctx()->current_connect_fe;
+ return Status::OK();
+}
+
+Status SchemaCatalogMetaCacheStatsScanner::_get_meta_cache_from_fe() {
+ TSchemaTableRequestParams schema_table_request_params;
+ for (int i = 0; i < _s_tbls_columns.size(); i++) {
+ schema_table_request_params.__isset.columns_name = true;
+
schema_table_request_params.columns_name.emplace_back(_s_tbls_columns[i].name);
+ }
+
schema_table_request_params.__set_current_user_ident(*_param->common_param->current_user_ident);
+
+ TFetchSchemaTableDataRequest request;
+
request.__set_schema_table_name(TSchemaTableName::CATALOG_META_CACHE_STATS);
+ request.__set_schema_table_params(schema_table_request_params);
+
+ TFetchSchemaTableDataResult result;
+
+ RETURN_IF_ERROR(ThriftRpcHelper::rpc<FrontendServiceClient>(
+ _fe_addr.hostname, _fe_addr.port,
+ [&request, &result](FrontendServiceConnection& client) {
+ client->fetchSchemaTableData(result, request);
+ },
+ _rpc_timeout));
+
+ Status status(Status::create(result.status));
+ if (!status.ok()) {
+ LOG(WARNING) << "fetch catalog meta cache stats from FE(" <<
_fe_addr.hostname
+ << ") failed, errmsg=" << status;
+ return status;
+ }
+ std::vector<TRow> result_data = result.data_batch;
+
+ _block = vectorized::Block::create_unique();
+ for (int i = 0; i < _s_tbls_columns.size(); ++i) {
+ TypeDescriptor descriptor(_s_tbls_columns[i].type);
+ auto data_type =
vectorized::DataTypeFactory::instance().create_data_type(descriptor, true);
+
_block->insert(vectorized::ColumnWithTypeAndName(data_type->create_column(),
data_type,
+
_s_tbls_columns[i].name));
+ }
+
+ _block->reserve(_block_rows_limit);
+
+ if (result_data.size() > 0) {
+ int col_size = result_data[0].column_value.size();
+ if (col_size != _s_tbls_columns.size()) {
+ return Status::InternalError<false>(
+ "catalog meta cache stats schema is not match for FE and
BE");
+ }
+ }
+
+ for (int i = 0; i < result_data.size(); i++) {
+ TRow row = result_data[i];
+ for (int j = 0; j < _s_tbls_columns.size(); j++) {
+ RETURN_IF_ERROR(insert_block_column(row.column_value[j], j,
_block.get(),
+ _s_tbls_columns[j].type));
+ }
+ }
+ return Status::OK();
+}
+
+Status
SchemaCatalogMetaCacheStatsScanner::get_next_block_internal(vectorized::Block*
block,
+ bool* eos) {
+ if (!_is_init) {
+ return Status::InternalError("Used before initialized.");
+ }
+
+ if (nullptr == block || nullptr == eos) {
+ return Status::InternalError("input pointer is nullptr.");
+ }
+
+ if (_block == nullptr) {
+ RETURN_IF_ERROR(_get_meta_cache_from_fe());
+ _total_rows = _block->rows();
+ }
+
+ if (_row_idx == _total_rows) {
+ *eos = true;
+ return Status::OK();
+ }
+
+ int current_batch_rows = std::min(_block_rows_limit, _total_rows -
_row_idx);
+ vectorized::MutableBlock mblock =
vectorized::MutableBlock::build_mutable_block(block);
+ RETURN_IF_ERROR(mblock.add_rows(_block.get(), _row_idx,
current_batch_rows));
+ _row_idx += current_batch_rows;
+
+ *eos = _row_idx == _total_rows;
+ return Status::OK();
+}
+
+} // namespace doris
diff --git
a/be/src/exec/schema_scanner/schema_catalog_meta_cache_stats_scanner.h
b/be/src/exec/schema_scanner/schema_catalog_meta_cache_stats_scanner.h
new file mode 100644
index 00000000000..dd6c908deb3
--- /dev/null
+++ b/be/src/exec/schema_scanner/schema_catalog_meta_cache_stats_scanner.h
@@ -0,0 +1,54 @@
+// 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 <vector>
+
+#include "common/status.h"
+#include "exec/schema_scanner.h"
+
+namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
+
+class SchemaCatalogMetaCacheStatsScanner : public SchemaScanner {
+ ENABLE_FACTORY_CREATOR(SchemaCatalogMetaCacheStatsScanner);
+
+public:
+ SchemaCatalogMetaCacheStatsScanner();
+ ~SchemaCatalogMetaCacheStatsScanner() override;
+
+ Status start(RuntimeState* state) override;
+ Status get_next_block_internal(vectorized::Block* block, bool* eos)
override;
+
+ static std::vector<SchemaScanner::ColumnDesc> _s_tbls_columns;
+
+private:
+ Status _get_meta_cache_from_fe();
+
+ TNetworkAddress _fe_addr;
+
+ int _block_rows_limit = 4096;
+ int _row_idx = 0;
+ int _total_rows = 0;
+ std::unique_ptr<vectorized::Block> _block = nullptr;
+ int _rpc_timeout = 3000;
+};
+}; // namespace doris
diff --git a/be/src/vec/exec/scan/vmeta_scanner.h
b/be/src/vec/exec/scan/vmeta_scanner.h
index 8256dff9b91..a9975300cdc 100644
--- a/be/src/vec/exec/scan/vmeta_scanner.h
+++ b/be/src/vec/exec/scan/vmeta_scanner.h
@@ -86,7 +86,6 @@ private:
TFetchSchemaTableDataRequest*
request);
Status _build_queries_metadata_request(const TMetaScanRange&
meta_scan_range,
TFetchSchemaTableDataRequest*
request);
-
bool _meta_eos;
TupleId _tuple_id;
TUserIdentity _user_identity;
diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
index 81e2c602c6e..9b61a734246 100644
--- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
@@ -2042,7 +2042,7 @@ public class Config extends ConfigBase {
* Decrease this value if FE's memory is small
*/
@ConfField(mutable = false, masterOnly = false)
- public static long max_external_file_cache_num = 100000;
+ public static long max_external_file_cache_num = 10000;
/**
* Max cache num of external table's schema
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java
b/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java
index 755ca3278ed..7fcb61e3d2d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java
@@ -84,7 +84,9 @@ public enum SchemaTableType {
SCH_WORKLOAD_GROUP_RESOURCE_USAGE("WORKLOAD_GROUP_RESOURCE_USAGE",
"WORKLOAD_GROUP_RESOURCE_USAGE",
TSchemaTableType.SCH_WORKLOAD_GROUP_RESOURCE_USAGE),
SCH_TABLE_PROPERTIES("TABLE_PROPERTIES", "TABLE_PROPERTIES",
- TSchemaTableType.SCH_TABLE_PROPERTIES);
+ TSchemaTableType.SCH_TABLE_PROPERTIES),
+ SCH_CATALOG_META_CACHE_STATISTICS("CATALOG_META_CACHE_STATISTICS",
"CATALOG_META_CACHE_STATISTICS",
+ TSchemaTableType.SCH_CATALOG_META_CACHE_STATISTICS);
private static final String dbName = "INFORMATION_SCHEMA";
private static SelectList fullSelectLists;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java
b/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java
index 683430548a6..174b2ca7d55 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java
@@ -557,6 +557,14 @@ public class SchemaTable extends Table {
.column("REMOTE_SCAN_BYTES_PER_SECOND",
ScalarType.createType(PrimitiveType.BIGINT))
.build())
)
+ .put("catalog_meta_cache_statistics",
+ new SchemaTable(SystemIdGenerator.getNextId(),
"catalog_meta_cache_statistics", TableType.SCHEMA,
+ builder().column("CATALOG_NAME",
ScalarType.createStringType())
+ .column("CACHE_NAME",
ScalarType.createStringType())
+ .column("METRIC_NAME",
ScalarType.createStringType())
+ .column("METRIC_VALUE",
ScalarType.createStringType())
+ .build())
+ )
.build();
private boolean fetchAllFe = false;
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalMetaCacheMgr.java
b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalMetaCacheMgr.java
index 513fc951672..8759f9f5ef2 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalMetaCacheMgr.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalMetaCacheMgr.java
@@ -36,6 +36,7 @@ import
org.apache.doris.nereids.exceptions.NotSupportedException;
import com.github.benmanes.caffeine.cache.CacheLoader;
import com.github.benmanes.caffeine.cache.RemovalListener;
+import com.github.benmanes.caffeine.cache.stats.CacheStats;
import com.google.common.collect.Maps;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -296,4 +297,15 @@ public class ExternalMetaCacheMgr {
maxSize, namesCacheLoader, metaObjCacheLoader,
removalListener);
return metaCache;
}
+
+ public static Map<String, String> getCacheStats(CacheStats cacheStats,
long estimatedSize) {
+ Map<String, String> stats = Maps.newHashMap();
+ stats.put("hit_ratio", String.valueOf(cacheStats.hitRate()));
+ stats.put("hit_count", String.valueOf(cacheStats.hitCount()));
+ stats.put("read_count", String.valueOf(cacheStats.hitCount() +
cacheStats.missCount()));
+ stats.put("eviction_count",
String.valueOf(cacheStats.evictionCount()));
+ stats.put("average_load_penalty",
String.valueOf(cacheStats.averageLoadPenalty()));
+ stats.put("estimated_size", String.valueOf(estimatedSize));
+ return stats;
+ }
}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java
b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java
index 99338fb87ad..3483ca155ba 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java
@@ -36,6 +36,7 @@ import org.apache.doris.common.util.CacheBulkLoader;
import org.apache.doris.common.util.LocationPath;
import org.apache.doris.common.util.Util;
import org.apache.doris.datasource.CacheException;
+import org.apache.doris.datasource.ExternalMetaCacheMgr;
import org.apache.doris.datasource.hive.AcidInfo.DeleteDeltaInfo;
import org.apache.doris.datasource.property.PropertyConverter;
import org.apache.doris.fs.FileSystemCache;
@@ -141,7 +142,7 @@ public class HiveMetaStoreCache {
OptionalLong.of(28800L),
OptionalLong.of(Config.external_cache_expire_time_minutes_after_access * 60L),
Config.max_hive_partition_table_cache_num,
- false,
+ true,
null);
partitionValuesCache = partitionValuesCacheFactory.buildCache(key ->
loadPartitionValues(key), null,
refreshExecutor);
@@ -150,7 +151,7 @@ public class HiveMetaStoreCache {
OptionalLong.of(28800L),
OptionalLong.of(Config.external_cache_expire_time_minutes_after_access * 60L),
Config.max_hive_partition_cache_num,
- false,
+ true,
null);
partitionCache = partitionCacheFactory.buildCache(new
CacheLoader<PartitionCacheKey, HivePartition>() {
@Override
@@ -183,7 +184,7 @@ public class HiveMetaStoreCache {
? fileMetaCacheTtlSecond : 28800L),
OptionalLong.of(Config.external_cache_expire_time_minutes_after_access * 60L),
Config.max_external_file_cache_num,
- false,
+ true,
null);
CacheLoader<FileCacheKey, FileCacheValue> loader = new
CacheBulkLoader<FileCacheKey, FileCacheValue>() {
@@ -1135,4 +1136,19 @@ public class HiveMetaStoreCache {
return copy;
}
}
+
+ /**
+ * get cache stats
+ * @return <cache name -> <metric name -> metric value>>
+ */
+ public Map<String, Map<String, String>> getStats() {
+ Map<String, Map<String, String>> res = Maps.newHashMap();
+ res.put("hive_partition_values_cache",
ExternalMetaCacheMgr.getCacheStats(partitionValuesCache.stats(),
+ partitionCache.estimatedSize()));
+ res.put("hive_partition_cache",
+ ExternalMetaCacheMgr.getCacheStats(partitionCache.stats(),
partitionCache.estimatedSize()));
+ res.put("hive_file_cache",
+ ExternalMetaCacheMgr.getCacheStats(fileCacheRef.get().stats(),
fileCacheRef.get().estimatedSize()));
+ return res;
+ }
}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedPartitionProcessor.java
b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedPartitionProcessor.java
index 2372b88e0db..62094b21c2b 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedPartitionProcessor.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiCachedPartitionProcessor.java
@@ -21,6 +21,7 @@ import org.apache.doris.common.CacheFactory;
import org.apache.doris.common.Config;
import org.apache.doris.common.util.Util;
import org.apache.doris.datasource.CacheException;
+import org.apache.doris.datasource.ExternalMetaCacheMgr;
import org.apache.doris.datasource.TablePartitionValues;
import org.apache.doris.datasource.TablePartitionValues.TablePartitionKey;
import org.apache.doris.datasource.hive.HMSExternalCatalog;
@@ -28,6 +29,7 @@ import org.apache.doris.datasource.hive.HMSExternalTable;
import com.github.benmanes.caffeine.cache.LoadingCache;
import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -37,6 +39,7 @@ import org.slf4j.LoggerFactory;
import java.util.Arrays;
import java.util.List;
+import java.util.Map;
import java.util.OptionalLong;
import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorService;
@@ -55,7 +58,7 @@ public class HudiCachedPartitionProcessor extends
HudiPartitionProcessor {
OptionalLong.of(28800L),
OptionalLong.of(Config.external_cache_expire_time_minutes_after_access * 60),
Config.max_external_table_cache_num,
- false,
+ true,
null);
this.partitionCache = partitionCacheFactory.buildCache(key -> new
TablePartitionValues(), null, executor);
}
@@ -167,4 +170,11 @@ public class HudiCachedPartitionProcessor extends
HudiPartitionProcessor {
throw new CacheException("Failed to get hudi partitions: " +
Util.getRootCauseMessage(e), e);
}
}
+
+ public Map<String, Map<String, String>> getCacheStats() {
+ Map<String, Map<String, String>> res = Maps.newHashMap();
+ res.put("hudi_partition_cache",
ExternalMetaCacheMgr.getCacheStats(partitionCache.stats(),
+ partitionCache.estimatedSize()));
+ return res;
+ }
}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java
b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java
index 13bd9650978..a35c73dc296 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataCache.java
@@ -22,6 +22,7 @@ import org.apache.doris.common.CacheFactory;
import org.apache.doris.common.Config;
import org.apache.doris.common.UserException;
import org.apache.doris.datasource.CatalogIf;
+import org.apache.doris.datasource.ExternalMetaCacheMgr;
import org.apache.doris.datasource.hive.HMSExternalCatalog;
import org.apache.doris.datasource.hive.HiveMetaStoreClientHelper;
import org.apache.doris.datasource.property.constants.HMSProperties;
@@ -31,6 +32,7 @@ import org.apache.doris.thrift.TIcebergMetadataParams;
import com.github.benmanes.caffeine.cache.LoadingCache;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.ManifestFiles;
import org.apache.iceberg.SerializableTable;
@@ -58,7 +60,7 @@ public class IcebergMetadataCache {
OptionalLong.of(28800L),
OptionalLong.of(Config.external_cache_expire_time_minutes_after_access * 60),
Config.max_external_table_cache_num,
- false,
+ true,
null);
this.snapshotListCache = snapshotListCacheFactory.buildCache(key ->
loadSnapshots(key), null, executor);
@@ -66,7 +68,7 @@ public class IcebergMetadataCache {
OptionalLong.of(28800L),
OptionalLong.of(Config.external_cache_expire_time_minutes_after_access * 60),
Config.max_external_table_cache_num,
- false,
+ true,
null);
this.tableCache = tableCacheFactory.buildCache(key -> loadTable(key),
null, executor);
}
@@ -246,4 +248,13 @@ public class IcebergMetadataCache {
return Objects.hash(catalog.getId(), dbName, tableName);
}
}
+
+ public Map<String, Map<String, String>> getCacheStats() {
+ Map<String, Map<String, String>> res = Maps.newHashMap();
+ res.put("iceberg_snapshot_cache",
ExternalMetaCacheMgr.getCacheStats(snapshotListCache.stats(),
+ snapshotListCache.estimatedSize()));
+ res.put("iceberg_table_cache",
ExternalMetaCacheMgr.getCacheStats(tableCache.stats(),
+ tableCache.estimatedSize()));
+ return res;
+ }
}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java
b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java
index 6a122d6640b..66794331377 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java
@@ -41,8 +41,12 @@ import org.apache.doris.common.util.NetUtils;
import org.apache.doris.common.util.TimeUtils;
import org.apache.doris.datasource.CatalogIf;
import org.apache.doris.datasource.ExternalCatalog;
+import org.apache.doris.datasource.ExternalMetaCacheMgr;
import org.apache.doris.datasource.InternalCatalog;
import org.apache.doris.datasource.hive.HMSExternalCatalog;
+import org.apache.doris.datasource.hive.HiveMetaStoreCache;
+import org.apache.doris.datasource.hudi.source.HudiCachedPartitionProcessor;
+import org.apache.doris.datasource.iceberg.IcebergExternalCatalog;
import org.apache.doris.datasource.iceberg.IcebergMetadataCache;
import org.apache.doris.datasource.maxcompute.MaxComputeExternalCatalog;
import org.apache.doris.job.common.JobType;
@@ -117,6 +121,8 @@ public class MetadataGenerator {
private static final ImmutableMap<String, Integer>
TABLE_PROPERTIES_COLUMN_TO_INDEX;
+ private static final ImmutableMap<String, Integer>
META_CACHE_STATS_COLUMN_TO_INDEX;
+
static {
ImmutableMap.Builder<String, Integer> activeQueriesbuilder = new
ImmutableMap.Builder();
List<Column> activeQueriesColList =
SchemaTable.TABLE_MAP.get("active_queries").getFullSchema();
@@ -164,6 +170,13 @@ public class MetadataGenerator {
propertiesBuilder.put(propertiesColList.get(i).getName().toLowerCase(), i);
}
TABLE_PROPERTIES_COLUMN_TO_INDEX = propertiesBuilder.build();
+
+ ImmutableMap.Builder<String, Integer> metaCacheBuilder = new
ImmutableMap.Builder();
+ List<Column> metaCacheColList =
SchemaTable.TABLE_MAP.get("catalog_meta_cache_statistics").getFullSchema();
+ for (int i = 0; i < metaCacheColList.size(); i++) {
+
metaCacheBuilder.put(metaCacheColList.get(i).getName().toLowerCase(), i);
+ }
+ META_CACHE_STATS_COLUMN_TO_INDEX = metaCacheBuilder.build();
}
public static TFetchSchemaTableDataResult
getMetadataTable(TFetchSchemaTableDataRequest request) throws TException {
@@ -255,6 +268,10 @@ public class MetadataGenerator {
result = tablePropertiesMetadataResult(schemaTableParams);
columnIndex = TABLE_PROPERTIES_COLUMN_TO_INDEX;
break;
+ case CATALOG_META_CACHE_STATS:
+ result = metaCacheStatsMetadataResult(schemaTableParams);
+ columnIndex = META_CACHE_STATS_COLUMN_TO_INDEX;
+ break;
default:
return errorResult("invalid schema table name.");
}
@@ -1250,4 +1267,50 @@ public class MetadataGenerator {
result.setStatus(new TStatus(TStatusCode.OK));
return result;
}
+
+ private static TFetchSchemaTableDataResult
metaCacheStatsMetadataResult(TSchemaTableRequestParams params) {
+ List<TRow> dataBatch = Lists.newArrayList();
+ TFetchSchemaTableDataResult result = new TFetchSchemaTableDataResult();
+ ExternalMetaCacheMgr mgr = Env.getCurrentEnv().getExtMetaCacheMgr();
+ for (CatalogIf catalogIf :
Env.getCurrentEnv().getCatalogMgr().getCopyOfCatalog()) {
+ if (catalogIf instanceof HMSExternalCatalog) {
+ HMSExternalCatalog catalog = (HMSExternalCatalog) catalogIf;
+ // 1. hive metastore cache
+ HiveMetaStoreCache cache = mgr.getMetaStoreCache(catalog);
+ if (cache != null) {
+ fillBatch(dataBatch, cache.getStats(), catalog.getName());
+ }
+ // 2. hudi cache
+ HudiCachedPartitionProcessor processor
+ = (HudiCachedPartitionProcessor)
mgr.getHudiPartitionProcess(catalog);
+ fillBatch(dataBatch, processor.getCacheStats(),
catalog.getName());
+ } else if (catalogIf instanceof IcebergExternalCatalog) {
+ // 3. iceberg cache
+ IcebergMetadataCache icebergCache =
mgr.getIcebergMetadataCache();
+ fillBatch(dataBatch, icebergCache.getCacheStats(),
catalogIf.getName());
+ }
+ }
+
+ result.setDataBatch(dataBatch);
+ result.setStatus(new TStatus(TStatusCode.OK));
+ return result;
+ }
+
+ private static void fillBatch(List<TRow> dataBatch, Map<String,
Map<String, String>> stats,
+ String catalogName) {
+ for (Map.Entry<String, Map<String, String>> entry : stats.entrySet()) {
+ String cacheName = entry.getKey();
+ Map<String, String> cacheStats = entry.getValue();
+ for (Map.Entry<String, String> cacheStatsEntry :
cacheStats.entrySet()) {
+ String metricName = cacheStatsEntry.getKey();
+ String metricValue = cacheStatsEntry.getValue();
+ TRow trow = new TRow();
+ trow.addToColumnValue(new TCell().setStringVal(catalogName));
// CATALOG_NAME
+ trow.addToColumnValue(new TCell().setStringVal(cacheName)); //
CACHE_NAME
+ trow.addToColumnValue(new TCell().setStringVal(metricName));
// METRIC_NAME
+ trow.addToColumnValue(new TCell().setStringVal(metricValue));
// METRIC_VALUE
+ dataBatch.add(trow);
+ }
+ }
+ }
}
diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift
index e11160caa7f..56222c2338a 100644
--- a/gensrc/thrift/Descriptors.thrift
+++ b/gensrc/thrift/Descriptors.thrift
@@ -137,7 +137,8 @@ enum TSchemaTableType {
SCH_WORKLOAD_GROUP_PRIVILEGES = 48,
SCH_WORKLOAD_GROUP_RESOURCE_USAGE = 49,
SCH_TABLE_PROPERTIES = 50,
- SCH_FILE_CACHE_STATISTICS = 51
+ SCH_FILE_CACHE_STATISTICS = 51,
+ SCH_CATALOG_META_CACHE_STATISTICS = 52;
}
enum THdfsCompression {
diff --git a/gensrc/thrift/FrontendService.thrift
b/gensrc/thrift/FrontendService.thrift
index 2cad1a55e42..765e7790e90 100644
--- a/gensrc/thrift/FrontendService.thrift
+++ b/gensrc/thrift/FrontendService.thrift
@@ -1004,6 +1004,7 @@ enum TSchemaTableName {
TABLE_OPTIONS = 6,
WORKLOAD_GROUP_PRIVILEGES = 7,
TABLE_PROPERTIES = 8,
+ CATALOG_META_CACHE_STATS = 9,
}
struct TMetadataTableRequestParams {
@@ -1018,6 +1019,7 @@ struct TMetadataTableRequestParams {
9: optional PlanNodes.TJobsMetadataParams jobs_metadata_params
10: optional PlanNodes.TTasksMetadataParams tasks_metadata_params
11: optional PlanNodes.TPartitionsMetadataParams partitions_metadata_params
+ 12: optional PlanNodes.TMetaCacheStatsParams meta_cache_stats_params
}
struct TSchemaTableRequestParams {
diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift
index 758ead76532..e53289c1b86 100644
--- a/gensrc/thrift/PlanNodes.thrift
+++ b/gensrc/thrift/PlanNodes.thrift
@@ -554,6 +554,9 @@ struct TQueriesMetadataParams {
6: optional TPartitionsMetadataParams partitions_params
}
+struct TMetaCacheStatsParams {
+}
+
struct TMetaScanRange {
1: optional Types.TMetadataType metadata_type
2: optional TIcebergMetadataParams iceberg_params
@@ -564,6 +567,7 @@ struct TMetaScanRange {
7: optional TJobsMetadataParams jobs_params
8: optional TTasksMetadataParams tasks_params
9: optional TPartitionsMetadataParams partitions_params
+ 10: optional TMetaCacheStatsParams meta_cache_stats_params
}
// Specification of an individual data range which is held in its entirety
diff --git a/gensrc/thrift/Types.thrift b/gensrc/thrift/Types.thrift
index e947dfc27c9..ee684a72df2 100644
--- a/gensrc/thrift/Types.thrift
+++ b/gensrc/thrift/Types.thrift
@@ -732,7 +732,7 @@ enum TMetadataType {
JOBS,
TASKS,
WORKLOAD_SCHED_POLICY,
- PARTITIONS
+ PARTITIONS;
}
enum TIcebergQueryType {
diff --git
a/regression-test/data/external_table_p0/jdbc/test_mariadb_jdbc_catalog.out
b/regression-test/data/external_table_p0/jdbc/test_mariadb_jdbc_catalog.out
index 902a0c66838..b6817f090f5 100644
--- a/regression-test/data/external_table_p0/jdbc/test_mariadb_jdbc_catalog.out
+++ b/regression-test/data/external_table_p0/jdbc/test_mariadb_jdbc_catalog.out
@@ -28,42 +28,7 @@ mariadb_jdbc_catalog
115 abg
-- !information_schema --
-active_queries
-backend_active_tasks
-character_sets
-collations
-column_privileges
-column_statistics
-columns
-engines
-events
-files
-global_variables
-key_column_usage
-metadata_name_ids
-parameters
-partitions
processlist
-profiling
-referential_constraints
-routines
-rowsets
-schema_privileges
-schemata
-session_variables
-statistics
-table_constraints
-table_options
-table_privileges
-table_properties
-tables
-triggers
-user_privileges
-views
-workload_group_privileges
-workload_group_resource_usage
-workload_groups
-workload_policy
-- !auto_default_t --
0
diff --git
a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out
b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out
index 2be6d4a141f..d697c8e5e60 100644
--- a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out
+++ b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out
@@ -192,42 +192,7 @@ bca 2022-11-02 2022-11-02 8012 vivo
2 2
-- !information_schema --
-active_queries
-backend_active_tasks
-character_sets
-collations
-column_privileges
-column_statistics
-columns
-engines
-events
-files
-global_variables
-key_column_usage
-metadata_name_ids
-parameters
-partitions
processlist
-profiling
-referential_constraints
-routines
-rowsets
-schema_privileges
-schemata
-session_variables
-statistics
-table_constraints
-table_options
-table_privileges
-table_properties
-tables
-triggers
-user_privileges
-views
-workload_group_privileges
-workload_group_resource_usage
-workload_groups
-workload_policy
-- !dt --
2023-06-17T10:00 2023-06-17T10:00:01.100 2023-06-17T10:00:02.220
2023-06-17T10:00:03.333 2023-06-17T10:00:04.444400
2023-06-17T10:00:05.555550 2023-06-17T10:00:06.666666
diff --git
a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.out
b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.out
index 08f8a3689d0..171cee4d958 100644
---
a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.out
+++
b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.out
@@ -160,42 +160,7 @@ bca 2022-11-02 2022-11-02 8012 vivo
123456789012345678901234567890123.12345
12345678901234567890123456789012.12345
1234567890123456789012345678901234.12345
123456789012345678901234567890123.12345
123456789012345678901234567890123456789012345678901234567890.12345
123456789012345678901234567890123456789012345678901234567890.12345
-- !information_schema --
-active_queries
-backend_active_tasks
-character_sets
-collations
-column_privileges
-column_statistics
-columns
-engines
-events
-files
-global_variables
-key_column_usage
-metadata_name_ids
-parameters
-partitions
processlist
-profiling
-referential_constraints
-routines
-rowsets
-schema_privileges
-schemata
-session_variables
-statistics
-table_constraints
-table_options
-table_privileges
-table_properties
-tables
-triggers
-user_privileges
-views
-workload_group_privileges
-workload_group_resource_usage
-workload_groups
-workload_policy
-- !test_insert1 --
doris1 18
diff --git
a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.out
b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.out
index 47ee7c60f38..736f2b57b4a 100644
---
a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.out
+++
b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.out
@@ -202,42 +202,7 @@ bca 2022-11-02 2022-11-02 8012 vivo
2 2
-- !information_schema --
-active_queries
-backend_active_tasks
-character_sets
-collations
-column_privileges
-column_statistics
-columns
-engines
-events
-files
-global_variables
-key_column_usage
-metadata_name_ids
-parameters
-partitions
processlist
-profiling
-referential_constraints
-routines
-rowsets
-schema_privileges
-schemata
-session_variables
-statistics
-table_constraints
-table_options
-table_privileges
-table_properties
-tables
-triggers
-user_privileges
-views
-workload_group_privileges
-workload_group_resource_usage
-workload_groups
-workload_policy
-- !dt --
2023-06-17T10:00 2023-06-17T10:00:01 2023-06-17T10:00:02
2023-06-17T10:00:03 2023-06-17T10:00:04 2023-06-17T10:00:05
2023-06-17T10:00:06
diff --git
a/regression-test/suites/external_table_p0/hive/test_external_catalog_hive.groovy
b/regression-test/suites/external_table_p0/hive/test_external_catalog_hive.groovy
index b920168198a..b71c72957e9 100644
---
a/regression-test/suites/external_table_p0/hive/test_external_catalog_hive.groovy
+++
b/regression-test/suites/external_table_p0/hive/test_external_catalog_hive.groovy
@@ -178,5 +178,9 @@ suite("test_external_catalog_hive",
"p0,external,hive,external_docker,external_d
"""
exception "Failed to init access controller: bound must be
positive"
}
+
+ // test catalog_meta_cache_statistics
+ sql """select * from
internal.information_schema.catalog_meta_cache_statistics;"""
+ sql """select * from
${catalog_name}.information_schema.catalog_meta_cache_statistics where
catalog_name="${catalog_name}";"""
}
}
diff --git
a/regression-test/suites/external_table_p0/iceberg/test_iceberg_table_stats.groovy
b/regression-test/suites/external_table_p0/iceberg/test_iceberg_table_stats.groovy
index 064139d22c9..a950f93909e 100644
---
a/regression-test/suites/external_table_p0/iceberg/test_iceberg_table_stats.groovy
+++
b/regression-test/suites/external_table_p0/iceberg/test_iceberg_table_stats.groovy
@@ -58,6 +58,10 @@ suite("test_iceberg_table_stats",
"p0,external,doris,external_docker,external_do
assert_stats("sample_mor_orc", "1000")
assert_stats("sample_mor_parquet", "1000")
+ // test catalog_meta_cache_statistics
+ sql """select * from
information_schema.catalog_meta_cache_statistics;"""
+ sql """select * from
information_schema.catalog_meta_cache_statistics where
catalog_name="${catalog_name}";"""
+
} finally {
}
}
diff --git
a/regression-test/suites/external_table_p0/jdbc/test_mariadb_jdbc_catalog.groovy
b/regression-test/suites/external_table_p0/jdbc/test_mariadb_jdbc_catalog.groovy
index d90ffe3e1cc..5281cdd20cc 100644
---
a/regression-test/suites/external_table_p0/jdbc/test_mariadb_jdbc_catalog.groovy
+++
b/regression-test/suites/external_table_p0/jdbc/test_mariadb_jdbc_catalog.groovy
@@ -68,7 +68,7 @@ suite("test_mariadb_jdbc_catalog",
"p0,external,mariadb,external_docker,external
sql """ insert into internal.${internal_db_name}.${inDorisTable}
select id, name from ${ex_tb0}; """
order_qt_in_tb """ select id, name from
internal.${internal_db_name}.${inDorisTable} order by id; """
- order_qt_information_schema """ show tables from information_schema;
"""
+ order_qt_information_schema """ show tables from information_schema
like "processlist"; """
order_qt_auto_default_t """insert into ${auto_default_t}(name)
values('a'); """
order_qt_dt """select * from ${dt}; """
diff --git
a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy
b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy
index d30d7fe9150..0be7d0ad396 100644
---
a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy
+++
b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy
@@ -168,7 +168,7 @@ suite("test_mysql_jdbc_catalog",
"p0,external,mysql,external_docker,external_doc
order_qt_ex_tb21_6 """ select `key`, `id` from ${ex_tb21} where `key`
= case when id = 1 then 1 else 0 end order by id;"""
order_qt_ex_tb21_7 """ select (`key` +1) as k, `id` from ${ex_tb21}
having abs(k) = 2 order by id;"""
order_qt_ex_tb21_8 """ select `key` as k, `id` from ${ex_tb21} having
abs(k) = 2 order by id;"""
- order_qt_information_schema """ show tables from information_schema;
"""
+ order_qt_information_schema """ show tables from information_schema
like "processlist"; """
order_qt_dt """select * from ${dt}; """
order_qt_dt_null """select * from ${dt_null} order by 1; """
order_qt_test_dz """select * from ${test_zd} order by 1; """
diff --git
a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.groovy
b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.groovy
index 24aac183eb8..e7ac412c0b5 100644
---
a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.groovy
+++
b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.groovy
@@ -113,7 +113,7 @@ suite("test_mysql_jdbc_catalog_nereids",
"p0,external,mysql,external_docker,exte
order_qt_ex_tb18 """ select * from ${ex_tb18} order by num_tinyint;
"""
order_qt_ex_tb19 """ select * from ${ex_tb19} order by date_value; """
order_qt_ex_tb20 """ select * from ${ex_tb20} order by
decimal_normal; """
- order_qt_information_schema """ show tables from information_schema;
"""
+ order_qt_information_schema """ show tables from information_schema
like "processlist"; """
// test insert
String uuid1 = UUID.randomUUID().toString();
diff --git
a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.groovy
b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.groovy
index fd1826d76e2..a74890a56d8 100644
---
a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.groovy
+++
b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.groovy
@@ -175,7 +175,7 @@ suite("test_mysql_jdbc_driver5_catalog",
"p0,external,mysql,external_docker,exte
order_qt_ex_tb21_6 """ select `key`, `id` from ${ex_tb21} where
`key` = case when id = 1 then 1 else 0 end order by id;"""
order_qt_ex_tb21_7 """ select (`key` +1) as k, `id` from
${ex_tb21} having abs(k) = 2 order by id;"""
order_qt_ex_tb21_8 """ select `key` as k, `id` from ${ex_tb21}
having abs(k) = 2 order by id;"""
- order_qt_information_schema """ show tables from
information_schema; """
+ order_qt_information_schema """ show tables from
information_schema like "processlist"; """
order_qt_dt """select * from ${dt}; """
order_qt_dt_null """select * from ${dt_null} order by 1; """
order_qt_test_dz """select * from ${test_zd} order by 1; """
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]